From 2917d4560b4a12818969185ec98cff4145d3db17 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Fri, 22 Nov 2024 12:28:05 +0800 Subject: [PATCH 1/7] *: update kvproto (#8816) ref tikv/pd#4399, ref pingcap/tidb#57338 Signed-off-by: Neil Shen --- .../server/resource_group_test.go | 28 ------------------- 1 file changed, 28 deletions(-) diff --git a/pkg/mcs/resourcemanager/server/resource_group_test.go b/pkg/mcs/resourcemanager/server/resource_group_test.go index 96325ea3653..96b15d14293 100644 --- a/pkg/mcs/resourcemanager/server/resource_group_test.go +++ b/pkg/mcs/resourcemanager/server/resource_group_test.go @@ -2,7 +2,6 @@ package server import ( "encoding/json" - "reflect" "testing" "github.com/brianvoe/gofakeit/v6" @@ -37,37 +36,10 @@ func TestPatchResourceGroup(t *testing.T) { } } -func resetSizeCache(obj any) { - resetSizeCacheRecursive(reflect.ValueOf(obj)) -} - -func resetSizeCacheRecursive(value reflect.Value) { - if value.Kind() == reflect.Ptr { - value = value.Elem() - } - - if value.Kind() != reflect.Struct { - return - } - - for i := range value.NumField() { - fieldValue := value.Field(i) - fieldType := value.Type().Field(i) - - if fieldType.Name == "XXX_sizecache" && fieldType.Type.Kind() == reflect.Int32 { - fieldValue.SetInt(0) - } else { - resetSizeCacheRecursive(fieldValue) - } - } -} - func TestClone(t *testing.T) { for i := 0; i <= 10; i++ { var rg ResourceGroup gofakeit.Struct(&rg) - // hack to reset XXX_sizecache, gofakeit will random set this field but proto clone will not copy this field. - resetSizeCache(&rg) rgClone := rg.Clone(true) require.EqualValues(t, &rg, rgClone) } From be4a366da715ff3eeb628d98248068364be8ef91 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Fri, 22 Nov 2024 13:58:38 +0800 Subject: [PATCH 2/7] tso/local: remove `allocatorPatroller`/`PriorityChecker` func (#8845) close tikv/pd#8802 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/tso/allocator_manager.go | 418 +---------------------------------- pkg/tso/filter.go | 5 - pkg/tso/local_allocator.go | 69 ------ server/api/tso.go | 38 +--- server/api/tso_test.go | 65 ------ 5 files changed, 3 insertions(+), 592 deletions(-) delete mode 100644 server/api/tso_test.go diff --git a/pkg/tso/allocator_manager.go b/pkg/tso/allocator_manager.go index a02d4884e17..821e1fc5d62 100644 --- a/pkg/tso/allocator_manager.go +++ b/pkg/tso/allocator_manager.go @@ -25,13 +25,11 @@ import ( "sync" "time" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" - "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/member" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/storage/endpoint" @@ -249,30 +247,6 @@ func (am *AllocatorManager) SetUpGlobalAllocator(ctx context.Context, leadership } } -// setUpLocalAllocator is used to set up an allocator, which will initialize the allocator and put it into allocator daemon. -// One TSO Allocator should only be set once, and may be initialized and reset multiple times depending on the election. -func (am *AllocatorManager) setUpLocalAllocator(parentCtx context.Context, dcLocation string, leadership *election.Leadership) { - am.mu.Lock() - defer am.mu.Unlock() - - if _, exist := am.mu.allocatorGroups[dcLocation]; exist { - return - } - allocator := NewLocalTSOAllocator(am, leadership, dcLocation) - // Create a new allocatorGroup - ctx, cancel := context.WithCancel(parentCtx) - am.mu.allocatorGroups[dcLocation] = &allocatorGroup{ - dcLocation: dcLocation, - ctx: ctx, - cancel: cancel, - leadership: leadership, - allocator: allocator, - } - // Start election of the Local TSO Allocator here - localTSOAllocator, _ := allocator.(*LocalTSOAllocator) - go am.allocatorLeaderLoop(parentCtx, localTSOAllocator) -} - // getGroupID returns the keyspace group ID of the allocator manager. func (am *AllocatorManager) getGroupID() uint32 { if am == nil { @@ -478,15 +452,6 @@ func (am *AllocatorManager) GetClusterDCLocationsNumber() int { return len(am.mu.clusterDCLocations) } -// compareAndSetMaxSuffix sets the max suffix sign if suffix is greater than am.mu.maxSuffix. -func (am *AllocatorManager) compareAndSetMaxSuffix(suffix int32) { - am.mu.Lock() - defer am.mu.Unlock() - if suffix > am.mu.maxSuffix { - am.mu.maxSuffix = suffix - } -} - // GetSuffixBits calculates the bits of suffix sign // by the max number of suffix so far, // which will be used in the TSO logical part. @@ -516,202 +481,11 @@ func (am *AllocatorManager) getLocalTSOAllocatorPath() string { return path.Join(am.rootPath, localTSOAllocatorEtcdPrefix) } -// similar logic with leaderLoop in server/server.go -func (am *AllocatorManager) allocatorLeaderLoop(ctx context.Context, allocator *LocalTSOAllocator) { - defer logutil.LogPanic() - defer log.Info("server is closed, return local tso allocator leader loop", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", allocator.GetDCLocation()), - zap.String("local-tso-allocator-name", am.member.Name())) - for { - select { - case <-ctx.Done(): - return - default: - } - - // Check whether the Local TSO Allocator has the leader already - allocatorLeader, rev, checkAgain := allocator.CheckAllocatorLeader() - if checkAgain { - continue - } - if allocatorLeader != nil { - log.Info("start to watch allocator leader", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.Stringer(fmt.Sprintf("%s-allocator-leader", allocator.GetDCLocation()), allocatorLeader), - zap.String("local-tso-allocator-name", am.member.Name())) - // WatchAllocatorLeader will keep looping and never return unless the Local TSO Allocator leader has changed. - allocator.WatchAllocatorLeader(ctx, allocatorLeader, rev) - log.Info("local tso allocator leader has changed, try to re-campaign a local tso allocator leader", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", allocator.GetDCLocation())) - } - - // Check the next-leader key - nextLeader, err := am.getNextLeaderID(allocator.GetDCLocation()) - if err != nil { - log.Error("get next leader from etcd failed", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", allocator.GetDCLocation()), - errs.ZapError(err)) - time.Sleep(200 * time.Millisecond) - continue - } - isNextLeader := false - if nextLeader != 0 { - if nextLeader != am.member.ID() { - log.Info("skip campaigning of the local tso allocator leader and check later", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("server-name", am.member.Name()), - zap.Uint64("server-id", am.member.ID()), - zap.Uint64("next-leader-id", nextLeader)) - time.Sleep(200 * time.Millisecond) - continue - } - isNextLeader = true - } - - // Make sure the leader is aware of this new dc-location in order to make the - // Global TSO synchronization can cover up this dc-location. - ok, dcLocationInfo, err := am.getDCLocationInfoFromLeader(ctx, allocator.GetDCLocation()) - if err != nil { - log.Error("get dc-location info from pd leader failed", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", allocator.GetDCLocation()), - errs.ZapError(err)) - // PD leader hasn't been elected out, wait for the campaign - if !longSleep(ctx, time.Second) { - return - } - continue - } - if !ok || dcLocationInfo.Suffix <= 0 || dcLocationInfo.MaxTs == nil { - log.Warn("pd leader is not aware of dc-location during allocatorLeaderLoop, wait next round", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", allocator.GetDCLocation()), - zap.Any("dc-location-info", dcLocationInfo), - zap.String("wait-duration", checkStep.String())) - // Because the checkStep is long, we use select here to check whether the ctx is done - // to prevent the leak of goroutine. - if !longSleep(ctx, checkStep) { - return - } - continue - } - - am.campaignAllocatorLeader(ctx, allocator, dcLocationInfo, isNextLeader) - } -} - -// longSleep is used to sleep the long wait duration while also watching the -// ctx.Done() to prevent the goroutine from leaking. This function returns -// true if the sleep is over, false if the ctx is done. -func longSleep(ctx context.Context, waitStep time.Duration) bool { - waitTicker := time.NewTicker(waitStep) - defer waitTicker.Stop() - select { - case <-ctx.Done(): - return false - case <-waitTicker.C: - return true - } -} - -func (am *AllocatorManager) campaignAllocatorLeader( - loopCtx context.Context, - allocator *LocalTSOAllocator, - dcLocationInfo *pdpb.GetDCLocationInfoResponse, - isNextLeader bool, -) { - logger := log.With( - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", allocator.GetDCLocation()), - zap.Any("dc-location-info", dcLocationInfo), - zap.String("name", am.member.Name()), - ) - logger.Info("start to campaign local tso allocator leader") - cmps := make([]clientv3.Cmp, 0) - nextLeaderKey := am.nextLeaderKey(allocator.GetDCLocation()) - if !isNextLeader { - cmps = append(cmps, clientv3.Compare(clientv3.CreateRevision(nextLeaderKey), "=", 0)) - } else { - nextLeaderValue := fmt.Sprintf("%v", am.member.ID()) - cmps = append(cmps, clientv3.Compare(clientv3.Value(nextLeaderKey), "=", nextLeaderValue)) - } - failpoint.Inject("injectNextLeaderKey", func(val failpoint.Value) { - if val.(bool) { - // In order not to campaign leader too often in tests - time.Sleep(5 * time.Second) - cmps = []clientv3.Cmp{ - clientv3.Compare(clientv3.Value(nextLeaderKey), "=", "mockValue"), - } - } - }) - if err := allocator.CampaignAllocatorLeader(am.leaderLease, cmps...); err != nil { - if err.Error() == errs.ErrEtcdTxnConflict.Error() { - logger.Info("failed to campaign local tso allocator leader due to txn conflict, another allocator may campaign successfully") - } else { - logger.Error("failed to campaign local tso allocator leader due to etcd error", errs.ZapError(err)) - } - return - } - - // Start keepalive the Local TSO Allocator leadership and enable Local TSO service. - ctx, cancel := context.WithCancel(loopCtx) - defer cancel() - defer am.ResetAllocatorGroup(allocator.GetDCLocation(), false) - // Maintain the Local TSO Allocator leader - go allocator.KeepAllocatorLeader(ctx) - - logger.Info("Complete campaign local tso allocator leader, begin to initialize the local TSO allocator") - if err := allocator.Initialize(int(dcLocationInfo.Suffix)); err != nil { - log.Error("failed to initialize the local TSO allocator", errs.ZapError(err)) - return - } - if dcLocationInfo.GetMaxTs().GetPhysical() != 0 { - if err := allocator.WriteTSO(dcLocationInfo.GetMaxTs()); err != nil { - log.Error("failed to write the max local TSO after member changed", errs.ZapError(err)) - return - } - } - am.compareAndSetMaxSuffix(dcLocationInfo.Suffix) - allocator.EnableAllocatorLeader() - // The next leader is me, delete it to finish campaigning - if err := am.deleteNextLeaderID(allocator.GetDCLocation()); err != nil { - logger.Warn("failed to delete next leader key after campaign local tso allocator leader", errs.ZapError(err)) - } - logger.Info("local tso allocator leader is ready to serve") - - leaderTicker := time.NewTicker(constant.LeaderTickInterval) - defer leaderTicker.Stop() - - for { - select { - case <-leaderTicker.C: - if !allocator.IsAllocatorLeader() { - logger.Info("no longer a local tso allocator leader because lease has expired, local tso allocator leader will step down") - return - } - case <-ctx.Done(): - // Server is closed and it should return nil. - logger.Info("server is closed, reset the local tso allocator") - return - } - } -} - // AllocatorDaemon is used to update every allocator's TSO and check whether we have // any new local allocator that needs to be set up. func (am *AllocatorManager) AllocatorDaemon(ctx context.Context) { log.Info("entering into allocator daemon", logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0)) - // allocatorPatroller should only work when enableLocalTSO is true to - // set up the new Local TSO Allocator in time. - var patrolTicker = &time.Ticker{} - if am.enableLocalTSO { - patrolTicker = time.NewTicker(patrolStep) - defer patrolTicker.Stop() - } tsTicker := time.NewTicker(am.updatePhysicalInterval) failpoint.Inject("fastUpdatePhysicalInterval", func() { tsTicker.Reset(time.Millisecond) @@ -722,21 +496,13 @@ func (am *AllocatorManager) AllocatorDaemon(ctx context.Context) { for { select { - case <-patrolTicker.C: - // Inspect the cluster dc-location info and set up the new Local TSO Allocator in time. - am.allocatorPatroller(ctx) case <-tsTicker.C: // Update the initialized TSO Allocator to advance TSO. am.allocatorUpdater() case <-checkerTicker.C: // Check and maintain the cluster's meta info about dc-location distribution. go am.ClusterDCLocationChecker() - // We won't have any Local TSO Allocator set up in PD without enabling Local TSO. - if am.enableLocalTSO { - // Check the election priority of every Local TSO Allocator this PD is holding. - go am.PriorityChecker() - } - // PS: ClusterDCLocationChecker and PriorityChecker are time consuming and low frequent to run, + // PS: ClusterDCLocationChecker are time consuming and low frequent to run, // we should run them concurrently to speed up the progress. case <-ctx.Done(): log.Info("exit allocator daemon", logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0)) @@ -788,33 +554,6 @@ func (am *AllocatorManager) updateAllocator(ag *allocatorGroup) { } } -// Check if we have any new dc-location configured, if yes, -// then set up the corresponding local allocator. -func (am *AllocatorManager) allocatorPatroller(serverCtx context.Context) { - // Collect all dc-locations - dcLocations := am.GetClusterDCLocations() - // Get all Local TSO Allocators - allocatorGroups := am.getAllocatorGroups(FilterDCLocation(GlobalDCLocation)) - // Set up the new one - for dcLocation := range dcLocations { - if slice.NoneOf(allocatorGroups, func(i int) bool { - return allocatorGroups[i].dcLocation == dcLocation - }) { - am.setUpLocalAllocator(serverCtx, dcLocation, election.NewLeadership( - am.member.Client(), - am.getAllocatorPath(dcLocation), - fmt.Sprintf("%s local allocator leader election", dcLocation), - )) - } - } - // Clean up the unused one - for _, ag := range allocatorGroups { - if _, exist := dcLocations[ag.dcLocation]; !exist { - am.deleteAllocatorGroup(ag.dcLocation) - } - } -} - // ClusterDCLocationChecker collects all dc-locations of a cluster, computes some related info // and stores them into the DCLocationInfo, then finally writes them into am.mu.clusterDCLocations. func (am *AllocatorManager) ClusterDCLocationChecker() { @@ -975,72 +714,6 @@ func (am *AllocatorManager) GetLocalTSOSuffixPath(dcLocation string) string { return path.Join(am.GetLocalTSOSuffixPathPrefix(), dcLocation) } -// PriorityChecker is used to check the election priority of a Local TSO Allocator. -// In the normal case, if we want to elect a Local TSO Allocator for a certain DC, -// such as dc-1, we need to make sure the follow priority rules: -// 1. The PD server with dc-location="dc-1" needs to be elected as the allocator -// leader with the highest priority. -// 2. If all PD servers with dc-location="dc-1" are down, then the other PD servers -// of DC could be elected. -func (am *AllocatorManager) PriorityChecker() { - defer logutil.LogPanic() - - serverID := am.member.ID() - myServerDCLocation := am.getServerDCLocation(serverID) - // Check all Local TSO Allocator followers to see if their priorities is higher than the leaders - // Filter out allocators with leadership and initialized - allocatorGroups := am.getAllocatorGroups(FilterDCLocation(GlobalDCLocation), FilterAvailableLeadership()) - for _, allocatorGroup := range allocatorGroups { - localTSOAllocator, _ := allocatorGroup.allocator.(*LocalTSOAllocator) - leaderServerID := localTSOAllocator.GetAllocatorLeader().GetMemberId() - // No leader, maybe the leader is not been watched yet - if leaderServerID == 0 { - continue - } - leaderServerDCLocation := am.getServerDCLocation(leaderServerID) - // For example, an allocator leader for dc-1 is elected by a server of dc-2, then the server of dc-1 will - // find this allocator's dc-location isn't the same with server of dc-2 but is same with itself. - if allocatorGroup.dcLocation != leaderServerDCLocation && allocatorGroup.dcLocation == myServerDCLocation { - log.Info("try to move the local tso allocator", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.Uint64("old-leader-id", leaderServerID), - zap.String("old-dc-location", leaderServerDCLocation), - zap.Uint64("next-leader-id", serverID), - zap.String("next-dc-location", myServerDCLocation)) - if err := am.transferLocalAllocator(allocatorGroup.dcLocation, am.member.ID()); err != nil { - log.Error("move the local tso allocator failed", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.Uint64("old-leader-id", leaderServerID), - zap.String("old-dc-location", leaderServerDCLocation), - zap.Uint64("next-leader-id", serverID), - zap.String("next-dc-location", myServerDCLocation), - errs.ZapError(err)) - continue - } - } - } - // Check next leader and resign - // Filter out allocators with leadership - allocatorGroups = am.getAllocatorGroups(FilterDCLocation(GlobalDCLocation), FilterUnavailableLeadership()) - for _, allocatorGroup := range allocatorGroups { - nextLeader, err := am.getNextLeaderID(allocatorGroup.dcLocation) - if err != nil { - log.Error("get next leader from etcd failed", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", allocatorGroup.dcLocation), - errs.ZapError(err)) - continue - } - // nextLeader is not empty and isn't same with the server ID, resign the leader - if nextLeader != 0 && nextLeader != serverID { - log.Info("next leader key found, resign current leader", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.Uint64("nextLeaderID", nextLeader)) - am.ResetAllocatorGroup(allocatorGroup.dcLocation, false) - } - } -} - // TransferAllocatorForDCLocation transfer local tso allocator to the target member for the given dcLocation func (am *AllocatorManager) TransferAllocatorForDCLocation(dcLocation string, memberID uint64) error { if dcLocation == GlobalDCLocation { @@ -1063,56 +736,6 @@ func (am *AllocatorManager) TransferAllocatorForDCLocation(dcLocation string, me return am.transferLocalAllocator(dcLocation, memberID) } -func (am *AllocatorManager) getServerDCLocation(serverID uint64) string { - am.mu.RLock() - defer am.mu.RUnlock() - for dcLocation, info := range am.mu.clusterDCLocations { - if slice.AnyOf(info.ServerIDs, func(i int) bool { return info.ServerIDs[i] == serverID }) { - return dcLocation - } - } - return "" -} - -func (am *AllocatorManager) getNextLeaderID(dcLocation string) (uint64, error) { - nextLeaderKey := am.nextLeaderKey(dcLocation) - nextLeaderValue, err := etcdutil.GetValue(am.member.Client(), nextLeaderKey) - if err != nil { - return 0, err - } - if len(nextLeaderValue) == 0 { - return 0, nil - } - return strconv.ParseUint(string(nextLeaderValue), 10, 64) -} - -func (am *AllocatorManager) deleteNextLeaderID(dcLocation string) error { - nextLeaderKey := am.nextLeaderKey(dcLocation) - resp, err := kv.NewSlowLogTxn(am.member.Client()). - Then(clientv3.OpDelete(nextLeaderKey)). - Commit() - if err != nil { - return errs.ErrEtcdKVDelete.Wrap(err).GenWithStackByCause() - } - if !resp.Succeeded { - return errs.ErrEtcdTxnConflict.FastGenByArgs() - } - return nil -} - -// deleteAllocatorGroup should only be used to remove the unused Local TSO Allocator from an unused dc-location. -// If you want to clear or reset a TSO allocator, use (*AllocatorManager).ResetAllocatorGroup. -func (am *AllocatorManager) deleteAllocatorGroup(dcLocation string) { - am.mu.Lock() - defer am.mu.Unlock() - if allocatorGroup, exist := am.mu.allocatorGroups[dcLocation]; exist { - allocatorGroup.allocator.Reset() - allocatorGroup.leadership.Reset() - allocatorGroup.cancel() - delete(am.mu.allocatorGroups, dcLocation) - } -} - // HandleRequest forwards TSO allocation requests to correct TSO Allocators. func (am *AllocatorManager) HandleRequest(ctx context.Context, dcLocation string, count uint32) (pdpb.Timestamp, error) { defer trace.StartRegion(ctx, "AllocatorManager.HandleRequest").End() @@ -1238,45 +861,6 @@ func (am *AllocatorManager) getOrCreateGRPCConn(ctx context.Context, addr string return conn, nil } -func (am *AllocatorManager) getDCLocationInfoFromLeader(ctx context.Context, dcLocation string) (bool, *pdpb.GetDCLocationInfoResponse, error) { - if am.IsLeader() { - info, ok := am.GetDCLocationInfo(dcLocation) - if !ok { - return false, &pdpb.GetDCLocationInfoResponse{}, nil - } - dcLocationInfo := &pdpb.GetDCLocationInfoResponse{Suffix: info.Suffix} - var err error - if dcLocationInfo.MaxTs, err = am.GetMaxLocalTSO(ctx); err != nil { - return false, &pdpb.GetDCLocationInfoResponse{}, err - } - return ok, dcLocationInfo, nil - } - - leaderAddr := am.GetLeaderAddr() - if len(leaderAddr) < 1 { - return false, &pdpb.GetDCLocationInfoResponse{}, fmt.Errorf("failed to get leader client url") - } - conn, err := am.getOrCreateGRPCConn(ctx, leaderAddr) - if err != nil { - return false, &pdpb.GetDCLocationInfoResponse{}, err - } - getCtx, cancel := context.WithTimeout(ctx, rpcTimeout) - defer cancel() - resp, err := pdpb.NewPDClient(conn).GetDCLocationInfo(getCtx, &pdpb.GetDCLocationInfoRequest{ - Header: &pdpb.RequestHeader{ - SenderId: am.member.ID(), - }, - DcLocation: dcLocation, - }) - if err != nil { - return false, &pdpb.GetDCLocationInfoResponse{}, err - } - if resp.GetHeader().GetError() != nil { - return false, &pdpb.GetDCLocationInfoResponse{}, errors.Errorf("get the dc-location info from leader failed: %s", resp.GetHeader().GetError().String()) - } - return resp.GetSuffix() != 0, resp, nil -} - // GetMaxLocalTSO will sync with the current Local TSO Allocators among the cluster to get the // max Local TSO. func (am *AllocatorManager) GetMaxLocalTSO(ctx context.Context) (*pdpb.Timestamp, error) { diff --git a/pkg/tso/filter.go b/pkg/tso/filter.go index 4e28e520889..c2e32a62106 100644 --- a/pkg/tso/filter.go +++ b/pkg/tso/filter.go @@ -24,11 +24,6 @@ func FilterUninitialized() func(ag *allocatorGroup) bool { return func(ag *allocatorGroup) bool { return !ag.allocator.IsInitialize() } } -// FilterAvailableLeadership will filter out the allocatorGroup whose leadership is available. -func FilterAvailableLeadership() func(ag *allocatorGroup) bool { - return func(ag *allocatorGroup) bool { return ag.leadership.Check() } -} - // FilterUnavailableLeadership will filter out the allocatorGroup whose leadership is unavailable. func FilterUnavailableLeadership() func(ag *allocatorGroup) bool { return func(ag *allocatorGroup) bool { return !ag.leadership.Check() } diff --git a/pkg/tso/local_allocator.go b/pkg/tso/local_allocator.go index 040b5891d12..e3456261a3a 100644 --- a/pkg/tso/local_allocator.go +++ b/pkg/tso/local_allocator.go @@ -26,11 +26,8 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" - "github.com/tikv/pd/pkg/utils/keypath" - "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -53,37 +50,6 @@ type LocalTSOAllocator struct { tsoAllocatorRoleGauge prometheus.Gauge } -// NewLocalTSOAllocator creates a new local TSO allocator. -func NewLocalTSOAllocator( - am *AllocatorManager, - leadership *election.Leadership, - dcLocation string, -) Allocator { - return &LocalTSOAllocator{ - allocatorManager: am, - leadership: leadership, - timestampOracle: newLocalTimestampOracle(am, leadership, dcLocation), - rootPath: leadership.GetLeaderKey(), - tsoAllocatorRoleGauge: tsoAllocatorRole.WithLabelValues(am.getGroupIDStr(), dcLocation), - } -} - -func newLocalTimestampOracle(am *AllocatorManager, leadership *election.Leadership, dcLocation string) *timestampOracle { - oracle := ×tampOracle{ - client: leadership.GetClient(), - keyspaceGroupID: am.kgID, - tsPath: keypath.KeyspaceGroupLocalTSPath(localTSOAllocatorEtcdPrefix, am.kgID, dcLocation), - storage: am.storage, - saveInterval: am.saveInterval, - updatePhysicalInterval: am.updatePhysicalInterval, - maxResetTSGap: am.maxResetTSGap, - dcLocation: dcLocation, - tsoMux: &tsoObject{}, - metrics: newTSOMetrics(am.getGroupIDStr(), dcLocation), - } - return oracle -} - // GetTimestampPath returns the timestamp path in etcd. func (lta *LocalTSOAllocator) GetTimestampPath() string { if lta == nil || lta.timestampOracle == nil { @@ -138,16 +104,6 @@ func (lta *LocalTSOAllocator) Reset() { lta.timestampOracle.ResetTimestamp() } -// setAllocatorLeader sets the current Local TSO Allocator leader. -func (lta *LocalTSOAllocator) setAllocatorLeader(member any) { - lta.allocatorLeader.Store(member) -} - -// unsetAllocatorLeader unsets the current Local TSO Allocator leader. -func (lta *LocalTSOAllocator) unsetAllocatorLeader() { - lta.allocatorLeader.Store(&pdpb.Member{}) -} - // GetAllocatorLeader returns the Local TSO Allocator leader. func (lta *LocalTSOAllocator) GetAllocatorLeader() *pdpb.Member { allocatorLeader := lta.allocatorLeader.Load() @@ -184,22 +140,6 @@ func (lta *LocalTSOAllocator) WriteTSO(maxTS *pdpb.Timestamp) error { return lta.timestampOracle.resetUserTimestamp(context.Background(), lta.leadership, tsoutil.GenerateTS(maxTS), true) } -// EnableAllocatorLeader sets the Local TSO Allocator itself to a leader. -func (lta *LocalTSOAllocator) EnableAllocatorLeader() { - lta.setAllocatorLeader(lta.allocatorManager.member.GetMember()) -} - -// CampaignAllocatorLeader is used to campaign a Local TSO Allocator's leadership. -func (lta *LocalTSOAllocator) CampaignAllocatorLeader(leaseTimeout int64, cmps ...clientv3.Cmp) error { - return lta.leadership.Campaign(leaseTimeout, lta.allocatorManager.member.MemberValue(), cmps...) -} - -// KeepAllocatorLeader is used to keep the PD leader's leadership. -func (lta *LocalTSOAllocator) KeepAllocatorLeader(ctx context.Context) { - defer logutil.LogPanic() - lta.leadership.Keep(ctx) -} - // IsAllocatorLeader returns whether the allocator is still a // Local TSO Allocator leader by checking its leadership's lease and leader info. func (lta *LocalTSOAllocator) IsAllocatorLeader() bool { @@ -251,15 +191,6 @@ func (lta *LocalTSOAllocator) CheckAllocatorLeader() (*pdpb.Member, int64, bool) return allocatorLeader, rev, false } -// WatchAllocatorLeader is used to watch the changes of the Local TSO Allocator leader. -func (lta *LocalTSOAllocator) WatchAllocatorLeader(serverCtx context.Context, allocatorLeader *pdpb.Member, revision int64) { - lta.setAllocatorLeader(allocatorLeader) - // Check the cluster dc-locations to update the max suffix bits - go lta.allocatorManager.ClusterDCLocationChecker() - lta.leadership.Watch(serverCtx, revision) - lta.unsetAllocatorLeader() -} - func (lta *LocalTSOAllocator) getMetrics() *tsoMetrics { return lta.timestampOracle.metrics } diff --git a/server/api/tso.go b/server/api/tso.go index ddac5b65bc4..95096257066 100644 --- a/server/api/tso.go +++ b/server/api/tso.go @@ -15,10 +15,8 @@ package api import ( - "fmt" "net/http" - "github.com/gorilla/mux" "github.com/tikv/pd/server" "github.com/unrolled/render" ) @@ -41,39 +39,7 @@ func newTSOHandler(svr *server.Server, rd *render.Render) *tsoHandler { // @Param name path string true "PD server name" // @Param body body object true "json params" // @Produce json -// @Success 200 {string} string "The transfer command is submitted." -// @Failure 400 {string} string "The input is invalid." -// @Failure 404 {string} string "The member does not exist." -// @Failure 500 {string} string "PD server failed to proceed the request." // @Router /tso/allocator/transfer/{name} [post] -func (h *tsoHandler) TransferLocalTSOAllocator(w http.ResponseWriter, r *http.Request) { - members, membersErr := getMembers(h.svr) - if membersErr != nil { - h.rd.JSON(w, http.StatusInternalServerError, membersErr.Error()) - return - } - name := mux.Vars(r)["name"] - dcLocation := r.URL.Query().Get("dcLocation") - if len(dcLocation) < 1 { - h.rd.JSON(w, http.StatusBadRequest, "dcLocation is undefined") - return - } - var memberID uint64 - for _, m := range members.GetMembers() { - if m.GetName() == name { - memberID = m.GetMemberId() - break - } - } - if memberID == 0 { - h.rd.JSON(w, http.StatusNotFound, fmt.Sprintf("not found, pd: %s", name)) - return - } - // TODO: support local tso forward in api service mode in the future. - err := h.svr.GetTSOAllocatorManager().TransferAllocatorForDCLocation(dcLocation, memberID) - if err != nil { - h.rd.JSON(w, http.StatusInternalServerError, err.Error()) - return - } - h.rd.JSON(w, http.StatusOK, "The transfer command is submitted.") +func (h *tsoHandler) TransferLocalTSOAllocator(w http.ResponseWriter, _ *http.Request) { + h.rd.JSON(w, http.StatusOK, "The transfer command is deprecated.") } diff --git a/server/api/tso_test.go b/server/api/tso_test.go deleted file mode 100644 index c0fbe27321d..00000000000 --- a/server/api/tso_test.go +++ /dev/null @@ -1,65 +0,0 @@ -// Copyright 2021 TiKV Project Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package api - -import ( - "fmt" - "testing" - "time" - - "github.com/stretchr/testify/suite" - tu "github.com/tikv/pd/pkg/utils/testutil" - "github.com/tikv/pd/server" - "github.com/tikv/pd/server/config" -) - -type tsoTestSuite struct { - suite.Suite - svr *server.Server - cleanup tu.CleanupFunc - urlPrefix string -} - -func TestTSOTestSuite(t *testing.T) { - suite.Run(t, new(tsoTestSuite)) -} - -func (suite *tsoTestSuite) SetupSuite() { - re := suite.Require() - suite.svr, suite.cleanup = mustNewServer(re, func(cfg *config.Config) { - cfg.EnableLocalTSO = true - cfg.Labels[config.ZoneLabel] = "dc-1" - }) - server.MustWaitLeader(re, []*server.Server{suite.svr}) - - addr := suite.svr.GetAddr() - suite.urlPrefix = fmt.Sprintf("%s%s/api/v1", addr, apiPrefix) -} - -func (suite *tsoTestSuite) TearDownSuite() { - suite.cleanup() -} - -func (suite *tsoTestSuite) TestTransferAllocator() { - re := suite.Require() - tu.Eventually(re, func() bool { - suite.svr.GetTSOAllocatorManager().ClusterDCLocationChecker() - _, err := suite.svr.GetTSOAllocatorManager().GetAllocator("dc-1") - return err == nil - }, tu.WithWaitFor(15*time.Second), tu.WithTickInterval(3*time.Second)) - addr := suite.urlPrefix + "/tso/allocator/transfer/pd1?dcLocation=dc-1" - err := tu.CheckPostJSON(testDialClient, addr, nil, tu.StatusOK(re)) - re.NoError(err) -} From 20c4157ed1540d6dc4d0b50a5cb656141e2d0f36 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Fri, 22 Nov 2024 16:33:47 +0800 Subject: [PATCH 3/7] client: separate the metrics package (#8833) ref tikv/pd#8690 Separate the client metrics package. Signed-off-by: JmPotato Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/client.go | 55 +++---- client/gc_client.go | 9 +- client/inner_client.go | 3 +- client/keyspace_client.go | 19 +-- client/meta_storage_client.go | 9 +- client/metrics.go | 251 -------------------------------- client/metrics/metrics.go | 262 ++++++++++++++++++++++++++++++++++ client/tso_client.go | 7 +- client/tso_dispatcher.go | 3 +- client/tso_request.go | 11 +- client/tso_stream.go | 13 +- 11 files changed, 331 insertions(+), 311 deletions(-) delete mode 100644 client/metrics.go create mode 100644 client/metrics/metrics.go diff --git a/client/client.go b/client/client.go index a9f5a6fae0f..bdde1a1b675 100644 --- a/client/client.go +++ b/client/client.go @@ -34,6 +34,7 @@ import ( "github.com/tikv/pd/client/caller" "github.com/tikv/pd/client/clients/metastorage" "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/utils/tlsutil" "go.uber.org/zap" @@ -555,7 +556,7 @@ func (c *client) UpdateOption(option opt.DynamicOption, value any) error { // GetAllMembers gets the members Info from PD. func (c *client) GetAllMembers(ctx context.Context) ([]*pdpb.Member, error) { start := time.Now() - defer func() { cmdDurationGetAllMembers.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationGetAllMembers.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() @@ -565,7 +566,7 @@ func (c *client) GetAllMembers(ctx context.Context) ([]*pdpb.Member, error) { return nil, errs.ErrClientGetProtoClient } resp, err := protoClient.GetMembers(ctx, req) - if err = c.respForErr(cmdFailDurationGetAllMembers, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationGetAllMembers, start, err, resp.GetHeader()); err != nil { return nil, err } return resp.GetMembers(), nil @@ -683,7 +684,7 @@ func (c *client) GetRegionFromMember(ctx context.Context, key []byte, memberURLs defer span.Finish() } start := time.Now() - defer func() { cmdDurationGetRegion.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationGetRegion.Observe(time.Since(start).Seconds()) }() var resp *pdpb.GetRegionResponse for _, url := range memberURLs { @@ -707,7 +708,7 @@ func (c *client) GetRegionFromMember(ctx context.Context, key []byte, memberURLs } if resp == nil { - cmdFailDurationGetRegion.Observe(time.Since(start).Seconds()) + metrics.CmdFailedDurationGetRegion.Observe(time.Since(start).Seconds()) c.inner.pdSvcDiscovery.ScheduleCheckMemberChanged() errorMsg := fmt.Sprintf("[pd] can't get region info from member URLs: %+v", memberURLs) return nil, errors.WithStack(errors.New(errorMsg)) @@ -722,7 +723,7 @@ func (c *client) GetRegion(ctx context.Context, key []byte, opts ...opt.GetRegio defer span.Finish() } start := time.Now() - defer func() { cmdDurationGetRegion.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationGetRegion.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() @@ -749,7 +750,7 @@ func (c *client) GetRegion(ctx context.Context, key []byte, opts ...opt.GetRegio resp, err = protoClient.GetRegion(cctx, req) } - if err = c.respForErr(cmdFailDurationGetRegion, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationGetRegion, start, err, resp.GetHeader()); err != nil { return nil, err } return handleRegionResponse(resp), nil @@ -762,7 +763,7 @@ func (c *client) GetPrevRegion(ctx context.Context, key []byte, opts ...opt.GetR defer span.Finish() } start := time.Now() - defer func() { cmdDurationGetPrevRegion.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationGetPrevRegion.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() @@ -789,7 +790,7 @@ func (c *client) GetPrevRegion(ctx context.Context, key []byte, opts ...opt.GetR resp, err = protoClient.GetPrevRegion(cctx, req) } - if err = c.respForErr(cmdFailDurationGetPrevRegion, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationGetPrevRegion, start, err, resp.GetHeader()); err != nil { return nil, err } return handleRegionResponse(resp), nil @@ -802,7 +803,7 @@ func (c *client) GetRegionByID(ctx context.Context, regionID uint64, opts ...opt defer span.Finish() } start := time.Now() - defer func() { cmdDurationGetRegionByID.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationGetRegionByID.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() @@ -829,7 +830,7 @@ func (c *client) GetRegionByID(ctx context.Context, regionID uint64, opts ...opt resp, err = protoClient.GetRegionByID(cctx, req) } - if err = c.respForErr(cmdFailedDurationGetRegionByID, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationGetRegionByID, start, err, resp.GetHeader()); err != nil { return nil, err } return handleRegionResponse(resp), nil @@ -842,7 +843,7 @@ func (c *client) ScanRegions(ctx context.Context, key, endKey []byte, limit int, defer span.Finish() } start := time.Now() - defer func() { cmdDurationScanRegions.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationScanRegions.Observe(time.Since(start).Seconds()) }() var cancel context.CancelFunc scanCtx := ctx @@ -879,7 +880,7 @@ func (c *client) ScanRegions(ctx context.Context, key, endKey []byte, limit int, resp, err = protoClient.ScanRegions(cctx, req) } - if err = c.respForErr(cmdFailedDurationScanRegions, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationScanRegions, start, err, resp.GetHeader()); err != nil { return nil, err } @@ -893,7 +894,7 @@ func (c *client) BatchScanRegions(ctx context.Context, ranges []KeyRange, limit defer span.Finish() } start := time.Now() - defer func() { cmdDurationBatchScanRegions.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationBatchScanRegions.Observe(time.Since(start).Seconds()) }() var cancel context.CancelFunc scanCtx := ctx @@ -933,7 +934,7 @@ func (c *client) BatchScanRegions(ctx context.Context, ranges []KeyRange, limit resp, err = protoClient.BatchScanRegions(cctx, req) } - if err = c.respForErr(cmdFailedDurationBatchScanRegions, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationBatchScanRegions, start, err, resp.GetHeader()); err != nil { return nil, err } @@ -993,7 +994,7 @@ func (c *client) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, e defer span.Finish() } start := time.Now() - defer func() { cmdDurationGetStore.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationGetStore.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() @@ -1007,7 +1008,7 @@ func (c *client) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, e } resp, err := protoClient.GetStore(ctx, req) - if err = c.respForErr(cmdFailedDurationGetStore, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationGetStore, start, err, resp.GetHeader()); err != nil { return nil, err } return handleStoreResponse(resp) @@ -1037,7 +1038,7 @@ func (c *client) GetAllStores(ctx context.Context, opts ...opt.GetStoreOption) ( defer span.Finish() } start := time.Now() - defer func() { cmdDurationGetAllStores.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationGetAllStores.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() @@ -1051,7 +1052,7 @@ func (c *client) GetAllStores(ctx context.Context, opts ...opt.GetStoreOption) ( } resp, err := protoClient.GetAllStores(ctx, req) - if err = c.respForErr(cmdFailedDurationGetAllStores, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationGetAllStores, start, err, resp.GetHeader()); err != nil { return nil, err } return resp.GetStores(), nil @@ -1064,7 +1065,7 @@ func (c *client) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint6 defer span.Finish() } start := time.Now() - defer func() { cmdDurationUpdateGCSafePoint.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationUpdateGCSafePoint.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() @@ -1078,7 +1079,7 @@ func (c *client) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint6 } resp, err := protoClient.UpdateGCSafePoint(ctx, req) - if err = c.respForErr(cmdFailedDurationUpdateGCSafePoint, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationUpdateGCSafePoint, start, err, resp.GetHeader()); err != nil { return 0, err } return resp.GetNewSafePoint(), nil @@ -1095,7 +1096,7 @@ func (c *client) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, } start := time.Now() - defer func() { cmdDurationUpdateServiceGCSafePoint.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationUpdateServiceGCSafePoint.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() @@ -1111,7 +1112,7 @@ func (c *client) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, } resp, err := protoClient.UpdateServiceGCSafePoint(ctx, req) - if err = c.respForErr(cmdFailedDurationUpdateServiceGCSafePoint, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationUpdateServiceGCSafePoint, start, err, resp.GetHeader()); err != nil { return 0, err } return resp.GetMinSafePoint(), nil @@ -1128,7 +1129,7 @@ func (c *client) ScatterRegion(ctx context.Context, regionID uint64) error { func (c *client) scatterRegionsWithGroup(ctx context.Context, regionID uint64, group string) error { start := time.Now() - defer func() { cmdDurationScatterRegion.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationScatterRegion.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() @@ -1167,7 +1168,7 @@ func (c *client) SplitAndScatterRegions(ctx context.Context, splitKeys [][]byte, defer span.Finish() } start := time.Now() - defer func() { cmdDurationSplitAndScatterRegions.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationSplitAndScatterRegions.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() options := &opt.RegionsOp{} @@ -1195,7 +1196,7 @@ func (c *client) GetOperator(ctx context.Context, regionID uint64) (*pdpb.GetOpe defer span.Finish() } start := time.Now() - defer func() { cmdDurationGetOperator.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationGetOperator.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() @@ -1217,7 +1218,7 @@ func (c *client) SplitRegions(ctx context.Context, splitKeys [][]byte, opts ...o defer span.Finish() } start := time.Now() - defer func() { cmdDurationSplitRegions.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationSplitRegions.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) defer cancel() options := &opt.RegionsOp{} @@ -1246,7 +1247,7 @@ func (c *client) requestHeader() *pdpb.RequestHeader { func (c *client) scatterRegionsWithOptions(ctx context.Context, regionsID []uint64, opts ...opt.RegionsOption) (*pdpb.ScatterRegionResponse, error) { start := time.Now() - defer func() { cmdDurationScatterRegions.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationScatterRegions.Observe(time.Since(start).Seconds()) }() options := &opt.RegionsOp{} for _, opt := range opts { opt(options) diff --git a/client/gc_client.go b/client/gc_client.go index 2b64cb91c4a..f30521905c3 100644 --- a/client/gc_client.go +++ b/client/gc_client.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/metrics" "go.uber.org/zap" ) @@ -39,7 +40,7 @@ func (c *client) UpdateGCSafePointV2(ctx context.Context, keyspaceID uint32, saf defer span.Finish() } start := time.Now() - defer func() { cmdDurationUpdateGCSafePointV2.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationUpdateGCSafePointV2.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) req := &pdpb.UpdateGCSafePointV2Request{ @@ -55,7 +56,7 @@ func (c *client) UpdateGCSafePointV2(ctx context.Context, keyspaceID uint32, saf resp, err := protoClient.UpdateGCSafePointV2(ctx, req) cancel() - if err = c.respForErr(cmdFailedDurationUpdateGCSafePointV2, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationUpdateGCSafePointV2, start, err, resp.GetHeader()); err != nil { return 0, err } return resp.GetNewSafePoint(), nil @@ -68,7 +69,7 @@ func (c *client) UpdateServiceSafePointV2(ctx context.Context, keyspaceID uint32 defer span.Finish() } start := time.Now() - defer func() { cmdDurationUpdateServiceSafePointV2.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationUpdateServiceSafePointV2.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) req := &pdpb.UpdateServiceSafePointV2Request{ @@ -85,7 +86,7 @@ func (c *client) UpdateServiceSafePointV2(ctx context.Context, keyspaceID uint32 } resp, err := protoClient.UpdateServiceSafePointV2(ctx, req) cancel() - if err = c.respForErr(cmdFailedDurationUpdateServiceSafePointV2, start, err, resp.GetHeader()); err != nil { + if err = c.respForErr(metrics.CmdFailedDurationUpdateServiceSafePointV2, start, err, resp.GetHeader()); err != nil { return 0, err } return resp.GetMinSafePoint(), nil diff --git a/client/inner_client.go b/client/inner_client.go index 47acda56e42..62fcd84dd5d 100644 --- a/client/inner_client.go +++ b/client/inner_client.go @@ -10,6 +10,7 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" "go.uber.org/zap" "google.golang.org/grpc" @@ -160,7 +161,7 @@ func (c *innerClient) close() { func (c *innerClient) setup() error { // Init the metrics. if c.option.InitMetrics { - initAndRegisterMetrics(c.option.MetricsLabels) + metrics.InitAndRegisterMetrics(c.option.MetricsLabels) } // Init the client base. diff --git a/client/keyspace_client.go b/client/keyspace_client.go index 3f8cea993c0..ce0cc0bc426 100644 --- a/client/keyspace_client.go +++ b/client/keyspace_client.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/metrics" ) // KeyspaceClient manages keyspace metadata. @@ -51,7 +52,7 @@ func (c *client) LoadKeyspace(ctx context.Context, name string) (*keyspacepb.Key defer span.Finish() } start := time.Now() - defer func() { cmdDurationLoadKeyspace.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationLoadKeyspace.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) req := &keyspacepb.LoadKeyspaceRequest{ Header: c.requestHeader(), @@ -66,13 +67,13 @@ func (c *client) LoadKeyspace(ctx context.Context, name string) (*keyspacepb.Key cancel() if err != nil { - cmdFailedDurationLoadKeyspace.Observe(time.Since(start).Seconds()) + metrics.CmdFailedDurationLoadKeyspace.Observe(time.Since(start).Seconds()) c.inner.pdSvcDiscovery.ScheduleCheckMemberChanged() return nil, err } if resp.Header.GetError() != nil { - cmdFailedDurationLoadKeyspace.Observe(time.Since(start).Seconds()) + metrics.CmdFailedDurationLoadKeyspace.Observe(time.Since(start).Seconds()) return nil, errors.Errorf("Load keyspace %s failed: %s", name, resp.Header.GetError().String()) } @@ -95,7 +96,7 @@ func (c *client) UpdateKeyspaceState(ctx context.Context, id uint32, state keysp defer span.Finish() } start := time.Now() - defer func() { cmdDurationUpdateKeyspaceState.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationUpdateKeyspaceState.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) req := &keyspacepb.UpdateKeyspaceStateRequest{ Header: c.requestHeader(), @@ -111,13 +112,13 @@ func (c *client) UpdateKeyspaceState(ctx context.Context, id uint32, state keysp cancel() if err != nil { - cmdFailedDurationUpdateKeyspaceState.Observe(time.Since(start).Seconds()) + metrics.CmdFailedDurationUpdateKeyspaceState.Observe(time.Since(start).Seconds()) c.inner.pdSvcDiscovery.ScheduleCheckMemberChanged() return nil, err } if resp.Header.GetError() != nil { - cmdFailedDurationUpdateKeyspaceState.Observe(time.Since(start).Seconds()) + metrics.CmdFailedDurationUpdateKeyspaceState.Observe(time.Since(start).Seconds()) return nil, errors.Errorf("Update state for keyspace id %d failed: %s", id, resp.Header.GetError().String()) } @@ -139,7 +140,7 @@ func (c *client) GetAllKeyspaces(ctx context.Context, startID uint32, limit uint defer span.Finish() } start := time.Now() - defer func() { cmdDurationGetAllKeyspaces.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationGetAllKeyspaces.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) req := &keyspacepb.GetAllKeyspacesRequest{ Header: c.requestHeader(), @@ -155,13 +156,13 @@ func (c *client) GetAllKeyspaces(ctx context.Context, startID uint32, limit uint cancel() if err != nil { - cmdDurationGetAllKeyspaces.Observe(time.Since(start).Seconds()) + metrics.CmdDurationGetAllKeyspaces.Observe(time.Since(start).Seconds()) c.inner.pdSvcDiscovery.ScheduleCheckMemberChanged() return nil, err } if resp.Header.GetError() != nil { - cmdDurationGetAllKeyspaces.Observe(time.Since(start).Seconds()) + metrics.CmdDurationGetAllKeyspaces.Observe(time.Since(start).Seconds()) return nil, errors.Errorf("Get all keyspaces metadata failed: %s", resp.Header.GetError().String()) } diff --git a/client/meta_storage_client.go b/client/meta_storage_client.go index 6409b6e7a46..e73982f981d 100644 --- a/client/meta_storage_client.go +++ b/client/meta_storage_client.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/kvproto/pkg/meta_storagepb" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/utils/grpcutil" ) @@ -62,7 +63,7 @@ func (c *innerClient) Put(ctx context.Context, key, value []byte, opts ...opt.Me defer span.Finish() } start := time.Now() - defer func() { cmdDurationPut.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationPut.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.option.Timeout) req := &meta_storagepb.PutRequest{ @@ -80,7 +81,7 @@ func (c *innerClient) Put(ctx context.Context, key, value []byte, opts ...opt.Me resp, err := cli.Put(ctx, req) cancel() - if err = c.respForMetaStorageErr(cmdFailedDurationPut, start, err, resp.GetHeader()); err != nil { + if err = c.respForMetaStorageErr(metrics.CmdFailedDurationPut, start, err, resp.GetHeader()); err != nil { return nil, err } return resp, nil @@ -101,7 +102,7 @@ func (c *innerClient) Get(ctx context.Context, key []byte, opts ...opt.MetaStora defer span.Finish() } start := time.Now() - defer func() { cmdDurationGet.Observe(time.Since(start).Seconds()) }() + defer func() { metrics.CmdDurationGet.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.option.Timeout) req := &meta_storagepb.GetRequest{ @@ -119,7 +120,7 @@ func (c *innerClient) Get(ctx context.Context, key []byte, opts ...opt.MetaStora resp, err := cli.Get(ctx, req) cancel() - if err = c.respForMetaStorageErr(cmdFailedDurationGet, start, err, resp.GetHeader()); err != nil { + if err = c.respForMetaStorageErr(metrics.CmdFailedDurationGet, start, err, resp.GetHeader()); err != nil { return nil, err } return resp, nil diff --git a/client/metrics.go b/client/metrics.go deleted file mode 100644 index fdcdd88f016..00000000000 --- a/client/metrics.go +++ /dev/null @@ -1,251 +0,0 @@ -// Copyright 2016 TiKV Project Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package pd - -import ( - "sync/atomic" - - "github.com/prometheus/client_golang/prometheus" -) - -// make sure register metrics only once -var initialized int32 - -func init() { - initMetrics(prometheus.Labels{}) - initCmdDurations() -} - -func initAndRegisterMetrics(constLabels prometheus.Labels) { - if atomic.CompareAndSwapInt32(&initialized, 0, 1) { - // init metrics with constLabels - initMetrics(constLabels) - initCmdDurations() - // register metrics - registerMetrics() - } -} - -var ( - cmdDuration *prometheus.HistogramVec - cmdFailedDuration *prometheus.HistogramVec - requestDuration *prometheus.HistogramVec - tsoBestBatchSize prometheus.Histogram - tsoBatchSize prometheus.Histogram - tsoBatchSendLatency prometheus.Histogram - requestForwarded *prometheus.GaugeVec - ongoingRequestCountGauge *prometheus.GaugeVec - estimateTSOLatencyGauge *prometheus.GaugeVec -) - -func initMetrics(constLabels prometheus.Labels) { - cmdDuration = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: "pd_client", - Subsystem: "cmd", - Name: "handle_cmds_duration_seconds", - Help: "Bucketed histogram of processing time (s) of handled success cmds.", - ConstLabels: constLabels, - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13), - }, []string{"type"}) - - cmdFailedDuration = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: "pd_client", - Subsystem: "cmd", - Name: "handle_failed_cmds_duration_seconds", - Help: "Bucketed histogram of processing time (s) of failed handled cmds.", - ConstLabels: constLabels, - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13), - }, []string{"type"}) - - requestDuration = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: "pd_client", - Subsystem: "request", - Name: "handle_requests_duration_seconds", - Help: "Bucketed histogram of processing time (s) of handled requests.", - ConstLabels: constLabels, - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13), - }, []string{"type"}) - - tsoBestBatchSize = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: "pd_client", - Subsystem: "request", - Name: "handle_tso_best_batch_size", - Help: "Bucketed histogram of the best batch size of handled requests.", - ConstLabels: constLabels, - Buckets: prometheus.ExponentialBuckets(1, 2, 13), - }) - - tsoBatchSize = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: "pd_client", - Subsystem: "request", - Name: "handle_tso_batch_size", - Help: "Bucketed histogram of the batch size of handled requests.", - ConstLabels: constLabels, - Buckets: []float64{1, 2, 4, 8, 10, 14, 18, 22, 26, 30, 35, 40, 45, 50, 60, 70, 80, 90, 100, 110, 120, 140, 160, 180, 200, 500, 1000}, - }) - - tsoBatchSendLatency = prometheus.NewHistogram( - prometheus.HistogramOpts{ - Namespace: "pd_client", - Subsystem: "request", - Name: "tso_batch_send_latency", - ConstLabels: constLabels, - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13), - Help: "tso batch send latency", - }) - - requestForwarded = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "pd_client", - Subsystem: "request", - Name: "forwarded_status", - Help: "The status to indicate if the request is forwarded", - ConstLabels: constLabels, - }, []string{"host", "delegate"}) - - ongoingRequestCountGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "pd_client", - Subsystem: "request", - Name: "ongoing_requests_count", - Help: "Current count of ongoing batch tso requests", - ConstLabels: constLabels, - }, []string{"stream"}) - estimateTSOLatencyGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "pd_client", - Subsystem: "request", - Name: "estimate_tso_latency", - Help: "Estimated latency of an RTT of getting TSO", - ConstLabels: constLabels, - }, []string{"stream"}) -} - -var ( - cmdDurationTSOWait prometheus.Observer - cmdDurationTSO prometheus.Observer - cmdDurationTSOAsyncWait prometheus.Observer - cmdDurationGetRegion prometheus.Observer - cmdDurationGetAllMembers prometheus.Observer - cmdDurationGetPrevRegion prometheus.Observer - cmdDurationGetRegionByID prometheus.Observer - cmdDurationScanRegions prometheus.Observer - cmdDurationBatchScanRegions prometheus.Observer - cmdDurationGetStore prometheus.Observer - cmdDurationGetAllStores prometheus.Observer - cmdDurationUpdateGCSafePoint prometheus.Observer - cmdDurationUpdateServiceGCSafePoint prometheus.Observer - cmdDurationScatterRegion prometheus.Observer - cmdDurationScatterRegions prometheus.Observer - cmdDurationGetOperator prometheus.Observer - cmdDurationSplitRegions prometheus.Observer - cmdDurationSplitAndScatterRegions prometheus.Observer - cmdDurationLoadKeyspace prometheus.Observer - cmdDurationUpdateKeyspaceState prometheus.Observer - cmdDurationGetAllKeyspaces prometheus.Observer - cmdDurationGet prometheus.Observer - cmdDurationPut prometheus.Observer - cmdDurationUpdateGCSafePointV2 prometheus.Observer - cmdDurationUpdateServiceSafePointV2 prometheus.Observer - - cmdFailDurationGetRegion prometheus.Observer - cmdFailDurationTSOWait prometheus.Observer - cmdFailDurationTSO prometheus.Observer - cmdFailDurationGetAllMembers prometheus.Observer - cmdFailDurationGetPrevRegion prometheus.Observer - cmdFailedDurationGetRegionByID prometheus.Observer - cmdFailedDurationScanRegions prometheus.Observer - cmdFailedDurationBatchScanRegions prometheus.Observer - cmdFailedDurationGetStore prometheus.Observer - cmdFailedDurationGetAllStores prometheus.Observer - cmdFailedDurationUpdateGCSafePoint prometheus.Observer - cmdFailedDurationUpdateServiceGCSafePoint prometheus.Observer - cmdFailedDurationLoadKeyspace prometheus.Observer - cmdFailedDurationUpdateKeyspaceState prometheus.Observer - cmdFailedDurationGet prometheus.Observer - cmdFailedDurationPut prometheus.Observer - cmdFailedDurationUpdateGCSafePointV2 prometheus.Observer - cmdFailedDurationUpdateServiceSafePointV2 prometheus.Observer - - requestDurationTSO prometheus.Observer - requestFailedDurationTSO prometheus.Observer -) - -func initCmdDurations() { - // WithLabelValues is a heavy operation, define variable to avoid call it every time. - cmdDurationTSOWait = cmdDuration.WithLabelValues("wait") - cmdDurationTSO = cmdDuration.WithLabelValues("tso") - cmdDurationTSOAsyncWait = cmdDuration.WithLabelValues("tso_async_wait") - cmdDurationGetRegion = cmdDuration.WithLabelValues("get_region") - cmdDurationGetAllMembers = cmdDuration.WithLabelValues("get_member_info") - cmdDurationGetPrevRegion = cmdDuration.WithLabelValues("get_prev_region") - cmdDurationGetRegionByID = cmdDuration.WithLabelValues("get_region_byid") - cmdDurationScanRegions = cmdDuration.WithLabelValues("scan_regions") - cmdDurationBatchScanRegions = cmdDuration.WithLabelValues("batch_scan_regions") - cmdDurationGetStore = cmdDuration.WithLabelValues("get_store") - cmdDurationGetAllStores = cmdDuration.WithLabelValues("get_all_stores") - cmdDurationUpdateGCSafePoint = cmdDuration.WithLabelValues("update_gc_safe_point") - cmdDurationUpdateServiceGCSafePoint = cmdDuration.WithLabelValues("update_service_gc_safe_point") - cmdDurationScatterRegion = cmdDuration.WithLabelValues("scatter_region") - cmdDurationScatterRegions = cmdDuration.WithLabelValues("scatter_regions") - cmdDurationGetOperator = cmdDuration.WithLabelValues("get_operator") - cmdDurationSplitRegions = cmdDuration.WithLabelValues("split_regions") - cmdDurationSplitAndScatterRegions = cmdDuration.WithLabelValues("split_and_scatter_regions") - cmdDurationLoadKeyspace = cmdDuration.WithLabelValues("load_keyspace") - cmdDurationUpdateKeyspaceState = cmdDuration.WithLabelValues("update_keyspace_state") - cmdDurationGetAllKeyspaces = cmdDuration.WithLabelValues("get_all_keyspaces") - cmdDurationGet = cmdDuration.WithLabelValues("get") - cmdDurationPut = cmdDuration.WithLabelValues("put") - cmdDurationUpdateGCSafePointV2 = cmdDuration.WithLabelValues("update_gc_safe_point_v2") - cmdDurationUpdateServiceSafePointV2 = cmdDuration.WithLabelValues("update_service_safe_point_v2") - - cmdFailDurationGetRegion = cmdFailedDuration.WithLabelValues("get_region") - cmdFailDurationTSOWait = cmdFailedDuration.WithLabelValues("wait") - cmdFailDurationTSO = cmdFailedDuration.WithLabelValues("tso") - cmdFailDurationGetAllMembers = cmdFailedDuration.WithLabelValues("get_member_info") - cmdFailDurationGetPrevRegion = cmdFailedDuration.WithLabelValues("get_prev_region") - cmdFailedDurationGetRegionByID = cmdFailedDuration.WithLabelValues("get_region_byid") - cmdFailedDurationScanRegions = cmdFailedDuration.WithLabelValues("scan_regions") - cmdFailedDurationBatchScanRegions = cmdFailedDuration.WithLabelValues("batch_scan_regions") - cmdFailedDurationGetStore = cmdFailedDuration.WithLabelValues("get_store") - cmdFailedDurationGetAllStores = cmdFailedDuration.WithLabelValues("get_all_stores") - cmdFailedDurationUpdateGCSafePoint = cmdFailedDuration.WithLabelValues("update_gc_safe_point") - cmdFailedDurationUpdateServiceGCSafePoint = cmdFailedDuration.WithLabelValues("update_service_gc_safe_point") - cmdFailedDurationLoadKeyspace = cmdFailedDuration.WithLabelValues("load_keyspace") - cmdFailedDurationUpdateKeyspaceState = cmdFailedDuration.WithLabelValues("update_keyspace_state") - cmdFailedDurationGet = cmdFailedDuration.WithLabelValues("get") - cmdFailedDurationPut = cmdFailedDuration.WithLabelValues("put") - cmdFailedDurationUpdateGCSafePointV2 = cmdFailedDuration.WithLabelValues("update_gc_safe_point_v2") - cmdFailedDurationUpdateServiceSafePointV2 = cmdFailedDuration.WithLabelValues("update_service_safe_point_v2") - - requestDurationTSO = requestDuration.WithLabelValues("tso") - requestFailedDurationTSO = requestDuration.WithLabelValues("tso-failed") -} - -func registerMetrics() { - prometheus.MustRegister(cmdDuration) - prometheus.MustRegister(cmdFailedDuration) - prometheus.MustRegister(requestDuration) - prometheus.MustRegister(tsoBestBatchSize) - prometheus.MustRegister(tsoBatchSize) - prometheus.MustRegister(tsoBatchSendLatency) - prometheus.MustRegister(requestForwarded) - prometheus.MustRegister(estimateTSOLatencyGauge) -} diff --git a/client/metrics/metrics.go b/client/metrics/metrics.go new file mode 100644 index 00000000000..da36217eb34 --- /dev/null +++ b/client/metrics/metrics.go @@ -0,0 +1,262 @@ +// Copyright 2016 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "sync/atomic" + + "github.com/prometheus/client_golang/prometheus" +) + +// make sure register metrics only once +var initialized int32 + +func init() { + initMetrics(prometheus.Labels{}) + initCmdDurations() +} + +// InitAndRegisterMetrics initializes and registers the metrics manually. +func InitAndRegisterMetrics(constLabels prometheus.Labels) { + if atomic.CompareAndSwapInt32(&initialized, 0, 1) { + // init metrics with constLabels + initMetrics(constLabels) + initCmdDurations() + // register metrics + registerMetrics() + } +} + +var ( + cmdDuration *prometheus.HistogramVec + cmdFailedDuration *prometheus.HistogramVec + requestDuration *prometheus.HistogramVec + + // TSOBestBatchSize is the histogram of the best batch size of TSO requests. + TSOBestBatchSize prometheus.Histogram + // TSOBatchSize is the histogram of the batch size of TSO requests. + TSOBatchSize prometheus.Histogram + // TSOBatchSendLatency is the histogram of the latency of sending TSO requests. + TSOBatchSendLatency prometheus.Histogram + // RequestForwarded is the gauge to indicate if the request is forwarded. + RequestForwarded *prometheus.GaugeVec + // OngoingRequestCountGauge is the gauge to indicate the count of ongoing TSO requests. + OngoingRequestCountGauge *prometheus.GaugeVec + // EstimateTSOLatencyGauge is the gauge to indicate the estimated latency of TSO requests. + EstimateTSOLatencyGauge *prometheus.GaugeVec +) + +func initMetrics(constLabels prometheus.Labels) { + cmdDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "pd_client", + Subsystem: "cmd", + Name: "handle_cmds_duration_seconds", + Help: "Bucketed histogram of processing time (s) of handled success cmds.", + ConstLabels: constLabels, + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13), + }, []string{"type"}) + + cmdFailedDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "pd_client", + Subsystem: "cmd", + Name: "handle_failed_cmds_duration_seconds", + Help: "Bucketed histogram of processing time (s) of failed handled cmds.", + ConstLabels: constLabels, + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13), + }, []string{"type"}) + + requestDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "pd_client", + Subsystem: "request", + Name: "handle_requests_duration_seconds", + Help: "Bucketed histogram of processing time (s) of handled requests.", + ConstLabels: constLabels, + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13), + }, []string{"type"}) + + TSOBestBatchSize = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "pd_client", + Subsystem: "request", + Name: "handle_tso_best_batch_size", + Help: "Bucketed histogram of the best batch size of handled requests.", + ConstLabels: constLabels, + Buckets: prometheus.ExponentialBuckets(1, 2, 13), + }) + + TSOBatchSize = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "pd_client", + Subsystem: "request", + Name: "handle_tso_batch_size", + Help: "Bucketed histogram of the batch size of handled requests.", + ConstLabels: constLabels, + Buckets: []float64{1, 2, 4, 8, 10, 14, 18, 22, 26, 30, 35, 40, 45, 50, 60, 70, 80, 90, 100, 110, 120, 140, 160, 180, 200, 500, 1000}, + }) + + TSOBatchSendLatency = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "pd_client", + Subsystem: "request", + Name: "tso_batch_send_latency", + ConstLabels: constLabels, + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13), + Help: "tso batch send latency", + }) + + RequestForwarded = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "pd_client", + Subsystem: "request", + Name: "forwarded_status", + Help: "The status to indicate if the request is forwarded", + ConstLabels: constLabels, + }, []string{"host", "delegate"}) + + OngoingRequestCountGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "pd_client", + Subsystem: "request", + Name: "ongoing_requests_count", + Help: "Current count of ongoing batch tso requests", + ConstLabels: constLabels, + }, []string{"stream"}) + EstimateTSOLatencyGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "pd_client", + Subsystem: "request", + Name: "estimate_tso_latency", + Help: "Estimated latency of an RTT of getting TSO", + ConstLabels: constLabels, + }, []string{"stream"}) +} + +// CmdDurationXXX and CmdFailedDurationXXX are the durations of the client commands. +var ( + CmdDurationTSOWait prometheus.Observer + CmdDurationTSO prometheus.Observer + CmdDurationTSOAsyncWait prometheus.Observer + CmdDurationGetRegion prometheus.Observer + CmdDurationGetAllMembers prometheus.Observer + CmdDurationGetPrevRegion prometheus.Observer + CmdDurationGetRegionByID prometheus.Observer + CmdDurationScanRegions prometheus.Observer + CmdDurationBatchScanRegions prometheus.Observer + CmdDurationGetStore prometheus.Observer + CmdDurationGetAllStores prometheus.Observer + CmdDurationUpdateGCSafePoint prometheus.Observer + CmdDurationUpdateServiceGCSafePoint prometheus.Observer + CmdDurationScatterRegion prometheus.Observer + CmdDurationScatterRegions prometheus.Observer + CmdDurationGetOperator prometheus.Observer + CmdDurationSplitRegions prometheus.Observer + CmdDurationSplitAndScatterRegions prometheus.Observer + CmdDurationLoadKeyspace prometheus.Observer + CmdDurationUpdateKeyspaceState prometheus.Observer + CmdDurationGetAllKeyspaces prometheus.Observer + CmdDurationGet prometheus.Observer + CmdDurationPut prometheus.Observer + CmdDurationUpdateGCSafePointV2 prometheus.Observer + CmdDurationUpdateServiceSafePointV2 prometheus.Observer + + CmdFailedDurationGetRegion prometheus.Observer + CmdFailedDurationTSOWait prometheus.Observer + CmdFailedDurationTSO prometheus.Observer + CmdFailedDurationGetAllMembers prometheus.Observer + CmdFailedDurationGetPrevRegion prometheus.Observer + CmdFailedDurationGetRegionByID prometheus.Observer + CmdFailedDurationScanRegions prometheus.Observer + CmdFailedDurationBatchScanRegions prometheus.Observer + CmdFailedDurationGetStore prometheus.Observer + CmdFailedDurationGetAllStores prometheus.Observer + CmdFailedDurationUpdateGCSafePoint prometheus.Observer + CmdFailedDurationUpdateServiceGCSafePoint prometheus.Observer + CmdFailedDurationLoadKeyspace prometheus.Observer + CmdFailedDurationUpdateKeyspaceState prometheus.Observer + CmdFailedDurationGet prometheus.Observer + CmdFailedDurationPut prometheus.Observer + CmdFailedDurationUpdateGCSafePointV2 prometheus.Observer + CmdFailedDurationUpdateServiceSafePointV2 prometheus.Observer + + // RequestDurationTSO records the durations of the successful TSO requests. + RequestDurationTSO prometheus.Observer + // RequestFailedDurationTSO records the durations of the failed TSO requests. + RequestFailedDurationTSO prometheus.Observer +) + +func initCmdDurations() { + // WithLabelValues is a heavy operation, define variable to avoid call it every time. + CmdDurationTSOWait = cmdDuration.WithLabelValues("wait") + CmdDurationTSO = cmdDuration.WithLabelValues("tso") + CmdDurationTSOAsyncWait = cmdDuration.WithLabelValues("tso_async_wait") + CmdDurationGetRegion = cmdDuration.WithLabelValues("get_region") + CmdDurationGetAllMembers = cmdDuration.WithLabelValues("get_member_info") + CmdDurationGetPrevRegion = cmdDuration.WithLabelValues("get_prev_region") + CmdDurationGetRegionByID = cmdDuration.WithLabelValues("get_region_byid") + CmdDurationScanRegions = cmdDuration.WithLabelValues("scan_regions") + CmdDurationBatchScanRegions = cmdDuration.WithLabelValues("batch_scan_regions") + CmdDurationGetStore = cmdDuration.WithLabelValues("get_store") + CmdDurationGetAllStores = cmdDuration.WithLabelValues("get_all_stores") + CmdDurationUpdateGCSafePoint = cmdDuration.WithLabelValues("update_gc_safe_point") + CmdDurationUpdateServiceGCSafePoint = cmdDuration.WithLabelValues("update_service_gc_safe_point") + CmdDurationScatterRegion = cmdDuration.WithLabelValues("scatter_region") + CmdDurationScatterRegions = cmdDuration.WithLabelValues("scatter_regions") + CmdDurationGetOperator = cmdDuration.WithLabelValues("get_operator") + CmdDurationSplitRegions = cmdDuration.WithLabelValues("split_regions") + CmdDurationSplitAndScatterRegions = cmdDuration.WithLabelValues("split_and_scatter_regions") + CmdDurationLoadKeyspace = cmdDuration.WithLabelValues("load_keyspace") + CmdDurationUpdateKeyspaceState = cmdDuration.WithLabelValues("update_keyspace_state") + CmdDurationGetAllKeyspaces = cmdDuration.WithLabelValues("get_all_keyspaces") + CmdDurationGet = cmdDuration.WithLabelValues("get") + CmdDurationPut = cmdDuration.WithLabelValues("put") + CmdDurationUpdateGCSafePointV2 = cmdDuration.WithLabelValues("update_gc_safe_point_v2") + CmdDurationUpdateServiceSafePointV2 = cmdDuration.WithLabelValues("update_service_safe_point_v2") + + CmdFailedDurationGetRegion = cmdFailedDuration.WithLabelValues("get_region") + CmdFailedDurationTSOWait = cmdFailedDuration.WithLabelValues("wait") + CmdFailedDurationTSO = cmdFailedDuration.WithLabelValues("tso") + CmdFailedDurationGetAllMembers = cmdFailedDuration.WithLabelValues("get_member_info") + CmdFailedDurationGetPrevRegion = cmdFailedDuration.WithLabelValues("get_prev_region") + CmdFailedDurationGetRegionByID = cmdFailedDuration.WithLabelValues("get_region_byid") + CmdFailedDurationScanRegions = cmdFailedDuration.WithLabelValues("scan_regions") + CmdFailedDurationBatchScanRegions = cmdFailedDuration.WithLabelValues("batch_scan_regions") + CmdFailedDurationGetStore = cmdFailedDuration.WithLabelValues("get_store") + CmdFailedDurationGetAllStores = cmdFailedDuration.WithLabelValues("get_all_stores") + CmdFailedDurationUpdateGCSafePoint = cmdFailedDuration.WithLabelValues("update_gc_safe_point") + CmdFailedDurationUpdateServiceGCSafePoint = cmdFailedDuration.WithLabelValues("update_service_gc_safe_point") + CmdFailedDurationLoadKeyspace = cmdFailedDuration.WithLabelValues("load_keyspace") + CmdFailedDurationUpdateKeyspaceState = cmdFailedDuration.WithLabelValues("update_keyspace_state") + CmdFailedDurationGet = cmdFailedDuration.WithLabelValues("get") + CmdFailedDurationPut = cmdFailedDuration.WithLabelValues("put") + CmdFailedDurationUpdateGCSafePointV2 = cmdFailedDuration.WithLabelValues("update_gc_safe_point_v2") + CmdFailedDurationUpdateServiceSafePointV2 = cmdFailedDuration.WithLabelValues("update_service_safe_point_v2") + + RequestDurationTSO = requestDuration.WithLabelValues("tso") + RequestFailedDurationTSO = requestDuration.WithLabelValues("tso-failed") +} + +func registerMetrics() { + prometheus.MustRegister(cmdDuration) + prometheus.MustRegister(cmdFailedDuration) + prometheus.MustRegister(requestDuration) + prometheus.MustRegister(TSOBestBatchSize) + prometheus.MustRegister(TSOBatchSize) + prometheus.MustRegister(TSOBatchSendLatency) + prometheus.MustRegister(RequestForwarded) + prometheus.MustRegister(EstimateTSOLatencyGauge) +} diff --git a/client/tso_client.go b/client/tso_client.go index 18e39dffd14..cdd85dd2479 100644 --- a/client/tso_client.go +++ b/client/tso_client.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/utils/grpcutil" "go.uber.org/zap" @@ -336,7 +337,7 @@ func (c *tsoClient) tryConnectToTSO( addr := trimHTTPPrefix(backupURL) // the goroutine is used to check the network and change back to the original stream go c.checkLeader(ctx, cancel, forwardedHostTrim, addr, url, updateAndClear) - requestForwarded.WithLabelValues(forwardedHostTrim, addr).Set(1) + metrics.RequestForwarded.WithLabelValues(forwardedHostTrim, addr).Set(1) updateAndClear(backupURL, &tsoConnectionContext{cctx, cancel, backupURL, stream}) return nil } @@ -355,7 +356,7 @@ func (c *tsoClient) checkLeader( defer func() { // cancel the forward stream forwardCancel() - requestForwarded.WithLabelValues(forwardedHostTrim, addr).Set(0) + metrics.RequestForwarded.WithLabelValues(forwardedHostTrim, addr).Set(0) }() cc, u := c.getTSOLeaderClientConn() var healthCli healthpb.HealthClient @@ -441,7 +442,7 @@ func (c *tsoClient) tryConnectToTSOWithProxy( if addr != leaderAddr { forwardedHostTrim := trimHTTPPrefix(forwardedHost) addrTrim := trimHTTPPrefix(addr) - requestForwarded.WithLabelValues(forwardedHostTrim, addrTrim).Set(1) + metrics.RequestForwarded.WithLabelValues(forwardedHostTrim, addrTrim).Set(1) } connectionCtxs.Store(addr, &tsoConnectionContext{cctx, cancel, addr, stream}) continue diff --git a/client/tso_dispatcher.go b/client/tso_dispatcher.go index 7d19a11c2d0..c696dc26b36 100644 --- a/client/tso_dispatcher.go +++ b/client/tso_dispatcher.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/retry" "github.com/tikv/pd/client/utils/timerutil" @@ -127,7 +128,7 @@ func newTSODispatcher( return newBatchController[*tsoRequest]( maxBatchSize*2, tsoRequestFinisher(0, 0, invalidStreamID), - tsoBestBatchSize, + metrics.TSOBestBatchSize, ) }, }, diff --git a/client/tso_request.go b/client/tso_request.go index 29654752cd0..d2048e4b3b1 100644 --- a/client/tso_request.go +++ b/client/tso_request.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/tikv/pd/client/metrics" ) // TSFuture is a future which promises to return a TSO. @@ -67,7 +68,7 @@ func (req *tsoRequest) waitCtx(ctx context.Context) (physical int64, logical int // If tso command duration is observed very high, the reason could be it // takes too long for Wait() be called. start := time.Now() - cmdDurationTSOAsyncWait.Observe(start.Sub(req.start).Seconds()) + metrics.CmdDurationTSOAsyncWait.Observe(start.Sub(req.start).Seconds()) select { case err = <-req.done: defer req.pool.Put(req) @@ -75,13 +76,13 @@ func (req *tsoRequest) waitCtx(ctx context.Context) (physical int64, logical int err = errors.WithStack(err) now := time.Now() if err != nil { - cmdFailDurationTSOWait.Observe(now.Sub(start).Seconds()) - cmdFailDurationTSO.Observe(now.Sub(req.start).Seconds()) + metrics.CmdFailedDurationTSOWait.Observe(now.Sub(start).Seconds()) + metrics.CmdFailedDurationTSO.Observe(now.Sub(req.start).Seconds()) return 0, 0, err } physical, logical = req.physical, req.logical - cmdDurationTSOWait.Observe(now.Sub(start).Seconds()) - cmdDurationTSO.Observe(now.Sub(req.start).Seconds()) + metrics.CmdDurationTSOWait.Observe(now.Sub(start).Seconds()) + metrics.CmdDurationTSO.Observe(now.Sub(req.start).Seconds()) return case <-ctx.Done(): return 0, 0, errors.WithStack(ctx.Err()) diff --git a/client/tso_stream.go b/client/tso_stream.go index 55bfd0b72b0..51ae5696dc4 100644 --- a/client/tso_stream.go +++ b/client/tso_stream.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/metrics" "go.uber.org/zap" "google.golang.org/grpc" ) @@ -243,7 +244,7 @@ func newTSOStream(ctx context.Context, serverURL string, stream grpcTSOStreamAda cancel: cancel, - ongoingRequestCountGauge: ongoingRequestCountGauge.WithLabelValues(streamID), + ongoingRequestCountGauge: metrics.OngoingRequestCountGauge.WithLabelValues(streamID), } s.wg.Add(1) go s.recvLoop(ctx) @@ -309,7 +310,7 @@ func (s *tsoStream) processRequests( log.Warn("failed to send RPC request through tsoStream", zap.String("stream", s.streamID), zap.Error(err)) return nil } - tsoBatchSendLatency.Observe(time.Since(batchStartTime).Seconds()) + metrics.TSOBatchSendLatency.Observe(time.Since(batchStartTime).Seconds()) s.ongoingRequestCountGauge.Set(float64(s.ongoingRequests.Add(1))) return nil } @@ -382,7 +383,7 @@ func (s *tsoStream) recvLoop(ctx context.Context) { micros := math.Exp(filteredValue) s.estimatedLatencyMicros.Store(uint64(micros)) // Update the metrics in seconds. - estimateTSOLatencyGauge.WithLabelValues(s.streamID).Set(micros * 1e-6) + metrics.EstimateTSOLatencyGauge.WithLabelValues(s.streamID).Set(micros * 1e-6) } recvLoop: @@ -413,7 +414,7 @@ recvLoop: // Note that it's also possible that the stream is broken due to network without being requested. In this // case, `Recv` may return an error while no request is pending. if hasReq { - requestFailedDurationTSO.Observe(latencySeconds) + metrics.RequestFailedDurationTSO.Observe(latencySeconds) } if err == io.EOF { finishWithErr = errors.WithStack(errs.ErrClientTSOStreamClosed) @@ -426,8 +427,8 @@ recvLoop: break recvLoop } - requestDurationTSO.Observe(latencySeconds) - tsoBatchSize.Observe(float64(res.count)) + metrics.RequestDurationTSO.Observe(latencySeconds) + metrics.TSOBatchSize.Observe(float64(res.count)) updateEstimatedLatency(currentReq.startTime, latency) if res.count != uint32(currentReq.count) { From ec77762762fe919b046a25e8d1dcf208dafae484 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Mon, 25 Nov 2024 12:08:23 +0800 Subject: [PATCH 4/7] *: independent the service discovery package (#8825) ref tikv/pd#8690 Make the service discovery module of the client a separate package. Signed-off-by: JmPotato --- client/client.go | 57 ++----- client/client_test.go | 51 ------ client/constants/constants.go | 32 ++++ client/errs/errno.go | 14 ++ client/errs/errs.go | 6 + client/http/client.go | 12 +- client/inner_client.go | 21 +-- .../mock_pd_service_discovery.go | 8 +- .../pd_service_discovery.go | 154 ++++++------------ .../pd_service_discovery_test.go | 117 +++++++++---- .../tso_service_discovery.go | 35 ++-- client/tso_client.go | 20 ++- client/tso_dispatcher.go | 11 +- client/tso_dispatcher_test.go | 5 +- client/tso_stream.go | 5 +- client/utils/tlsutil/url.go | 88 ++++++++++ tests/integrations/client/client_test.go | 35 ++-- .../resourcemanager/resource_manager_test.go | 3 +- tests/integrations/mcs/tso/server_test.go | 8 +- tests/integrations/tso/client_test.go | 23 +-- tools/pd-simulator/simulator/client.go | 4 +- tools/pd-simulator/simulator/drive.go | 4 +- 22 files changed, 396 insertions(+), 317 deletions(-) create mode 100644 client/constants/constants.go rename client/{ => servicediscovery}/mock_pd_service_discovery.go (92%) rename client/{ => servicediscovery}/pd_service_discovery.go (89%) rename client/{ => servicediscovery}/pd_service_discovery_test.go (69%) rename client/{ => servicediscovery}/tso_service_discovery.go (95%) create mode 100644 client/utils/tlsutil/url.go diff --git a/client/client.go b/client/client.go index bdde1a1b675..6781182a44b 100644 --- a/client/client.go +++ b/client/client.go @@ -33,28 +33,15 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/tikv/pd/client/caller" "github.com/tikv/pd/client/clients/metastorage" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" + sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/client/utils/tlsutil" "go.uber.org/zap" ) -const ( - // defaultKeyspaceID is the default key space id. - // Valid keyspace id range is [0, 0xFFFFFF](uint24max, or 16777215) - // ​0 is reserved for default keyspace with the name "DEFAULT", It's initialized - // when PD bootstrap and reserved for users who haven't been assigned keyspace. - defaultKeyspaceID = uint32(0) - maxKeyspaceID = uint32(0xFFFFFF) - // nullKeyspaceID is used for api v1 or legacy path where is keyspace agnostic. - nullKeyspaceID = uint32(0xFFFFFFFF) - // defaultKeySpaceGroupID is the default key space group id. - // We also reserved 0 for the keyspace group for the same purpose. - defaultKeySpaceGroupID = uint32(0) - defaultKeyspaceName = "DEFAULT" -) - // Region contains information of a region's meta and its peers. type Region struct { Meta *metapb.Region @@ -175,7 +162,7 @@ type Client interface { // syncing leader from server. GetLeaderURL() string // GetServiceDiscovery returns ServiceDiscovery - GetServiceDiscovery() ServiceDiscovery + GetServiceDiscovery() sd.ServiceDiscovery // UpdateOption updates the client option. UpdateOption(option opt.DynamicOption, value any) error @@ -184,19 +171,6 @@ type Client interface { Close() } -var ( - // errUnmatchedClusterID is returned when found a PD with a different cluster ID. - errUnmatchedClusterID = errors.New("[pd] unmatched cluster id") - // errFailInitClusterID is returned when failed to load clusterID from all supplied PD addresses. - errFailInitClusterID = errors.New("[pd] failed to get cluster id") - // errClosing is returned when request is canceled when client is closing. - errClosing = errors.New("[pd] closing") - // errTSOLength is returned when the number of response timestamps is inconsistent with request. - errTSOLength = errors.New("[pd] tso length in rpc response is incorrect") - // errInvalidRespHeader is returned when the response doesn't contain service mode info unexpectedly. - errNoServiceModeReturned = errors.New("[pd] no service mode returned") -) - var _ Client = (*client)(nil) // serviceModeKeeper is for service mode switching. @@ -206,7 +180,7 @@ type serviceModeKeeper struct { sync.RWMutex serviceMode pdpb.ServiceMode tsoClient *tsoClient - tsoSvcDiscovery ServiceDiscovery + tsoSvcDiscovery sd.ServiceDiscovery } func (k *serviceModeKeeper) close() { @@ -289,7 +263,7 @@ func NewClientWithContext( security SecurityOption, opts ...opt.ClientOption, ) (Client, error) { return createClientWithKeyspace(ctx, callerComponent, - nullKeyspaceID, svrAddrs, security, opts...) + constants.NullKeyspaceID, svrAddrs, security, opts...) } // NewClientWithKeyspace creates a client with context and the specified keyspace id. @@ -300,9 +274,9 @@ func NewClientWithKeyspace( keyspaceID uint32, svrAddrs []string, security SecurityOption, opts ...opt.ClientOption, ) (Client, error) { - if keyspaceID < defaultKeyspaceID || keyspaceID > maxKeyspaceID { + if keyspaceID < constants.DefaultKeyspaceID || keyspaceID > constants.MaxKeyspaceID { return nil, errors.Errorf("invalid keyspace id %d. It must be in the range of [%d, %d]", - keyspaceID, defaultKeyspaceID, maxKeyspaceID) + keyspaceID, constants.DefaultKeyspaceID, constants.MaxKeyspaceID) } return createClientWithKeyspace(ctx, callerComponent, keyspaceID, svrAddrs, security, opts...) @@ -392,7 +366,7 @@ type apiContextV2 struct { // NewAPIContextV2 creates a API context with the specified keyspace name for V2. func NewAPIContextV2(keyspaceName string) APIContext { if len(keyspaceName) == 0 { - keyspaceName = defaultKeyspaceName + keyspaceName = constants.DefaultKeyspaceName } return &apiContextV2{keyspaceName: keyspaceName} } @@ -452,7 +426,7 @@ func newClientWithKeyspaceName( inner: &innerClient{ // Create a PD service discovery with null keyspace id, then query the real id with the keyspace name, // finally update the keyspace id to the PD service discovery for the following interactions. - keyspaceID: nullKeyspaceID, + keyspaceID: constants.NullKeyspaceID, updateTokenConnectionCh: make(chan struct{}, 1), ctx: clientCtx, cancel: clientCancel, @@ -511,7 +485,7 @@ func (c *client) GetLeaderURL() string { } // GetServiceDiscovery returns the client-side service discovery object -func (c *client) GetServiceDiscovery() ServiceDiscovery { +func (c *client) GetServiceDiscovery() sd.ServiceDiscovery { return c.inner.pdSvcDiscovery } @@ -1277,17 +1251,6 @@ func (c *client) scatterRegionsWithOptions(ctx context.Context, regionsID []uint return resp, nil } -const ( - httpSchemePrefix = "http://" - httpsSchemePrefix = "https://" -) - -func trimHTTPPrefix(str string) string { - str = strings.TrimPrefix(str, httpSchemePrefix) - str = strings.TrimPrefix(str, httpsSchemePrefix) - return str -} - // LoadGlobalConfig implements the RPCClient interface. func (c *client) LoadGlobalConfig(ctx context.Context, names []string, configPath string) ([]GlobalConfigItem, int64, error) { ctx, cancel := context.WithTimeout(ctx, c.inner.option.Timeout) diff --git a/client/client_test.go b/client/client_test.go index 234bb2da10a..8b4cc2242ca 100644 --- a/client/client_test.go +++ b/client/client_test.go @@ -20,14 +20,12 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/tikv/pd/client/caller" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/utils/testutil" "github.com/tikv/pd/client/utils/tsoutil" "go.uber.org/goleak" - "google.golang.org/grpc" ) func TestMain(m *testing.M) { @@ -43,36 +41,6 @@ func TestTSLessEqual(t *testing.T) { re.True(tsoutil.TSLessEqual(9, 6, 9, 8)) } -func TestUpdateURLs(t *testing.T) { - re := require.New(t) - members := []*pdpb.Member{ - {Name: "pd4", ClientUrls: []string{"tmp://pd4"}}, - {Name: "pd1", ClientUrls: []string{"tmp://pd1"}}, - {Name: "pd3", ClientUrls: []string{"tmp://pd3"}}, - {Name: "pd2", ClientUrls: []string{"tmp://pd2"}}, - } - getURLs := func(ms []*pdpb.Member) (urls []string) { - for _, m := range ms { - urls = append(urls, m.GetClientUrls()[0]) - } - return - } - cli := &pdServiceDiscovery{option: opt.NewOption()} - cli.urls.Store([]string{}) - cli.updateURLs(members[1:]) - re.Equal(getURLs([]*pdpb.Member{members[1], members[3], members[2]}), cli.GetServiceURLs()) - cli.updateURLs(members[1:]) - re.Equal(getURLs([]*pdpb.Member{members[1], members[3], members[2]}), cli.GetServiceURLs()) - cli.updateURLs(members) - re.Equal(getURLs([]*pdpb.Member{members[1], members[3], members[2], members[0]}), cli.GetServiceURLs()) - cli.updateURLs(members[1:]) - re.Equal(getURLs([]*pdpb.Member{members[1], members[3], members[2]}), cli.GetServiceURLs()) - cli.updateURLs(members[2:]) - re.Equal(getURLs([]*pdpb.Member{members[3], members[2]}), cli.GetServiceURLs()) - cli.updateURLs(members[3:]) - re.Equal(getURLs([]*pdpb.Member{members[3]}), cli.GetServiceURLs()) -} - const testClientURL = "tmp://test.url:5255" func TestClientCtx(t *testing.T) { @@ -95,25 +63,6 @@ func TestClientWithRetry(t *testing.T) { re.Less(time.Since(start), time.Second*10) } -func TestGRPCDialOption(t *testing.T) { - re := require.New(t) - start := time.Now() - ctx, cancel := context.WithTimeout(context.TODO(), 500*time.Millisecond) - defer cancel() - cli := &pdServiceDiscovery{ - checkMembershipCh: make(chan struct{}, 1), - ctx: ctx, - cancel: cancel, - tlsCfg: nil, - option: opt.NewOption(), - } - cli.urls.Store([]string{testClientURL}) - cli.option.GRPCDialOptions = []grpc.DialOption{grpc.WithBlock()} - err := cli.updateMember() - re.Error(err) - re.Greater(time.Since(start), 500*time.Millisecond) -} - func TestTsoRequestWait(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) diff --git a/client/constants/constants.go b/client/constants/constants.go new file mode 100644 index 00000000000..10963dd10b6 --- /dev/null +++ b/client/constants/constants.go @@ -0,0 +1,32 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package constants + +const ( + // DefaultKeyspaceID is the default keyspace ID. + // Valid keyspace id range is [0, 0xFFFFFF](uint24max, or 16777215) + // ​0 is reserved for default keyspace with the name "DEFAULT", It's initialized + // when PD bootstrap and reserved for users who haven't been assigned keyspace. + DefaultKeyspaceID = uint32(0) + // MaxKeyspaceID is the maximum keyspace ID. + MaxKeyspaceID = uint32(0xFFFFFF) + // NullKeyspaceID is used for API v1 or legacy path where is keyspace agnostic. + NullKeyspaceID = uint32(0xFFFFFFFF) + // DefaultKeyspaceGroupID is the default key space group id. + // We also reserved 0 for the keyspace group for the same purpose. + DefaultKeyspaceGroupID = uint32(0) + // DefaultKeyspaceName is the default keyspace name. + DefaultKeyspaceName = "DEFAULT" +) diff --git a/client/errs/errno.go b/client/errs/errno.go index 95c6bffdfa4..df8b677525a 100644 --- a/client/errs/errno.go +++ b/client/errs/errno.go @@ -36,6 +36,20 @@ const ( NotPrimaryErr = "not primary" ) +// internal errors +var ( + // ErrUnmatchedClusterID is returned when found a PD with a different cluster ID. + ErrUnmatchedClusterID = errors.New("[pd] unmatched cluster id") + // ErrFailInitClusterID is returned when failed to load clusterID from all supplied PD addresses. + ErrFailInitClusterID = errors.New("[pd] failed to get cluster id") + // ErrClosing is returned when request is canceled when client is closing. + ErrClosing = errors.New("[pd] closing") + // ErrTSOLength is returned when the number of response timestamps is inconsistent with request. + ErrTSOLength = errors.New("[pd] tso length in rpc response is incorrect") + // ErrNoServiceModeReturned is returned when the response doesn't contain service mode info unexpectedly. + ErrNoServiceModeReturned = errors.New("[pd] no service mode returned") +) + // client errors var ( ErrClientGetProtoClient = errors.Normalize("failed to get proto client", errors.RFCCodeText("PD:client:ErrClientGetProtoClient")) diff --git a/client/errs/errs.go b/client/errs/errs.go index da333efda4c..67a5dd8ec92 100644 --- a/client/errs/errs.go +++ b/client/errs/errs.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" + "google.golang.org/grpc/codes" ) // IsLeaderChange will determine whether there is a leader/primary change. @@ -38,6 +39,11 @@ func IsLeaderChange(err error) bool { strings.Contains(errMsg, NotPrimaryErr) } +// IsNetworkError returns true if the error is a network error. +func IsNetworkError(code codes.Code) bool { + return code == codes.Unavailable || code == codes.DeadlineExceeded +} + // ZapError is used to make the log output easier. func ZapError(err error, causeError ...error) zap.Field { if err == nil { diff --git a/client/http/client.go b/client/http/client.go index 123ca616422..9c522d87286 100644 --- a/client/http/client.go +++ b/client/http/client.go @@ -26,9 +26,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" - pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/retry" + sd "github.com/tikv/pd/client/servicediscovery" "go.uber.org/zap" ) @@ -56,7 +56,7 @@ type clientInner struct { ctx context.Context cancel context.CancelFunc - sd pd.ServiceDiscovery + sd sd.ServiceDiscovery // source is used to mark the source of the client creation, // it will also be used in the caller ID of the inner client. @@ -74,7 +74,7 @@ func newClientInner(ctx context.Context, cancel context.CancelFunc, source strin return &clientInner{ctx: ctx, cancel: cancel, source: source} } -func (ci *clientInner) init(sd pd.ServiceDiscovery) { +func (ci *clientInner) init(sd sd.ServiceDiscovery) { // Init the HTTP client if it's not configured. if ci.cli == nil { ci.cli = &http.Client{Timeout: defaultTimeout} @@ -305,7 +305,7 @@ func WithMetrics( // NewClientWithServiceDiscovery creates a PD HTTP client with the given PD service discovery. func NewClientWithServiceDiscovery( source string, - sd pd.ServiceDiscovery, + sd sd.ServiceDiscovery, opts ...ClientOption, ) Client { ctx, cancel := context.WithCancel(context.Background()) @@ -330,7 +330,7 @@ func NewClient( for _, opt := range opts { opt(c) } - sd := pd.NewDefaultPDServiceDiscovery(ctx, cancel, pdAddrs, c.inner.tlsConf) + sd := sd.NewDefaultPDServiceDiscovery(ctx, cancel, pdAddrs, c.inner.tlsConf) if err := sd.Init(); err != nil { log.Error("[pd] init service discovery failed", zap.String("source", source), zap.Strings("pd-addrs", pdAddrs), zap.Error(err)) @@ -430,7 +430,7 @@ func newClientWithMockServiceDiscovery( for _, opt := range opts { opt(c) } - sd := pd.NewMockPDServiceDiscovery(pdAddrs, c.inner.tlsConf) + sd := sd.NewMockPDServiceDiscovery(pdAddrs, c.inner.tlsConf) if err := sd.Init(); err != nil { log.Error("[pd] init mock service discovery failed", zap.String("source", source), zap.Strings("pd-addrs", pdAddrs), zap.Error(err)) diff --git a/client/inner_client.go b/client/inner_client.go index 62fcd84dd5d..467d6b66352 100644 --- a/client/inner_client.go +++ b/client/inner_client.go @@ -12,6 +12,7 @@ import ( "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" + sd "github.com/tikv/pd/client/servicediscovery" "go.uber.org/zap" "google.golang.org/grpc" ) @@ -24,7 +25,7 @@ const ( type innerClient struct { keyspaceID uint32 svrUrls []string - pdSvcDiscovery *pdServiceDiscovery + pdSvcDiscovery sd.ServiceDiscovery tokenDispatcher *tokenDispatcher // For service mode switching. @@ -40,8 +41,8 @@ type innerClient struct { option *opt.Option } -func (c *innerClient) init(updateKeyspaceIDCb updateKeyspaceIDFunc) error { - c.pdSvcDiscovery = newPDServiceDiscovery( +func (c *innerClient) init(updateKeyspaceIDCb sd.UpdateKeyspaceIDFunc) error { + c.pdSvcDiscovery = sd.NewPDServiceDiscovery( c.ctx, c.cancel, &c.wg, c.setServiceMode, updateKeyspaceIDCb, c.keyspaceID, c.svrUrls, c.tlsCfg, c.option) if err := c.setup(); err != nil { @@ -83,14 +84,14 @@ func (c *innerClient) resetTSOClientLocked(mode pdpb.ServiceMode) { // Re-create a new TSO client. var ( newTSOCli *tsoClient - newTSOSvcDiscovery ServiceDiscovery + newTSOSvcDiscovery sd.ServiceDiscovery ) switch mode { case pdpb.ServiceMode_PD_SVC_MODE: newTSOCli = newTSOClient(c.ctx, c.option, c.pdSvcDiscovery, &pdTSOStreamBuilderFactory{}) case pdpb.ServiceMode_API_SVC_MODE: - newTSOSvcDiscovery = newTSOServiceDiscovery( + newTSOSvcDiscovery = sd.NewTSOServiceDiscovery( c.ctx, c, c.pdSvcDiscovery, c.keyspaceID, c.tlsCfg, c.option) // At this point, the keyspace group isn't known yet. Starts from the default keyspace group, @@ -152,7 +153,7 @@ func (c *innerClient) close() { c.pdSvcDiscovery.Close() if c.tokenDispatcher != nil { - tokenErr := errors.WithStack(errClosing) + tokenErr := errors.WithStack(errs.ErrClosing) c.tokenDispatcher.tokenBatchController.revokePendingTokenRequest(tokenErr) c.tokenDispatcher.dispatcherCancel() } @@ -179,10 +180,10 @@ func (c *innerClient) setup() error { // getClientAndContext returns the leader pd client and the original context. If leader is unhealthy, it returns // follower pd client and the context which holds forward information. -func (c *innerClient) getRegionAPIClientAndContext(ctx context.Context, allowFollower bool) (ServiceClient, context.Context) { - var serviceClient ServiceClient +func (c *innerClient) getRegionAPIClientAndContext(ctx context.Context, allowFollower bool) (sd.ServiceClient, context.Context) { + var serviceClient sd.ServiceClient if allowFollower { - serviceClient = c.pdSvcDiscovery.getServiceClientByKind(regionAPIKind) + serviceClient = c.pdSvcDiscovery.GetServiceClientByKind(sd.UniversalAPIKind) if serviceClient != nil { return serviceClient, serviceClient.BuildGRPCTargetContext(ctx, !allowFollower) } @@ -202,7 +203,7 @@ func (c *innerClient) gRPCErrorHandler(err error) { } func (c *innerClient) getOrCreateGRPCConn() (*grpc.ClientConn, error) { - cc, err := c.pdSvcDiscovery.GetOrCreateGRPCConn(c.pdSvcDiscovery.getLeaderURL()) + cc, err := c.pdSvcDiscovery.GetOrCreateGRPCConn(c.pdSvcDiscovery.GetServingURL()) if err != nil { return nil, err } diff --git a/client/mock_pd_service_discovery.go b/client/servicediscovery/mock_pd_service_discovery.go similarity index 92% rename from client/mock_pd_service_discovery.go rename to client/servicediscovery/mock_pd_service_discovery.go index 16462b0b1e6..87b74ae2136 100644 --- a/client/mock_pd_service_discovery.go +++ b/client/servicediscovery/mock_pd_service_discovery.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package pd +package servicediscovery import ( "crypto/tls" @@ -62,6 +62,9 @@ func (*mockPDServiceDiscovery) GetClusterID() uint64 { return 0 } // GetKeyspaceID implements the ServiceDiscovery interface. func (*mockPDServiceDiscovery) GetKeyspaceID() uint32 { return 0 } +// SetKeyspaceID implements the ServiceDiscovery interface. +func (*mockPDServiceDiscovery) SetKeyspaceID(uint32) {} + // GetKeyspaceGroupID implements the ServiceDiscovery interface. func (*mockPDServiceDiscovery) GetKeyspaceGroupID() uint32 { return 0 } @@ -83,6 +86,9 @@ func (*mockPDServiceDiscovery) GetBackupURLs() []string { return nil } // GetServiceClient implements the ServiceDiscovery interface. func (*mockPDServiceDiscovery) GetServiceClient() ServiceClient { return nil } +// GetServiceClientByKind implements the ServiceDiscovery interface. +func (*mockPDServiceDiscovery) GetServiceClientByKind(APIKind) ServiceClient { return nil } + // GetOrCreateGRPCConn implements the ServiceDiscovery interface. func (*mockPDServiceDiscovery) GetOrCreateGRPCConn(string) (*grpc.ClientConn, error) { return nil, nil diff --git a/client/pd_service_discovery.go b/client/servicediscovery/pd_service_discovery.go similarity index 89% rename from client/pd_service_discovery.go rename to client/servicediscovery/pd_service_discovery.go index 0bdc6868c65..2d106559b76 100644 --- a/client/pd_service_discovery.go +++ b/client/servicediscovery/pd_service_discovery.go @@ -12,13 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -package pd +package servicediscovery import ( "context" "crypto/tls" "fmt" - "net/url" "reflect" "sort" "strings" @@ -30,10 +29,12 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/retry" "github.com/tikv/pd/client/utils/grpcutil" + "github.com/tikv/pd/client/utils/tlsutil" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/codes" @@ -42,23 +43,29 @@ import ( ) const ( - memberUpdateInterval = time.Minute - serviceModeUpdateInterval = 3 * time.Second - updateMemberTimeout = time.Second // Use a shorter timeout to recover faster from network isolation. - updateMemberBackOffBaseTime = 100 * time.Millisecond - - httpScheme = "http" - httpsScheme = "https" + // MemberUpdateInterval is the interval to update the member list. + MemberUpdateInterval = time.Minute + // UpdateMemberTimeout is the timeout to update the member list. + // Use a shorter timeout to recover faster from network isolation. + UpdateMemberTimeout = time.Second + // UpdateMemberBackOffBaseTime is the base time to back off when updating the member list. + UpdateMemberBackOffBaseTime = 100 * time.Millisecond + + serviceModeUpdateInterval = 3 * time.Second ) // MemberHealthCheckInterval might be changed in the unit to shorten the testing time. var MemberHealthCheckInterval = time.Second -type apiKind int +// APIKind defines how this API should be handled. +type APIKind int const ( - forwardAPIKind apiKind = iota - regionAPIKind + // ForwardAPIKind means this API should be forwarded from the followers to the leader. + ForwardAPIKind APIKind = iota + // UniversalAPIKind means this API can be handled by both the leader and the followers. + UniversalAPIKind + apiKindCount ) @@ -80,6 +87,8 @@ type ServiceDiscovery interface { GetClusterID() uint64 // GetKeyspaceID returns the ID of the keyspace GetKeyspaceID() uint32 + // SetKeyspaceID sets the ID of the keyspace + SetKeyspaceID(id uint32) // GetKeyspaceGroupID returns the ID of the keyspace group GetKeyspaceGroupID() uint32 // GetServiceURLs returns the URLs of the servers providing the service @@ -101,6 +110,8 @@ type ServiceDiscovery interface { // If the leader ServiceClient meets network problem, // it returns a follower/secondary ServiceClient which can forward the request to leader. GetServiceClient() ServiceClient + // GetServiceClientByKind tries to get the ServiceClient with the given API kind. + GetServiceClientByKind(kind APIKind) ServiceClient // GetAllServiceClients tries to get all ServiceClient. // If the leader is not nil, it will put the leader service client first in the slice. GetAllServiceClients() []ServiceClient @@ -221,17 +232,13 @@ func (c *pdServiceClient) checkNetworkAvailable(ctx context.Context) { } }) rpcErr, ok := status.FromError(err) - if (ok && isNetworkError(rpcErr.Code())) || resp.GetStatus() != healthpb.HealthCheckResponse_SERVING { + if (ok && errs.IsNetworkError(rpcErr.Code())) || resp.GetStatus() != healthpb.HealthCheckResponse_SERVING { c.networkFailure.Store(true) } else { c.networkFailure.Store(false) } } -func isNetworkError(code codes.Code) bool { - return code == codes.Unavailable || code == codes.DeadlineExceeded -} - // GetClientConn implements ServiceClient. func (c *pdServiceClient) GetClientConn() *grpc.ClientConn { if c == nil { @@ -383,18 +390,19 @@ func (c *pdServiceBalancer) get() (ret ServiceClient) { return } -type updateKeyspaceIDFunc func() error +// UpdateKeyspaceIDFunc is the function type for updating the keyspace ID. +type UpdateKeyspaceIDFunc func() error type tsoLeaderURLUpdatedFunc func(string) error -// tsoEventSource subscribes to events related to changes in the TSO leader/primary from the service discovery. -type tsoEventSource interface { +// TSOEventSource subscribes to events related to changes in the TSO leader/primary from the service discovery. +type TSOEventSource interface { // SetTSOLeaderURLUpdatedCallback adds a callback which will be called when the TSO leader/primary is updated. SetTSOLeaderURLUpdatedCallback(callback tsoLeaderURLUpdatedFunc) } var ( _ ServiceDiscovery = (*pdServiceDiscovery)(nil) - _ tsoEventSource = (*pdServiceDiscovery)(nil) + _ TSOEventSource = (*pdServiceDiscovery)(nil) ) // pdServiceDiscovery is the service discovery client of PD/API service which is quorum based @@ -433,7 +441,7 @@ type pdServiceDiscovery struct { cancel context.CancelFunc closeOnce sync.Once - updateKeyspaceIDFunc updateKeyspaceIDFunc + updateKeyspaceIDFunc UpdateKeyspaceIDFunc keyspaceID uint32 tlsCfg *tls.Config // Client option. @@ -444,20 +452,20 @@ type pdServiceDiscovery struct { func NewDefaultPDServiceDiscovery( ctx context.Context, cancel context.CancelFunc, urls []string, tlsCfg *tls.Config, -) *pdServiceDiscovery { +) ServiceDiscovery { var wg sync.WaitGroup - return newPDServiceDiscovery(ctx, cancel, &wg, nil, nil, defaultKeyspaceID, urls, tlsCfg, opt.NewOption()) + return NewPDServiceDiscovery(ctx, cancel, &wg, nil, nil, constants.DefaultKeyspaceID, urls, tlsCfg, opt.NewOption()) } -// newPDServiceDiscovery returns a new PD service discovery-based client. -func newPDServiceDiscovery( +// NewPDServiceDiscovery returns a new PD service discovery-based client. +func NewPDServiceDiscovery( ctx context.Context, cancel context.CancelFunc, wg *sync.WaitGroup, serviceModeUpdateCb func(pdpb.ServiceMode), - updateKeyspaceIDFunc updateKeyspaceIDFunc, + updateKeyspaceIDFunc UpdateKeyspaceIDFunc, keyspaceID uint32, urls []string, tlsCfg *tls.Config, option *opt.Option, -) *pdServiceDiscovery { +) ServiceDiscovery { pdsd := &pdServiceDiscovery{ checkMembershipCh: make(chan struct{}, 1), ctx: ctx, @@ -470,7 +478,7 @@ func newPDServiceDiscovery( tlsCfg: tlsCfg, option: option, } - urls = addrsToURLs(urls, tlsCfg) + urls = tlsutil.AddrsToURLs(urls, tlsCfg) pdsd.urls.Store(urls) return pdsd } @@ -493,7 +501,7 @@ func (c *pdServiceDiscovery) Init() error { // 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.keyspaceID == nullKeyspaceID && c.updateKeyspaceIDFunc != nil { + if c.keyspaceID == constants.NullKeyspaceID && c.updateKeyspaceIDFunc != nil { if err := c.initRetry(c.updateKeyspaceIDFunc); err != nil { return err } @@ -534,10 +542,10 @@ func (c *pdServiceDiscovery) updateMemberLoop() { ctx, cancel := context.WithCancel(c.ctx) defer cancel() - ticker := time.NewTicker(memberUpdateInterval) + ticker := time.NewTicker(MemberUpdateInterval) defer ticker.Stop() - bo := retry.InitialBackoffer(updateMemberBackOffBaseTime, updateMemberTimeout, updateMemberBackOffBaseTime) + bo := retry.InitialBackoffer(UpdateMemberBackOffBaseTime, UpdateMemberTimeout, UpdateMemberBackOffBaseTime) for { select { case <-ctx.Done(): @@ -546,9 +554,6 @@ func (c *pdServiceDiscovery) updateMemberLoop() { case <-ticker.C: case <-c.checkMembershipCh: } - failpoint.Inject("skipUpdateMember", func() { - failpoint.Continue() - }) if err := bo.Exec(ctx, c.updateMember); err != nil { log.Error("[pd] failed to update member", zap.Strings("urls", c.GetServiceURLs()), errs.ZapError(err)) } @@ -663,7 +668,7 @@ func (c *pdServiceDiscovery) SetKeyspaceID(keyspaceID uint32) { // GetKeyspaceGroupID returns the ID of the keyspace group func (*pdServiceDiscovery) GetKeyspaceGroupID() uint32 { // PD/API service only supports the default keyspace group - return defaultKeySpaceGroupID + return constants.DefaultKeyspaceGroupID } // DiscoverMicroservice discovers the microservice with the specified type and returns the server urls. @@ -733,8 +738,8 @@ func (c *pdServiceDiscovery) getLeaderServiceClient() *pdServiceClient { return leader.(*pdServiceClient) } -// getServiceClientByKind returns ServiceClient of the specific kind. -func (c *pdServiceDiscovery) getServiceClientByKind(kind apiKind) ServiceClient { +// GetServiceClientByKind returns ServiceClient of the specific kind. +func (c *pdServiceDiscovery) GetServiceClientByKind(kind APIKind) ServiceClient { client := c.apiCandidateNodes[kind].get() if client == nil { return nil @@ -746,7 +751,7 @@ func (c *pdServiceDiscovery) getServiceClientByKind(kind apiKind) ServiceClient func (c *pdServiceDiscovery) GetServiceClient() ServiceClient { leaderClient := c.getLeaderServiceClient() if c.option.EnableForwarding && !leaderClient.Available() { - if followerClient := c.getServiceClientByKind(forwardAPIKind); followerClient != nil { + if followerClient := c.GetServiceClientByKind(ForwardAPIKind); followerClient != nil { log.Debug("[pd] use follower client", zap.String("url", followerClient.GetURL())) return followerClient } @@ -833,17 +838,14 @@ func (c *pdServiceDiscovery) initClusterID() error { clusterID = members.GetHeader().GetClusterId() continue } - failpoint.Inject("skipClusterIDCheck", func() { - failpoint.Continue() - }) // All URLs passed in should have the same cluster ID. if members.GetHeader().GetClusterId() != clusterID { - return errors.WithStack(errUnmatchedClusterID) + return errors.WithStack(errs.ErrUnmatchedClusterID) } } // Failed to init the cluster ID. if clusterID == 0 { - return errors.WithStack(errFailInitClusterID) + return errors.WithStack(errs.ErrFailInitClusterID) } c.clusterID = clusterID return nil @@ -869,7 +871,7 @@ func (c *pdServiceDiscovery) checkServiceModeChanged() error { return err } if clusterInfo == nil || len(clusterInfo.ServiceModes) == 0 { - return errors.WithStack(errNoServiceModeReturned) + return errors.WithStack(errs.ErrNoServiceModeReturned) } if c.serviceModeUpdateCb != nil { c.serviceModeUpdateCb(clusterInfo.ServiceModes[0]) @@ -878,14 +880,8 @@ func (c *pdServiceDiscovery) checkServiceModeChanged() error { } func (c *pdServiceDiscovery) updateMember() error { - for i, url := range c.GetServiceURLs() { - failpoint.Inject("skipFirstUpdateMember", func() { - if i == 0 { - failpoint.Continue() - } - }) - - members, err := c.getMembers(c.ctx, url, updateMemberTimeout) + for _, url := range c.GetServiceURLs() { + members, err := c.getMembers(c.ctx, url, UpdateMemberTimeout) // Check the cluster ID. updatedClusterID := members.GetHeader().GetClusterId() if err == nil && updatedClusterID != c.clusterID { @@ -1016,7 +1012,7 @@ func (c *pdServiceDiscovery) updateFollowers(members []*pdpb.Member, leaderID ui followerURLs = append(followerURLs, member.GetClientUrls()...) // FIXME: How to safely compare urls(also for leader)? For now, only allows one client url. - url := pickMatchedURL(member.GetClientUrls(), c.tlsCfg) + url := tlsutil.PickMatchedURL(member.GetClientUrls(), c.tlsCfg) if client, ok := c.followers.Load(url); ok { if client.(*pdServiceClient).GetClientConn() == nil { conn, err := c.GetOrCreateGRPCConn(url) @@ -1053,7 +1049,7 @@ func (c *pdServiceDiscovery) updateFollowers(members []*pdpb.Member, leaderID ui func (c *pdServiceDiscovery) updateServiceClient(members []*pdpb.Member, leader *pdpb.Member) error { // FIXME: How to safely compare leader urls? For now, only allows one client url. - leaderURL := pickMatchedURL(leader.GetClientUrls(), c.tlsCfg) + leaderURL := tlsutil.PickMatchedURL(leader.GetClientUrls(), c.tlsCfg) leaderChanged, err := c.switchLeader(leaderURL) followerChanged := c.updateFollowers(members, leader.GetMemberId(), leaderURL) // don't need to recreate balancer if no changes. @@ -1082,51 +1078,3 @@ func (c *pdServiceDiscovery) updateServiceClient(members []*pdpb.Member, leader func (c *pdServiceDiscovery) GetOrCreateGRPCConn(url string) (*grpc.ClientConn, error) { return grpcutil.GetOrCreateGRPCConn(c.ctx, &c.clientConns, url, c.tlsCfg, c.option.GRPCDialOptions...) } - -func addrsToURLs(addrs []string, tlsCfg *tls.Config) []string { - // Add default schema "http://" to addrs. - urls := make([]string, 0, len(addrs)) - for _, addr := range addrs { - urls = append(urls, modifyURLScheme(addr, tlsCfg)) - } - return urls -} - -func modifyURLScheme(url string, tlsCfg *tls.Config) string { - if tlsCfg == nil { - if strings.HasPrefix(url, httpsSchemePrefix) { - url = httpSchemePrefix + strings.TrimPrefix(url, httpsSchemePrefix) - } else if !strings.HasPrefix(url, httpSchemePrefix) { - url = httpSchemePrefix + url - } - } else { - if strings.HasPrefix(url, httpSchemePrefix) { - url = httpsSchemePrefix + strings.TrimPrefix(url, httpSchemePrefix) - } else if !strings.HasPrefix(url, httpsSchemePrefix) { - url = httpsSchemePrefix + url - } - } - return url -} - -// pickMatchedURL picks the matched URL based on the TLS config. -// Note: please make sure the URLs are valid. -func pickMatchedURL(urls []string, tlsCfg *tls.Config) string { - for _, uStr := range urls { - u, err := url.Parse(uStr) - if err != nil { - continue - } - if tlsCfg != nil && u.Scheme == httpsScheme { - return uStr - } - if tlsCfg == nil && u.Scheme == httpScheme { - return uStr - } - } - ret := modifyURLScheme(urls[0], tlsCfg) - log.Warn("[pd] no matched url found", zap.Strings("urls", urls), - zap.Bool("tls-enabled", tlsCfg != nil), - zap.String("attempted-url", ret)) - return ret -} diff --git a/client/pd_service_discovery_test.go b/client/servicediscovery/pd_service_discovery_test.go similarity index 69% rename from client/pd_service_discovery_test.go rename to client/servicediscovery/pd_service_discovery_test.go index e553b087d34..45faf2aa7f1 100644 --- a/client/pd_service_discovery_test.go +++ b/client/servicediscovery/pd_service_discovery_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package pd +package servicediscovery import ( "context" @@ -30,8 +30,10 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/utils/grpcutil" "github.com/tikv/pd/client/utils/testutil" + "github.com/tikv/pd/client/utils/tlsutil" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" pb "google.golang.org/grpc/examples/helloworld/helloworld" @@ -142,12 +144,12 @@ func (suite *serviceClientTestSuite) SetupSuite() { followerConn, err2 := grpc.Dial(suite.followerServer.addr, grpc.WithTransportCredentials(insecure.NewCredentials())) if err1 == nil && err2 == nil { suite.followerClient = newPDServiceClient( - modifyURLScheme(suite.followerServer.addr, nil), - modifyURLScheme(suite.leaderServer.addr, nil), + tlsutil.ModifyURLScheme(suite.followerServer.addr, nil), + tlsutil.ModifyURLScheme(suite.leaderServer.addr, nil), followerConn, false) suite.leaderClient = newPDServiceClient( - modifyURLScheme(suite.leaderServer.addr, nil), - modifyURLScheme(suite.leaderServer.addr, nil), + tlsutil.ModifyURLScheme(suite.leaderServer.addr, nil), + tlsutil.ModifyURLScheme(suite.leaderServer.addr, nil), leaderConn, true) suite.followerServer.server.leaderConn = suite.leaderClient.GetClientConn() suite.followerServer.server.leaderAddr = suite.leaderClient.GetURL() @@ -173,8 +175,8 @@ func (suite *serviceClientTestSuite) TearDownSuite() { func (suite *serviceClientTestSuite) TestServiceClient() { re := suite.Require() - leaderAddress := modifyURLScheme(suite.leaderServer.addr, nil) - followerAddress := modifyURLScheme(suite.followerServer.addr, nil) + leaderAddress := tlsutil.ModifyURLScheme(suite.leaderServer.addr, nil) + followerAddress := tlsutil.ModifyURLScheme(suite.followerServer.addr, nil) follower := suite.followerClient leader := suite.leaderClient @@ -188,12 +190,12 @@ func (suite *serviceClientTestSuite) TestServiceClient() { re.False(follower.IsConnectedToLeader()) re.True(leader.IsConnectedToLeader()) - re.NoError(failpoint.Enable("github.com/tikv/pd/client/unreachableNetwork1", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1", "return(true)")) follower.(*pdServiceClient).checkNetworkAvailable(suite.ctx) leader.(*pdServiceClient).checkNetworkAvailable(suite.ctx) re.False(follower.Available()) re.False(leader.Available()) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/unreachableNetwork1")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1")) follower.(*pdServiceClient).checkNetworkAvailable(suite.ctx) leader.(*pdServiceClient).checkNetworkAvailable(suite.ctx) @@ -235,7 +237,7 @@ func (suite *serviceClientTestSuite) TestServiceClient() { followerAPIClient := newPDServiceAPIClient(follower, regionAPIErrorFn) leaderAPIClient := newPDServiceAPIClient(leader, regionAPIErrorFn) - re.NoError(failpoint.Enable("github.com/tikv/pd/client/fastCheckAvailable", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/fastCheckAvailable", "return(true)")) re.True(followerAPIClient.Available()) re.True(leaderAPIClient.Available()) @@ -267,7 +269,7 @@ func (suite *serviceClientTestSuite) TestServiceClient() { re.True(followerAPIClient.Available()) re.True(leaderAPIClient.Available()) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/fastCheckAvailable")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/fastCheckAvailable")) } func (suite *serviceClientTestSuite) TestServiceClientBalancer() { @@ -308,17 +310,17 @@ func (suite *serviceClientTestSuite) TestServiceClientBalancer() { func TestServiceClientScheme(t *testing.T) { re := require.New(t) - cli := newPDServiceClient(modifyURLScheme("127.0.0.1:2379", nil), modifyURLScheme("127.0.0.1:2379", nil), nil, false) + cli := newPDServiceClient(tlsutil.ModifyURLScheme("127.0.0.1:2379", nil), tlsutil.ModifyURLScheme("127.0.0.1:2379", nil), nil, false) re.Equal("http://127.0.0.1:2379", cli.GetURL()) - cli = newPDServiceClient(modifyURLScheme("https://127.0.0.1:2379", nil), modifyURLScheme("127.0.0.1:2379", nil), nil, false) + cli = newPDServiceClient(tlsutil.ModifyURLScheme("https://127.0.0.1:2379", nil), tlsutil.ModifyURLScheme("127.0.0.1:2379", nil), nil, false) re.Equal("http://127.0.0.1:2379", cli.GetURL()) - cli = newPDServiceClient(modifyURLScheme("http://127.0.0.1:2379", nil), modifyURLScheme("127.0.0.1:2379", nil), nil, false) + cli = newPDServiceClient(tlsutil.ModifyURLScheme("http://127.0.0.1:2379", nil), tlsutil.ModifyURLScheme("127.0.0.1:2379", nil), nil, false) re.Equal("http://127.0.0.1:2379", cli.GetURL()) - cli = newPDServiceClient(modifyURLScheme("127.0.0.1:2379", &tls.Config{}), modifyURLScheme("127.0.0.1:2379", &tls.Config{}), nil, false) + cli = newPDServiceClient(tlsutil.ModifyURLScheme("127.0.0.1:2379", &tls.Config{}), tlsutil.ModifyURLScheme("127.0.0.1:2379", &tls.Config{}), nil, false) re.Equal("https://127.0.0.1:2379", cli.GetURL()) - cli = newPDServiceClient(modifyURLScheme("https://127.0.0.1:2379", &tls.Config{}), modifyURLScheme("127.0.0.1:2379", &tls.Config{}), nil, false) + cli = newPDServiceClient(tlsutil.ModifyURLScheme("https://127.0.0.1:2379", &tls.Config{}), tlsutil.ModifyURLScheme("127.0.0.1:2379", &tls.Config{}), nil, false) re.Equal("https://127.0.0.1:2379", cli.GetURL()) - cli = newPDServiceClient(modifyURLScheme("http://127.0.0.1:2379", &tls.Config{}), modifyURLScheme("127.0.0.1:2379", &tls.Config{}), nil, false) + cli = newPDServiceClient(tlsutil.ModifyURLScheme("http://127.0.0.1:2379", &tls.Config{}), tlsutil.ModifyURLScheme("127.0.0.1:2379", &tls.Config{}), nil, false) re.Equal("https://127.0.0.1:2379", cli.GetURL()) } @@ -336,48 +338,97 @@ func TestSchemeFunction(t *testing.T) { "http://127.0.0.1:2379", "https://127.0.0.1:2379", } - urls := addrsToURLs(endpoints1, tlsCfg) + urls := tlsutil.AddrsToURLs(endpoints1, tlsCfg) for _, u := range urls { re.Equal("https://tc-pd:2379", u) } - urls = addrsToURLs(endpoints2, tlsCfg) + urls = tlsutil.AddrsToURLs(endpoints2, tlsCfg) for _, u := range urls { re.Equal("https://127.0.0.1:2379", u) } - urls = addrsToURLs(endpoints1, nil) + urls = tlsutil.AddrsToURLs(endpoints1, nil) for _, u := range urls { re.Equal("http://tc-pd:2379", u) } - urls = addrsToURLs(endpoints2, nil) + urls = tlsutil.AddrsToURLs(endpoints2, nil) for _, u := range urls { re.Equal("http://127.0.0.1:2379", u) } - re.Equal("https://127.0.0.1:2379", modifyURLScheme("https://127.0.0.1:2379", tlsCfg)) - re.Equal("https://127.0.0.1:2379", modifyURLScheme("http://127.0.0.1:2379", tlsCfg)) - re.Equal("https://127.0.0.1:2379", modifyURLScheme("127.0.0.1:2379", tlsCfg)) - re.Equal("https://tc-pd:2379", modifyURLScheme("tc-pd:2379", tlsCfg)) - re.Equal("http://127.0.0.1:2379", modifyURLScheme("https://127.0.0.1:2379", nil)) - re.Equal("http://127.0.0.1:2379", modifyURLScheme("http://127.0.0.1:2379", nil)) - re.Equal("http://127.0.0.1:2379", modifyURLScheme("127.0.0.1:2379", nil)) - re.Equal("http://tc-pd:2379", modifyURLScheme("tc-pd:2379", nil)) + re.Equal("https://127.0.0.1:2379", tlsutil.ModifyURLScheme("https://127.0.0.1:2379", tlsCfg)) + re.Equal("https://127.0.0.1:2379", tlsutil.ModifyURLScheme("http://127.0.0.1:2379", tlsCfg)) + re.Equal("https://127.0.0.1:2379", tlsutil.ModifyURLScheme("127.0.0.1:2379", tlsCfg)) + re.Equal("https://tc-pd:2379", tlsutil.ModifyURLScheme("tc-pd:2379", tlsCfg)) + re.Equal("http://127.0.0.1:2379", tlsutil.ModifyURLScheme("https://127.0.0.1:2379", nil)) + re.Equal("http://127.0.0.1:2379", tlsutil.ModifyURLScheme("http://127.0.0.1:2379", nil)) + re.Equal("http://127.0.0.1:2379", tlsutil.ModifyURLScheme("127.0.0.1:2379", nil)) + re.Equal("http://tc-pd:2379", tlsutil.ModifyURLScheme("tc-pd:2379", nil)) urls = []string{ "http://127.0.0.1:2379", "https://127.0.0.1:2379", } - re.Equal("https://127.0.0.1:2379", pickMatchedURL(urls, tlsCfg)) + re.Equal("https://127.0.0.1:2379", tlsutil.PickMatchedURL(urls, tlsCfg)) urls = []string{ "http://127.0.0.1:2379", } - re.Equal("https://127.0.0.1:2379", pickMatchedURL(urls, tlsCfg)) + re.Equal("https://127.0.0.1:2379", tlsutil.PickMatchedURL(urls, tlsCfg)) urls = []string{ "http://127.0.0.1:2379", "https://127.0.0.1:2379", } - re.Equal("http://127.0.0.1:2379", pickMatchedURL(urls, nil)) + re.Equal("http://127.0.0.1:2379", tlsutil.PickMatchedURL(urls, nil)) urls = []string{ "https://127.0.0.1:2379", } - re.Equal("http://127.0.0.1:2379", pickMatchedURL(urls, nil)) + re.Equal("http://127.0.0.1:2379", tlsutil.PickMatchedURL(urls, nil)) +} + +func TestUpdateURLs(t *testing.T) { + re := require.New(t) + members := []*pdpb.Member{ + {Name: "pd4", ClientUrls: []string{"tmp://pd4"}}, + {Name: "pd1", ClientUrls: []string{"tmp://pd1"}}, + {Name: "pd3", ClientUrls: []string{"tmp://pd3"}}, + {Name: "pd2", ClientUrls: []string{"tmp://pd2"}}, + } + getURLs := func(ms []*pdpb.Member) (urls []string) { + for _, m := range ms { + urls = append(urls, m.GetClientUrls()[0]) + } + return + } + cli := &pdServiceDiscovery{option: opt.NewOption()} + cli.urls.Store([]string{}) + cli.updateURLs(members[1:]) + re.Equal(getURLs([]*pdpb.Member{members[1], members[3], members[2]}), cli.GetServiceURLs()) + cli.updateURLs(members[1:]) + re.Equal(getURLs([]*pdpb.Member{members[1], members[3], members[2]}), cli.GetServiceURLs()) + cli.updateURLs(members) + re.Equal(getURLs([]*pdpb.Member{members[1], members[3], members[2], members[0]}), cli.GetServiceURLs()) + cli.updateURLs(members[1:]) + re.Equal(getURLs([]*pdpb.Member{members[1], members[3], members[2]}), cli.GetServiceURLs()) + cli.updateURLs(members[2:]) + re.Equal(getURLs([]*pdpb.Member{members[3], members[2]}), cli.GetServiceURLs()) + cli.updateURLs(members[3:]) + re.Equal(getURLs([]*pdpb.Member{members[3]}), cli.GetServiceURLs()) +} + +func TestGRPCDialOption(t *testing.T) { + re := require.New(t) + start := time.Now() + ctx, cancel := context.WithTimeout(context.TODO(), 500*time.Millisecond) + defer cancel() + cli := &pdServiceDiscovery{ + checkMembershipCh: make(chan struct{}, 1), + ctx: ctx, + cancel: cancel, + tlsCfg: nil, + option: opt.NewOption(), + } + cli.urls.Store([]string{"tmp://test.url:5255"}) + cli.option.GRPCDialOptions = []grpc.DialOption{grpc.WithBlock()} + err := cli.updateMember() + re.Error(err) + re.Greater(time.Since(start), 500*time.Millisecond) } diff --git a/client/tso_service_discovery.go b/client/servicediscovery/tso_service_discovery.go similarity index 95% rename from client/tso_service_discovery.go rename to client/servicediscovery/tso_service_discovery.go index 7d5b761e68c..81ef69c5545 100644 --- a/client/tso_service_discovery.go +++ b/client/servicediscovery/tso_service_discovery.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package pd +package servicediscovery import ( "context" @@ -30,6 +30,7 @@ import ( "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" "github.com/tikv/pd/client/clients/metastorage" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/utils/grpcutil" @@ -52,8 +53,10 @@ const ( tsoQueryRetryInterval = 500 * time.Millisecond ) -var _ ServiceDiscovery = (*tsoServiceDiscovery)(nil) -var _ tsoEventSource = (*tsoServiceDiscovery)(nil) +var ( + _ ServiceDiscovery = (*tsoServiceDiscovery)(nil) + _ TSOEventSource = (*tsoServiceDiscovery)(nil) +) // keyspaceGroupSvcDiscovery is used for discovering the serving endpoints of the keyspace // group to which the keyspace belongs @@ -154,8 +157,8 @@ type tsoServiceDiscovery struct { option *opt.Option } -// newTSOServiceDiscovery returns a new client-side service discovery for the independent TSO service. -func newTSOServiceDiscovery( +// NewTSOServiceDiscovery returns a new client-side service discovery for the independent TSO service. +func NewTSOServiceDiscovery( ctx context.Context, metacli metastorage.Client, apiSvcDiscovery ServiceDiscovery, keyspaceID uint32, tlsCfg *tls.Config, option *opt.Option, ) ServiceDiscovery { @@ -179,7 +182,7 @@ func newTSOServiceDiscovery( c.tsoServerDiscovery = &tsoServerDiscovery{urls: make([]string, 0)} // Start with the default keyspace group. The actual keyspace group, to which the keyspace belongs, // will be discovered later. - c.defaultDiscoveryKey = fmt.Sprintf(tsoSvcDiscoveryFormat, c.clusterID, defaultKeySpaceGroupID) + c.defaultDiscoveryKey = fmt.Sprintf(tsoSvcDiscoveryFormat, c.clusterID, constants.DefaultKeyspaceGroupID) log.Info("created tso service discovery", zap.Uint64("cluster-id", c.clusterID), @@ -249,7 +252,7 @@ func (c *tsoServiceDiscovery) startCheckMemberLoop() { ctx, cancel := context.WithCancel(c.ctx) defer cancel() - ticker := time.NewTicker(memberUpdateInterval) + ticker := time.NewTicker(MemberUpdateInterval) defer ticker.Stop() for { @@ -279,13 +282,18 @@ 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 { c.keyspaceGroupSD.RLock() defer c.keyspaceGroupSD.RUnlock() if c.keyspaceGroupSD.group == nil { - return defaultKeySpaceGroupID + return constants.DefaultKeyspaceGroupID } return c.keyspaceGroupSD.group.Id } @@ -375,6 +383,11 @@ func (c *tsoServiceDiscovery) GetServiceClient() ServiceClient { return c.apiSvcDiscovery.GetServiceClient() } +// GetServiceClientByKind implements ServiceDiscovery +func (c *tsoServiceDiscovery) GetServiceClientByKind(kind APIKind) ServiceClient { + return c.apiSvcDiscovery.GetServiceClientByKind(kind) +} + // GetAllServiceClients implements ServiceDiscovery func (c *tsoServiceDiscovery) GetAllServiceClients() []ServiceClient { return c.apiSvcDiscovery.GetAllServiceClients() @@ -419,7 +432,7 @@ func (c *tsoServiceDiscovery) updateMember() error { keyspaceID := c.GetKeyspaceID() var keyspaceGroup *tsopb.KeyspaceGroup if len(tsoServerURL) > 0 { - keyspaceGroup, err = c.findGroupByKeyspaceID(keyspaceID, tsoServerURL, updateMemberTimeout) + keyspaceGroup, err = c.findGroupByKeyspaceID(keyspaceID, tsoServerURL, UpdateMemberTimeout) if err != nil { if c.tsoServerDiscovery.countFailure() { log.Error("[tso] failed to find the keyspace group", @@ -456,7 +469,7 @@ func (c *tsoServiceDiscovery) updateMember() error { } members[0].IsPrimary = true keyspaceGroup = &tsopb.KeyspaceGroup{ - Id: defaultKeySpaceGroupID, + Id: constants.DefaultKeyspaceGroupID, Members: members, } } @@ -541,7 +554,7 @@ func (c *tsoServiceDiscovery) findGroupByKeyspaceID( Header: &tsopb.RequestHeader{ ClusterId: c.clusterID, KeyspaceId: keyspaceID, - KeyspaceGroupId: defaultKeySpaceGroupID, + KeyspaceGroupId: constants.DefaultKeyspaceGroupID, }, KeyspaceId: keyspaceID, }) diff --git a/client/tso_client.go b/client/tso_client.go index cdd85dd2479..1d0a6385647 100644 --- a/client/tso_client.go +++ b/client/tso_client.go @@ -28,7 +28,9 @@ import ( "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" + sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/client/utils/grpcutil" + "github.com/tikv/pd/client/utils/tlsutil" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/codes" @@ -72,7 +74,7 @@ type tsoClient struct { wg sync.WaitGroup option *opt.Option - svcDiscovery ServiceDiscovery + svcDiscovery sd.ServiceDiscovery tsoStreamBuilderFactory // leaderURL is the URL of the TSO leader. leaderURL atomic.Value @@ -86,7 +88,7 @@ type tsoClient struct { // newTSOClient returns a new TSO client. func newTSOClient( ctx context.Context, option *opt.Option, - svcDiscovery ServiceDiscovery, factory tsoStreamBuilderFactory, + svcDiscovery sd.ServiceDiscovery, factory tsoStreamBuilderFactory, ) *tsoClient { ctx, cancel := context.WithCancel(ctx) c := &tsoClient{ @@ -106,7 +108,7 @@ func newTSOClient( }, } - eventSrc := svcDiscovery.(tsoEventSource) + eventSrc := svcDiscovery.(sd.TSOEventSource) eventSrc.SetTSOLeaderURLUpdatedCallback(c.updateTSOLeaderURL) c.svcDiscovery.AddServiceURLsSwitchedCallback(c.scheduleUpdateTSOConnectionCtxs) @@ -115,7 +117,7 @@ func newTSOClient( func (c *tsoClient) getOption() *opt.Option { return c.option } -func (c *tsoClient) getServiceDiscovery() ServiceDiscovery { return c.svcDiscovery } +func (c *tsoClient) getServiceDiscovery() sd.ServiceDiscovery { return c.svcDiscovery } func (c *tsoClient) getDispatcher() *tsoDispatcher { return c.dispatcher.Load() @@ -303,7 +305,7 @@ func (c *tsoClient) tryConnectToTSO( // There is no need to wait for the transport layer timeout which can reduce the time of unavailability. // But it conflicts with the retry mechanism since we use the error code to decide if it is caused by network error. // And actually the `Canceled` error can be regarded as a kind of network error in some way. - if rpcErr, ok := status.FromError(err); ok && (isNetworkError(rpcErr.Code()) || rpcErr.Code() == codes.Canceled) { + if rpcErr, ok := status.FromError(err); ok && (errs.IsNetworkError(rpcErr.Code()) || rpcErr.Code() == codes.Canceled) { networkErrNum++ } } @@ -333,8 +335,8 @@ func (c *tsoClient) tryConnectToTSO( cctx = grpcutil.BuildForwardContext(cctx, forwardedHost) stream, err = c.tsoStreamBuilderFactory.makeBuilder(backupClientConn).build(cctx, cancel, c.option.Timeout) if err == nil { - forwardedHostTrim := trimHTTPPrefix(forwardedHost) - addr := trimHTTPPrefix(backupURL) + forwardedHostTrim := tlsutil.TrimHTTPPrefix(forwardedHost) + addr := tlsutil.TrimHTTPPrefix(backupURL) // the goroutine is used to check the network and change back to the original stream go c.checkLeader(ctx, cancel, forwardedHostTrim, addr, url, updateAndClear) metrics.RequestForwarded.WithLabelValues(forwardedHostTrim, addr).Set(1) @@ -440,8 +442,8 @@ func (c *tsoClient) tryConnectToTSOWithProxy( stream, err := tsoStreamBuilder.build(cctx, cancel, c.option.Timeout) if err == nil { if addr != leaderAddr { - forwardedHostTrim := trimHTTPPrefix(forwardedHost) - addrTrim := trimHTTPPrefix(addr) + forwardedHostTrim := tlsutil.TrimHTTPPrefix(forwardedHost) + addrTrim := tlsutil.TrimHTTPPrefix(addr) metrics.RequestForwarded.WithLabelValues(forwardedHostTrim, addrTrim).Set(1) } connectionCtxs.Store(addr, &tsoConnectionContext{cctx, cancel, addr, stream}) diff --git a/client/tso_dispatcher.go b/client/tso_dispatcher.go index c696dc26b36..1123e59dbdd 100644 --- a/client/tso_dispatcher.go +++ b/client/tso_dispatcher.go @@ -32,6 +32,7 @@ import ( "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/retry" + sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/client/utils/timerutil" "github.com/tikv/pd/client/utils/tsoutil" "go.uber.org/zap" @@ -70,7 +71,7 @@ type tsoInfo struct { type tsoServiceProvider interface { getOption() *opt.Option - getServiceDiscovery() ServiceDiscovery + getServiceDiscovery() sd.ServiceDiscovery updateConnectionCtxs(ctx context.Context, connectionCtxs *sync.Map) bool } @@ -179,7 +180,7 @@ func (td *tsoDispatcher) revokePendingRequests(err error) { func (td *tsoDispatcher) close() { td.cancel() - tsoErr := errors.WithStack(errClosing) + tsoErr := errors.WithStack(errs.ErrClosing) td.revokePendingRequests(tsoErr) } @@ -210,7 +211,7 @@ func (td *tsoDispatcher) handleDispatcher(wg *sync.WaitGroup) { // If you encounter this failure, please check the stack in the logs to see if it's a panic. log.Fatal("batched tso requests not cleared when exiting the tso dispatcher loop", zap.Any("panic", recover())) } - tsoErr := errors.WithStack(errClosing) + tsoErr := errors.WithStack(errs.ErrClosing) td.revokePendingRequests(tsoErr) wg.Done() }() @@ -233,7 +234,7 @@ func (td *tsoDispatcher) handleDispatcher(wg *sync.WaitGroup) { <-batchingTimer.C defer batchingTimer.Stop() - bo := retry.InitialBackoffer(updateMemberBackOffBaseTime, updateMemberTimeout, updateMemberBackOffBaseTime) + bo := retry.InitialBackoffer(sd.UpdateMemberBackOffBaseTime, sd.UpdateMemberTimeout, sd.UpdateMemberBackOffBaseTime) tsoBatchLoop: for { select { @@ -494,7 +495,7 @@ func (td *tsoDispatcher) connectionCtxsUpdater() { if enableTSOFollowerProxy && updateTicker.C == nil { // Because the TSO Follower Proxy is enabled, // the periodic check needs to be performed. - setNewUpdateTicker(time.NewTicker(memberUpdateInterval)) + setNewUpdateTicker(time.NewTicker(sd.MemberUpdateInterval)) } else if !enableTSOFollowerProxy && updateTicker.C != nil { // Because the TSO Follower Proxy is disabled, // the periodic check needs to be turned off. diff --git a/client/tso_dispatcher_test.go b/client/tso_dispatcher_test.go index 84bc6a4dc99..6cb963df3df 100644 --- a/client/tso_dispatcher_test.go +++ b/client/tso_dispatcher_test.go @@ -27,6 +27,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/tikv/pd/client/opt" + sd "github.com/tikv/pd/client/servicediscovery" "go.uber.org/zap/zapcore" ) @@ -47,8 +48,8 @@ func (m *mockTSOServiceProvider) getOption() *opt.Option { return m.option } -func (*mockTSOServiceProvider) getServiceDiscovery() ServiceDiscovery { - return NewMockPDServiceDiscovery([]string{mockStreamURL}, nil) +func (*mockTSOServiceProvider) getServiceDiscovery() sd.ServiceDiscovery { + return sd.NewMockPDServiceDiscovery([]string{mockStreamURL}, nil) } func (m *mockTSOServiceProvider) updateConnectionCtxs(ctx context.Context, connectionCtxs *sync.Map) bool { diff --git a/client/tso_stream.go b/client/tso_stream.go index 51ae5696dc4..ce3c513ac46 100644 --- a/client/tso_stream.go +++ b/client/tso_stream.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "go.uber.org/zap" @@ -144,7 +145,7 @@ func (s pdTSOStreamAdapter) Recv() (tsoRequestResult, error) { physical: resp.GetTimestamp().GetPhysical(), logical: resp.GetTimestamp().GetLogical(), count: resp.GetCount(), - respKeyspaceGroupID: defaultKeySpaceGroupID, + respKeyspaceGroupID: constants.DefaultKeyspaceGroupID, }, nil } @@ -432,7 +433,7 @@ recvLoop: updateEstimatedLatency(currentReq.startTime, latency) if res.count != uint32(currentReq.count) { - finishWithErr = errors.WithStack(errTSOLength) + finishWithErr = errors.WithStack(errs.ErrTSOLength) break recvLoop } diff --git a/client/utils/tlsutil/url.go b/client/utils/tlsutil/url.go new file mode 100644 index 00000000000..ccc312d195b --- /dev/null +++ b/client/utils/tlsutil/url.go @@ -0,0 +1,88 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package tlsutil + +import ( + "crypto/tls" + "net/url" + "strings" + + "github.com/pingcap/log" + "go.uber.org/zap" +) + +const ( + httpScheme = "http" + httpsScheme = "https" + httpSchemePrefix = "http://" + httpsSchemePrefix = "https://" +) + +// AddrsToURLs converts a list of addresses to a list of URLs. +func AddrsToURLs(addrs []string, tlsCfg *tls.Config) []string { + // Add default schema "http://" to addrs. + urls := make([]string, 0, len(addrs)) + for _, addr := range addrs { + urls = append(urls, ModifyURLScheme(addr, tlsCfg)) + } + return urls +} + +// ModifyURLScheme modifies the scheme of the URL based on the TLS config. +func ModifyURLScheme(url string, tlsCfg *tls.Config) string { + if tlsCfg == nil { + if strings.HasPrefix(url, httpsSchemePrefix) { + url = httpSchemePrefix + strings.TrimPrefix(url, httpsSchemePrefix) + } else if !strings.HasPrefix(url, httpSchemePrefix) { + url = httpSchemePrefix + url + } + } else { + if strings.HasPrefix(url, httpSchemePrefix) { + url = httpsSchemePrefix + strings.TrimPrefix(url, httpSchemePrefix) + } else if !strings.HasPrefix(url, httpsSchemePrefix) { + url = httpsSchemePrefix + url + } + } + return url +} + +// PickMatchedURL picks the matched URL based on the TLS config. +// Note: please make sure the URLs are valid. +func PickMatchedURL(urls []string, tlsCfg *tls.Config) string { + for _, uStr := range urls { + u, err := url.Parse(uStr) + if err != nil { + continue + } + if tlsCfg != nil && u.Scheme == httpsScheme { + return uStr + } + if tlsCfg == nil && u.Scheme == httpScheme { + return uStr + } + } + ret := ModifyURLScheme(urls[0], tlsCfg) + log.Warn("[pd] no matched url found", zap.Strings("urls", urls), + zap.Bool("tls-enabled", tlsCfg != nil), + zap.String("attempted-url", ret)) + return ret +} + +// TrimHTTPPrefix trims the HTTP/HTTPS prefix from the string. +func TrimHTTPPrefix(str string) string { + str = strings.TrimPrefix(str, httpSchemePrefix) + str = strings.TrimPrefix(str, httpsSchemePrefix) + return str +} diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index bc69367c72a..79f981f3bb3 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -41,6 +41,7 @@ import ( "github.com/tikv/pd/client/caller" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/retry" + sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/mock/mockid" @@ -83,7 +84,7 @@ func TestClientLeaderChange(t *testing.T) { } cli := setupCli(ctx, re, endpointsWithWrongURL) defer cli.Close() - innerCli, ok := cli.(interface{ GetServiceDiscovery() pd.ServiceDiscovery }) + innerCli, ok := cli.(interface{ GetServiceDiscovery() sd.ServiceDiscovery }) re.True(ok) var ts1, ts2 uint64 @@ -324,7 +325,7 @@ func TestTSOFollowerProxy(t *testing.T) { func TestTSOFollowerProxyWithTSOService(t *testing.T) { re := require.New(t) - re.NoError(failpoint.Enable("github.com/tikv/pd/client/fastUpdateServiceMode", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/fastUpdateServiceMode", `return(true)`)) ctx, cancel := context.WithCancel(context.Background()) defer cancel() cluster, err := tests.NewTestAPICluster(ctx, 1) @@ -346,7 +347,7 @@ func TestTSOFollowerProxyWithTSOService(t *testing.T) { // TSO service does not support the follower proxy, so enabling it should fail. err = cli.UpdateOption(opt.EnableTSOFollowerProxy, true) re.Error(err) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/fastUpdateServiceMode")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/fastUpdateServiceMode")) } // TestUnavailableTimeAfterLeaderIsReady is used to test https://github.com/tikv/pd/issues/5207 @@ -451,7 +452,7 @@ func TestFollowerForwardAndHandleTestSuite(t *testing.T) { func (suite *followerForwardAndHandleTestSuite) SetupSuite() { re := suite.Require() suite.ctx, suite.clean = context.WithCancel(context.Background()) - pd.MemberHealthCheckInterval = 100 * time.Millisecond + sd.MemberHealthCheckInterval = 100 * time.Millisecond cluster, err := tests.NewTestCluster(suite.ctx, 3) re.NoError(err) suite.cluster = cluster @@ -497,13 +498,13 @@ func (suite *followerForwardAndHandleTestSuite) TestGetRegionByFollowerForwardin cli := setupCli(ctx, re, suite.endpoints, opt.WithForwardingOption(true)) defer cli.Close() - re.NoError(failpoint.Enable("github.com/tikv/pd/client/unreachableNetwork1", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1", "return(true)")) time.Sleep(200 * time.Millisecond) r, err := cli.GetRegion(context.Background(), []byte("a")) re.NoError(err) re.NotNil(r) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/unreachableNetwork1")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1")) time.Sleep(200 * time.Millisecond) r, err = cli.GetRegion(context.Background(), []byte("a")) re.NoError(err) @@ -719,7 +720,7 @@ func (suite *followerForwardAndHandleTestSuite) TestGetRegionFromFollower() { // because we can't check whether this request is processed by followers from response, // we can disable forward and make network problem for leader. - re.NoError(failpoint.Enable("github.com/tikv/pd/client/unreachableNetwork1", fmt.Sprintf("return(\"%s\")", leader.GetAddr()))) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1", fmt.Sprintf("return(\"%s\")", leader.GetAddr()))) time.Sleep(150 * time.Millisecond) cnt = 0 for range 100 { @@ -730,11 +731,11 @@ func (suite *followerForwardAndHandleTestSuite) TestGetRegionFromFollower() { re.Equal(resp.Meta.Id, suite.regionID) } re.Equal(100, cnt) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/unreachableNetwork1")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1")) // make network problem for follower. follower := cluster.GetServer(cluster.GetFollower()) - re.NoError(failpoint.Enable("github.com/tikv/pd/client/unreachableNetwork1", fmt.Sprintf("return(\"%s\")", follower.GetAddr()))) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1", fmt.Sprintf("return(\"%s\")", follower.GetAddr()))) time.Sleep(100 * time.Millisecond) cnt = 0 for range 100 { @@ -745,7 +746,7 @@ func (suite *followerForwardAndHandleTestSuite) TestGetRegionFromFollower() { re.Equal(resp.Meta.Id, suite.regionID) } re.Equal(100, cnt) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/unreachableNetwork1")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1")) // follower client failed will retry by leader service client. re.NoError(failpoint.Enable("github.com/tikv/pd/server/followerHandleError", "return(true)")) @@ -761,8 +762,8 @@ func (suite *followerForwardAndHandleTestSuite) TestGetRegionFromFollower() { re.NoError(failpoint.Disable("github.com/tikv/pd/server/followerHandleError")) // test after being healthy - re.NoError(failpoint.Enable("github.com/tikv/pd/client/unreachableNetwork1", fmt.Sprintf("return(\"%s\")", leader.GetAddr()))) - re.NoError(failpoint.Enable("github.com/tikv/pd/client/fastCheckAvailable", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1", fmt.Sprintf("return(\"%s\")", leader.GetAddr()))) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/fastCheckAvailable", "return(true)")) time.Sleep(100 * time.Millisecond) cnt = 0 for range 100 { @@ -773,8 +774,8 @@ func (suite *followerForwardAndHandleTestSuite) TestGetRegionFromFollower() { re.Equal(resp.Meta.Id, suite.regionID) } re.Equal(100, cnt) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/unreachableNetwork1")) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/fastCheckAvailable")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/fastCheckAvailable")) } func (suite *followerForwardAndHandleTestSuite) TestGetTSFuture() { @@ -857,7 +858,7 @@ func setupCli(ctx context.Context, re *require.Assertions, endpoints []string, o return cli } -func waitLeader(re *require.Assertions, cli pd.ServiceDiscovery, leader *tests.TestServer) { +func waitLeader(re *require.Assertions, cli sd.ServiceDiscovery, leader *tests.TestServer) { testutil.Eventually(re, func() bool { cli.ScheduleCheckMemberChanged() return cli.GetServingURL() == leader.GetConfig().ClientUrls && leader.GetAddr() == cli.GetServingURL() @@ -1778,7 +1779,7 @@ func (suite *clientTestSuite) TestMemberUpdateBackOff() { endpoints := runServer(re, cluster) cli := setupCli(ctx, re, endpoints) defer cli.Close() - innerCli, ok := cli.(interface{ GetServiceDiscovery() pd.ServiceDiscovery }) + innerCli, ok := cli.(interface{ GetServiceDiscovery() sd.ServiceDiscovery }) re.True(ok) leader := cluster.GetLeader() @@ -1801,7 +1802,7 @@ func (suite *clientTestSuite) TestMemberUpdateBackOff() { re.NoError(failpoint.Disable("github.com/tikv/pd/client/retry/backOffExecute")) } -func waitLeaderChange(re *require.Assertions, cluster *tests.TestCluster, old string, cli pd.ServiceDiscovery) string { +func waitLeaderChange(re *require.Assertions, cluster *tests.TestCluster, old string, cli sd.ServiceDiscovery) string { var leader string testutil.Eventually(re, func() bool { cli.ScheduleCheckMemberChanged() diff --git a/tests/integrations/mcs/resourcemanager/resource_manager_test.go b/tests/integrations/mcs/resourcemanager/resource_manager_test.go index 5688ea8a8ac..15bec8ea8fd 100644 --- a/tests/integrations/mcs/resourcemanager/resource_manager_test.go +++ b/tests/integrations/mcs/resourcemanager/resource_manager_test.go @@ -34,6 +34,7 @@ import ( pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/caller" "github.com/tikv/pd/client/resource_group/controller" + sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/pkg/mcs/resourcemanager/server" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" @@ -142,7 +143,7 @@ func (suite *resourceManagerClientTestSuite) SetupSuite() { } func waitLeader(re *require.Assertions, cli pd.Client, leaderAddr string) { - innerCli, ok := cli.(interface{ GetServiceDiscovery() pd.ServiceDiscovery }) + innerCli, ok := cli.(interface{ GetServiceDiscovery() sd.ServiceDiscovery }) re.True(ok) re.NotNil(innerCli) testutil.Eventually(re, func() bool { diff --git a/tests/integrations/mcs/tso/server_test.go b/tests/integrations/mcs/tso/server_test.go index 8624454aec3..a78d61bf429 100644 --- a/tests/integrations/mcs/tso/server_test.go +++ b/tests/integrations/mcs/tso/server_test.go @@ -245,7 +245,7 @@ func NewAPIServerForward(re *require.Assertions) APIServerForward { re.NoError(suite.pdLeader.BootstrapCluster()) suite.addRegions() - re.NoError(failpoint.Enable("github.com/tikv/pd/client/usePDServiceMode", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/usePDServiceMode", "return(true)")) suite.pdClient, err = pd.NewClientWithContext(context.Background(), caller.TestComponent, []string{suite.backendEndpoints}, pd.SecurityOption{}, opt.WithMaxErrorRetry(1)) @@ -267,7 +267,7 @@ func (suite *APIServerForward) ShutDown() { } suite.cluster.Destroy() suite.cancel() - re.NoError(failpoint.Disable("github.com/tikv/pd/client/usePDServiceMode")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/usePDServiceMode")) } func TestForwardTSORelated(t *testing.T) { @@ -593,7 +593,7 @@ func (suite *CommonTestSuite) TestBootstrapDefaultKeyspaceGroup() { // If `EnableTSODynamicSwitching` is disabled, the PD should not provide TSO service after the TSO server is stopped. func TestTSOServiceSwitch(t *testing.T) { re := require.New(t) - re.NoError(failpoint.Enable("github.com/tikv/pd/client/fastUpdateServiceMode", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/fastUpdateServiceMode", `return(true)`)) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -666,7 +666,7 @@ func TestTSOServiceSwitch(t *testing.T) { // Verify PD is now providing TSO service and timestamps are monotonically increasing re.NoError(checkTSOMonotonic(ctx, pdClient, &globalLastTS, 10)) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/fastUpdateServiceMode")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/fastUpdateServiceMode")) } func checkTSOMonotonic(ctx context.Context, pdClient pd.Client, globalLastTS *uint64, count int) error { diff --git a/tests/integrations/tso/client_test.go b/tests/integrations/tso/client_test.go index 121a61b1986..422d578326a 100644 --- a/tests/integrations/tso/client_test.go +++ b/tests/integrations/tso/client_test.go @@ -31,6 +31,7 @@ import ( pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/caller" "github.com/tikv/pd/client/opt" + sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/client/utils/testutil" bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/utils/constant" @@ -154,7 +155,7 @@ func (suite *tsoClientTestSuite) SetupTest() { caller.TestComponent, suite.getBackendEndpoints(), pd.SecurityOption{}, opt.WithForwardingOption(true)) re.NoError(err) - innerClient, ok := client.(interface{ GetServiceDiscovery() pd.ServiceDiscovery }) + innerClient, ok := client.(interface{ GetServiceDiscovery() sd.ServiceDiscovery }) re.True(ok) re.Equal(constant.NullKeyspaceID, innerClient.GetServiceDiscovery().GetKeyspaceID()) re.Equal(constant.DefaultKeyspaceGroupID, innerClient.GetServiceDiscovery().GetKeyspaceGroupID()) @@ -266,11 +267,11 @@ func (suite *tsoClientTestSuite) TestDiscoverTSOServiceWithLegacyPath() { failpointValue := fmt.Sprintf(`return(%d)`, keyspaceID) // Simulate the case that the server has lower version than the client and returns no tso addrs // in the GetClusterInfo RPC. - re.NoError(failpoint.Enable("github.com/tikv/pd/client/serverReturnsNoTSOAddrs", `return(true)`)) - re.NoError(failpoint.Enable("github.com/tikv/pd/client/unexpectedCallOfFindGroupByKeyspaceID", failpointValue)) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/serverReturnsNoTSOAddrs", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/unexpectedCallOfFindGroupByKeyspaceID", failpointValue)) defer func() { - re.NoError(failpoint.Disable("github.com/tikv/pd/client/serverReturnsNoTSOAddrs")) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/unexpectedCallOfFindGroupByKeyspaceID")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/serverReturnsNoTSOAddrs")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/unexpectedCallOfFindGroupByKeyspaceID")) }() ctx, cancel := context.WithCancel(suite.ctx) @@ -320,14 +321,14 @@ func (suite *tsoClientTestSuite) TestGetMinTS() { } wg.Wait() - re.NoError(failpoint.Enable("github.com/tikv/pd/client/unreachableNetwork1", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1", "return(true)")) time.Sleep(time.Second) testutil.Eventually(re, func() bool { var err error _, _, err = suite.clients[0].GetMinTS(suite.ctx) return err == nil }) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/unreachableNetwork1")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1")) } // More details can be found in this issue: https://github.com/tikv/pd/issues/4884 @@ -487,10 +488,10 @@ func TestMixedTSODeployment(t *testing.T) { re := require.New(t) re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastUpdatePhysicalInterval", "return(true)")) - re.NoError(failpoint.Enable("github.com/tikv/pd/client/skipUpdateServiceMode", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/skipUpdateServiceMode", "return(true)")) defer func() { re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastUpdatePhysicalInterval")) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/skipUpdateServiceMode")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/skipUpdateServiceMode")) }() ctx, cancel := context.WithCancel(context.Background()) @@ -550,7 +551,7 @@ func TestUpgradingAPIandTSOClusters(t *testing.T) { backendEndpoints := pdLeader.GetAddr() // Create a pd client in PD mode to let the API leader to forward requests to the TSO cluster. - re.NoError(failpoint.Enable("github.com/tikv/pd/client/usePDServiceMode", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/usePDServiceMode", "return(true)")) pdClient, err := pd.NewClientWithContext(context.Background(), caller.TestComponent, []string{backendEndpoints}, pd.SecurityOption{}, opt.WithMaxErrorRetry(1)) @@ -579,7 +580,7 @@ func TestUpgradingAPIandTSOClusters(t *testing.T) { tsoCluster.Destroy() apiCluster.Destroy() cancel() - re.NoError(failpoint.Disable("github.com/tikv/pd/client/usePDServiceMode")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/usePDServiceMode")) } func checkTSO( diff --git a/tools/pd-simulator/simulator/client.go b/tools/pd-simulator/simulator/client.go index 39c2633cec5..4de2ea52f88 100644 --- a/tools/pd-simulator/simulator/client.go +++ b/tools/pd-simulator/simulator/client.go @@ -26,8 +26,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - pd "github.com/tikv/pd/client" pdHttp "github.com/tikv/pd/client/http" + sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/typeutil" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" @@ -64,7 +64,7 @@ var ( // PDHTTPClient is a client for PD HTTP API. PDHTTPClient pdHttp.Client // SD is a service discovery for PD. - SD pd.ServiceDiscovery + SD sd.ServiceDiscovery clusterID atomic.Uint64 ) diff --git a/tools/pd-simulator/simulator/drive.go b/tools/pd-simulator/simulator/drive.go index 22d4175ecc6..c8c325cfca6 100644 --- a/tools/pd-simulator/simulator/drive.go +++ b/tools/pd-simulator/simulator/drive.go @@ -30,8 +30,8 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/prometheus/client_golang/prometheus/promhttp" - pd "github.com/tikv/pd/client" pdHttp "github.com/tikv/pd/client/http" + sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/tools/pd-simulator/simulator/cases" @@ -163,7 +163,7 @@ func (d *Driver) allocID() error { func (d *Driver) updateNodesClient() error { urls := strings.Split(d.pdAddr, ",") ctx, cancel := context.WithCancel(context.Background()) - SD = pd.NewDefaultPDServiceDiscovery(ctx, cancel, urls, nil) + SD = sd.NewDefaultPDServiceDiscovery(ctx, cancel, urls, nil) if err := SD.Init(); err != nil { return err } From 19c54c819ca20821ff3f96b4431ee3d462471f65 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Mon, 25 Nov 2024 15:00:59 +0800 Subject: [PATCH 5/7] etcd/path: change leaderPath/dcLocationPath/memberPath/allocalIDPath to absolute path (#8789) ref tikv/pd#8582 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/id/id.go | 61 ++++---- pkg/id/id_test.go | 25 ++- pkg/keyspace/keyspace.go | 2 - pkg/mcs/scheduling/server/server.go | 6 +- pkg/member/member.go | 40 ++--- pkg/member/participant.go | 39 +++-- pkg/tso/keyspace_group_manager.go | 7 +- pkg/utils/keypath/key_path.go | 7 - pkg/utils/keypath/key_path_v2.go | 101 ++++++++++++ server/server.go | 21 +-- tests/integrations/realcluster/cluster.go | 6 +- .../realcluster/cluster_id_test.go | 5 +- .../download_integration_test_binaries.sh | 8 + .../integrations/realcluster/etcd_key_test.go | 146 ++++++++++++++++++ tests/integrations/realcluster/util.go | 11 ++ 15 files changed, 363 insertions(+), 122 deletions(-) create mode 100644 pkg/utils/keypath/key_path_v2.go create mode 100644 tests/integrations/realcluster/etcd_key_test.go diff --git a/pkg/id/id.go b/pkg/id/id.go index 0225dedd4f1..eb2788fc656 100644 --- a/pkg/id/id.go +++ b/pkg/id/id.go @@ -15,19 +15,27 @@ package id import ( - "path" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" + "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) +type label string + +const ( + // DefaultLabel is the default label for id allocator. + DefaultLabel label = "idalloc" + // KeyspaceLabel is the label for keyspace id allocator. + KeyspaceLabel label = "keyspace-idAlloc" +) + // Allocator is the allocator to generate unique ID. type Allocator interface { // SetBase set base id @@ -48,13 +56,11 @@ type allocatorImpl struct { base uint64 end uint64 - client *clientv3.Client - rootPath string - allocPath string - label string - member string - step uint64 - metrics *metrics + client *clientv3.Client + label label + member string + step uint64 + metrics *metrics } // metrics is a collection of idAllocator's metrics. @@ -64,24 +70,20 @@ type metrics struct { // AllocatorParams are parameters needed to create a new ID Allocator. type AllocatorParams struct { - Client *clientv3.Client - RootPath string - AllocPath string // AllocPath specifies path to the persistent window boundary. - Label string // Label used to label metrics and logs. - Member string // Member value, used to check if current pd leader. - Step uint64 // Step size of each persistent window boundary increment, default 1000. + Client *clientv3.Client + Label label // Label used to label metrics and logs. + Member string // Member value, used to check if current pd leader. + Step uint64 // Step size of each persistent window boundary increment, default 1000. } // NewAllocator creates a new ID Allocator. func NewAllocator(params *AllocatorParams) Allocator { allocator := &allocatorImpl{ - client: params.Client, - rootPath: params.RootPath, - allocPath: params.AllocPath, - label: params.Label, - member: params.Member, - step: params.Step, - metrics: &metrics{idGauge: idGauge.WithLabelValues(params.Label)}, + client: params.Client, + label: params.Label, + member: params.Member, + step: params.Step, + metrics: &metrics{idGauge: idGauge.WithLabelValues(string(params.Label))}, } if allocator.step == 0 { allocator.step = defaultAllocStep @@ -127,9 +129,14 @@ func (alloc *allocatorImpl) Rebase() error { } func (alloc *allocatorImpl) rebaseLocked(checkCurrEnd bool) error { - key := alloc.getAllocIDPath() + var key string + if alloc.label == KeyspaceLabel { + key = keypath.KeyspaceAllocIDPath() + } else { + key = keypath.AllocIDPath() + } - leaderPath := path.Join(alloc.rootPath, "leader") + leaderPath := keypath.LeaderPath(nil) var ( cmps = []clientv3.Cmp{clientv3.Compare(clientv3.Value(leaderPath), "=", alloc.member)} end uint64 @@ -173,10 +180,6 @@ func (alloc *allocatorImpl) rebaseLocked(checkCurrEnd bool) error { // please do not reorder the first field, it's need when getting the new-end // see: https://docs.pingcap.com/tidb/dev/pd-recover#get-allocated-id-from-pd-log log.Info("idAllocator allocates a new id", zap.Uint64("new-end", end), zap.Uint64("new-base", alloc.base), - zap.String("label", alloc.label), zap.Bool("check-curr-end", checkCurrEnd)) + zap.String("label", string(alloc.label)), zap.Bool("check-curr-end", checkCurrEnd)) return nil } - -func (alloc *allocatorImpl) getAllocIDPath() string { - return path.Join(alloc.rootPath, alloc.allocPath) -} diff --git a/pkg/id/id_test.go b/pkg/id/id_test.go index d46ac5a963e..e08c0e93367 100644 --- a/pkg/id/id_test.go +++ b/pkg/id/id_test.go @@ -16,7 +16,6 @@ package id import ( "context" - "strconv" "sync" "testing" @@ -25,10 +24,7 @@ import ( ) const ( - rootPath = "/pd" - leaderPath = "/pd/leader" - allocPath = "alloc_id" - label = "idalloc" + leaderPath = "/pd/0/leader" memberVal = "member" step = uint64(500) ) @@ -44,24 +40,25 @@ func TestMultipleAllocator(t *testing.T) { _, err := client.Put(context.Background(), leaderPath, memberVal) re.NoError(err) + var i uint64 wg := sync.WaitGroup{} - for i := range 3 { - iStr := strconv.Itoa(i) + fn := func(label label) { wg.Add(1) - // All allocators share rootPath and memberVal, but they have different allocPaths, labels and steps. + // Different allocators have different labels and steps. allocator := NewAllocator(&AllocatorParams{ - Client: client, - RootPath: rootPath, - AllocPath: allocPath + iStr, - Label: label + iStr, - Member: memberVal, - Step: step * uint64(i), // allocator 0, 1, 2 should have step size 1000 (default), 500, 1000 respectively. + Client: client, + Label: label, + Member: memberVal, + Step: step * i, // allocator 0, 1 should have step size 1000 (default), 500 respectively. }) go func(re *require.Assertions, allocator Allocator) { defer wg.Done() testAllocator(re, allocator) }(re, allocator) + i++ } + fn(DefaultLabel) + fn(KeyspaceLabel) wg.Wait() } diff --git a/pkg/keyspace/keyspace.go b/pkg/keyspace/keyspace.go index afb60d7bb3f..08bb51da936 100644 --- a/pkg/keyspace/keyspace.go +++ b/pkg/keyspace/keyspace.go @@ -42,8 +42,6 @@ const ( // AllocStep set idAllocator's step when write persistent window boundary. // Use a lower value for denser idAllocation in the event of frequent pd leader change. AllocStep = uint64(100) - // AllocLabel is used to label keyspace idAllocator's metrics. - AllocLabel = "keyspace-idAlloc" // regionLabelIDPrefix is used to prefix the keyspace region label. regionLabelIDPrefix = "keyspaces/" // regionLabelKey is the key for keyspace id in keyspace region label. diff --git a/pkg/mcs/scheduling/server/server.go b/pkg/mcs/scheduling/server/server.go index a613b54b82d..f615a78fe05 100644 --- a/pkg/mcs/scheduling/server/server.go +++ b/pkg/mcs/scheduling/server/server.go @@ -452,13 +452,15 @@ func (s *Server) startServer() (err error) { uniqueName := s.cfg.GetAdvertiseListenAddr() uniqueID := memberutil.GenerateUniqueID(uniqueName) log.Info("joining primary election", zap.String("participant-name", uniqueName), zap.Uint64("participant-id", uniqueID)) - s.participant = member.NewParticipant(s.GetClient(), constant.SchedulingServiceName) + s.participant = member.NewParticipant(s.GetClient(), keypath.MsParam{ + ServiceName: constant.SchedulingServiceName, + }) p := &schedulingpb.Participant{ Name: uniqueName, Id: uniqueID, // id is unique among all participants ListenUrls: []string{s.cfg.GetAdvertiseListenAddr()}, } - s.participant.InitInfo(p, keypath.SchedulingSvcRootPath(), constant.PrimaryKey, "primary election") + s.participant.InitInfo(p, keypath.SchedulingSvcRootPath(), "primary election") s.service = &Service{Server: s} s.AddServiceReadyCallback(s.startCluster) diff --git a/pkg/member/member.go b/pkg/member/member.go index 2dc8be52031..89ba59d385f 100644 --- a/pkg/member/member.go +++ b/pkg/member/member.go @@ -34,6 +34,7 @@ import ( "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" + "github.com/tikv/pd/pkg/utils/keypath" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" "go.uber.org/zap" @@ -41,8 +42,7 @@ import ( const ( // The timeout to wait transfer etcd leader to complete. - moveLeaderTimeout = 5 * time.Second - dcLocationConfigEtcdPrefix = "dc-location" + moveLeaderTimeout = 5 * time.Second // If the campaign times is more than this value in `campaignTimesRecordTimeout`, the PD will resign and campaign again. campaignLeaderFrequencyTimes = 3 ) @@ -160,8 +160,8 @@ func (m *EmbeddedEtcdMember) EnableLeader() { } // GetLeaderPath returns the path of the PD leader. -func (m *EmbeddedEtcdMember) GetLeaderPath() string { - return path.Join(m.rootPath, "leader") +func (*EmbeddedEtcdMember) GetLeaderPath() string { + return keypath.LeaderPath(nil) } // GetLeadership returns the leadership of the PD member. @@ -384,13 +384,13 @@ func (m *EmbeddedEtcdMember) getMemberLeaderPriorityPath(id uint64) string { } // GetDCLocationPathPrefix returns the dc-location path prefix of the cluster. -func (m *EmbeddedEtcdMember) GetDCLocationPathPrefix() string { - return path.Join(m.rootPath, dcLocationConfigEtcdPrefix) +func (*EmbeddedEtcdMember) GetDCLocationPathPrefix() string { + return keypath.Prefix(keypath.DCLocationPath(nil, 0)) } // GetDCLocationPath returns the dc-location path of a member with the given member ID. -func (m *EmbeddedEtcdMember) GetDCLocationPath(id uint64) string { - return path.Join(m.GetDCLocationPathPrefix(), fmt.Sprint(id)) +func (*EmbeddedEtcdMember) GetDCLocationPath(id uint64) string { + return keypath.DCLocationPath(nil, id) } // SetMemberLeaderPriority saves a member's priority to be elected as the etcd leader. @@ -452,13 +452,9 @@ func (m *EmbeddedEtcdMember) GetMemberLeaderPriority(id uint64) (int, error) { return int(priority), nil } -func (m *EmbeddedEtcdMember) getMemberBinaryDeployPath(id uint64) string { - return path.Join(m.rootPath, fmt.Sprintf("member/%d/deploy_path", id)) -} - // GetMemberDeployPath loads a member's binary deploy path. func (m *EmbeddedEtcdMember) GetMemberDeployPath(id uint64) (string, error) { - key := m.getMemberBinaryDeployPath(id) + key := keypath.MemberBinaryDeployPath(id) res, err := etcdutil.EtcdKVGet(m.client, key) if err != nil { return "", err @@ -471,7 +467,7 @@ func (m *EmbeddedEtcdMember) GetMemberDeployPath(id uint64) (string, error) { // SetMemberDeployPath saves a member's binary deploy path. func (m *EmbeddedEtcdMember) SetMemberDeployPath(id uint64) error { - key := m.getMemberBinaryDeployPath(id) + key := keypath.MemberBinaryDeployPath(id) txn := kv.NewSlowLogTxn(m.client) execPath, err := os.Executable() deployPath := filepath.Dir(execPath) @@ -488,17 +484,9 @@ func (m *EmbeddedEtcdMember) SetMemberDeployPath(id uint64) error { return nil } -func (m *EmbeddedEtcdMember) getMemberGitHashPath(id uint64) string { - return path.Join(m.rootPath, fmt.Sprintf("member/%d/git_hash", id)) -} - -func (m *EmbeddedEtcdMember) getMemberBinaryVersionPath(id uint64) string { - return path.Join(m.rootPath, fmt.Sprintf("member/%d/binary_version", id)) -} - // GetMemberBinaryVersion loads a member's binary version. func (m *EmbeddedEtcdMember) GetMemberBinaryVersion(id uint64) (string, error) { - key := m.getMemberBinaryVersionPath(id) + key := keypath.MemberBinaryVersionPath(id) res, err := etcdutil.EtcdKVGet(m.client, key) if err != nil { return "", err @@ -511,7 +499,7 @@ func (m *EmbeddedEtcdMember) GetMemberBinaryVersion(id uint64) (string, error) { // GetMemberGitHash loads a member's git hash. func (m *EmbeddedEtcdMember) GetMemberGitHash(id uint64) (string, error) { - key := m.getMemberGitHashPath(id) + key := keypath.MemberGitHashPath(id) res, err := etcdutil.EtcdKVGet(m.client, key) if err != nil { return "", err @@ -524,7 +512,7 @@ func (m *EmbeddedEtcdMember) GetMemberGitHash(id uint64) (string, error) { // SetMemberBinaryVersion saves a member's binary version. func (m *EmbeddedEtcdMember) SetMemberBinaryVersion(id uint64, releaseVersion string) error { - key := m.getMemberBinaryVersionPath(id) + key := keypath.MemberBinaryVersionPath(id) txn := kv.NewSlowLogTxn(m.client) res, err := txn.Then(clientv3.OpPut(key, releaseVersion)).Commit() if err != nil { @@ -538,7 +526,7 @@ func (m *EmbeddedEtcdMember) SetMemberBinaryVersion(id uint64, releaseVersion st // SetMemberGitHash saves a member's git hash. func (m *EmbeddedEtcdMember) SetMemberGitHash(id uint64, gitHash string) error { - key := m.getMemberGitHashPath(id) + key := keypath.MemberGitHashPath(id) txn := kv.NewSlowLogTxn(m.client) res, err := txn.Then(clientv3.OpPut(key, gitHash)).Commit() if err != nil { diff --git a/pkg/member/participant.go b/pkg/member/participant.go index a513152d9b2..93d852f0fb0 100644 --- a/pkg/member/participant.go +++ b/pkg/member/participant.go @@ -16,8 +16,6 @@ package member import ( "context" - "fmt" - "path" "sync/atomic" "time" @@ -28,6 +26,7 @@ import ( "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/etcdutil" + "github.com/tikv/pd/pkg/utils/keypath" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -48,14 +47,13 @@ type participant interface { // EmbeddedEtcdMember, Participant relies on etcd for election, but it's decoupled // from the embedded etcd. It implements Member interface. type Participant struct { + keypath.MsParam leadership *election.Leadership // stored as member type - leader atomic.Value - client *clientv3.Client - rootPath string - leaderPath string - member participant - serviceName string + leader atomic.Value + client *clientv3.Client + rootPath string + member participant // memberValue is the serialized string of `member`. It will be saved in the // leader key when this participant is successfully elected as the leader of // the group. Every write will use it to check the leadership. @@ -70,15 +68,15 @@ type Participant struct { } // NewParticipant create a new Participant. -func NewParticipant(client *clientv3.Client, serviceName string) *Participant { +func NewParticipant(client *clientv3.Client, msParam keypath.MsParam) *Participant { return &Participant{ - client: client, - serviceName: serviceName, + MsParam: msParam, + client: client, } } -// InitInfo initializes the member info. The leader key is path.Join(rootPath, leaderName) -func (m *Participant) InitInfo(p participant, rootPath string, leaderName string, purpose string) { +// InitInfo initializes the member info. +func (m *Participant) InitInfo(p participant, rootPath string, purpose string) { data, err := p.Marshal() if err != nil { // can't fail, so panic here. @@ -87,10 +85,9 @@ func (m *Participant) InitInfo(p participant, rootPath string, leaderName string m.member = p m.memberValue = string(data) m.rootPath = rootPath - m.leaderPath = path.Join(rootPath, leaderName) m.leadership = election.NewLeadership(m.client, m.GetLeaderPath(), purpose) m.lastLeaderUpdatedTime.Store(time.Now()) - log.Info("participant joining election", zap.String("participant-info", p.String()), zap.String("leader-path", m.leaderPath)) + log.Info("participant joining election", zap.String("participant-info", p.String()), zap.String("leader-path", m.GetLeaderPath())) } // ID returns the unique ID for this participant in the election group @@ -143,7 +140,7 @@ func (m *Participant) GetLeaderID() uint64 { func (m *Participant) GetLeader() participant { leader := m.leader.Load() if leader == nil { - return NewParticipantByService(m.serviceName) + return NewParticipantByService(m.ServiceName) } return leader.(participant) } @@ -156,7 +153,7 @@ func (m *Participant) setLeader(member participant) { // unsetLeader unsets the member's leader. func (m *Participant) unsetLeader() { - leader := NewParticipantByService(m.serviceName) + leader := NewParticipantByService(m.ServiceName) m.leader.Store(leader) m.lastLeaderUpdatedTime.Store(time.Now()) } @@ -168,7 +165,7 @@ func (m *Participant) EnableLeader() { // GetLeaderPath returns the path of the leader. func (m *Participant) GetLeaderPath() string { - return m.leaderPath + return keypath.LeaderPath(&m.MsParam) } // GetLastLeaderUpdatedTime returns the last time when the leader is updated. @@ -207,7 +204,7 @@ func (*Participant) PreCheckLeader() error { // getPersistentLeader gets the corresponding leader from etcd by given leaderPath (as the key). func (m *Participant) getPersistentLeader() (participant, int64, error) { - leader := NewParticipantByService(m.serviceName) + leader := NewParticipantByService(m.ServiceName) ok, rev, err := etcdutil.GetProtoMsgWithModRev(m.client, m.GetLeaderPath(), leader) if err != nil { return nil, 0, err @@ -281,12 +278,12 @@ func (m *Participant) IsSameLeader(leader participant) bool { // GetDCLocationPathPrefix returns the dc-location path prefix of the cluster. func (m *Participant) GetDCLocationPathPrefix() string { - return path.Join(m.rootPath, dcLocationConfigEtcdPrefix) + return keypath.Prefix(keypath.DCLocationPath(&m.MsParam, 0)) } // GetDCLocationPath returns the dc-location path of a member with the given member ID. func (m *Participant) GetDCLocationPath(id uint64) string { - return path.Join(m.GetDCLocationPathPrefix(), fmt.Sprint(id)) + return keypath.DCLocationPath(&m.MsParam, id) } func (m *Participant) campaignCheck() bool { diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 4c49153814d..07731204e3d 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -751,13 +751,16 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroup(group *endpoint.KeyspaceGro zap.String("participant-name", uniqueName), zap.Uint64("participant-id", uniqueID)) // Initialize the participant info to join the primary election. - participant := member.NewParticipant(kgm.etcdClient, constant.TSOServiceName) + participant := member.NewParticipant(kgm.etcdClient, keypath.MsParam{ + ServiceName: constant.TSOServiceName, + GroupID: group.ID, + }) p := &tsopb.Participant{ Name: uniqueName, Id: uniqueID, // id is unique among all participants ListenUrls: []string{kgm.cfg.GetAdvertiseListenAddr()}, } - participant.InitInfo(p, keypath.KeyspaceGroupsElectionPath(kgm.tsoSvcRootPath, group.ID), constant.PrimaryKey, "keyspace group primary election") + participant.InitInfo(p, keypath.KeyspaceGroupsElectionPath(kgm.tsoSvcRootPath, group.ID), "keyspace group primary election") // If the keyspace group is in split, we should ensure that the primary elected by the new keyspace group // is always on the same TSO Server node as the primary of the old keyspace group, and this constraint cannot // be broken until the entire split process is completed. diff --git a/pkg/utils/keypath/key_path.go b/pkg/utils/keypath/key_path.go index 10934d1be9e..ba0e7111cf2 100644 --- a/pkg/utils/keypath/key_path.go +++ b/pkg/utils/keypath/key_path.go @@ -53,7 +53,6 @@ const ( keyspacePrefix = "keyspaces" keyspaceMetaInfix = "meta" keyspaceIDInfix = "id" - keyspaceAllocID = "alloc_id" gcSafePointInfix = "gc_safe_point" serviceSafePointInfix = "service_safe_point" regionPathPrefix = "raft/r" @@ -285,12 +284,6 @@ func KeyspaceIDPath(name string) string { return path.Join(keyspacePrefix, keyspaceIDInfix, name) } -// KeyspaceIDAlloc returns the path of the keyspace id's persistent window boundary. -// Path: keyspaces/alloc_id -func KeyspaceIDAlloc() string { - return path.Join(keyspacePrefix, keyspaceAllocID) -} - // EncodeKeyspaceID from uint32 to string. // It adds extra padding to make encoded ID ordered. // Encoded ID can be decoded directly with strconv.ParseUint. diff --git a/pkg/utils/keypath/key_path_v2.go b/pkg/utils/keypath/key_path_v2.go new file mode 100644 index 00000000000..395e311e64c --- /dev/null +++ b/pkg/utils/keypath/key_path_v2.go @@ -0,0 +1,101 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package keypath + +import ( + "fmt" + "path" +) + +const ( + leaderPathFormat = "/pd/%d/leader" // "/pd/{cluster_id}/leader" + dcLocationPathFormat = "/pd/%d/dc-location/%d" // "/pd/{cluster_id}/dc-location/{member_id}" + memberBinaryDeployPathFormat = "/pd/%d/member/%d/deploy_path" // "/pd/{cluster_id}/member/{member_id}/deploy_path" + memberGitHashPath = "/pd/%d/member/%d/git_hash" // "/pd/{cluster_id}/member/{member_id}/git_hash" + memberBinaryVersionPathFormat = "/pd/%d/member/%d/binary_version" // "/pd/{cluster_id}/member/{member_id}/binary_version" + allocIDPathFormat = "/pd/%d/alloc_id" // "/pd/{cluster_id}/alloc_id" + keyspaceAllocIDPathFormat = "/pd/%d/keyspaces/alloc_id" // "/pd/{cluster_id}/keyspaces/alloc_id" + + msLeaderPathFormat = "/ms/%d/%s/primary" // "/ms/{cluster_id}/{service_name}/primary" + msTsoDefaultLeaderPathFormat = "/ms/%d/tso/00000/primary" // "/ms/{cluster_id}/tso/00000/primary" + msTsoKespaceLeaderPathFormat = "/ms/%d/tso/keyspace_groups/election/%05d/primary" // "/ms/{cluster_id}/tso/keyspace_groups/election/{group_id}/primary" + msDCLocationPathFormat = "/ms/%d/%s/dc-location/%d" // "/ms/{cluster_id}/{service_name}/dc-location/{member_id}" + msTsoDefaultDCLocationPath = "/ms/%d/tso/00000/dc-location/%d" // "/ms/{cluster_id}/tso/00000/dc-location/{member_id}" + msTsoKespaceDCLocationPath = "/ms/%d/tso/keyspace_groups/election/%05d/dc-location/%d" // "/ms/{cluster_id}/tso/keyspace_groups/election/{group_id}/dc-location/{member_id}" +) + +// MsParam is the parameter of micro service. +type MsParam struct { + ServiceName string + GroupID uint32 // only used for tso keyspace group +} + +// Prefix returns the parent directory of the given path. +func Prefix(str string) string { + return path.Dir(str) +} + +// LeaderPath returns the leader path. +func LeaderPath(p *MsParam) string { + if p == nil || p.ServiceName == "" { + return fmt.Sprintf(leaderPathFormat, ClusterID()) + } + if p.ServiceName == "tso" { + if p.GroupID == 0 { + return fmt.Sprintf(msTsoDefaultLeaderPathFormat, ClusterID()) + } + return fmt.Sprintf(msTsoKespaceLeaderPathFormat, ClusterID(), p.GroupID) + } + return fmt.Sprintf(msLeaderPathFormat, ClusterID(), p.ServiceName) +} + +// DCLocationPath returns the dc-location path. +func DCLocationPath(p *MsParam, memberID uint64) string { + if p == nil || p.ServiceName == "" { + return fmt.Sprintf(dcLocationPathFormat, ClusterID(), memberID) + } + if p.ServiceName == "tso" { + if p.GroupID == 0 { + return fmt.Sprintf(msTsoDefaultDCLocationPath, ClusterID(), memberID) + } + return fmt.Sprintf(msTsoKespaceDCLocationPath, ClusterID(), p.GroupID, memberID) + } + return fmt.Sprintf(msDCLocationPathFormat, ClusterID(), p.ServiceName, memberID) +} + +// MemberBinaryDeployPath returns the member binary deploy path. +func MemberBinaryDeployPath(id uint64) string { + return fmt.Sprintf(memberBinaryDeployPathFormat, ClusterID(), id) +} + +// MemberGitHashPath returns the member git hash path. +func MemberGitHashPath(id uint64) string { + return fmt.Sprintf(memberGitHashPath, ClusterID(), id) +} + +// MemberBinaryVersionPath returns the member binary version path. +func MemberBinaryVersionPath(id uint64) string { + return fmt.Sprintf(memberBinaryVersionPathFormat, ClusterID(), id) +} + +// AllocIDPath returns the alloc id path. +func AllocIDPath() string { + return fmt.Sprintf(allocIDPathFormat, ClusterID()) +} + +// KeyspaceAllocIDPath returns the keyspace alloc id path. +func KeyspaceAllocIDPath() string { + return fmt.Sprintf(keyspaceAllocIDPathFormat, ClusterID()) +} diff --git a/server/server.go b/server/server.go index 029c85694c3..ed61b14834f 100644 --- a/server/server.go +++ b/server/server.go @@ -93,9 +93,6 @@ const ( // pdRootPath for all pd servers. pdRootPath = "/pd" pdAPIPrefix = "/pd/" - // idAllocPath for idAllocator to save persistent window's end. - idAllocPath = "alloc_id" - idAllocLabel = "idalloc" recoveringMarkPath = "cluster/markers/snapshot-recovering" @@ -446,11 +443,9 @@ func (s *Server) startServer(ctx context.Context) error { return err } s.idAllocator = id.NewAllocator(&id.AllocatorParams{ - Client: s.client, - RootPath: s.rootPath, - AllocPath: idAllocPath, - Label: idAllocLabel, - Member: s.member.MemberValue(), + Client: s.client, + Label: id.DefaultLabel, + Member: s.member.MemberValue(), }) s.encryptionKeyManager, err = encryption.NewManager(s.client, &s.cfg.Security.Encryption) if err != nil { @@ -487,12 +482,10 @@ func (s *Server) startServer(ctx context.Context) error { s.basicCluster = core.NewBasicCluster() s.cluster = cluster.NewRaftCluster(ctx, s.GetMember(), s.GetBasicCluster(), s.GetStorage(), syncer.NewRegionSyncer(s), s.client, s.httpClient, s.tsoAllocatorManager) keyspaceIDAllocator := id.NewAllocator(&id.AllocatorParams{ - Client: s.client, - RootPath: s.rootPath, - AllocPath: keypath.KeyspaceIDAlloc(), - Label: keyspace.AllocLabel, - Member: s.member.MemberValue(), - Step: keyspace.AllocStep, + Client: s.client, + Label: id.KeyspaceLabel, + Member: s.member.MemberValue(), + Step: keyspace.AllocStep, }) if s.IsAPIServiceMode() { s.keyspaceGroupManager = keyspace.NewKeyspaceGroupManager(s.ctx, s.storage, s.client) diff --git a/tests/integrations/realcluster/cluster.go b/tests/integrations/realcluster/cluster.go index 5068dc3b73b..b90ec3cb4d9 100644 --- a/tests/integrations/realcluster/cluster.go +++ b/tests/integrations/realcluster/cluster.go @@ -87,6 +87,9 @@ func (s *clusterSuite) stopCluster(t *testing.T) { } func (s *clusterSuite) tag() string { + if s.ms { + return fmt.Sprintf("pd_real_cluster_test_ms_%s_%d", s.suiteName, s.clusterCnt) + } return fmt.Sprintf("pd_real_cluster_test_%s_%d", s.suiteName, s.clusterCnt) } @@ -181,5 +184,6 @@ func waitTiupReady(t *testing.T, tag string) { zap.String("tag", tag), zap.Error(err)) time.Sleep(time.Duration(interval) * time.Second) } - require.Failf(t, "TiUP is not ready", "tag: %s", tag) + // this check can trigger the cleanup function + require.NotZero(t, 1, "TiUP is not ready", "tag: %s", tag) } diff --git a/tests/integrations/realcluster/cluster_id_test.go b/tests/integrations/realcluster/cluster_id_test.go index 518991fef97..914915d0947 100644 --- a/tests/integrations/realcluster/cluster_id_test.go +++ b/tests/integrations/realcluster/cluster_id_test.go @@ -16,7 +16,6 @@ package realcluster import ( "context" - "os/exec" "strings" "testing" @@ -57,10 +56,8 @@ func (s *clusterIDSuite) TestClientClusterID() { } func getPDEndpoints(t *testing.T) []string { - cmd := exec.Command("sh", "-c", "ps -ef | grep tikv-server | awk -F '--pd-endpoints=' '{print $2}' | awk '{print $1}'") - bytes, err := cmd.Output() + pdAddrsForEachTikv, err := runCommandWithOutput("ps -ef | grep tikv-server | awk -F '--pd-endpoints=' '{print $2}' | awk '{print $1}'") require.NoError(t, err) - pdAddrsForEachTikv := strings.Split(string(bytes), "\n") var pdAddrs []string for _, addr := range pdAddrsForEachTikv { // length of addr is less than 5 means it must not be a valid address diff --git a/tests/integrations/realcluster/download_integration_test_binaries.sh b/tests/integrations/realcluster/download_integration_test_binaries.sh index 8d4cc3411a8..9009750ca99 100644 --- a/tests/integrations/realcluster/download_integration_test_binaries.sh +++ b/tests/integrations/realcluster/download_integration_test_binaries.sh @@ -28,6 +28,8 @@ tidb_download_url="${file_server_url}/download/builds/pingcap/tidb/${tidb_sha1}/ tikv_download_url="${file_server_url}/download/builds/pingcap/tikv/${tikv_sha1}/centos7/tikv-server.tar.gz" tiflash_download_url="${file_server_url}/download/builds/pingcap/tiflash/${branch}/${tiflash_sha1}/centos7/tiflash.tar.gz" +ETCD_VERSION=v3.5.15 + set -o nounset # See https://misc.flogisoft.com/bash/tip_colors_and_formatting. @@ -69,6 +71,12 @@ function main() { mv third_bin/tiflash third_bin/_tiflash mv third_bin/_tiflash/* third_bin && rm -rf third_bin/_tiflash + # etcdctl + curl -L https://github.com/etcd-io/etcd/releases/download/${ETCD_VERSION}/etcd-${ETCD_VERSION}-linux-amd64.tar.gz -o etcd-${ETCD_VERSION}-linux-amd64.tar.gz + tar -xvf etcd-${ETCD_VERSION}-linux-amd64.tar.gz + sudo mv etcd-${ETCD_VERSION}-linux-amd64/etcdctl /usr/local/bin/ + etcdctl version + chmod +x third_bin/* rm -rf tmp rm -rf third_bin/bin diff --git a/tests/integrations/realcluster/etcd_key_test.go b/tests/integrations/realcluster/etcd_key_test.go new file mode 100644 index 00000000000..e546a9ae9b1 --- /dev/null +++ b/tests/integrations/realcluster/etcd_key_test.go @@ -0,0 +1,146 @@ +// Copyright 2024 TiKV Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package realcluster + +import ( + "fmt" + "slices" + "testing" + + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "github.com/tikv/pd/client/utils/testutil" +) + +type etcdKeySuite struct { + clusterSuite +} + +func TestEtcdKey(t *testing.T) { + suite.Run(t, &etcdKeySuite{ + clusterSuite: clusterSuite{ + suiteName: "etcd_key", + }, + }) +} + +func TestMSEtcdKey(t *testing.T) { + suite.Run(t, &etcdKeySuite{ + clusterSuite: clusterSuite{ + suiteName: "etcd_key", + ms: true, + }, + }) +} + +var ( + // The keys that prefix is `/pd`. + pdKeys = []string{ + "", + "/pd//alloc_id", + "/pd//config", + // If not call `UpdateGCSafePoint`, this key will not exist. + // "/pd//gc/safe_point", + "/pd//gc/safe_point/service/gc_worker", + "/pd//keyspaces/id/DEFAULT", + "/pd//keyspaces/meta/", + "/pd//leader", + "/pd//member//binary_version", + "/pd//member//deploy_path", + "/pd//member//git_hash", + "/pd//raft", + "/pd//raft/min_resolved_ts", + "/pd//raft/r/", + "/pd//raft/s/", + "/pd//raft/status/raft_bootstrap_time", + "/pd//region_label/keyspaces/", + "/pd//rule_group/tiflash", + "/pd//rules/-c", // Why -c? See https://github.com/tikv/pd/pull/8789#discussion_r1853341293 + "/pd//scheduler_config/balance-hot-region-scheduler", + "/pd//scheduler_config/balance-leader-scheduler", + "/pd//scheduler_config/balance-region-scheduler", + "/pd//scheduler_config/evict-slow-store-scheduler", + "/pd//timestamp", + "/pd//tso/keyspace_groups/membership/", // ms + "/pd/cluster_id", + } + // The keys that prefix is `/ms`. + msKeys = []string{ + "", + "/ms//scheduling/primary", + "/ms//scheduling/primary/expected_primary", + "/ms//scheduling/registry/http://...:", + "/ms//tso//primary", + "/ms//tso//primary/expected_primary", + "/ms//tso/registry/http://...:", + } + // These keys with `/pd` are only in `ms` mode. + pdMSKeys = []string{ + "/pd//tso/keyspace_groups/membership/", + } +) + +func (s *etcdKeySuite) TestEtcdKey() { + var keysBackup []string + if !s.ms { + keysBackup = pdKeys + pdKeys = slices.DeleteFunc(pdKeys, func(s string) bool { + return slices.Contains(pdMSKeys, s) + }) + defer func() { + pdKeys = keysBackup + }() + } + t := s.T() + endpoints := getPDEndpoints(t) + + testutil.Eventually(require.New(t), func() bool { + keys, err := getEtcdKey(endpoints[0], "/pd") + if err != nil { + return false + } + return checkEtcdKey(t, keys, pdKeys) + }) + + if s.ms { + testutil.Eventually(require.New(t), func() bool { + keys, err := getEtcdKey(endpoints[0], "/ms") + if err != nil { + return false + } + return checkEtcdKey(t, keys, msKeys) + }) + } +} + +func getEtcdKey(endpoints, prefix string) ([]string, error) { + // `sed 's/[0-9]*//g'` is used to remove the number in the etcd key, such as the cluster id. + etcdCmd := fmt.Sprintf("etcdctl --endpoints=%s get %s --prefix --keys-only | sed 's/[0-9]*//g' | sort | uniq", + endpoints, prefix) + return runCommandWithOutput(etcdCmd) +} + +func checkEtcdKey(t *testing.T, keys, expectedKeys []string) bool { + for i, key := range keys { + if len(key) == 0 { + continue + } + if expectedKeys[i] != key { + t.Logf("expected key: %s, got key: %s", expectedKeys[i], key) + return false + } + } + return true +} diff --git a/tests/integrations/realcluster/util.go b/tests/integrations/realcluster/util.go index 789ceaa29c2..a33f9c071c6 100644 --- a/tests/integrations/realcluster/util.go +++ b/tests/integrations/realcluster/util.go @@ -17,6 +17,7 @@ package realcluster import ( "os" "os/exec" + "strings" "time" ) @@ -40,6 +41,16 @@ func runCommand(name string, args ...string) error { return cmd.Run() } +func runCommandWithOutput(cmdStr string) ([]string, error) { + cmd := exec.Command("sh", "-c", cmdStr) + bytes, err := cmd.Output() + if err != nil { + return nil, err + } + output := strings.Split(string(bytes), "\n") + return output, nil +} + func fileExists(path string) bool { _, err := os.Stat(path) return !os.IsNotExist(err) From 21a04cb23711a12680d8d47b5e9c6a6460ce87c1 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Mon, 25 Nov 2024 15:27:08 +0800 Subject: [PATCH 6/7] tso: remove local tso code part 2 (#8846) ref tikv/pd#8802 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- errors.toml | 10 - pkg/errs/errno.go | 2 - pkg/tso/allocator_manager.go | 389 ----------------------------------- pkg/tso/global_allocator.go | 350 +------------------------------ server/api/member_test.go | 8 +- server/config/config.go | 6 - server/grpc_service.go | 51 +---- server/server.go | 20 -- tests/cluster.go | 65 ------ 9 files changed, 7 insertions(+), 894 deletions(-) diff --git a/errors.toml b/errors.toml index a7e039564d7..9bfd4a79190 100644 --- a/errors.toml +++ b/errors.toml @@ -901,16 +901,6 @@ error = ''' reset user timestamp failed, %s ''' -["PD:tso:ErrSetLocalTSOConfig"] -error = ''' -set local tso config failed, %s -''' - -["PD:tso:ErrSyncMaxTS"] -error = ''' -sync max ts failed, %s -''' - ["PD:tso:ErrUpdateTimestamp"] error = ''' update timestamp failed, %s diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index 29065c7c13d..e5c23cffde2 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -40,10 +40,8 @@ var ( // tso errors var ( - ErrSetLocalTSOConfig = errors.Normalize("set local tso config failed, %s", errors.RFCCodeText("PD:tso:ErrSetLocalTSOConfig")) ErrGetAllocator = errors.Normalize("get allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetAllocator")) ErrGetLocalAllocator = errors.Normalize("get local allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetLocalAllocator")) - ErrSyncMaxTS = errors.Normalize("sync max ts failed, %s", errors.RFCCodeText("PD:tso:ErrSyncMaxTS")) ErrResetUserTimestamp = errors.Normalize("reset user timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrResetUserTimestamp")) ErrGenerateTimestamp = errors.Normalize("generate timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrGenerateTimestamp")) ErrUpdateTimestamp = errors.Normalize("update timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrUpdateTimestamp")) diff --git a/pkg/tso/allocator_manager.go b/pkg/tso/allocator_manager.go index 821e1fc5d62..ea1eeeeb18a 100644 --- a/pkg/tso/allocator_manager.go +++ b/pkg/tso/allocator_manager.go @@ -33,7 +33,6 @@ import ( "github.com/tikv/pd/pkg/member" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/storage/endpoint" - "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/logutil" @@ -315,64 +314,6 @@ func (am *AllocatorManager) GetMember() ElectionMember { return am.member } -// SetLocalTSOConfig receives the zone label of this PD server and write it into etcd as dc-location -// to make the whole cluster know the DC-level topology for later Local TSO Allocator campaign. -func (am *AllocatorManager) SetLocalTSOConfig(dcLocation string) error { - serverName := am.member.Name() - serverID := am.member.ID() - if err := am.checkDCLocationUpperLimit(dcLocation); err != nil { - log.Error("check dc-location upper limit failed", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID != 0), - zap.Int("upper-limit", int(math.Pow(2, MaxSuffixBits))-1), - zap.String("dc-location", dcLocation), - zap.String("server-name", serverName), - zap.Uint64("server-id", serverID), - errs.ZapError(err)) - return err - } - // The key-value pair in etcd will be like: serverID -> dcLocation - dcLocationKey := am.member.GetDCLocationPath(serverID) - resp, err := kv. - NewSlowLogTxn(am.member.Client()). - Then(clientv3.OpPut(dcLocationKey, dcLocation)). - Commit() - if err != nil { - return errs.ErrEtcdTxnInternal.Wrap(err).GenWithStackByCause() - } - if !resp.Succeeded { - log.Warn("write dc-location configuration into etcd failed", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", dcLocation), - zap.String("server-name", serverName), - zap.Uint64("server-id", serverID)) - return errs.ErrEtcdTxnConflict.FastGenByArgs() - } - log.Info("write dc-location configuration into etcd", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", dcLocation), - zap.String("server-name", serverName), - zap.Uint64("server-id", serverID)) - go am.ClusterDCLocationChecker() - return nil -} - -func (am *AllocatorManager) checkDCLocationUpperLimit(dcLocation string) error { - clusterDCLocations, err := am.GetClusterDCLocationsFromEtcd() - if err != nil { - return err - } - // It's ok to add a new PD to the old dc-location. - if _, ok := clusterDCLocations[dcLocation]; ok { - return nil - } - // Check whether the dc-location number meets the upper limit 2**(LogicalBits-1)-1, - // which includes 1 global and 2**(LogicalBits-1) local - if len(clusterDCLocations) == int(math.Pow(2, MaxSuffixBits))-1 { - return errs.ErrSetLocalTSOConfig.FastGenByArgs("the number of dc-location meets the upper limit") - } - return nil -} - // GetClusterDCLocationsFromEtcd fetches dcLocation topology from etcd func (am *AllocatorManager) GetClusterDCLocationsFromEtcd() (clusterDCLocations map[string][]uint64, err error) { resp, err := etcdutil.EtcdKVGet( @@ -413,26 +354,6 @@ func (am *AllocatorManager) GetDCLocationInfo(dcLocation string) (DCLocationInfo return infoPtr.clone(), true } -// CleanUpDCLocation cleans up certain server's DCLocationInfo -func (am *AllocatorManager) CleanUpDCLocation() error { - serverID := am.member.ID() - dcLocationKey := am.member.GetDCLocationPath(serverID) - // remove dcLocationKey from etcd - if resp, err := kv. - NewSlowLogTxn(am.member.Client()). - Then(clientv3.OpDelete(dcLocationKey)). - Commit(); err != nil { - return errs.ErrEtcdTxnInternal.Wrap(err).GenWithStackByCause() - } else if !resp.Succeeded { - return errs.ErrEtcdTxnConflict.FastGenByArgs() - } - log.Info("delete the dc-location key previously written in etcd", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.Uint64("server-id", serverID)) - go am.ClusterDCLocationChecker() - return nil -} - // GetClusterDCLocations returns all dc-locations of a cluster with a copy of map, // which satisfies dcLocation -> DCLocationInfo. func (am *AllocatorManager) GetClusterDCLocations() map[string]DCLocationInfo { @@ -467,20 +388,6 @@ func CalSuffixBits(maxSuffix int32) int { return int(math.Ceil(math.Log2(float64(maxSuffix + 1)))) } -func (am *AllocatorManager) getAllocatorPath(dcLocation string) string { - // For backward compatibility, the global timestamp's store path will still use the old one - if dcLocation == GlobalDCLocation { - return am.rootPath - } - return path.Join(am.getLocalTSOAllocatorPath(), dcLocation) -} - -// Add a prefix to the root path to prevent being conflicted -// with other system key paths such as leader, member, alloc_id, raft, etc. -func (am *AllocatorManager) getLocalTSOAllocatorPath() string { - return path.Join(am.rootPath, localTSOAllocatorEtcdPrefix) -} - // AllocatorDaemon is used to update every allocator's TSO and check whether we have // any new local allocator that needs to be set up. func (am *AllocatorManager) AllocatorDaemon(ctx context.Context) { @@ -491,19 +398,12 @@ func (am *AllocatorManager) AllocatorDaemon(ctx context.Context) { tsTicker.Reset(time.Millisecond) }) defer tsTicker.Stop() - checkerTicker := time.NewTicker(PriorityCheck) - defer checkerTicker.Stop() for { select { case <-tsTicker.C: // Update the initialized TSO Allocator to advance TSO. am.allocatorUpdater() - case <-checkerTicker.C: - // Check and maintain the cluster's meta info about dc-location distribution. - go am.ClusterDCLocationChecker() - // PS: ClusterDCLocationChecker are time consuming and low frequent to run, - // we should run them concurrently to speed up the progress. case <-ctx.Done(): log.Info("exit allocator daemon", logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0)) return @@ -554,188 +454,6 @@ func (am *AllocatorManager) updateAllocator(ag *allocatorGroup) { } } -// ClusterDCLocationChecker collects all dc-locations of a cluster, computes some related info -// and stores them into the DCLocationInfo, then finally writes them into am.mu.clusterDCLocations. -func (am *AllocatorManager) ClusterDCLocationChecker() { - defer logutil.LogPanic() - // Wait for the group leader to be elected out. - if !am.member.IsLeaderElected() { - return - } - newClusterDCLocations, err := am.GetClusterDCLocationsFromEtcd() - if err != nil { - log.Error("get cluster dc-locations from etcd failed", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - errs.ZapError(err)) - return - } - am.mu.Lock() - // Clean up the useless dc-locations - for dcLocation := range am.mu.clusterDCLocations { - if _, ok := newClusterDCLocations[dcLocation]; !ok { - delete(am.mu.clusterDCLocations, dcLocation) - } - } - // May be used to rollback the updating after - newDCLocations := make([]string, 0) - // Update the new dc-locations - for dcLocation, serverIDs := range newClusterDCLocations { - if _, ok := am.mu.clusterDCLocations[dcLocation]; !ok { - am.mu.clusterDCLocations[dcLocation] = &DCLocationInfo{ - ServerIDs: serverIDs, - Suffix: -1, - } - newDCLocations = append(newDCLocations, dcLocation) - } - } - // Only leader can write the TSO suffix to etcd in order to make it consistent in the cluster - if am.IsLeader() { - for dcLocation, info := range am.mu.clusterDCLocations { - if info.Suffix > 0 { - continue - } - suffix, err := am.getOrCreateLocalTSOSuffix(dcLocation) - if err != nil { - log.Warn("get or create the local tso suffix failed", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", dcLocation), - errs.ZapError(err)) - continue - } - if suffix > am.mu.maxSuffix { - am.mu.maxSuffix = suffix - } - am.mu.clusterDCLocations[dcLocation].Suffix = suffix - } - } else { - // Follower should check and update the am.mu.maxSuffix - maxSuffix, err := am.getMaxLocalTSOSuffix() - if err != nil { - log.Error("get the max local tso suffix from etcd failed", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - errs.ZapError(err)) - // Rollback the new dc-locations we update before - for _, dcLocation := range newDCLocations { - delete(am.mu.clusterDCLocations, dcLocation) - } - } else if maxSuffix > am.mu.maxSuffix { - am.mu.maxSuffix = maxSuffix - } - } - am.mu.Unlock() -} - -// getOrCreateLocalTSOSuffix will check whether we have the Local TSO suffix written into etcd. -// If not, it will write a number into etcd according to the its joining order. -// If yes, it will just return the previous persisted one. -func (am *AllocatorManager) getOrCreateLocalTSOSuffix(dcLocation string) (int32, error) { - // Try to get the suffix from etcd - dcLocationSuffix, err := am.getDCLocationSuffixMapFromEtcd() - if err != nil { - return -1, nil - } - var maxSuffix int32 - for curDCLocation, suffix := range dcLocationSuffix { - // If we already have the suffix persisted in etcd before, - // just use it as the result directly. - if curDCLocation == dcLocation { - return suffix, nil - } - if suffix > maxSuffix { - maxSuffix = suffix - } - } - maxSuffix++ - localTSOSuffixKey := am.GetLocalTSOSuffixPath(dcLocation) - // The Local TSO suffix is determined by the joining order of this dc-location. - localTSOSuffixValue := strconv.FormatInt(int64(maxSuffix), 10) - txnResp, err := kv.NewSlowLogTxn(am.member.Client()). - If(clientv3.Compare(clientv3.CreateRevision(localTSOSuffixKey), "=", 0)). - Then(clientv3.OpPut(localTSOSuffixKey, localTSOSuffixValue)). - Commit() - if err != nil { - return -1, errs.ErrEtcdTxnInternal.Wrap(err).GenWithStackByCause() - } - if !txnResp.Succeeded { - log.Warn("write local tso suffix into etcd failed", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", dcLocation), - zap.String("local-tso-suffix", localTSOSuffixValue), - zap.String("server-name", am.member.Name()), - zap.Uint64("server-id", am.member.ID())) - return -1, errs.ErrEtcdTxnConflict.FastGenByArgs() - } - return maxSuffix, nil -} - -func (am *AllocatorManager) getDCLocationSuffixMapFromEtcd() (map[string]int32, error) { - resp, err := etcdutil.EtcdKVGet( - am.member.Client(), - am.GetLocalTSOSuffixPathPrefix(), - clientv3.WithPrefix()) - if err != nil { - return nil, err - } - dcLocationSuffix := make(map[string]int32) - for _, kv := range resp.Kvs { - suffix, err := strconv.ParseInt(string(kv.Value), 10, 32) - if err != nil { - return nil, err - } - splitKey := strings.Split(string(kv.Key), "/") - dcLocation := splitKey[len(splitKey)-1] - dcLocationSuffix[dcLocation] = int32(suffix) - } - return dcLocationSuffix, nil -} - -func (am *AllocatorManager) getMaxLocalTSOSuffix() (int32, error) { - // Try to get the suffix from etcd - dcLocationSuffix, err := am.getDCLocationSuffixMapFromEtcd() - if err != nil { - return -1, err - } - var maxSuffix int32 - for _, suffix := range dcLocationSuffix { - if suffix > maxSuffix { - maxSuffix = suffix - } - } - return maxSuffix, nil -} - -// GetLocalTSOSuffixPathPrefix returns the etcd key prefix of the Local TSO suffix for the given dc-location. -func (am *AllocatorManager) GetLocalTSOSuffixPathPrefix() string { - return path.Join(am.rootPath, localTSOSuffixEtcdPrefix) -} - -// GetLocalTSOSuffixPath returns the etcd key of the Local TSO suffix for the given dc-location. -func (am *AllocatorManager) GetLocalTSOSuffixPath(dcLocation string) string { - return path.Join(am.GetLocalTSOSuffixPathPrefix(), dcLocation) -} - -// TransferAllocatorForDCLocation transfer local tso allocator to the target member for the given dcLocation -func (am *AllocatorManager) TransferAllocatorForDCLocation(dcLocation string, memberID uint64) error { - if dcLocation == GlobalDCLocation { - return fmt.Errorf("dc-location %v should be transferred by transfer leader", dcLocation) - } - dcLocationsInfo := am.GetClusterDCLocations() - _, ok := dcLocationsInfo[dcLocation] - if !ok { - return fmt.Errorf("dc-location %v haven't been discovered yet", dcLocation) - } - allocator, err := am.GetAllocator(dcLocation) - if err != nil { - return err - } - localTSOAllocator, _ := allocator.(*LocalTSOAllocator) - leaderServerID := localTSOAllocator.GetAllocatorLeader().GetMemberId() - if leaderServerID == memberID { - return nil - } - return am.transferLocalAllocator(dcLocation, memberID) -} - // HandleRequest forwards TSO allocation requests to correct TSO Allocators. func (am *AllocatorManager) HandleRequest(ctx context.Context, dcLocation string, count uint32) (pdpb.Timestamp, error) { defer trace.StartRegion(ctx, "AllocatorManager.HandleRequest").End() @@ -841,113 +559,6 @@ func (am *AllocatorManager) GetLocalAllocatorLeaders() (map[string]*pdpb.Member, return localAllocatorLeaderMember, nil } -func (am *AllocatorManager) getOrCreateGRPCConn(ctx context.Context, addr string) (*grpc.ClientConn, error) { - conn, ok := am.getGRPCConn(addr) - if ok { - return conn, nil - } - tlsCfg, err := am.securityConfig.ToTLSConfig() - if err != nil { - return nil, err - } - ctxWithTimeout, cancel := context.WithTimeout(ctx, dialTimeout) - defer cancel() - cc, err := grpcutil.GetClientConn(ctxWithTimeout, addr, tlsCfg) - if err != nil { - return nil, err - } - am.setGRPCConn(cc, addr) - conn, _ = am.getGRPCConn(addr) - return conn, nil -} - -// GetMaxLocalTSO will sync with the current Local TSO Allocators among the cluster to get the -// max Local TSO. -func (am *AllocatorManager) GetMaxLocalTSO(ctx context.Context) (*pdpb.Timestamp, error) { - // Sync the max local TSO from the other Local TSO Allocators who has been initialized - clusterDCLocations := am.GetClusterDCLocations() - for dcLocation := range clusterDCLocations { - allocatorGroup, ok := am.getAllocatorGroup(dcLocation) - if !(ok && allocatorGroup.leadership.Check()) { - delete(clusterDCLocations, dcLocation) - } - } - maxTSO := &pdpb.Timestamp{} - if len(clusterDCLocations) == 0 { - return maxTSO, nil - } - globalAllocator, err := am.GetAllocator(GlobalDCLocation) - if err != nil { - return nil, err - } - if err := globalAllocator.(*GlobalTSOAllocator).SyncMaxTS(ctx, clusterDCLocations, maxTSO, false); err != nil { - return nil, err - } - return maxTSO, nil -} - -func (am *AllocatorManager) getGRPCConn(addr string) (*grpc.ClientConn, bool) { - am.localAllocatorConn.RLock() - defer am.localAllocatorConn.RUnlock() - conn, ok := am.localAllocatorConn.clientConns[addr] - return conn, ok -} - -func (am *AllocatorManager) setGRPCConn(newConn *grpc.ClientConn, addr string) { - am.localAllocatorConn.Lock() - defer am.localAllocatorConn.Unlock() - if _, ok := am.localAllocatorConn.clientConns[addr]; ok { - newConn.Close() - log.Debug("use old connection", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("target", newConn.Target()), - zap.String("state", newConn.GetState().String())) - return - } - am.localAllocatorConn.clientConns[addr] = newConn -} - -func (am *AllocatorManager) transferLocalAllocator(dcLocation string, serverID uint64) error { - nextLeaderKey := am.nextLeaderKey(dcLocation) - // Grant a etcd lease with checkStep * 1.5 - nextLeaderLease := clientv3.NewLease(am.member.Client()) - ctx, cancel := context.WithTimeout(am.member.Client().Ctx(), etcdutil.DefaultRequestTimeout) - leaseResp, err := nextLeaderLease.Grant(ctx, int64(checkStep.Seconds()*1.5)) - cancel() - if err != nil { - err = errs.ErrEtcdGrantLease.Wrap(err).GenWithStackByCause() - log.Error("failed to grant the lease of the next leader key", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", dcLocation), - zap.Uint64("serverID", serverID), - errs.ZapError(err)) - return err - } - resp, err := kv.NewSlowLogTxn(am.member.Client()). - If(clientv3.Compare(clientv3.CreateRevision(nextLeaderKey), "=", 0)). - Then(clientv3.OpPut(nextLeaderKey, fmt.Sprint(serverID), clientv3.WithLease(leaseResp.ID))). - Commit() - if err != nil { - err = errs.ErrEtcdTxnInternal.Wrap(err).GenWithStackByCause() - log.Error("failed to write next leader key into etcd", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", dcLocation), zap.Uint64("serverID", serverID), - errs.ZapError(err)) - return err - } - if !resp.Succeeded { - log.Warn("write next leader id into etcd unsuccessfully", - logutil.CondUint32("keyspace-group-id", am.kgID, am.kgID > 0), - zap.String("dc-location", dcLocation)) - return errs.ErrEtcdTxnConflict.GenWithStack("write next leader id into etcd unsuccessfully") - } - return nil -} - -func (am *AllocatorManager) nextLeaderKey(dcLocation string) string { - return path.Join(am.getAllocatorPath(dcLocation), "next-leader") -} - // EnableLocalTSO returns the value of AllocatorManager.enableLocalTSO. func (am *AllocatorManager) EnableLocalTSO() bool { return am.enableLocalTSO diff --git a/pkg/tso/global_allocator.go b/pkg/tso/global_allocator.go index d44297b803e..cb599875dbd 100644 --- a/pkg/tso/global_allocator.go +++ b/pkg/tso/global_allocator.go @@ -24,7 +24,6 @@ import ( "sync/atomic" "time" - "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" @@ -33,13 +32,9 @@ import ( mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/member" - "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - "github.com/tikv/pd/pkg/utils/tsoutil" - "github.com/tikv/pd/pkg/utils/typeutil" "go.uber.org/zap" - "google.golang.org/grpc" ) // Allocator is a Timestamp Oracle allocator. @@ -85,10 +80,6 @@ type GlobalTSOAllocator struct { // expectedPrimaryLease is used to store the expected primary lease. expectedPrimaryLease atomic.Value // store as *election.LeaderLease timestampOracle *timestampOracle - // syncRTT is the RTT duration a SyncMaxTS RPC call will cost, - // which is used to estimate the MaxTS in a Global TSO generation - // to reduce the gRPC network IO latency. - syncRTT atomic.Value // store as int64 milliseconds // pre-initialized metrics tsoAllocatorRoleGauge prometheus.Gauge } @@ -142,19 +133,6 @@ func (gta *GlobalTSOAllocator) getGroupID() uint32 { return gta.am.getGroupID() } -func (gta *GlobalTSOAllocator) setSyncRTT(rtt int64) { - gta.syncRTT.Store(rtt) - gta.getMetrics().globalTSOSyncRTTGauge.Set(float64(rtt)) -} - -func (gta *GlobalTSOAllocator) getSyncRTT() int64 { - syncRTT := gta.syncRTT.Load() - if syncRTT == nil { - return 0 - } - return syncRTT.(int64) -} - // GetTimestampPath returns the timestamp path in etcd. func (gta *GlobalTSOAllocator) GetTimestampPath() string { if gta == nil || gta.timestampOracle == nil { @@ -163,24 +141,6 @@ func (gta *GlobalTSOAllocator) GetTimestampPath() string { return gta.timestampOracle.GetTimestampPath() } -func (gta *GlobalTSOAllocator) estimateMaxTS(ctx context.Context, count uint32, suffixBits int) (*pdpb.Timestamp, bool, error) { - physical, logical, lastUpdateTime := gta.timestampOracle.generateTSO(ctx, int64(count), 0) - if physical == 0 { - return &pdpb.Timestamp{}, false, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory isn't initialized") - } - estimatedMaxTSO := &pdpb.Timestamp{ - Physical: physical + time.Since(lastUpdateTime).Milliseconds() + 2*gta.getSyncRTT(), // TODO: make the coefficient of RTT configurable - Logical: logical, - } - // Precheck to make sure the logical part won't overflow after being differentiated. - // If precheckLogical returns false, it means the logical part is overflow, - // we need to wait a updatePhysicalInterval and retry the estimation later. - if !gta.precheckLogical(estimatedMaxTSO, suffixBits) { - return nil, true, nil - } - return estimatedMaxTSO, false, nil -} - // Initialize will initialize the created global TSO allocator. func (gta *GlobalTSOAllocator) Initialize(int) error { gta.tsoAllocatorRoleGauge.Set(1) @@ -208,321 +168,15 @@ func (gta *GlobalTSOAllocator) SetTSO(tso uint64, ignoreSmaller, skipUpperBoundC // Make sure you have initialized the TSO allocator before calling this method. // Basically, there are two ways to generate a Global TSO: // 1. The old way to generate a normal TSO from memory directly, which makes the TSO service node become single point. -// 2. The new way to generate a Global TSO by synchronizing with all other Local TSO Allocators. -// -// And for the new way, there are two different strategies: -// 1. Collect the max Local TSO from all Local TSO Allocator leaders and write it back to them as MaxTS. -// 2. Estimate a MaxTS and try to write it to all Local TSO Allocator leaders directly to reduce the RTT. -// During the process, if the estimated MaxTS is not accurate, it will fallback to the collecting way. +// 2. Deprecated: The new way to generate a Global TSO by synchronizing with all other Local TSO Allocators. func (gta *GlobalTSOAllocator) GenerateTSO(ctx context.Context, count uint32) (pdpb.Timestamp, error) { defer trace.StartRegion(ctx, "GlobalTSOAllocator.GenerateTSO").End() if !gta.member.GetLeadership().Check() { gta.getMetrics().notLeaderEvent.Inc() return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs(fmt.Sprintf("requested pd %s of cluster", errs.NotLeaderErr)) } - // To check if we have any dc-location configured in the cluster - dcLocationMap := gta.am.GetClusterDCLocations() - // No dc-locations configured in the cluster, use the normal Global TSO generation way. - // (without synchronization with other Local TSO Allocators) - if len(dcLocationMap) == 0 { - return gta.timestampOracle.getTS(ctx, gta.member.GetLeadership(), count, 0) - } - ctx1 := ctx - - // Have dc-locations configured in the cluster, use the Global TSO generation way. - // (whit synchronization with other Local TSO Allocators) - ctx, cancel := context.WithCancel(gta.ctx) - defer cancel() - for range maxRetryCount { - var ( - err error - shouldRetry, skipCheck bool - globalTSOResp pdpb.Timestamp - estimatedMaxTSO *pdpb.Timestamp - suffixBits = gta.am.GetSuffixBits() - ) - // TODO: add a switch to control whether to enable the MaxTSO estimation. - // 1. Estimate a MaxTS among all Local TSO Allocator leaders according to the RTT. - estimatedMaxTSO, shouldRetry, err = gta.estimateMaxTS(ctx1, count, suffixBits) - if err != nil { - log.Error("global tso allocator estimates MaxTS failed", - logutil.CondUint32("keyspace-group-id", gta.getGroupID(), gta.getGroupID() > 0), - errs.ZapError(err)) - continue - } - if shouldRetry { - time.Sleep(gta.timestampOracle.updatePhysicalInterval) - continue - } - SETTING_PHASE: - // 2. Send the MaxTSO to all Local TSO Allocators leaders to make sure the subsequent Local TSOs will be bigger than it. - // It's not safe to skip check at the first time here because the estimated maxTSO may not be big enough, - // we need to validate it first before we write it into every Local TSO Allocator's memory. - globalTSOResp = *estimatedMaxTSO - if err = gta.SyncMaxTS(ctx, dcLocationMap, &globalTSOResp, skipCheck); err != nil { - log.Error("global tso allocator synchronizes MaxTS failed", - logutil.CondUint32("keyspace-group-id", gta.getGroupID(), gta.getGroupID() > 0), - errs.ZapError(err)) - continue - } - // 3. If skipCheck is false and the maxTSO is bigger than estimatedMaxTSO, - // we need to redo the setting phase with the bigger one and skip the check safely. - if !skipCheck && tsoutil.CompareTimestamp(&globalTSOResp, estimatedMaxTSO) > 0 { - gta.getMetrics().globalTSOSyncEvent.Inc() - *estimatedMaxTSO = globalTSOResp - // Re-add the count and check the overflow. - estimatedMaxTSO.Logical += int64(count) - if !gta.precheckLogical(estimatedMaxTSO, suffixBits) { - estimatedMaxTSO.Physical += UpdateTimestampGuard.Milliseconds() - estimatedMaxTSO.Logical = int64(count) - } - skipCheck = true - goto SETTING_PHASE - } - // Is skipCheck is false and globalTSOResp remains the same, it means the estimatedTSO is valid. - if !skipCheck && tsoutil.CompareTimestamp(&globalTSOResp, estimatedMaxTSO) == 0 { - gta.getMetrics().globalTSOEstimateEvent.Inc() - } - // 4. Persist MaxTS into memory, and etcd if needed - var currentGlobalTSO *pdpb.Timestamp - if currentGlobalTSO, err = gta.getCurrentTSO(ctx1); err != nil { - log.Error("global tso allocator gets the current global tso in memory failed", - logutil.CondUint32("keyspace-group-id", gta.getGroupID(), gta.getGroupID() > 0), - errs.ZapError(err)) - continue - } - if tsoutil.CompareTimestamp(currentGlobalTSO, &globalTSOResp) < 0 { - gta.getMetrics().globalTSOPersistEvent.Inc() - // Update the Global TSO in memory - if err = gta.timestampOracle.resetUserTimestamp(ctx1, gta.member.GetLeadership(), tsoutil.GenerateTS(&globalTSOResp), true); err != nil { - gta.getMetrics().errGlobalTSOPersistEvent.Inc() - log.Error("global tso allocator update the global tso in memory failed", - logutil.CondUint32("keyspace-group-id", gta.getGroupID(), gta.getGroupID() > 0), - errs.ZapError(err)) - continue - } - } - // 5. Check leadership again before we returning the response. - if !gta.member.GetLeadership().Check() { - gta.getMetrics().notLeaderAnymoreEvent.Inc() - return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs(fmt.Sprintf("requested %s anymore", errs.NotLeaderErr)) - } - // 6. Calibrate the logical part to make the TSO unique globally by giving it a unique suffix in the whole cluster - globalTSOResp.Logical = gta.timestampOracle.calibrateLogical(globalTSOResp.GetLogical(), suffixBits) - globalTSOResp.SuffixBits = uint32(suffixBits) - return globalTSOResp, nil - } - gta.getMetrics().exceededMaxRetryEvent.Inc() - return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("global tso allocator maximum number of retries exceeded") -} -// Only used for test -var globalTSOOverflowFlag = true - -func (gta *GlobalTSOAllocator) precheckLogical(maxTSO *pdpb.Timestamp, suffixBits int) bool { - failpoint.Inject("globalTSOOverflow", func() { - if globalTSOOverflowFlag { - maxTSO.Logical = maxLogical - globalTSOOverflowFlag = false - } - }) - // Make sure the physical time is not empty again. - if maxTSO.GetPhysical() == 0 { - return false - } - // Check if the logical part will reach the overflow condition after being differentiated. - if caliLogical := gta.timestampOracle.calibrateLogical(maxTSO.Logical, suffixBits); caliLogical >= maxLogical { - log.Error("estimated logical part outside of max logical interval, please check ntp time", - logutil.CondUint32("keyspace-group-id", gta.getGroupID(), gta.getGroupID() > 0), - zap.Reflect("max-tso", maxTSO), errs.ZapError(errs.ErrLogicOverflow)) - gta.getMetrics().precheckLogicalOverflowEvent.Inc() - return false - } - return true -} - -const ( - dialTimeout = 3 * time.Second - rpcTimeout = 3 * time.Second - // TODO: maybe make syncMaxRetryCount configurable - syncMaxRetryCount = 2 -) - -type syncResp struct { - rpcRes *pdpb.SyncMaxTSResponse - err error - rtt time.Duration -} - -// SyncMaxTS is used to sync MaxTS with all Local TSO Allocator leaders in dcLocationMap. -// If maxTSO is the biggest TSO among all Local TSO Allocators, it will be written into -// each allocator and remains the same after the synchronization. -// If not, it will be replaced with the new max Local TSO and return. -func (gta *GlobalTSOAllocator) SyncMaxTS( - ctx context.Context, - dcLocationMap map[string]DCLocationInfo, - maxTSO *pdpb.Timestamp, - skipCheck bool, -) error { - defer trace.StartRegion(ctx, "GlobalTSOAllocator.SyncMaxTS").End() - originalMaxTSO := *maxTSO - for i := range syncMaxRetryCount { - // Collect all allocator leaders' client URLs - allocatorLeaders := make(map[string]*pdpb.Member) - for dcLocation := range dcLocationMap { - allocator, err := gta.am.GetAllocator(dcLocation) - if err != nil { - return err - } - allocatorLeader := allocator.(*LocalTSOAllocator).GetAllocatorLeader() - if allocatorLeader.GetMemberId() == 0 { - return errs.ErrSyncMaxTS.FastGenByArgs(fmt.Sprintf("%s does not have the local allocator leader yet", dcLocation)) - } - allocatorLeaders[dcLocation] = allocatorLeader - } - leaderURLs := make([]string, 0) - for _, allocator := range allocatorLeaders { - // Check if its client URLs are empty - if len(allocator.GetClientUrls()) < 1 { - continue - } - leaderURL := allocator.GetClientUrls()[0] - if slice.NoneOf(leaderURLs, func(i int) bool { return leaderURLs[i] == leaderURL }) { - leaderURLs = append(leaderURLs, leaderURL) - } - } - // Prepare to make RPC requests concurrently - respCh := make(chan *syncResp, len(leaderURLs)) - wg := sync.WaitGroup{} - request := &pdpb.SyncMaxTSRequest{ - Header: &pdpb.RequestHeader{ - SenderId: gta.am.member.ID(), - }, - SkipCheck: skipCheck, - MaxTs: maxTSO, - } - for _, leaderURL := range leaderURLs { - leaderConn, err := gta.am.getOrCreateGRPCConn(ctx, leaderURL) - if err != nil { - return err - } - // Send SyncMaxTSRequest to all allocator leaders concurrently. - wg.Add(1) - go func(ctx context.Context, conn *grpc.ClientConn, respCh chan<- *syncResp) { - defer logutil.LogPanic() - defer wg.Done() - syncMaxTSResp := &syncResp{} - syncCtx, cancel := context.WithTimeout(ctx, rpcTimeout) - startTime := time.Now() - syncMaxTSResp.rpcRes, syncMaxTSResp.err = pdpb.NewPDClient(conn).SyncMaxTS(syncCtx, request) - // Including RPC request -> RPC processing -> RPC response - syncMaxTSResp.rtt = time.Since(startTime) - cancel() - respCh <- syncMaxTSResp - if syncMaxTSResp.err != nil { - log.Error("sync max ts rpc failed, got an error", - logutil.CondUint32("keyspace-group-id", gta.getGroupID(), gta.getGroupID() > 0), - zap.String("local-allocator-leader-url", leaderConn.Target()), - errs.ZapError(err)) - return - } - if syncMaxTSResp.rpcRes.GetHeader().GetError() != nil { - log.Error("sync max ts rpc failed, got an error", - logutil.CondUint32("keyspace-group-id", gta.getGroupID(), gta.getGroupID() > 0), - zap.String("local-allocator-leader-url", leaderConn.Target()), - errs.ZapError(errors.New(syncMaxTSResp.rpcRes.GetHeader().GetError().String()))) - return - } - }(ctx, leaderConn, respCh) - } - wg.Wait() - close(respCh) - var ( - errList []error - syncedDCs []string - maxTSORtt time.Duration - ) - // Iterate each response to handle the error and compare MaxTSO. - for resp := range respCh { - if resp.err != nil { - errList = append(errList, resp.err) - } - // If any error occurs, just jump out of the loop. - if len(errList) != 0 { - break - } - if resp.rpcRes == nil { - return errs.ErrSyncMaxTS.FastGenByArgs("got nil response") - } - if skipCheck { - // Set all the Local TSOs to the maxTSO unconditionally, so the MaxLocalTS in response should be nil. - if resp.rpcRes.GetMaxLocalTs() != nil { - return errs.ErrSyncMaxTS.FastGenByArgs("got non-nil max local ts in the second sync phase") - } - syncedDCs = append(syncedDCs, resp.rpcRes.GetSyncedDcs()...) - } else { - // Compare and get the max one - if tsoutil.CompareTimestamp(resp.rpcRes.GetMaxLocalTs(), maxTSO) > 0 { - *maxTSO = *(resp.rpcRes.GetMaxLocalTs()) - if resp.rtt > maxTSORtt { - maxTSORtt = resp.rtt - } - } - syncedDCs = append(syncedDCs, resp.rpcRes.GetSyncedDcs()...) - } - } - // We need to collect all info needed to ensure the consistency of TSO. - // So if any error occurs, the synchronization process will fail directly. - if len(errList) != 0 { - return errs.ErrSyncMaxTS.FastGenWithCause(errList) - } - // Check whether all dc-locations have been considered during the synchronization and retry once if any dc-location missed. - if ok, unsyncedDCs := gta.checkSyncedDCs(dcLocationMap, syncedDCs); !ok { - log.Info("unsynced dc-locations found, will retry", - logutil.CondUint32("keyspace-group-id", gta.getGroupID(), gta.getGroupID() > 0), - zap.Bool("skip-check", skipCheck), - zap.Strings("synced-DCs", syncedDCs), - zap.Strings("unsynced-DCs", unsyncedDCs)) - if i < syncMaxRetryCount-1 { - // maxTSO should remain the same. - *maxTSO = originalMaxTSO - // To make sure we have the latest dc-location info - gta.am.ClusterDCLocationChecker() - continue - } - return errs.ErrSyncMaxTS.FastGenByArgs( - fmt.Sprintf("unsynced dc-locations found, skip-check: %t, synced dc-locations: %+v, unsynced dc-locations: %+v", - skipCheck, syncedDCs, unsyncedDCs)) - } - // Update the sync RTT to help estimate MaxTS later. - if maxTSORtt != 0 { - gta.setSyncRTT(maxTSORtt.Milliseconds()) - } - } - return nil -} - -func (gta *GlobalTSOAllocator) checkSyncedDCs(dcLocationMap map[string]DCLocationInfo, syncedDCs []string) (bool, []string) { - var unsyncedDCs []string - for dcLocation := range dcLocationMap { - if slice.NoneOf(syncedDCs, func(i int) bool { return syncedDCs[i] == dcLocation }) { - unsyncedDCs = append(unsyncedDCs, dcLocation) - } - } - log.Debug("check unsynced dc-locations", - logutil.CondUint32("keyspace-group-id", gta.getGroupID(), gta.getGroupID() > 0), - zap.Strings("unsynced-DCs", unsyncedDCs), - zap.Strings("synced-DCs", syncedDCs)) - return len(unsyncedDCs) == 0, unsyncedDCs -} - -func (gta *GlobalTSOAllocator) getCurrentTSO(ctx context.Context) (*pdpb.Timestamp, error) { - defer trace.StartRegion(ctx, "GlobalTSOAllocator.getCurrentTSO").End() - currentPhysical, currentLogical := gta.timestampOracle.getTSO() - if currentPhysical == typeutil.ZeroTime { - return &pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory isn't initialized") - } - return tsoutil.GenerateTimestamp(currentPhysical, uint64(currentLogical)), nil + return gta.timestampOracle.getTS(ctx, gta.member.GetLeadership(), count, 0) } // Reset is used to reset the TSO allocator. diff --git a/server/api/member_test.go b/server/api/member_test.go index ad4812249c7..b56e84b2a1a 100644 --- a/server/api/member_test.go +++ b/server/api/member_test.go @@ -45,12 +45,7 @@ func TestMemberTestSuite(t *testing.T) { } func (suite *memberTestSuite) SetupSuite() { - suite.cfgs, suite.servers, suite.clean = mustNewCluster(suite.Require(), 3, func(cfg *config.Config) { - cfg.EnableLocalTSO = true - cfg.Labels = map[string]string{ - config.ZoneLabel: "dc-1", - } - }) + suite.cfgs, suite.servers, suite.clean = mustNewCluster(suite.Require(), 3) } func (suite *memberTestSuite) TearDownSuite() { @@ -76,7 +71,6 @@ func checkListResponse(re *require.Assertions, body []byte, cfgs []*config.Confi if member.GetName() != cfg.Name { continue } - re.Equal("dc-1", member.DcLocation) relaxEqualStings(re, member.ClientUrls, strings.Split(cfg.ClientUrls, ",")) relaxEqualStings(re, member.PeerUrls, strings.Split(cfg.PeerUrls, ",")) } diff --git a/server/config/config.go b/server/config/config.go index 1ccafe7248a..ed37d899390 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -256,12 +256,6 @@ const ( defaultEnableTSODynamicSwitching = false ) -// Special keys for Labels -const ( - // ZoneLabel is the name of the key which indicates DC location of this PD server. - ZoneLabel = "zone" -) - var ( defaultEnableTelemetry = false defaultRuntimeServices = []string{} diff --git a/server/grpc_service.go b/server/grpc_service.go index d5fd8ae3e32..3aa07e841ff 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -2756,54 +2756,11 @@ func scatterRegions(cluster *cluster.RaftCluster, regionsID []uint64, group stri return percentage, nil } -// GetDCLocationInfo gets the dc-location info of the given dc-location from PD leader's TSO allocator manager. -func (s *GrpcServer) GetDCLocationInfo(ctx context.Context, request *pdpb.GetDCLocationInfoRequest) (*pdpb.GetDCLocationInfoResponse, error) { - // TODO: support local tso forward in api service mode in the future. - var err error - if err = s.validateInternalRequest(request.GetHeader(), false); err != nil { - return nil, err - } - if !s.member.IsLeader() { - return nil, ErrNotLeader - } - if s.GetServiceMiddlewarePersistOptions().IsGRPCRateLimitEnabled() { - fName := currentFunction() - limiter := s.GetGRPCRateLimiter() - if done, err := limiter.Allow(fName); err == nil { - defer done() - } else { - return &pdpb.GetDCLocationInfoResponse{ - Header: wrapErrorToHeader(pdpb.ErrorType_UNKNOWN, err.Error()), - }, nil - } - } - am := s.GetTSOAllocatorManager() - info, ok := am.GetDCLocationInfo(request.GetDcLocation()) - if !ok { - am.ClusterDCLocationChecker() - return &pdpb.GetDCLocationInfoResponse{ - Header: wrapErrorToHeader(pdpb.ErrorType_UNKNOWN, - fmt.Sprintf("dc-location %s is not found", request.GetDcLocation())), - }, nil - } - resp := &pdpb.GetDCLocationInfoResponse{ +// Deprecated +func (*GrpcServer) GetDCLocationInfo(_ context.Context, _ *pdpb.GetDCLocationInfoRequest) (*pdpb.GetDCLocationInfoResponse, error) { + return &pdpb.GetDCLocationInfoResponse{ Header: wrapHeader(), - Suffix: info.Suffix, - } - // Because the number of suffix bits is changing dynamically according to the dc-location number, - // there is a corner case may cause the Local TSO is not unique while member changing. - // Example: - // t1: xxxxxxxxxxxxxxx1 | 11 - // t2: xxxxxxxxxxxxxxx | 111 - // So we will force the newly added Local TSO Allocator to have a Global TSO synchronization - // when it becomes the Local TSO Allocator leader. - // Please take a look at https://github.com/tikv/pd/issues/3260 for more details. - if resp.MaxTs, err = am.GetMaxLocalTSO(ctx); err != nil { - return &pdpb.GetDCLocationInfoResponse{ - Header: wrapErrorToHeader(pdpb.ErrorType_UNKNOWN, err.Error()), - }, nil - } - return resp, nil + }, nil } // validateInternalRequest checks if server is closed, which is used to validate diff --git a/server/server.go b/server/server.go index ed61b14834f..e2c9b31580d 100644 --- a/server/server.go +++ b/server/server.go @@ -466,18 +466,6 @@ func (s *Server) startServer(ctx context.Context) error { s.tsoProtoFactory = &tsoutil.TSOProtoFactory{} s.pdProtoFactory = &tsoutil.PDProtoFactory{} s.tsoAllocatorManager = tso.NewAllocatorManager(s.ctx, constant.DefaultKeyspaceGroupID, s.member, s.rootPath, s.storage, s) - // When disabled the Local TSO, we should clean up the Local TSO Allocator's meta info written in etcd if it exists. - if !s.cfg.EnableLocalTSO { - if err = s.tsoAllocatorManager.CleanUpDCLocation(); err != nil { - return err - } - } - if zone, exist := s.cfg.Labels[config.ZoneLabel]; exist && zone != "" && s.cfg.EnableLocalTSO { - if err = s.tsoAllocatorManager.SetLocalTSOConfig(zone); err != nil { - return err - } - } - s.gcSafePointManager = gc.NewSafePointManager(s.storage, s.cfg.PDServerCfg) s.basicCluster = core.NewBasicCluster() s.cluster = cluster.NewRaftCluster(ctx, s.GetMember(), s.GetBasicCluster(), s.GetStorage(), syncer.NewRegionSyncer(s), s.client, s.httpClient, s.tsoAllocatorManager) @@ -1630,10 +1618,6 @@ func (s *Server) leaderLoop() { log.Error("reload config failed", errs.ZapError(err)) continue } - if !s.IsAPIServiceMode() { - // Check the cluster dc-location after the PD leader is elected - go s.tsoAllocatorManager.ClusterDCLocationChecker() - } syncer := s.cluster.GetRegionSyncer() if s.persistOptions.IsUseRegionStorage() { syncer.StartSyncWithLeader(leader.GetListenUrls()[0]) @@ -1745,10 +1729,6 @@ func (s *Server) campaignLeader() { // EnableLeader to accept the remaining service, such as GetStore, GetRegion. s.member.EnableLeader() member.ServiceMemberGauge.WithLabelValues(s.mode).Set(1) - if !s.IsAPIServiceMode() { - // Check the cluster dc-location after the PD leader is elected. - go s.tsoAllocatorManager.ClusterDCLocationChecker() - } defer resetLeaderOnce.Do(func() { // as soon as cancel the leadership keepalive, then other member have chance // to be new leader. diff --git a/tests/cluster.go b/tests/cluster.go index 6ad15e3291f..7996e289cf5 100644 --- a/tests/cluster.go +++ b/tests/cluster.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/autoscaling" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/dashboard" @@ -41,7 +40,6 @@ import ( "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/api" "github.com/tikv/pd/server/apiv2" @@ -719,56 +717,6 @@ func (c *TestCluster) ResignLeader() error { return errors.New("no leader") } -// WaitAllocatorLeader is used to get the Local TSO Allocator leader. -// If it exceeds the maximum number of loops, it will return an empty string. -func (c *TestCluster) WaitAllocatorLeader(dcLocation string, ops ...WaitOption) string { - option := &WaitOp{ - retryTimes: WaitLeaderRetryTimes, - waitInterval: WaitLeaderCheckInterval, - } - for _, op := range ops { - op(option) - } - for range option.retryTimes { - counter := make(map[string]int) - running := 0 - for _, s := range c.servers { - if s.state == Running && s.GetTSOAllocatorManager().EnableLocalTSO() { - running++ - } - serverName := s.GetAllocatorLeader(dcLocation).GetName() - if serverName != "" { - counter[serverName]++ - } - } - for serverName, num := range counter { - if num == running && c.GetServer(serverName).IsAllocatorLeader(dcLocation) { - return serverName - } - } - time.Sleep(option.waitInterval) - } - return "" -} - -// WaitAllLeaders will block and wait for the election of PD leader and all Local TSO Allocator leaders. -func (c *TestCluster) WaitAllLeaders(re *require.Assertions, dcLocations map[string]string) { - c.WaitLeader() - c.CheckClusterDCLocation() - // Wait for each DC's Local TSO Allocator leader - wg := sync.WaitGroup{} - for _, dcLocation := range dcLocations { - wg.Add(1) - go func(dc string) { - testutil.Eventually(re, func() bool { - return c.WaitAllocatorLeader(dc) != "" - }) - wg.Done() - }(dcLocation) - } - wg.Wait() -} - // GetCluster returns PD cluster. func (c *TestCluster) GetCluster() *metapb.Cluster { leader := c.GetLeader() @@ -853,19 +801,6 @@ func (c *TestCluster) Destroy() { } } -// CheckClusterDCLocation will force the cluster to do the dc-location check in order to speed up the test. -func (c *TestCluster) CheckClusterDCLocation() { - wg := sync.WaitGroup{} - for _, server := range c.GetServers() { - wg.Add(1) - go func(s *TestServer) { - s.GetTSOAllocatorManager().ClusterDCLocationChecker() - wg.Done() - }(server) - } - wg.Wait() -} - // CheckTSOUnique will check whether the TSO is unique among the cluster in the past and present. func (c *TestCluster) CheckTSOUnique(ts uint64) bool { c.tsPool.Lock() From 52b42b25ba8f8f75943200b76d48565b8aace2b8 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Mon, 25 Nov 2024 15:42:23 +0800 Subject: [PATCH 7/7] client: fix trace (#8847) close tikv/pd#8534 Signed-off-by: Ryan Leung Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/tso_dispatcher.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/client/tso_dispatcher.go b/client/tso_dispatcher.go index 1123e59dbdd..fd9a17405e6 100644 --- a/client/tso_dispatcher.go +++ b/client/tso_dispatcher.go @@ -546,10 +546,10 @@ func (td *tsoDispatcher) processRequests( } } defer func() { - for i := range spans { + for i := len(spans) - 1; i >= 0; i-- { spans[i].Finish() } - for i := range traceRegions { + for i := len(traceRegions) - 1; i >= 0; i-- { traceRegions[i].End() } }()