diff --git a/.github/workflows/tso-consistency-test.yaml b/.github/workflows/tso-consistency-test.yaml deleted file mode 100644 index 5fb0fd70552..00000000000 --- a/.github/workflows/tso-consistency-test.yaml +++ /dev/null @@ -1,17 +0,0 @@ -name: TSO Consistency Test -on: - # Only run when the new code is merged into master. - push: - branches: - - master -jobs: - tso-consistency-test: - runs-on: ubuntu-latest - steps: - - name: Checkout code - uses: actions/checkout@v4 - - uses: actions/setup-go@v5 - with: - go-version: '1.23' - - name: Make TSO Consistency Test - run: make test-tso-consistency diff --git a/Makefile b/Makefile index 51a1b04b71c..f7f1cba1102 100644 --- a/Makefile +++ b/Makefile @@ -243,7 +243,7 @@ SUBMODULES := $(filter $(shell find . -iname "go.mod" -exec dirname {} \;),\ test: install-tools # testing all pkgs... @$(FAILPOINT_ENABLE) - CGO_ENABLED=1 go test -tags tso_function_test,deadlock -timeout 20m -race -cover $(TEST_PKGS) || { $(FAILPOINT_DISABLE); exit 1; } + CGO_ENABLED=1 go test -tags deadlock -timeout 20m -race -cover $(TEST_PKGS) || { $(FAILPOINT_DISABLE); exit 1; } @$(FAILPOINT_DISABLE) basic-test: install-tools @@ -257,24 +257,12 @@ ci-test-job: install-tools dashboard-ui pd-ut ./scripts/ci-subtask.sh $(JOB_INDEX) || { $(FAILPOINT_DISABLE); exit 1; } @$(FAILPOINT_DISABLE) -TSO_INTEGRATION_TEST_PKGS := $(PD_PKG)/tests/server/tso - -test-tso: install-tools - # testing TSO function & consistency... - @$(FAILPOINT_ENABLE) - CGO_ENABLED=1 go test -race -tags without_dashboard,tso_full_test,deadlock $(TSO_INTEGRATION_TEST_PKGS) || { $(FAILPOINT_DISABLE); exit 1; } - @$(FAILPOINT_DISABLE) +TSO_FUNCTION_TEST_PKGS := $(PD_PKG)/tests/server/tso test-tso-function: install-tools # testing TSO function... @$(FAILPOINT_ENABLE) - CGO_ENABLED=1 go test -race -tags without_dashboard,tso_function_test,deadlock $(TSO_INTEGRATION_TEST_PKGS) || { $(FAILPOINT_DISABLE); exit 1; } - @$(FAILPOINT_DISABLE) - -test-tso-consistency: install-tools - # testing TSO consistency... - @$(FAILPOINT_ENABLE) - CGO_ENABLED=1 go test -race -tags without_dashboard,tso_consistency_test,deadlock $(TSO_INTEGRATION_TEST_PKGS) || { $(FAILPOINT_DISABLE); exit 1; } + CGO_ENABLED=1 go test -race -tags without_dashboard,deadlock $(TSO_FUNCTION_TEST_PKGS) || { $(FAILPOINT_DISABLE); exit 1; } @$(FAILPOINT_DISABLE) REAL_CLUSTER_TEST_PATH := $(ROOT_PATH)/tests/integrations/realcluster @@ -302,7 +290,7 @@ test-with-cover-parallel: install-tools dashboard-ui split split: # todo: it will remove server/api,/tests and tso packages after daily CI integrate all verify CI. - go list ./... | grep -v -E "github.com/tikv/pd/server/api|github.com/tikv/pd/tests/client|github.com/tikv/pd/tests/server/tso" > packages.list;\ + go list ./... | grep -v -E "github.com/tikv/pd/server/api|github.com/tikv/pd/tests/client|$(TSO_FUNCTION_TEST_PKGS)" > packages.list;\ split packages.list -n r/${TASK_COUNT} packages_unit_ -a 1 --numeric-suffixes=1;\ cat packages_unit_${TASK_ID} |tr "\n" " " >package.list;\ rm packages*; diff --git a/client/client.go b/client/client.go index 4ddfd7ffaf3..330e1bd4974 100644 --- a/client/client.go +++ b/client/client.go @@ -34,7 +34,6 @@ import ( "github.com/tikv/pd/client/caller" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/utils/tlsutil" - "github.com/tikv/pd/client/utils/tsoutil" "go.uber.org/zap" ) @@ -590,18 +589,20 @@ func (c *client) getClientAndContext(ctx context.Context) (pdpb.PDClient, contex // GetTSAsync implements the TSOClient interface. func (c *client) GetTSAsync(ctx context.Context) TSFuture { - return c.GetLocalTSAsync(ctx, globalDCLocation) -} - -// GetLocalTSAsync implements the TSOClient interface. -func (c *client) GetLocalTSAsync(ctx context.Context, dcLocation string) TSFuture { - defer trace.StartRegion(ctx, "pdclient.GetLocalTSAsync").End() + defer trace.StartRegion(ctx, "pdclient.GetTSAsync").End() if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span = span.Tracer().StartSpan("pdclient.GetLocalTSAsync", opentracing.ChildOf(span.Context())) + span = span.Tracer().StartSpan("pdclient.GetTSAsync", opentracing.ChildOf(span.Context())) defer span.Finish() } + return c.inner.dispatchTSORequestWithRetry(ctx) +} - return c.inner.dispatchTSORequestWithRetry(ctx, dcLocation) +// GetLocalTSAsync implements the TSOClient interface. +// +// Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the +// parameters passed in, this method will default to returning the global TSO. +func (c *client) GetLocalTSAsync(ctx context.Context, _ string) TSFuture { + return c.GetTSAsync(ctx) } // GetTS implements the TSOClient interface. @@ -611,9 +612,11 @@ func (c *client) GetTS(ctx context.Context) (physical int64, logical int64, err } // GetLocalTS implements the TSOClient interface. -func (c *client) GetLocalTS(ctx context.Context, dcLocation string) (physical int64, logical int64, err error) { - resp := c.GetLocalTSAsync(ctx, dcLocation) - return resp.Wait() +// +// Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the +// parameters passed in, this method will default to returning the global TSO. +func (c *client) GetLocalTS(ctx context.Context, _ string) (physical int64, logical int64, err error) { + return c.GetTS(ctx) } // GetMinTS implements the TSOClient interface. @@ -659,7 +662,7 @@ func (c *client) GetMinTS(ctx context.Context) (physical int64, logical int64, e } minTS := resp.GetTimestamp() - return minTS.Physical, tsoutil.AddLogical(minTS.Logical, 0, minTS.SuffixBits), nil + return minTS.Physical, minTS.Logical, nil } func handleRegionResponse(res *pdpb.GetRegionResponse) *Region { @@ -1444,16 +1447,6 @@ func (c *client) respForErr(observer prometheus.Observer, start time.Time, err e return nil } -// GetTSOAllocators returns {dc-location -> TSO allocator leader URL} connection map -// For test only. -func (c *client) GetTSOAllocators() *sync.Map { - tsoClient := c.inner.getTSOClient() - if tsoClient == nil { - return nil - } - return tsoClient.GetTSOAllocators() -} - // WithCallerID implements the RPCClient interface. func (c *client) WithCallerID(callerID caller.ID) RPCClient { newClient := *c diff --git a/client/inner_client.go b/client/inner_client.go index d841c8b5286..1bbbdd0eb88 100644 --- a/client/inner_client.go +++ b/client/inner_client.go @@ -207,7 +207,7 @@ func (c *innerClient) getOrCreateGRPCConn() (*grpc.ClientConn, error) { return cc, err } -func (c *innerClient) dispatchTSORequestWithRetry(ctx context.Context, dcLocation string) TSFuture { +func (c *innerClient) dispatchTSORequestWithRetry(ctx context.Context) TSFuture { var ( retryable bool err error @@ -226,7 +226,7 @@ func (c *innerClient) dispatchTSORequestWithRetry(ctx context.Context, dcLocatio } // Get a new request from the pool if it's nil or not from the current pool. if req == nil || req.pool != tsoClient.tsoReqPool { - req = tsoClient.getTSORequest(ctx, dcLocation) + req = tsoClient.getTSORequest(ctx) } retryable, err = tsoClient.dispatchRequest(req) if !retryable { diff --git a/client/pd_service_discovery.go b/client/pd_service_discovery.go index 94c08f5c10d..c961e9e42fd 100644 --- a/client/pd_service_discovery.go +++ b/client/pd_service_discovery.go @@ -17,6 +17,7 @@ package pd import ( "context" "crypto/tls" + "fmt" "net/url" "reflect" "sort" @@ -40,7 +41,6 @@ import ( ) const ( - globalDCLocation = "global" memberUpdateInterval = time.Minute serviceModeUpdateInterval = 3 * time.Second updateMemberTimeout = time.Second // Use a shorter timeout to recover faster from network isolation. @@ -383,21 +383,17 @@ func (c *pdServiceBalancer) get() (ret ServiceClient) { } type updateKeyspaceIDFunc func() error -type tsoLocalServURLsUpdatedFunc func(map[string]string) error -type tsoGlobalServURLUpdatedFunc func(string) error +type tsoLeaderURLUpdatedFunc func(string) error -type tsoAllocatorEventSource interface { - // SetTSOLocalServURLsUpdatedCallback adds a callback which will be called when the local tso - // allocator leader list is updated. - SetTSOLocalServURLsUpdatedCallback(callback tsoLocalServURLsUpdatedFunc) - // SetTSOGlobalServURLUpdatedCallback adds a callback which will be called when the global tso - // allocator leader is updated. - SetTSOGlobalServURLUpdatedCallback(callback tsoGlobalServURLUpdatedFunc) +// 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) - _ tsoAllocatorEventSource = (*pdServiceDiscovery)(nil) + _ ServiceDiscovery = (*pdServiceDiscovery)(nil) + _ tsoEventSource = (*pdServiceDiscovery)(nil) ) // pdServiceDiscovery is the service discovery client of PD/API service which is quorum based @@ -426,12 +422,8 @@ type pdServiceDiscovery struct { // membersChangedCbs will be called after there is any membership change in the // leader and followers membersChangedCbs []func() - // tsoLocalAllocLeadersUpdatedCb will be called when the local tso allocator - // leader list is updated. The input is a map {DC Location -> Leader URL} - tsoLocalAllocLeadersUpdatedCb tsoLocalServURLsUpdatedFunc - // tsoGlobalAllocLeaderUpdatedCb will be called when the global tso allocator - // leader is updated. - tsoGlobalAllocLeaderUpdatedCb tsoGlobalServURLUpdatedFunc + // tsoLeaderUpdatedCb will be called when the TSO leader is updated. + tsoLeaderUpdatedCb tsoLeaderURLUpdatedFunc checkMembershipCh chan struct{} @@ -801,22 +793,15 @@ func (c *pdServiceDiscovery) AddServiceURLsSwitchedCallback(callbacks ...func()) c.membersChangedCbs = append(c.membersChangedCbs, callbacks...) } -// SetTSOLocalServURLsUpdatedCallback adds a callback which will be called when the local tso -// allocator leader list is updated. -func (c *pdServiceDiscovery) SetTSOLocalServURLsUpdatedCallback(callback tsoLocalServURLsUpdatedFunc) { - c.tsoLocalAllocLeadersUpdatedCb = callback -} - -// SetTSOGlobalServURLUpdatedCallback adds a callback which will be called when the global tso -// allocator leader is updated. -func (c *pdServiceDiscovery) SetTSOGlobalServURLUpdatedCallback(callback tsoGlobalServURLUpdatedFunc) { +// SetTSOLeaderURLUpdatedCallback adds a callback which will be called when the TSO leader is updated. +func (c *pdServiceDiscovery) SetTSOLeaderURLUpdatedCallback(callback tsoLeaderURLUpdatedFunc) { url := c.getLeaderURL() if len(url) > 0 { if err := callback(url); err != nil { - log.Error("[tso] failed to call back when tso global service url update", zap.String("url", url), errs.ZapError(err)) + log.Error("[tso] failed to call back when tso leader url update", zap.String("url", url), errs.ZapError(err)) } } - c.tsoGlobalAllocLeaderUpdatedCb = callback + c.tsoLeaderUpdatedCb = callback } // getLeaderURL returns the leader URL. @@ -901,19 +886,16 @@ func (c *pdServiceDiscovery) updateMember() error { members, err := c.getMembers(c.ctx, url, updateMemberTimeout) // Check the cluster ID. - if err == nil && members.GetHeader().GetClusterId() != c.clusterID { - err = errs.ErrClientUpdateMember.FastGenByArgs("cluster id does not match") + updatedClusterID := members.GetHeader().GetClusterId() + if err == nil && updatedClusterID != c.clusterID { + log.Warn("[pd] cluster id does not match", + zap.Uint64("updated-cluster-id", updatedClusterID), + zap.Uint64("expected-cluster-id", c.clusterID)) + err = errs.ErrClientUpdateMember.FastGenByArgs(fmt.Sprintf("cluster id does not match: %d != %d", updatedClusterID, c.clusterID)) } - // Check the TSO Allocator Leader. - var errTSO error - if err == nil { - if members.GetLeader() == nil || len(members.GetLeader().GetClientUrls()) == 0 { - err = errs.ErrClientGetLeader.FastGenByArgs("leader url doesn't exist") - } - // Still need to update TsoAllocatorLeaders, even if there is no PD leader - errTSO = c.switchTSOAllocatorLeaders(members.GetTsoAllocatorLeaders()) + if err == nil && (members.GetLeader() == nil || len(members.GetLeader().GetClientUrls()) == 0) { + err = errs.ErrClientGetLeader.FastGenByArgs("leader url doesn't exist") } - // Failed to get members if err != nil { log.Info("[pd] cannot update member from this url", @@ -926,15 +908,9 @@ func (c *pdServiceDiscovery) updateMember() error { continue } } - c.updateURLs(members.GetMembers()) - if err := c.updateServiceClient(members.GetMembers(), members.GetLeader()); err != nil { - return err - } - // If `switchLeader` succeeds but `switchTSOAllocatorLeader` has an error, - // the error of `switchTSOAllocatorLeader` will be returned. - return errTSO + return c.updateServiceClient(members.GetMembers(), members.GetLeader()) } return errs.ErrClientGetMember.FastGenByArgs() } @@ -1009,13 +985,12 @@ func (c *pdServiceDiscovery) switchLeader(url string) (bool, error) { newConn, err := c.GetOrCreateGRPCConn(url) // If gRPC connect is created successfully or leader is new, still saves. if url != oldLeader.GetURL() || newConn != nil { - // Set PD leader and Global TSO Allocator (which is also the PD leader) leaderClient := newPDServiceClient(url, url, newConn, true) c.leader.Store(leaderClient) } // Run callbacks - if c.tsoGlobalAllocLeaderUpdatedCb != nil { - if err := c.tsoGlobalAllocLeaderUpdatedCb(url); err != nil { + if c.tsoLeaderUpdatedCb != nil { + if err := c.tsoLeaderUpdatedCb(url); err != nil { return true, err } } @@ -1102,30 +1077,6 @@ func (c *pdServiceDiscovery) updateServiceClient(members []*pdpb.Member, leader return err } -func (c *pdServiceDiscovery) switchTSOAllocatorLeaders(allocatorMap map[string]*pdpb.Member) error { - if len(allocatorMap) == 0 { - return nil - } - - allocMap := make(map[string]string) - // Switch to the new one - for dcLocation, member := range allocatorMap { - if len(member.GetClientUrls()) == 0 { - continue - } - allocMap[dcLocation] = member.GetClientUrls()[0] - } - - // Run the callback to reflect any possible change in the local tso allocators. - if c.tsoLocalAllocLeadersUpdatedCb != nil { - if err := c.tsoLocalAllocLeadersUpdatedCb(allocMap); err != nil { - return err - } - } - - return nil -} - // GetOrCreateGRPCConn returns the corresponding grpc client connection of the given URL. func (c *pdServiceDiscovery) GetOrCreateGRPCConn(url string) (*grpc.ClientConn, error) { return grpcutil.GetOrCreateGRPCConn(c.ctx, &c.clientConns, url, c.tlsCfg, c.option.gRPCDialOptions...) diff --git a/client/tso_client.go b/client/tso_client.go index d52a221c542..584c5df6134 100644 --- a/client/tso_client.go +++ b/client/tso_client.go @@ -16,10 +16,10 @@ package pd import ( "context" - "fmt" "math/rand" "runtime/trace" "sync" + "sync/atomic" "time" "github.com/pingcap/errors" @@ -35,7 +35,6 @@ import ( ) const ( - tsoDispatcherCheckInterval = time.Minute // defaultMaxTSOBatchSize is the default max size of the TSO request batch. defaultMaxTSOBatchSize = 10000 // retryInterval and maxRetryTimes are used to control the retry interval and max retry times. @@ -49,13 +48,20 @@ type TSOClient interface { GetTS(ctx context.Context) (int64, int64, error) // GetTSAsync gets a timestamp from PD or TSO microservice, without block the caller. GetTSAsync(ctx context.Context) TSFuture - // GetLocalTS gets a local timestamp from PD or TSO microservice. - GetLocalTS(ctx context.Context, dcLocation string) (int64, int64, error) - // GetLocalTSAsync gets a local timestamp from PD or TSO microservice, without block the caller. - GetLocalTSAsync(ctx context.Context, dcLocation string) TSFuture // GetMinTS gets a timestamp from PD or the minimal timestamp across all keyspace groups from // the TSO microservice. GetMinTS(ctx context.Context) (int64, int64, error) + + // GetLocalTS gets a local timestamp from PD or TSO microservice. + // + // Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the + // parameters passed in, this method will default to returning the global TSO. + GetLocalTS(ctx context.Context, _ string) (int64, int64, error) + // GetLocalTSAsync gets a local timestamp from PD or TSO microservice, without block the caller. + // + // Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the + // parameters passed in, this method will default to returning the global TSO. + GetLocalTSAsync(ctx context.Context, _ string) TSFuture } type tsoClient struct { @@ -66,19 +72,13 @@ type tsoClient struct { svcDiscovery ServiceDiscovery tsoStreamBuilderFactory - // tsoAllocators defines the mapping {dc-location -> TSO allocator leader URL} - tsoAllocators sync.Map // Store as map[string]string - // tsoAllocServingURLSwitchedCallback will be called when any global/local - // tso allocator leader is switched. - tsoAllocServingURLSwitchedCallback []func() + // leaderURL is the URL of the TSO leader. + leaderURL atomic.Value // tsoReqPool is the pool to recycle `*tsoRequest`. tsoReqPool *sync.Pool - // tsoDispatcher is used to dispatch different TSO requests to - // the corresponding dc-location TSO channel. - tsoDispatcher sync.Map // Same as map[string]*tsoDispatcher - - checkTSODispatcherCh chan struct{} + // dispatcher is used to dispatch the TSO requests to the channel. + dispatcher atomic.Pointer[tsoDispatcher] } // newTSOClient returns a new TSO client. @@ -102,13 +102,11 @@ func newTSOClient( } }, }, - checkTSODispatcherCh: make(chan struct{}, 1), } - eventSrc := svcDiscovery.(tsoAllocatorEventSource) - eventSrc.SetTSOLocalServURLsUpdatedCallback(c.updateTSOLocalServURLs) - eventSrc.SetTSOGlobalServURLUpdatedCallback(c.updateTSOGlobalServURL) - c.svcDiscovery.AddServiceURLsSwitchedCallback(c.scheduleUpdateAllTSOConnectionCtxs) + eventSrc := svcDiscovery.(tsoEventSource) + eventSrc.SetTSOLeaderURLUpdatedCallback(c.updateTSOLeaderURL) + c.svcDiscovery.AddServiceURLsSwitchedCallback(c.scheduleUpdateTSOConnectionCtxs) return c } @@ -117,36 +115,15 @@ func (c *tsoClient) getOption() *option { return c.option } func (c *tsoClient) getServiceDiscovery() ServiceDiscovery { return c.svcDiscovery } +func (c *tsoClient) getDispatcher() *tsoDispatcher { + return c.dispatcher.Load() +} + func (c *tsoClient) setup() { if err := c.svcDiscovery.CheckMemberChanged(); err != nil { log.Warn("[tso] failed to check member changed", errs.ZapError(err)) } - c.updateTSODispatcher() - - // Start the daemons. - c.wg.Add(1) - go c.tsoDispatcherCheckLoop() -} - -func (c *tsoClient) tsoDispatcherCheckLoop() { - log.Info("[tso] start tso dispatcher check loop") - defer log.Info("[tso] exit tso dispatcher check loop") - defer c.wg.Done() - - loopCtx, loopCancel := context.WithCancel(c.ctx) - defer loopCancel() - - ticker := time.NewTicker(tsoDispatcherCheckInterval) - defer ticker.Stop() - for { - c.updateTSODispatcher() - select { - case <-ticker.C: - case <-c.checkTSODispatcherCh: - case <-loopCtx.Done(): - return - } - } + c.tryCreateTSODispatcher() } // close closes the TSO client @@ -160,40 +137,16 @@ func (c *tsoClient) close() { c.wg.Wait() log.Info("[tso] close tso client") - c.closeTSODispatcher() + c.getDispatcher().close() log.Info("[tso] tso client is closed") } -func (c *tsoClient) scheduleCheckTSODispatcher() { - select { - case c.checkTSODispatcherCh <- struct{}{}: - default: - } -} - -// scheduleUpdateAllTSOConnectionCtxs update the TSO connection contexts for all dc-locations. -func (c *tsoClient) scheduleUpdateAllTSOConnectionCtxs() { - c.tsoDispatcher.Range(func(_, dispatcher any) bool { - dispatcher.(*tsoDispatcher).scheduleUpdateConnectionCtxs() - return true - }) -} - -// scheduleUpdateTSOConnectionCtxs update the TSO connection contexts for the given dc-location. -func (c *tsoClient) scheduleUpdateTSOConnectionCtxs(dcLocation string) { - dispatcher, ok := c.getTSODispatcher(dcLocation) - if !ok { - return - } - dispatcher.scheduleUpdateConnectionCtxs() -} - -// TSO Follower Proxy only supports the Global TSO proxy now. -func (c *tsoClient) allowTSOFollowerProxy(dc string) bool { - return dc == globalDCLocation && c.option.getEnableTSOFollowerProxy() +// scheduleUpdateTSOConnectionCtxs update the TSO connection contexts. +func (c *tsoClient) scheduleUpdateTSOConnectionCtxs() { + c.getDispatcher().scheduleUpdateConnectionCtxs() } -func (c *tsoClient) getTSORequest(ctx context.Context, dcLocation string) *tsoRequest { +func (c *tsoClient) getTSORequest(ctx context.Context) *tsoRequest { req := c.tsoReqPool.Get().(*tsoRequest) // Set needed fields in the request before using it. req.start = time.Now() @@ -202,121 +155,41 @@ func (c *tsoClient) getTSORequest(ctx context.Context, dcLocation string) *tsoRe req.clientCtx = c.ctx req.physical = 0 req.logical = 0 - req.dcLocation = dcLocation req.streamID = "" return req } -func (c *tsoClient) getTSODispatcher(dcLocation string) (*tsoDispatcher, bool) { - dispatcher, ok := c.tsoDispatcher.Load(dcLocation) - if !ok || dispatcher == nil { - return nil, false +func (c *tsoClient) getLeaderURL() string { + url := c.leaderURL.Load() + if url == nil { + return "" } - return dispatcher.(*tsoDispatcher), true -} - -// GetTSOAllocators returns {dc-location -> TSO allocator leader URL} connection map -func (c *tsoClient) GetTSOAllocators() *sync.Map { - return &c.tsoAllocators + return url.(string) } -// GetTSOAllocatorServingURLByDCLocation returns the tso allocator of the given dcLocation -func (c *tsoClient) GetTSOAllocatorServingURLByDCLocation(dcLocation string) (string, bool) { - url, exist := c.tsoAllocators.Load(dcLocation) - if !exist { - return "", false - } - return url.(string), true -} - -// GetTSOAllocatorClientConnByDCLocation returns the TSO allocator gRPC client connection of the given dcLocation. -func (c *tsoClient) GetTSOAllocatorClientConnByDCLocation(dcLocation string) (*grpc.ClientConn, string) { - url, ok := c.tsoAllocators.Load(dcLocation) - if !ok { - log.Fatal("[tso] the allocator leader should exist", zap.String("dc-location", dcLocation)) +// getTSOLeaderClientConn returns the TSO leader gRPC client connection. +func (c *tsoClient) getTSOLeaderClientConn() (*grpc.ClientConn, string) { + url := c.getLeaderURL() + if len(url) == 0 { + log.Fatal("[tso] the tso leader should exist") } cc, ok := c.svcDiscovery.GetClientConns().Load(url) if !ok { - return nil, url.(string) + return nil, url } - return cc.(*grpc.ClientConn), url.(string) + return cc.(*grpc.ClientConn), url } -// AddTSOAllocatorServingURLSwitchedCallback adds callbacks which will be called -// when any global/local tso allocator service endpoint is switched. -func (c *tsoClient) AddTSOAllocatorServingURLSwitchedCallback(callbacks ...func()) { - c.tsoAllocServingURLSwitchedCallback = append(c.tsoAllocServingURLSwitchedCallback, callbacks...) -} - -func (c *tsoClient) updateTSOLocalServURLs(allocatorMap map[string]string) error { - if len(allocatorMap) == 0 { - return nil - } - - updated := false - - // Switch to the new one - for dcLocation, url := range allocatorMap { - if len(url) == 0 { - continue - } - oldURL, exist := c.GetTSOAllocatorServingURLByDCLocation(dcLocation) - if exist && url == oldURL { - continue - } - updated = true - if _, err := c.svcDiscovery.GetOrCreateGRPCConn(url); err != nil { - log.Warn("[tso] failed to connect dc tso allocator serving url", - zap.String("dc-location", dcLocation), - zap.String("serving-url", url), - errs.ZapError(err)) - return err - } - c.tsoAllocators.Store(dcLocation, url) - log.Info("[tso] switch dc tso local allocator serving url", - zap.String("dc-location", dcLocation), - zap.String("new-url", url), - zap.String("old-url", oldURL)) - // Should trigger the update of the connection contexts once the allocator leader is switched. - c.scheduleUpdateTSOConnectionCtxs(dcLocation) - } - - // Garbage collection of the old TSO allocator primaries - c.gcAllocatorServingURL(allocatorMap) - - if updated { - c.scheduleCheckTSODispatcher() - } - - return nil -} - -func (c *tsoClient) updateTSOGlobalServURL(url string) error { - c.tsoAllocators.Store(globalDCLocation, url) - log.Info("[tso] switch dc tso global allocator serving url", - zap.String("dc-location", globalDCLocation), - zap.String("new-url", url)) - c.scheduleUpdateTSOConnectionCtxs(globalDCLocation) - c.scheduleCheckTSODispatcher() +func (c *tsoClient) updateTSOLeaderURL(url string) error { + c.leaderURL.Store(url) + log.Info("[tso] switch the tso leader serving url", zap.String("new-url", url)) + // Try to create the TSO dispatcher if it is not created yet. + c.tryCreateTSODispatcher() + // Update the TSO connection contexts after the dispatcher is ready. + c.scheduleUpdateTSOConnectionCtxs() return nil } -func (c *tsoClient) gcAllocatorServingURL(curAllocatorMap map[string]string) { - // Clean up the old TSO allocators - c.tsoAllocators.Range(func(dcLocationKey, _ any) bool { - dcLocation := dcLocationKey.(string) - // Skip the Global TSO Allocator - if dcLocation == globalDCLocation { - return true - } - if _, exist := curAllocatorMap[dcLocation]; !exist { - log.Info("[tso] delete unused tso allocator", zap.String("dc-location", dcLocation)) - c.tsoAllocators.Delete(dcLocation) - } - return true - }) -} - // backupClientConn gets a grpc client connection of the current reachable and healthy // backup service endpoints randomly. Backup service endpoints are followers in a // quorum-based cluster or secondaries in a primary/secondary configured cluster. @@ -354,28 +227,27 @@ type tsoConnectionContext struct { stream *tsoStream } -// updateConnectionCtxs will choose the proper way to update the connections for the given dc-location. +// updateConnectionCtxs will choose the proper way to update the connections. // It will return a bool to indicate whether the update is successful. -func (c *tsoClient) updateConnectionCtxs(ctx context.Context, dc string, connectionCtxs *sync.Map) bool { +func (c *tsoClient) updateConnectionCtxs(ctx context.Context, connectionCtxs *sync.Map) bool { // Normal connection creating, it will be affected by the `enableForwarding`. createTSOConnection := c.tryConnectToTSO - if c.allowTSOFollowerProxy(dc) { + if c.option.getEnableTSOFollowerProxy() { createTSOConnection = c.tryConnectToTSOWithProxy } - if err := createTSOConnection(ctx, dc, connectionCtxs); err != nil { - log.Error("[tso] update connection contexts failed", zap.String("dc", dc), errs.ZapError(err)) + if err := createTSOConnection(ctx, connectionCtxs); err != nil { + log.Error("[tso] update connection contexts failed", errs.ZapError(err)) return false } return true } -// tryConnectToTSO will try to connect to the TSO allocator leader. If the connection becomes unreachable +// tryConnectToTSO will try to connect to the TSO leader. If the connection becomes unreachable // and enableForwarding is true, it will create a new connection to a follower to do the forwarding, // while a new daemon will be created also to switch back to a normal leader connection ASAP the // connection comes back to normal. func (c *tsoClient) tryConnectToTSO( ctx context.Context, - dc string, connectionCtxs *sync.Map, ) error { var ( @@ -405,7 +277,7 @@ func (c *tsoClient) tryConnectToTSO( // Retry several times before falling back to the follower when the network problem happens for range maxRetryTimes { c.svcDiscovery.ScheduleCheckMemberChanged() - cc, url = c.GetTSOAllocatorClientConnByDCLocation(dc) + cc, url = c.getTSOLeaderClientConn() if _, ok := connectionCtxs.Load(url); ok { // Just trigger the clean up of the stale connection contexts. updateAndClear(url, nil) @@ -448,10 +320,10 @@ func (c *tsoClient) tryConnectToTSO( // encounter the network error backupClientConn, backupURL := c.backupClientConn() if backupClientConn != nil { - log.Info("[tso] fall back to use follower to forward tso stream", zap.String("dc", dc), zap.String("follower-url", backupURL)) - forwardedHost, ok := c.GetTSOAllocatorServingURLByDCLocation(dc) - if !ok { - return errors.Errorf("cannot find the allocator leader in %s", dc) + log.Info("[tso] fall back to use follower to forward tso stream", zap.String("follower-url", backupURL)) + forwardedHost := c.getLeaderURL() + if len(forwardedHost) == 0 { + return errors.Errorf("cannot find the tso leader") } // create the follower stream @@ -462,7 +334,7 @@ func (c *tsoClient) tryConnectToTSO( forwardedHostTrim := trimHTTPPrefix(forwardedHost) addr := trimHTTPPrefix(backupURL) // the goroutine is used to check the network and change back to the original stream - go c.checkAllocator(ctx, cancel, dc, forwardedHostTrim, addr, url, updateAndClear) + go c.checkLeader(ctx, cancel, forwardedHostTrim, addr, url, updateAndClear) requestForwarded.WithLabelValues(forwardedHostTrim, addr).Set(1) updateAndClear(backupURL, &tsoConnectionContext{cctx, cancel, backupURL, stream}) return nil @@ -473,10 +345,10 @@ func (c *tsoClient) tryConnectToTSO( return err } -func (c *tsoClient) checkAllocator( +func (c *tsoClient) checkLeader( ctx context.Context, forwardCancel context.CancelFunc, - dc, forwardedHostTrim, addr, url string, + forwardedHostTrim, addr, url string, updateAndClear func(newAddr string, connectionCtx *tsoConnectionContext), ) { defer func() { @@ -484,14 +356,14 @@ func (c *tsoClient) checkAllocator( forwardCancel() requestForwarded.WithLabelValues(forwardedHostTrim, addr).Set(0) }() - cc, u := c.GetTSOAllocatorClientConnByDCLocation(dc) + cc, u := c.getTSOLeaderClientConn() var healthCli healthpb.HealthClient ticker := time.NewTicker(time.Second) defer ticker.Stop() for { - // the pd/allocator leader change, we need to re-establish the stream + // the tso leader change, we need to re-establish the stream if u != url { - log.Info("[tso] the leader of the allocator leader is changed", zap.String("dc", dc), zap.String("origin", url), zap.String("new", u)) + log.Info("[tso] the tso leader is changed", zap.String("origin", url), zap.String("new", u)) return } if healthCli == nil && cc != nil { @@ -505,11 +377,11 @@ func (c *tsoClient) checkAllocator( }) healthCancel() if err == nil && resp.GetStatus() == healthpb.HealthCheckResponse_SERVING { - // create a stream of the original allocator + // create a stream of the original tso leader cctx, cancel := context.WithCancel(ctx) stream, err := c.tsoStreamBuilderFactory.makeBuilder(cc).build(cctx, cancel, c.option.timeout) if err == nil && stream != nil { - log.Info("[tso] recover the original tso stream since the network has become normal", zap.String("dc", dc), zap.String("url", url)) + log.Info("[tso] recover the original tso stream since the network has become normal", zap.String("url", url)) updateAndClear(url, &tsoConnectionContext{cctx, cancel, url, stream}) return } @@ -519,9 +391,8 @@ func (c *tsoClient) checkAllocator( case <-ctx.Done(): return case <-ticker.C: - // To ensure we can get the latest allocator leader - // and once the leader is changed, we can exit this function. - cc, u = c.GetTSOAllocatorClientConnByDCLocation(dc) + // To ensure we can get the latest tso leader and once it's changed, we can exit this function. + cc, u = c.getTSOLeaderClientConn() } } } @@ -530,21 +401,19 @@ func (c *tsoClient) checkAllocator( // a TSO proxy to reduce the pressure of the main serving service endpoint. func (c *tsoClient) tryConnectToTSOWithProxy( ctx context.Context, - dc string, connectionCtxs *sync.Map, ) error { tsoStreamBuilders := c.getAllTSOStreamBuilders() leaderAddr := c.svcDiscovery.GetServingURL() - forwardedHost, ok := c.GetTSOAllocatorServingURLByDCLocation(dc) - if !ok { - return errors.Errorf("cannot find the allocator leader in %s", dc) + forwardedHost := c.getLeaderURL() + if len(forwardedHost) == 0 { + return errors.Errorf("cannot find the tso leader") } // GC the stale one. connectionCtxs.Range(func(addr, cc any) bool { addrStr := addr.(string) if _, ok := tsoStreamBuilders[addrStr]; !ok { log.Info("[tso] remove the stale tso stream", - zap.String("dc", dc), zap.String("addr", addrStr)) cc.(*tsoConnectionContext).cancel() connectionCtxs.Delete(addr) @@ -553,16 +422,16 @@ func (c *tsoClient) tryConnectToTSOWithProxy( }) // Update the missing one. for addr, tsoStreamBuilder := range tsoStreamBuilders { - if _, ok = connectionCtxs.Load(addr); ok { + _, ok := connectionCtxs.Load(addr) + if ok { continue } - log.Info("[tso] try to create tso stream", - zap.String("dc", dc), zap.String("addr", addr)) + log.Info("[tso] try to create tso stream", zap.String("addr", addr)) cctx, cancel := context.WithCancel(ctx) // Do not proxy the leader client. if addr != leaderAddr { log.Info("[tso] use follower to forward tso stream to do the proxy", - zap.String("dc", dc), zap.String("addr", addr)) + zap.String("addr", addr)) cctx = grpcutil.BuildForwardContext(cctx, forwardedHost) } // Create the TSO stream. @@ -577,7 +446,7 @@ func (c *tsoClient) tryConnectToTSOWithProxy( continue } log.Error("[tso] create the tso stream failed", - zap.String("dc", dc), zap.String("addr", addr), errs.ZapError(err)) + zap.String("addr", addr), errs.ZapError(err)) cancel() } return nil @@ -609,57 +478,31 @@ func (c *tsoClient) getAllTSOStreamBuilders() map[string]tsoStreamBuilder { return streamBuilders } -func (c *tsoClient) createTSODispatcher(dcLocation string) { - dispatcher := newTSODispatcher(c.ctx, dcLocation, defaultMaxTSOBatchSize, c) - if _, ok := c.tsoDispatcher.LoadOrStore(dcLocation, dispatcher); !ok { - // Create a new dispatcher for the dc-location to handle the TSO requests. - c.wg.Add(1) - go dispatcher.handleDispatcher(&c.wg) - } else { +// tryCreateTSODispatcher will try to create the TSO dispatcher if it is not created yet. +func (c *tsoClient) tryCreateTSODispatcher() { + // The dispatcher is already created. + if c.getDispatcher() != nil { + return + } + // The TSO leader is not ready. + url := c.getLeaderURL() + if len(url) == 0 { + return + } + dispatcher := newTSODispatcher(c.ctx, defaultMaxTSOBatchSize, c) + c.wg.Add(1) + go dispatcher.handleDispatcher(&c.wg) + // Try to set the dispatcher atomically. + if swapped := c.dispatcher.CompareAndSwap(nil, dispatcher); !swapped { dispatcher.close() } } -func (c *tsoClient) closeTSODispatcher() { - c.tsoDispatcher.Range(func(_, dispatcherInterface any) bool { - if dispatcherInterface != nil { - dispatcherInterface.(*tsoDispatcher).close() - } - return true - }) -} - -func (c *tsoClient) updateTSODispatcher() { - // Set up the new TSO dispatcher and batch controller. - c.GetTSOAllocators().Range(func(dcLocationKey, _ any) bool { - dcLocation := dcLocationKey.(string) - if _, ok := c.getTSODispatcher(dcLocation); !ok { - c.createTSODispatcher(dcLocation) - } - return true - }) - // Clean up the unused TSO dispatcher - c.tsoDispatcher.Range(func(dcLocationKey, dispatcher any) bool { - dcLocation := dcLocationKey.(string) - // Skip the Global TSO Allocator - if dcLocation == globalDCLocation { - return true - } - if _, exist := c.GetTSOAllocators().Load(dcLocation); !exist { - log.Info("[tso] delete unused tso dispatcher", zap.String("dc-location", dcLocation)) - c.tsoDispatcher.Delete(dcLocation) - dispatcher.(*tsoDispatcher).close() - } - return true - }) -} - // dispatchRequest will send the TSO request to the corresponding TSO dispatcher. func (c *tsoClient) dispatchRequest(request *tsoRequest) (bool, error) { - dispatcher, ok := c.getTSODispatcher(request.dcLocation) - if !ok { - err := errs.ErrClientGetTSO.FastGenByArgs(fmt.Sprintf("unknown dc-location %s to the client", request.dcLocation)) - log.Error("[tso] dispatch tso request error", zap.String("dc-location", request.dcLocation), errs.ZapError(err)) + if c.getDispatcher() == nil { + err := errs.ErrClientGetTSO.FastGenByArgs("tso dispatcher is not ready") + log.Error("[tso] dispatch tso request error", errs.ZapError(err)) c.svcDiscovery.ScheduleCheckMemberChanged() // New dispatcher could be created in the meantime, which is retryable. return true, err @@ -681,7 +524,7 @@ func (c *tsoClient) dispatchRequest(request *tsoRequest) (bool, error) { failpoint.Inject("delayDispatchTSORequest", func() { time.Sleep(time.Second) }) - dispatcher.push(request) + c.getDispatcher().push(request) } // Check the contexts again to make sure the request is not been sent to a closed dispatcher. // Never retry on these conditions to prevent unexpected data race. diff --git a/client/tso_dispatcher.go b/client/tso_dispatcher.go index a59dcaa7c61..3d77610179d 100644 --- a/client/tso_dispatcher.go +++ b/client/tso_dispatcher.go @@ -69,7 +69,7 @@ type tsoInfo struct { type tsoServiceProvider interface { getOption() *option getServiceDiscovery() ServiceDiscovery - updateConnectionCtxs(ctx context.Context, dc string, connectionCtxs *sync.Map) bool + updateConnectionCtxs(ctx context.Context, connectionCtxs *sync.Map) bool } const dispatcherCheckRPCConcurrencyInterval = time.Second * 5 @@ -77,7 +77,6 @@ const dispatcherCheckRPCConcurrencyInterval = time.Second * 5 type tsoDispatcher struct { ctx context.Context cancel context.CancelFunc - dc string provider tsoServiceProvider // URL -> *connectionContext @@ -102,7 +101,6 @@ type tsoDispatcher struct { func newTSODispatcher( ctx context.Context, - dc string, maxBatchSize int, provider tsoServiceProvider, ) *tsoDispatcher { @@ -119,7 +117,6 @@ func newTSODispatcher( td := &tsoDispatcher{ ctx: dispatcherCtx, cancel: dispatcherCancel, - dc: dc, provider: provider, connectionCtxs: &sync.Map{}, tsoRequestCh: tsoRequestCh, @@ -128,7 +125,7 @@ func newTSODispatcher( New: func() any { return newBatchController[*tsoRequest]( maxBatchSize*2, - tsoRequestFinisher(0, 0, 0, invalidStreamID), + tsoRequestFinisher(0, 0, invalidStreamID), tsoBestBatchSize, ) }, @@ -141,15 +138,15 @@ func newTSODispatcher( } func (td *tsoDispatcher) watchTSDeadline() { - log.Info("[tso] start tso deadline watcher", zap.String("dc-location", td.dc)) - defer log.Info("[tso] exit tso deadline watcher", zap.String("dc-location", td.dc)) + log.Info("[tso] start tso deadline watcher") + defer log.Info("[tso] exit tso deadline watcher") for { select { case d := <-td.tsDeadlineCh: select { case <-d.timer.C: log.Error("[tso] tso request is canceled due to timeout", - zap.String("dc-location", td.dc), errs.ZapError(errs.ErrClientGetTSOTimeout)) + errs.ZapError(errs.ErrClientGetTSOTimeout)) d.cancel() timerutil.GlobalTimerPool.Put(d.timer) case <-d.done: @@ -191,7 +188,6 @@ func (td *tsoDispatcher) push(request *tsoRequest) { func (td *tsoDispatcher) handleDispatcher(wg *sync.WaitGroup) { var ( ctx = td.ctx - dc = td.dc provider = td.provider svcDiscovery = provider.getServiceDiscovery() option = provider.getOption() @@ -199,10 +195,10 @@ func (td *tsoDispatcher) handleDispatcher(wg *sync.WaitGroup) { tsoBatchController *batchController[*tsoRequest] ) - log.Info("[tso] tso dispatcher created", zap.String("dc-location", dc)) + log.Info("[tso] tso dispatcher created") // Clean up the connectionCtxs when the dispatcher exits. defer func() { - log.Info("[tso] exit tso dispatcher", zap.String("dc-location", dc)) + log.Info("[tso] exit tso dispatcher") // Cancel all connections. connectionCtxs.Range(func(_, cc any) bool { cc.(*tsoConnectionContext).cancel() @@ -256,7 +252,7 @@ tsoBatchLoop: if err = td.checkTSORPCConcurrency(ctx, maxBatchWaitInterval, currentBatchStartTime); err != nil { // checkTSORPCConcurrency can only fail due to `ctx` being invalidated. log.Info("[tso] stop checking tso rpc concurrency configurations due to context canceled", - zap.String("dc-location", dc), zap.Error(err)) + zap.Error(err)) return } @@ -265,11 +261,9 @@ tsoBatchLoop: // otherwise the upper caller may get blocked on waiting for the results. if err = tsoBatchController.fetchPendingRequests(ctx, td.tsoRequestCh, td.tokenCh, maxBatchWaitInterval); err != nil { if err == context.Canceled { - log.Info("[tso] stop fetching the pending tso requests due to context canceled", - zap.String("dc-location", dc)) + log.Info("[tso] stop fetching the pending tso requests due to context canceled") } else { log.Error("[tso] fetch pending tso requests error", - zap.String("dc-location", dc), zap.Error(errs.ErrClientGetTSO.FastGenByArgs(err.Error()))) } return @@ -296,8 +290,8 @@ tsoBatchLoop: } // Check stream and retry if necessary. if stream == nil { - log.Info("[tso] tso stream is not ready", zap.String("dc", dc)) - if provider.updateConnectionCtxs(ctx, dc, connectionCtxs) { + log.Info("[tso] tso stream is not ready") + if provider.updateConnectionCtxs(ctx, connectionCtxs) { continue streamChoosingLoop } timer := time.NewTimer(retryInterval) @@ -309,7 +303,7 @@ tsoBatchLoop: return case <-streamLoopTimer.C: err = errs.ErrClientCreateTSOStream.FastGenByArgs(errs.RetryTimeoutErr) - log.Error("[tso] create tso stream error", zap.String("dc-location", dc), errs.ZapError(err)) + log.Error("[tso] create tso stream error", errs.ZapError(err)) svcDiscovery.ScheduleCheckMemberChanged() // Finish the collected requests if the stream is failed to be created. td.cancelCollectedRequests(tsoBatchController, invalidStreamID, errors.WithStack(err)) @@ -322,7 +316,7 @@ tsoBatchLoop: } select { case <-streamCtx.Done(): - log.Info("[tso] tso stream is canceled", zap.String("dc", dc), zap.String("stream-url", streamURL)) + log.Info("[tso] tso stream is canceled", zap.String("stream-url", streamURL)) // Set `stream` to nil and remove this stream from the `connectionCtxs` due to being canceled. connectionCtxs.Delete(streamURL) cancel() @@ -388,7 +382,7 @@ tsoBatchLoop: if err != nil { // There should not be other kinds of errors. log.Info("[tso] stop fetching the pending tso requests due to context canceled", - zap.String("dc-location", dc), zap.Error(err)) + zap.Error(err)) td.cancelCollectedRequests(tsoBatchController, invalidStreamID, errors.WithStack(ctx.Err())) return } @@ -405,7 +399,7 @@ tsoBatchLoop: case td.tsDeadlineCh <- dl: } // processRequests guarantees that the collected requests could be finished properly. - err = td.processRequests(stream, dc, tsoBatchController, done) + err = td.processRequests(stream, tsoBatchController, done) // If error happens during tso stream handling, reset stream and run the next trial. if err == nil { // A nil error returned by `processRequests` indicates that the request batch is started successfully. @@ -440,7 +434,6 @@ func (td *tsoDispatcher) handleProcessRequestError(ctx context.Context, bo *retr svcDiscovery.ScheduleCheckMemberChanged() log.Error("[tso] getTS error after processing requests", - zap.String("dc-location", td.dc), zap.String("stream-url", streamURL), zap.Error(errs.ErrClientGetTSO.FastGenByArgs(err.Error()))) // Set `stream` to nil and remove this stream from the `connectionCtxs` due to error. @@ -460,24 +453,23 @@ func (td *tsoDispatcher) handleProcessRequestError(ctx context.Context, bo *retr // will cancel the current stream, then the EOF error caused by cancel() // should not trigger the updateConnectionCtxs here. // So we should only call it when the leader changes. - td.provider.updateConnectionCtxs(ctx, td.dc, td.connectionCtxs) + td.provider.updateConnectionCtxs(ctx, td.connectionCtxs) } return true } -// updateConnectionCtxs updates the `connectionCtxs` for the specified DC location regularly. +// updateConnectionCtxs updates the `connectionCtxs` regularly. func (td *tsoDispatcher) connectionCtxsUpdater() { var ( ctx = td.ctx - dc = td.dc connectionCtxs = td.connectionCtxs provider = td.provider option = td.provider.getOption() updateTicker = &time.Ticker{} ) - log.Info("[tso] start tso connection contexts updater", zap.String("dc-location", dc)) + log.Info("[tso] start tso connection contexts updater") setNewUpdateTicker := func(ticker *time.Ticker) { if updateTicker.C != nil { updateTicker.Stop() @@ -488,19 +480,14 @@ func (td *tsoDispatcher) connectionCtxsUpdater() { defer setNewUpdateTicker(nil) for { - provider.updateConnectionCtxs(ctx, dc, connectionCtxs) + provider.updateConnectionCtxs(ctx, connectionCtxs) select { case <-ctx.Done(): - log.Info("[tso] exit tso connection contexts updater", zap.String("dc-location", dc)) + log.Info("[tso] exit tso connection contexts updater") return case <-option.enableTSOFollowerProxyCh: - // TODO: implement support of TSO Follower Proxy for the Local TSO. - if dc != globalDCLocation { - continue - } enableTSOFollowerProxy := option.getEnableTSOFollowerProxy() log.Info("[tso] tso follower proxy status changed", - zap.String("dc-location", dc), zap.Bool("enable", enableTSOFollowerProxy)) if enableTSOFollowerProxy && updateTicker.C == nil { // Because the TSO Follower Proxy is enabled, @@ -541,7 +528,7 @@ func chooseStream(connectionCtxs *sync.Map) (connectionCtx *tsoConnectionContext // `close(done)` will be called at the same time when finishing the requests. // If this function returns a non-nil error, the requests will always be canceled synchronously. func (td *tsoDispatcher) processRequests( - stream *tsoStream, dcLocation string, tbc *batchController[*tsoRequest], done chan struct{}, + stream *tsoStream, tbc *batchController[*tsoRequest], done chan struct{}, ) error { // `done` must be guaranteed to be eventually called. var ( @@ -596,15 +583,15 @@ func (td *tsoDispatcher) processRequests( sourceStreamID: stream.streamID, } // `logical` is the largest ts's logical part here, we need to do the subtracting before we finish each TSO request. - firstLogical := tsoutil.AddLogical(result.logical, -int64(result.count)+1, result.suffixBits) + firstLogical := result.logical - int64(result.count) + 1 // Do the check before releasing the token. td.checkMonotonicity(tsoInfoBeforeReq, curTSOInfo, firstLogical) - td.doneCollectedRequests(tbc, result.physical, firstLogical, result.suffixBits, stream.streamID) + td.doneCollectedRequests(tbc, result.physical, firstLogical, stream.streamID) } err := stream.processRequests( clusterID, keyspaceID, reqKeyspaceGroupID, - dcLocation, count, tbc.extraBatchingStartTime, cb) + count, tbc.extraBatchingStartTime, cb) if err != nil { close(done) @@ -614,11 +601,11 @@ func (td *tsoDispatcher) processRequests( return nil } -func tsoRequestFinisher(physical, firstLogical int64, suffixBits uint32, streamID string) finisherFunc[*tsoRequest] { +func tsoRequestFinisher(physical, firstLogical int64, streamID string) finisherFunc[*tsoRequest] { return func(idx int, tsoReq *tsoRequest, err error) { // Retrieve the request context before the request is done to trace without race. requestCtx := tsoReq.requestCtx - tsoReq.physical, tsoReq.logical = physical, tsoutil.AddLogical(firstLogical, int64(idx), suffixBits) + tsoReq.physical, tsoReq.logical = physical, firstLogical+int64(idx) tsoReq.streamID = streamID tsoReq.tryDone(err) trace.StartRegion(requestCtx, "pdclient.tsoReqDequeue").End() @@ -627,12 +614,12 @@ func tsoRequestFinisher(physical, firstLogical int64, suffixBits uint32, streamI func (td *tsoDispatcher) cancelCollectedRequests(tbc *batchController[*tsoRequest], streamID string, err error) { td.tokenCh <- struct{}{} - tbc.finishCollectedRequests(tsoRequestFinisher(0, 0, 0, streamID), err) + tbc.finishCollectedRequests(tsoRequestFinisher(0, 0, streamID), err) } -func (td *tsoDispatcher) doneCollectedRequests(tbc *batchController[*tsoRequest], physical, firstLogical int64, suffixBits uint32, streamID string) { +func (td *tsoDispatcher) doneCollectedRequests(tbc *batchController[*tsoRequest], physical, firstLogical int64, streamID string) { td.tokenCh <- struct{}{} - tbc.finishCollectedRequests(tsoRequestFinisher(physical, firstLogical, suffixBits, streamID), nil) + tbc.finishCollectedRequests(tsoRequestFinisher(physical, firstLogical, streamID), nil) } // checkMonotonicity checks whether the monotonicity of the TSO allocation is violated. @@ -650,7 +637,6 @@ func (td *tsoDispatcher) checkMonotonicity( if lastTSOInfo != nil { if lastTSOInfo.respKeyspaceGroupID != curTSOInfo.respKeyspaceGroupID { log.Info("[tso] keyspace group changed", - zap.String("dc-location", td.dc), zap.Uint32("old-group-id", lastTSOInfo.respKeyspaceGroupID), zap.Uint32("new-group-id", curTSOInfo.respKeyspaceGroupID)) } @@ -660,7 +646,6 @@ func (td *tsoDispatcher) checkMonotonicity( // last time. if tsoutil.TSLessEqual(curTSOInfo.physical, firstLogical, lastTSOInfo.physical, lastTSOInfo.logical) { log.Panic("[tso] timestamp fallback", - zap.String("dc-location", td.dc), zap.Uint32("keyspace", keyspaceID), zap.String("last-ts", fmt.Sprintf("(%d, %d)", lastTSOInfo.physical, lastTSOInfo.logical)), zap.String("cur-ts", fmt.Sprintf("(%d, %d)", curTSOInfo.physical, firstLogical)), diff --git a/client/tso_dispatcher_test.go b/client/tso_dispatcher_test.go index 194c9bde455..2eb30066532 100644 --- a/client/tso_dispatcher_test.go +++ b/client/tso_dispatcher_test.go @@ -50,7 +50,7 @@ func (*mockTSOServiceProvider) getServiceDiscovery() ServiceDiscovery { return NewMockPDServiceDiscovery([]string{mockStreamURL}, nil) } -func (m *mockTSOServiceProvider) updateConnectionCtxs(ctx context.Context, _dc string, connectionCtxs *sync.Map) bool { +func (m *mockTSOServiceProvider) updateConnectionCtxs(ctx context.Context, connectionCtxs *sync.Map) bool { // Avoid concurrent updating in the background updating goroutine and active updating in the dispatcher loop when // stream is missing. m.updateConnMu.Lock() @@ -102,14 +102,13 @@ func (s *testTSODispatcherSuite) SetupTest() { created.Store(true) return s.stream } - s.dispatcher = newTSODispatcher(context.Background(), globalDCLocation, defaultMaxTSOBatchSize, newMockTSOServiceProvider(s.option, createStream)) + s.dispatcher = newTSODispatcher(context.Background(), defaultMaxTSOBatchSize, newMockTSOServiceProvider(s.option, createStream)) s.reqPool = &sync.Pool{ New: func() any { return &tsoRequest{ - done: make(chan error, 1), - physical: 0, - logical: 0, - dcLocation: globalDCLocation, + done: make(chan error, 1), + physical: 0, + logical: 0, } }, } @@ -331,10 +330,9 @@ func BenchmarkTSODispatcherHandleRequests(b *testing.B) { reqPool := &sync.Pool{ New: func() any { return &tsoRequest{ - done: make(chan error, 1), - physical: 0, - logical: 0, - dcLocation: globalDCLocation, + done: make(chan error, 1), + physical: 0, + logical: 0, } }, } @@ -349,7 +347,7 @@ func BenchmarkTSODispatcherHandleRequests(b *testing.B) { return req } - dispatcher := newTSODispatcher(ctx, globalDCLocation, defaultMaxTSOBatchSize, newMockTSOServiceProvider(newOption(), nil)) + dispatcher := newTSODispatcher(ctx, defaultMaxTSOBatchSize, newMockTSOServiceProvider(newOption(), nil)) var wg sync.WaitGroup wg.Add(1) diff --git a/client/tso_request.go b/client/tso_request.go index 441e92a4390..29654752cd0 100644 --- a/client/tso_request.go +++ b/client/tso_request.go @@ -40,7 +40,6 @@ type tsoRequest struct { done chan error physical int64 logical int64 - dcLocation string // The identifier of the RPC stream in which the request is processed. streamID string diff --git a/client/tso_service_discovery.go b/client/tso_service_discovery.go index 634b4211e38..b8debf05efe 100644 --- a/client/tso_service_discovery.go +++ b/client/tso_service_discovery.go @@ -51,7 +51,7 @@ const ( ) var _ ServiceDiscovery = (*tsoServiceDiscovery)(nil) -var _ tsoAllocatorEventSource = (*tsoServiceDiscovery)(nil) +var _ tsoEventSource = (*tsoServiceDiscovery)(nil) // keyspaceGroupSvcDiscovery is used for discovering the serving endpoints of the keyspace // group to which the keyspace belongs @@ -136,11 +136,8 @@ type tsoServiceDiscovery struct { // URL -> a gRPC connection clientConns sync.Map // Store as map[string]*grpc.ClientConn - // localAllocPrimariesUpdatedCb will be called when the local tso allocator primary list is updated. - // The input is a map {DC Location -> Leader URL} - localAllocPrimariesUpdatedCb tsoLocalServURLsUpdatedFunc - // globalAllocPrimariesUpdatedCb will be called when the local tso allocator primary list is updated. - globalAllocPrimariesUpdatedCb tsoGlobalServURLUpdatedFunc + // tsoLeaderUpdatedCb will be called when the TSO leader is updated. + tsoLeaderUpdatedCb tsoLeaderURLUpdatedFunc checkMembershipCh chan struct{} @@ -360,22 +357,15 @@ func (*tsoServiceDiscovery) AddServingURLSwitchedCallback(...func()) {} // in a primary/secondary configured cluster is changed. func (*tsoServiceDiscovery) AddServiceURLsSwitchedCallback(...func()) {} -// SetTSOLocalServURLsUpdatedCallback adds a callback which will be called when the local tso -// allocator leader list is updated. -func (c *tsoServiceDiscovery) SetTSOLocalServURLsUpdatedCallback(callback tsoLocalServURLsUpdatedFunc) { - c.localAllocPrimariesUpdatedCb = callback -} - -// SetTSOGlobalServURLUpdatedCallback adds a callback which will be called when the global tso -// allocator leader is updated. -func (c *tsoServiceDiscovery) SetTSOGlobalServURLUpdatedCallback(callback tsoGlobalServURLUpdatedFunc) { +// SetTSOLeaderURLUpdatedCallback adds a callback which will be called when the TSO leader is updated. +func (c *tsoServiceDiscovery) SetTSOLeaderURLUpdatedCallback(callback tsoLeaderURLUpdatedFunc) { url := c.getPrimaryURL() if len(url) > 0 { if err := callback(url); err != nil { log.Error("[tso] failed to call back when tso global service url update", zap.String("url", url), errs.ZapError(err)) } } - c.globalAllocPrimariesUpdatedCb = callback + c.tsoLeaderUpdatedCb = callback } // GetServiceClient implements ServiceDiscovery @@ -404,8 +394,8 @@ func (c *tsoServiceDiscovery) getSecondaryURLs() []string { func (c *tsoServiceDiscovery) afterPrimarySwitched(oldPrimary, newPrimary string) error { // Run callbacks - if c.globalAllocPrimariesUpdatedCb != nil { - if err := c.globalAllocPrimariesUpdatedCb(newPrimary); err != nil { + if c.tsoLeaderUpdatedCb != nil { + if err := c.tsoLeaderUpdatedCb(newPrimary); err != nil { return err } } diff --git a/client/tso_stream.go b/client/tso_stream.go index 142ad71c6b9..55bfd0b72b0 100644 --- a/client/tso_stream.go +++ b/client/tso_stream.go @@ -110,13 +110,11 @@ func checkStreamTimeout(ctx context.Context, cancel context.CancelFunc, done cha type tsoRequestResult struct { physical, logical int64 count uint32 - suffixBits uint32 respKeyspaceGroupID uint32 } type grpcTSOStreamAdapter interface { - Send(clusterID uint64, keyspaceID, keyspaceGroupID uint32, dcLocation string, - count int64) error + Send(clusterID uint64, keyspaceID, keyspaceGroupID uint32, count int64) error Recv() (tsoRequestResult, error) } @@ -125,13 +123,12 @@ type pdTSOStreamAdapter struct { } // Send implements the grpcTSOStreamAdapter interface. -func (s pdTSOStreamAdapter) Send(clusterID uint64, _, _ uint32, dcLocation string, count int64) error { +func (s pdTSOStreamAdapter) Send(clusterID uint64, _, _ uint32, count int64) error { req := &pdpb.TsoRequest{ Header: &pdpb.RequestHeader{ ClusterId: clusterID, }, - Count: uint32(count), - DcLocation: dcLocation, + Count: uint32(count), } return s.stream.Send(req) } @@ -146,7 +143,6 @@ func (s pdTSOStreamAdapter) Recv() (tsoRequestResult, error) { physical: resp.GetTimestamp().GetPhysical(), logical: resp.GetTimestamp().GetLogical(), count: resp.GetCount(), - suffixBits: resp.GetTimestamp().GetSuffixBits(), respKeyspaceGroupID: defaultKeySpaceGroupID, }, nil } @@ -156,15 +152,14 @@ type tsoTSOStreamAdapter struct { } // Send implements the grpcTSOStreamAdapter interface. -func (s tsoTSOStreamAdapter) Send(clusterID uint64, keyspaceID, keyspaceGroupID uint32, dcLocation string, count int64) error { +func (s tsoTSOStreamAdapter) Send(clusterID uint64, keyspaceID, keyspaceGroupID uint32, count int64) error { req := &tsopb.TsoRequest{ Header: &tsopb.RequestHeader{ ClusterId: clusterID, KeyspaceId: keyspaceID, KeyspaceGroupId: keyspaceGroupID, }, - Count: uint32(count), - DcLocation: dcLocation, + Count: uint32(count), } return s.stream.Send(req) } @@ -179,7 +174,6 @@ func (s tsoTSOStreamAdapter) Recv() (tsoRequestResult, error) { physical: resp.GetTimestamp().GetPhysical(), logical: resp.GetTimestamp().GetLogical(), count: resp.GetCount(), - suffixBits: resp.GetTimestamp().GetSuffixBits(), respKeyspaceGroupID: resp.GetHeader().GetKeyspaceGroupId(), }, nil } @@ -268,7 +262,7 @@ func (s *tsoStream) getServerURL() string { // It's guaranteed that the `callback` will be called, but when the request is failed to be scheduled, the callback // will be ignored. func (s *tsoStream) processRequests( - clusterID uint64, keyspaceID, keyspaceGroupID uint32, dcLocation string, count int64, batchStartTime time.Time, callback onFinishedCallback, + clusterID uint64, keyspaceID, keyspaceGroupID uint32, count int64, batchStartTime time.Time, callback onFinishedCallback, ) error { start := time.Now() @@ -305,7 +299,7 @@ func (s *tsoStream) processRequests( } s.state.Store(prevState) - if err := s.stream.Send(clusterID, keyspaceID, keyspaceGroupID, dcLocation, count); err != nil { + if err := s.stream.Send(clusterID, keyspaceID, keyspaceGroupID, count); err != nil { // As the request is already put into `pendingRequests`, the request should finally be canceled by the recvLoop. // So skip returning error here to avoid // if err == io.EOF { diff --git a/client/tso_stream_test.go b/client/tso_stream_test.go index 6595ed2c13a..a842befb550 100644 --- a/client/tso_stream_test.go +++ b/client/tso_stream_test.go @@ -76,7 +76,7 @@ func newMockTSOStreamImpl(ctx context.Context, resultMode resultMode) *mockTSOSt } } -func (s *mockTSOStreamImpl) Send(clusterID uint64, _keyspaceID, keyspaceGroupID uint32, _dcLocation string, count int64) error { +func (s *mockTSOStreamImpl) Send(clusterID uint64, _keyspaceID, keyspaceGroupID uint32, count int64) error { select { case <-s.ctx.Done(): return s.ctx.Err() @@ -210,7 +210,6 @@ func (s *mockTSOStreamImpl) autoGenResult(count int64) resultMsg { physical: s.resGenPhysical, logical: s.resGenLogical, count: uint32(count), - suffixBits: 0, respKeyspaceGroupID: 0, }, } @@ -225,7 +224,6 @@ func (s *mockTSOStreamImpl) returnResult(physical int64, logical int64, count ui physical: physical, logical: logical, count: count, - suffixBits: 0, respKeyspaceGroupID: s.keyspaceID, }, } @@ -305,10 +303,9 @@ func (s *testTSOStreamSuite) getResult(ch <-chan callbackInvocation) callbackInv func (s *testTSOStreamSuite) processRequestWithResultCh(count int64) (<-chan callbackInvocation, error) { ch := make(chan callbackInvocation, 1) - err := s.stream.processRequests(1, 2, 3, globalDCLocation, count, time.Now(), func(result tsoRequestResult, reqKeyspaceGroupID uint32, err error) { + err := s.stream.processRequests(1, 2, 3, count, time.Now(), func(result tsoRequestResult, reqKeyspaceGroupID uint32, err error) { if err == nil { s.re.Equal(uint32(3), reqKeyspaceGroupID) - s.re.Equal(uint32(0), result.suffixBits) } ch <- callbackInvocation{ result: result, @@ -357,7 +354,7 @@ func (s *testTSOStreamSuite) TestTSOStreamBasic() { // After an error from the (simulated) RPC stream, the tsoStream should be in a broken status and can't accept // new request anymore. - err := s.stream.processRequests(1, 2, 3, globalDCLocation, 1, time.Now(), func(_result tsoRequestResult, _reqKeyspaceGroupID uint32, _err error) { + err := s.stream.processRequests(1, 2, 3, 1, time.Now(), func(_result tsoRequestResult, _reqKeyspaceGroupID uint32, _err error) { panic("unreachable") }) s.re.Error(err) @@ -621,7 +618,7 @@ func BenchmarkTSOStreamSendRecv(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - err := stream.processRequests(1, 1, 1, globalDCLocation, 1, now, func(result tsoRequestResult, _ uint32, err error) { + err := stream.processRequests(1, 1, 1, 1, now, func(result tsoRequestResult, _ uint32, err error) { if err != nil { panic(err) } diff --git a/client/utils/tsoutil/tsoutil.go b/client/utils/tsoutil/tsoutil.go index ffc449640ac..34256373b36 100644 --- a/client/utils/tsoutil/tsoutil.go +++ b/client/utils/tsoutil/tsoutil.go @@ -18,11 +18,6 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" ) -// AddLogical shifts the count before we add it to the logical part. -func AddLogical(logical, count int64, suffixBits uint32) int64 { - return logical + count<", - Short: "Run services, for example, tso, resource_manager", + Short: "Run services, for example, tso, scheduling", } cmd.AddCommand(NewTSOServiceCommand()) - cmd.AddCommand(NewResourceManagerServiceCommand()) cmd.AddCommand(NewSchedulingServiceCommand()) cmd.AddCommand(NewAPIServiceCommand()) return cmd @@ -129,27 +126,6 @@ func NewSchedulingServiceCommand() *cobra.Command { return cmd } -// NewResourceManagerServiceCommand returns the resource manager service command. -func NewResourceManagerServiceCommand() *cobra.Command { - cmd := &cobra.Command{ - Use: rmMode, - Short: "Run the resource manager service", - Run: resource_manager.CreateServerWrapper, - } - cmd.Flags().StringP("name", "", "", "human-readable name for this resource manager member") - cmd.Flags().BoolP("version", "V", false, "print version information and exit") - cmd.Flags().StringP("config", "", "", "config file") - cmd.Flags().StringP("backend-endpoints", "", "", "url for etcd client") - cmd.Flags().StringP("listen-addr", "", "", "listen address for resource management service") - cmd.Flags().StringP("advertise-listen-addr", "", "", "advertise urls for listen address (default '${listen-addr}')") - cmd.Flags().StringP("cacert", "", "", "path of file that contains list of trusted TLS CAs") - cmd.Flags().StringP("cert", "", "", "path of file that contains X509 certificate in PEM format") - cmd.Flags().StringP("key", "", "", "path of file that contains X509 key in PEM format") - cmd.Flags().StringP("log-level", "L", "", "log level: debug, info, warn, error, fatal (default 'info')") - cmd.Flags().StringP("log-file", "", "", "log file path") - return cmd -} - // NewAPIServiceCommand returns the API service command. func NewAPIServiceCommand() *cobra.Command { cmd := &cobra.Command{ diff --git a/pd.code-workspace b/pd.code-workspace index c5767e86b22..28722d4fa61 100644 --- a/pd.code-workspace +++ b/pd.code-workspace @@ -17,5 +17,7 @@ "path": "tools" } ], - "settings": {} + "settings": { + "makefile.configureOnOpen": false + } } \ No newline at end of file diff --git a/pkg/mcs/discovery/discover.go b/pkg/mcs/discovery/discover.go index 083059780b4..d4234df893d 100644 --- a/pkg/mcs/discovery/discover.go +++ b/pkg/mcs/discovery/discover.go @@ -46,7 +46,7 @@ func Discover(cli *clientv3.Client, serviceName string) ([]string, error) { // GetMSMembers returns all the members of the specified service name. func GetMSMembers(serviceName string, client *clientv3.Client) ([]ServiceRegistryEntry, error) { switch serviceName { - case constant.TSOServiceName, constant.SchedulingServiceName, constant.ResourceManagerServiceName: + case constant.TSOServiceName, constant.SchedulingServiceName: servicePath := keypath.ServicePath(serviceName) resps, err := kv.NewSlowLogTxn(client).Then(clientv3.OpGet(servicePath, clientv3.WithPrefix())).Commit() if err != nil { diff --git a/pkg/mcs/resourcemanager/server/apis/v1/api.go b/pkg/mcs/resourcemanager/server/apis/v1/api.go index 3fd94e637d0..891072e898f 100644 --- a/pkg/mcs/resourcemanager/server/apis/v1/api.go +++ b/pkg/mcs/resourcemanager/server/apis/v1/api.go @@ -26,12 +26,10 @@ import ( "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" - "github.com/pingcap/log" rmserver "github.com/tikv/pd/pkg/mcs/resourcemanager/server" "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/apiutil/multiservicesapi" - "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/reflectutil" ) @@ -84,17 +82,10 @@ func NewService(srv *rmserver.Service) *Service { apiHandlerEngine: apiHandlerEngine, root: endpoint, } - s.RegisterAdminRouter() s.RegisterRouter() return s } -// RegisterAdminRouter registers the router of the TSO admin handler. -func (s *Service) RegisterAdminRouter() { - router := s.root.Group("admin") - router.PUT("/log", changeLogLevel) -} - // RegisterRouter registers the router of the service. func (s *Service) RegisterRouter() { configEndpoint := s.root.Group("/config") @@ -113,22 +104,6 @@ func (s *Service) handler() http.Handler { }) } -func changeLogLevel(c *gin.Context) { - svr := c.MustGet(multiservicesapi.ServiceContextKey).(*rmserver.Service) - var level string - if err := c.Bind(&level); err != nil { - c.String(http.StatusBadRequest, err.Error()) - return - } - - if err := svr.SetLogLevel(level); err != nil { - c.String(http.StatusBadRequest, err.Error()) - return - } - log.SetLevel(logutil.StringToZapLogLevel(level)) - c.String(http.StatusOK, "The log level is updated.") -} - // postResourceGroup // // @Tags ResourceManager diff --git a/pkg/mcs/resourcemanager/server/grpc_service.go b/pkg/mcs/resourcemanager/server/grpc_service.go index 93243eb05be..21681bc0759 100644 --- a/pkg/mcs/resourcemanager/server/grpc_service.go +++ b/pkg/mcs/resourcemanager/server/grpc_service.go @@ -54,8 +54,7 @@ func (dummyRestService) ServeHTTP(w http.ResponseWriter, _ *http.Request) { // Service is the gRPC service for resource manager. type Service struct { - ctx context.Context - *Server + ctx context.Context manager *Manager // settings } diff --git a/pkg/mcs/resourcemanager/server/install/install.go b/pkg/mcs/resourcemanager/server/install/install.go index 8573d5e52eb..8cd7e0eac85 100644 --- a/pkg/mcs/resourcemanager/server/install/install.go +++ b/pkg/mcs/resourcemanager/server/install/install.go @@ -28,5 +28,5 @@ func init() { // Install registers the API group and grpc service. func Install(register *registry.ServiceRegistry) { - register.RegisterService("ResourceManager", server.NewService[*server.Server]) + register.RegisterService("ResourceManager", server.NewService[*server.Manager]) } diff --git a/pkg/mcs/resourcemanager/server/server.go b/pkg/mcs/resourcemanager/server/server.go deleted file mode 100644 index 05b01094801..00000000000 --- a/pkg/mcs/resourcemanager/server/server.go +++ /dev/null @@ -1,422 +0,0 @@ -// Copyright 2023 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 server - -import ( - "context" - "net/http" - "os" - "os/signal" - "runtime" - "sync" - "sync/atomic" - "syscall" - "time" - - grpcprometheus "github.com/grpc-ecosystem/go-grpc-prometheus" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/diagnosticspb" - "github.com/pingcap/kvproto/pkg/resource_manager" - "github.com/pingcap/log" - "github.com/pingcap/sysutil" - "github.com/spf13/cobra" - bs "github.com/tikv/pd/pkg/basicserver" - "github.com/tikv/pd/pkg/errs" - "github.com/tikv/pd/pkg/mcs/discovery" - "github.com/tikv/pd/pkg/mcs/server" - "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/utils/apiutil" - "github.com/tikv/pd/pkg/utils/grpcutil" - "github.com/tikv/pd/pkg/utils/keypath" - "github.com/tikv/pd/pkg/utils/logutil" - "github.com/tikv/pd/pkg/utils/memberutil" - "github.com/tikv/pd/pkg/utils/metricutil" - "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" - "google.golang.org/grpc" -) - -var _ bs.Server = (*Server)(nil) - -const serviceName = "Resource Manager" - -// Server is the resource manager server, and it implements bs.Server. -type Server struct { - *server.BaseServer - diagnosticspb.DiagnosticsServer - // Server state. 0 is not running, 1 is running. - isRunning int64 - - serverLoopCtx context.Context - serverLoopCancel func() - serverLoopWg sync.WaitGroup - - cfg *Config - - // for the primary election of resource manager - participant *member.Participant - - service *Service - - // primaryCallbacks will be called after the server becomes leader. - primaryCallbacks []func(context.Context) error - - // for service registry - serviceID *discovery.ServiceRegistryEntry - serviceRegister *discovery.ServiceRegister -} - -// Name returns the unique name for this server in the resource manager cluster. -func (s *Server) Name() string { - return s.cfg.Name -} - -// GetAddr returns the server address. -func (s *Server) GetAddr() string { - return s.cfg.ListenAddr -} - -// GetAdvertiseListenAddr returns the advertise address of the server. -func (s *Server) GetAdvertiseListenAddr() string { - return s.cfg.AdvertiseListenAddr -} - -// SetLogLevel sets log level. -func (s *Server) SetLogLevel(level string) error { - if !logutil.IsLevelLegal(level) { - return errors.Errorf("log level %s is illegal", level) - } - s.cfg.Log.Level = level - log.SetLevel(logutil.StringToZapLogLevel(level)) - log.Warn("log level changed", zap.String("level", log.GetLevel().String())) - return nil -} - -// Run runs the Resource Manager server. -func (s *Server) Run() (err error) { - if err = utils.InitClient(s); err != nil { - return err - } - - if s.serviceID, s.serviceRegister, err = utils.Register(s, constant.ResourceManagerServiceName); err != nil { - return err - } - - return s.startServer() -} - -func (s *Server) startServerLoop() { - s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.Context()) - s.serverLoopWg.Add(1) - go s.primaryElectionLoop() -} - -func (s *Server) primaryElectionLoop() { - defer logutil.LogPanic() - defer s.serverLoopWg.Done() - - for { - select { - case <-s.serverLoopCtx.Done(): - log.Info("server is closed, exit resource manager primary election loop") - return - default: - } - - primary, checkAgain := s.participant.CheckLeader() - if checkAgain { - continue - } - if primary != nil { - log.Info("start to watch the primary", zap.Stringer("resource-manager-primary", primary)) - // Watch will keep looping and never return unless the primary/leader has changed. - primary.Watch(s.serverLoopCtx) - log.Info("the resource manager primary has changed, try to re-campaign a primary") - } - - s.campaignLeader() - } -} - -func (s *Server) campaignLeader() { - log.Info("start to campaign the primary/leader", zap.String("campaign-resource-manager-primary-name", s.participant.Name())) - if err := s.participant.CampaignLeader(s.Context(), s.cfg.LeaderLease); err != nil { - if err.Error() == errs.ErrEtcdTxnConflict.Error() { - log.Info("campaign resource manager primary meets error due to txn conflict, another server may campaign successfully", - zap.String("campaign-resource-manager-primary-name", s.participant.Name())) - } else { - log.Error("campaign resource manager primary meets error due to etcd error", - zap.String("campaign-resource-manager-primary-name", s.participant.Name()), - errs.ZapError(err)) - } - return - } - - // Start keepalive the leadership and enable Resource Manager service. - ctx, cancel := context.WithCancel(s.serverLoopCtx) - var resetLeaderOnce sync.Once - defer resetLeaderOnce.Do(func() { - cancel() - s.participant.ResetLeader() - member.ServiceMemberGauge.WithLabelValues(serviceName).Set(0) - }) - - // maintain the leadership, after this, Resource Manager could be ready to provide service. - s.participant.KeepLeader(ctx) - log.Info("campaign resource manager primary ok", zap.String("campaign-resource-manager-primary-name", s.participant.Name())) - - log.Info("triggering the primary callback functions") - for _, cb := range s.primaryCallbacks { - if err := cb(ctx); err != nil { - log.Error("failed to trigger the primary callback function", errs.ZapError(err)) - } - } - - s.participant.EnableLeader() - member.ServiceMemberGauge.WithLabelValues(serviceName).Set(1) - log.Info("resource manager primary is ready to serve", zap.String("resource-manager-primary-name", s.participant.Name())) - - leaderTicker := time.NewTicker(constant.LeaderTickInterval) - defer leaderTicker.Stop() - - for { - select { - case <-leaderTicker.C: - if !s.participant.IsLeader() { - log.Info("no longer a primary/leader because lease has expired, the resource manager primary/leader will step down") - return - } - case <-ctx.Done(): - // Server is closed and it should return nil. - log.Info("server is closed") - return - } - } -} - -// Close closes the server. -func (s *Server) Close() { - if !atomic.CompareAndSwapInt64(&s.isRunning, 1, 0) { - // server is already closed - return - } - - log.Info("closing resource manager server ...") - if err := s.serviceRegister.Deregister(); err != nil { - log.Error("failed to deregister the service", errs.ZapError(err)) - } - utils.StopHTTPServer(s) - utils.StopGRPCServer(s) - s.GetListener().Close() - s.CloseClientConns() - s.serverLoopCancel() - s.serverLoopWg.Wait() - - if s.GetClient() != nil { - if err := s.GetClient().Close(); err != nil { - log.Error("close etcd client meet error", errs.ZapError(errs.ErrCloseEtcdClient, err)) - } - } - - if s.GetHTTPClient() != nil { - s.GetHTTPClient().CloseIdleConnections() - } - - log.Info("resource manager server is closed") -} - -// GetControllerConfig returns the controller config. -func (s *Server) GetControllerConfig() *ControllerConfig { - return &s.cfg.Controller -} - -// IsServing returns whether the server is the leader, if there is embedded etcd, or the primary otherwise. -func (s *Server) IsServing() bool { - return !s.IsClosed() && s.participant.IsLeader() -} - -// IsClosed checks if the server loop is closed -func (s *Server) IsClosed() bool { - return s != nil && atomic.LoadInt64(&s.isRunning) == 0 -} - -// AddServiceReadyCallback adds callbacks when the server becomes the leader, if there is embedded etcd, or the primary otherwise. -func (s *Server) AddServiceReadyCallback(callbacks ...func(context.Context) error) { - s.primaryCallbacks = append(s.primaryCallbacks, callbacks...) -} - -// GetBackendEndpoints returns the backend endpoints. -func (s *Server) GetBackendEndpoints() string { - return s.cfg.BackendEndpoints -} - -// ServerLoopWgDone decreases the server loop wait group. -func (s *Server) ServerLoopWgDone() { - s.serverLoopWg.Done() -} - -// ServerLoopWgAdd increases the server loop wait group. -func (s *Server) ServerLoopWgAdd(n int) { - s.serverLoopWg.Add(n) -} - -// SetUpRestHandler sets up the REST handler. -func (s *Server) SetUpRestHandler() (http.Handler, apiutil.APIServiceGroup) { - return SetUpRestHandler(s.service) -} - -// RegisterGRPCService registers the grpc service. -func (s *Server) RegisterGRPCService(grpcServer *grpc.Server) { - s.service.RegisterGRPCService(grpcServer) -} - -// GetTLSConfig gets the security config. -func (s *Server) GetTLSConfig() *grpcutil.TLSConfig { - return &s.cfg.Security.TLSConfig -} - -// GetLeaderListenUrls gets service endpoints from the leader in election group. -func (s *Server) GetLeaderListenUrls() []string { - return s.participant.GetLeaderListenUrls() -} - -func (s *Server) startServer() (err error) { - // The independent Resource Manager service still reuses PD version info since PD and Resource Manager are just - // different service modes provided by the same pd-server binary - bs.ServerInfoGauge.WithLabelValues(versioninfo.PDReleaseVersion, versioninfo.PDGitHash).Set(float64(time.Now().Unix())) - bs.ServerMaxProcsGauge.Set(float64(runtime.GOMAXPROCS(0))) - - 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.ResourceManagerServiceName) - p := &resource_manager.Participant{ - Name: uniqueName, - Id: uniqueID, // id is unique among all participants - ListenUrls: []string{s.cfg.GetAdvertiseListenAddr()}, - } - s.participant.InitInfo(p, keypath.ResourceManagerSvcRootPath(), constant.PrimaryKey, "primary election") - - s.service = &Service{ - ctx: s.Context(), - manager: NewManager[*Server](s), - } - - if err := s.InitListener(s.GetTLSConfig(), s.cfg.GetListenAddr()); err != nil { - return err - } - - serverReadyChan := make(chan struct{}) - defer close(serverReadyChan) - s.serverLoopWg.Add(1) - go utils.StartGRPCAndHTTPServers(s, serverReadyChan, s.GetListener()) - <-serverReadyChan - - // Run callbacks - log.Info("triggering the start callback functions") - for _, cb := range s.GetStartCallbacks() { - cb() - } - // The start callback function will initialize storage, which will be used in service ready callback. - // We should make sure the calling sequence is right. - s.startServerLoop() - - atomic.StoreInt64(&s.isRunning, 1) - return nil -} - -// CreateServer creates the Server -func CreateServer(ctx context.Context, cfg *Config) *Server { - svr := &Server{ - BaseServer: server.NewBaseServer(ctx), - DiagnosticsServer: sysutil.NewDiagnosticsServer(cfg.Log.File.Filename), - cfg: cfg, - } - return svr -} - -// CreateServerWrapper encapsulates the configuration/log/metrics initialization and create the server -func CreateServerWrapper(cmd *cobra.Command, args []string) { - err := cmd.Flags().Parse(args) - if err != nil { - cmd.Println(err) - return - } - cfg := NewConfig() - flagSet := cmd.Flags() - err = cfg.Parse(flagSet) - defer logutil.LogPanic() - - if err != nil { - cmd.Println(err) - return - } - - if printVersion, err := flagSet.GetBool("version"); err != nil { - cmd.Println(err) - return - } else if printVersion { - versioninfo.Print() - utils.Exit(0) - } - - // New zap logger - err = logutil.SetupLogger(cfg.Log, &cfg.Logger, &cfg.LogProps, cfg.Security.RedactInfoLog) - if err == nil { - log.ReplaceGlobals(cfg.Logger, cfg.LogProps) - } else { - log.Fatal("initialize logger error", errs.ZapError(err)) - } - // Flushing any buffered log entries - log.Sync() - versioninfo.Log(serviceName) - log.Info("resource manager config", zap.Reflect("config", cfg)) - - grpcprometheus.EnableHandlingTimeHistogram() - metricutil.Push(&cfg.Metric) - - ctx, cancel := context.WithCancel(context.Background()) - svr := CreateServer(ctx, cfg) - - sc := make(chan os.Signal, 1) - signal.Notify(sc, - syscall.SIGHUP, - syscall.SIGINT, - syscall.SIGTERM, - syscall.SIGQUIT) - - var sig os.Signal - go func() { - sig = <-sc - cancel() - }() - - if err := svr.Run(); err != nil { - log.Fatal("run server failed", errs.ZapError(err)) - } - - <-ctx.Done() - log.Info("got signal to exit", zap.String("signal", sig.String())) - - svr.Close() - switch sig { - case syscall.SIGTERM: - utils.Exit(0) - default: - utils.Exit(1) - } -} diff --git a/pkg/mcs/resourcemanager/server/testutil.go b/pkg/mcs/resourcemanager/server/testutil.go deleted file mode 100644 index 3577301258c..00000000000 --- a/pkg/mcs/resourcemanager/server/testutil.go +++ /dev/null @@ -1,79 +0,0 @@ -// Copyright 2023 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 server - -import ( - "context" - "os" - - "github.com/pingcap/log" - "github.com/spf13/pflag" - "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/logutil" - "github.com/tikv/pd/pkg/utils/testutil" -) - -// NewTestServer creates a resource manager server for testing. -func NewTestServer(ctx context.Context, re *require.Assertions, cfg *Config) (*Server, testutil.CleanupFunc, error) { - // New zap logger - err := logutil.SetupLogger(cfg.Log, &cfg.Logger, &cfg.LogProps, cfg.Security.RedactInfoLog) - re.NoError(err) - log.ReplaceGlobals(cfg.Logger, cfg.LogProps) - // Flushing any buffered log entries - log.Sync() - - s := CreateServer(ctx, cfg) - if err = s.Run(); err != nil { - return nil, nil, err - } - - cleanup := func() { - s.Close() - os.RemoveAll(cfg.DataDir) - } - return s, cleanup, nil -} - -// GenerateConfig generates a new config with the given options. -func GenerateConfig(c *Config) (*Config, error) { - arguments := []string{ - "--name=" + c.Name, - "--listen-addr=" + c.ListenAddr, - "--advertise-listen-addr=" + c.AdvertiseListenAddr, - "--backend-endpoints=" + c.BackendEndpoints, - } - - flagSet := pflag.NewFlagSet("test", pflag.ContinueOnError) - flagSet.StringP("name", "", "", "human-readable name for this resource manager member") - flagSet.BoolP("version", "V", false, "print version information and exit") - flagSet.StringP("config", "", "", "config file") - flagSet.StringP("backend-endpoints", "", "", "url for etcd client") - flagSet.StringP("listen-addr", "", "", "listen address for resource manager service") - flagSet.StringP("advertise-listen-addr", "", "", "advertise urls for listen address (default '${listen-addr}')") - flagSet.StringP("cacert", "", "", "path of file that contains list of trusted TLS CAs") - flagSet.StringP("cert", "", "", "path of file that contains X509 certificate in PEM format") - flagSet.StringP("key", "", "", "path of file that contains X509 key in PEM format") - err := flagSet.Parse(arguments) - if err != nil { - return nil, err - } - cfg := NewConfig() - err = cfg.Parse(flagSet) - if err != nil { - return nil, err - } - - return cfg, nil -} diff --git a/pkg/mcs/utils/constant/constant.go b/pkg/mcs/utils/constant/constant.go index e8700ffbbc6..87fcf29f678 100644 --- a/pkg/mcs/utils/constant/constant.go +++ b/pkg/mcs/utils/constant/constant.go @@ -61,8 +61,6 @@ const ( APIServiceName = "api" // TSOServiceName is the name of tso server. TSOServiceName = "tso" - // ResourceManagerServiceName is the name of resource manager server. - ResourceManagerServiceName = "resource_manager" // SchedulingServiceName is the name of scheduling server. SchedulingServiceName = "scheduling" // KeyspaceGroupsKey is the path component of keyspace groups. diff --git a/pkg/member/participant.go b/pkg/member/participant.go index 5ef997341de..a513152d9b2 100644 --- a/pkg/member/participant.go +++ b/pkg/member/participant.go @@ -21,7 +21,6 @@ import ( "sync/atomic" "time" - "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" @@ -328,8 +327,6 @@ func NewParticipantByService(serviceName string) (p participant) { p = &tsopb.Participant{} case constant.SchedulingServiceName: p = &schedulingpb.Participant{} - case constant.ResourceManagerServiceName: - p = &resource_manager.Participant{} } return p } diff --git a/pkg/storage/storage_tso_test.go b/pkg/storage/storage_tso_test.go index 15a17e6a41d..07ea79df47c 100644 --- a/pkg/storage/storage_tso_test.go +++ b/pkg/storage/storage_tso_test.go @@ -37,34 +37,6 @@ func TestSaveLoadTimestamp(t *testing.T) { re.Equal(expectedTS, ts) } -func TestGlobalLocalTimestamp(t *testing.T) { - re := require.New(t) - storage, clean := newTestStorage(t) - defer clean() - ltaKey := "lta" - dc1LocationKey, dc2LocationKey := "dc1", "dc2" - localTS1 := time.Now().Round(0) - l1 := path.Join(ltaKey, dc1LocationKey, keypath.TimestampKey) - l2 := path.Join(ltaKey, dc2LocationKey, keypath.TimestampKey) - - err := storage.SaveTimestamp(l1, localTS1) - re.NoError(err) - globalTS := time.Now().Round(0) - err = storage.SaveTimestamp(keypath.TimestampKey, globalTS) - re.NoError(err) - localTS2 := time.Now().Round(0) - err = storage.SaveTimestamp(l2, localTS2) - re.NoError(err) - // return the max ts between global and local - ts, err := storage.LoadTimestamp("") - re.NoError(err) - re.Equal(localTS2, ts) - // return the local ts for a given dc location - ts, err = storage.LoadTimestamp(l1) - re.NoError(err) - re.Equal(localTS1, ts) -} - func TestTimestampTxn(t *testing.T) { re := require.New(t) storage, clean := newTestStorage(t) diff --git a/pkg/utils/keypath/key_path.go b/pkg/utils/keypath/key_path.go index 5f3aafeca36..10934d1be9e 100644 --- a/pkg/utils/keypath/key_path.go +++ b/pkg/utils/keypath/key_path.go @@ -317,12 +317,6 @@ func GetCompiledKeyspaceGroupIDRegexp() *regexp.Regexp { return regexp.MustCompile(pattern) } -// ResourceManagerSvcRootPath returns the root path of resource manager service. -// Path: /ms/{cluster_id}/resource_manager -func ResourceManagerSvcRootPath() string { - return svcRootPath(constant.ResourceManagerServiceName) -} - // SchedulingSvcRootPath returns the root path of scheduling service. // Path: /ms/{cluster_id}/scheduling func SchedulingSvcRootPath() string { diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index aef36eaec97..b0e600f9c7f 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -31,8 +31,6 @@ import ( "time" "github.com/docker/go-units" - "github.com/opentracing/basictracer-go" - "github.com/opentracing/opentracing-go" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/meta_storagepb" "github.com/pingcap/kvproto/pkg/metapb" @@ -41,14 +39,11 @@ import ( "github.com/stretchr/testify/suite" pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/caller" - clierrs "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/retry" "github.com/tikv/pd/pkg/core" - "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/mock/mockid" "github.com/tikv/pd/pkg/storage/endpoint" - "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/testutil" @@ -253,70 +248,6 @@ func TestGetTSAfterTransferLeader(t *testing.T) { re.NoError(err) } -func TestTSOAllocatorLeader(t *testing.T) { - re := require.New(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-2", - "pd3": "dc-3", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - re.NoError(err) - defer cluster.Destroy() - - err = cluster.RunInitialServers() - re.NoError(err) - cluster.WaitAllLeaders(re, dcLocationConfig) - - var ( - testServers = cluster.GetServers() - endpoints = make([]string, 0, len(testServers)) - endpointsMap = make(map[string]string) - ) - for _, s := range testServers { - endpoints = append(endpoints, s.GetConfig().AdvertiseClientUrls) - endpointsMap[s.GetServer().GetMemberInfo().GetName()] = s.GetConfig().AdvertiseClientUrls - } - var allocatorLeaderMap = make(map[string]string) - for _, dcLocation := range dcLocationConfig { - var pdName string - testutil.Eventually(re, func() bool { - pdName = cluster.WaitAllocatorLeader(dcLocation) - return len(pdName) > 0 - }) - allocatorLeaderMap[dcLocation] = pdName - } - cli := setupCli(ctx, re, endpoints) - defer cli.Close() - innerCli, ok := cli.(interface{ GetServiceDiscovery() pd.ServiceDiscovery }) - re.True(ok) - - // Check allocator leaders URL map. - cli.Close() - for dcLocation, url := range getTSOAllocatorServingEndpointURLs(cli.(TSOAllocatorsGetter)) { - if dcLocation == tso.GlobalDCLocation { - urls := innerCli.GetServiceDiscovery().GetServiceURLs() - sort.Strings(urls) - sort.Strings(endpoints) - re.Equal(endpoints, urls) - continue - } - pdName, exist := allocatorLeaderMap[dcLocation] - re.True(exist) - re.NotEmpty(pdName) - pdURL, exist := endpointsMap[pdName] - re.True(exist) - re.NotEmpty(pdURL) - re.Equal(pdURL, url) - } -} - func TestTSOFollowerProxy(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) @@ -481,144 +412,6 @@ func TestUnavailableTimeAfterLeaderIsReady(t *testing.T) { re.Less(maxUnavailableTime.UnixMilli(), leaderReadyTime.Add(1*time.Second).UnixMilli()) } -// TODO: migrate the Local/Global TSO tests to TSO integration test folder. -func TestGlobalAndLocalTSO(t *testing.T) { - re := require.New(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-2", - "pd3": "dc-3", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - re.NoError(err) - defer cluster.Destroy() - - endpoints := runServer(re, cluster) - cli := setupCli(ctx, re, endpoints) - defer cli.Close() - - // Wait for all nodes becoming healthy. - time.Sleep(time.Second * 5) - - // Join a new dc-location - pd4, err := cluster.Join(ctx, func(conf *config.Config, _ string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = "dc-4" - }) - re.NoError(err) - err = pd4.Run() - re.NoError(err) - dcLocationConfig["pd4"] = "dc-4" - cluster.CheckClusterDCLocation() - cluster.WaitAllLeaders(re, dcLocationConfig) - - // Test a nonexistent dc-location for Local TSO - p, l, err := cli.GetLocalTS(context.TODO(), "nonexistent-dc") - re.Equal(int64(0), p) - re.Equal(int64(0), l, int64(0)) - re.Error(err) - re.Contains(err.Error(), "unknown dc-location") - - wg := &sync.WaitGroup{} - requestGlobalAndLocalTSO(re, wg, dcLocationConfig, cli) - - // assert global tso after resign leader - re.NoError(failpoint.Enable("github.com/tikv/pd/client/skipUpdateMember", `return(true)`)) - err = cluster.ResignLeader() - re.NoError(err) - re.NotEmpty(cluster.WaitLeader()) - _, _, err = cli.GetTS(ctx) - re.Error(err) - re.True(clierrs.IsLeaderChange(err)) - _, _, err = cli.GetTS(ctx) - re.NoError(err) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/skipUpdateMember")) - - recorder := basictracer.NewInMemoryRecorder() - tracer := basictracer.New(recorder) - span := tracer.StartSpan("trace") - ctx = opentracing.ContextWithSpan(ctx, span) - future := cli.GetLocalTSAsync(ctx, "error-dc") - spans := recorder.GetSpans() - re.Len(spans, 1) - _, _, err = future.Wait() - re.Error(err) - spans = recorder.GetSpans() - re.Len(spans, 1) - _, _, err = cli.GetTS(ctx) - re.NoError(err) - spans = recorder.GetSpans() - re.Len(spans, 3) - - // Test the TSO follower proxy while enabling the Local TSO. - cli.UpdateOption(pd.EnableTSOFollowerProxy, true) - // Sleep a while here to prevent from canceling the ongoing TSO request. - time.Sleep(time.Millisecond * 50) - requestGlobalAndLocalTSO(re, wg, dcLocationConfig, cli) - cli.UpdateOption(pd.EnableTSOFollowerProxy, false) - time.Sleep(time.Millisecond * 50) - requestGlobalAndLocalTSO(re, wg, dcLocationConfig, cli) -} - -func requestGlobalAndLocalTSO( - re *require.Assertions, - wg *sync.WaitGroup, - dcLocationConfig map[string]string, - cli pd.Client, -) { - for _, dcLocation := range dcLocationConfig { - wg.Add(tsoRequestConcurrencyNumber) - for range tsoRequestConcurrencyNumber { - go func(dc string) { - defer wg.Done() - var lastTS uint64 - for range tsoRequestRound { - globalPhysical1, globalLogical1, err := cli.GetTS(context.TODO()) - // The allocator leader may be changed due to the environment issue. - if err != nil { - re.ErrorContains(err, errs.NotLeaderErr) - } - globalTS1 := tsoutil.ComposeTS(globalPhysical1, globalLogical1) - localPhysical, localLogical, err := cli.GetLocalTS(context.TODO(), dc) - if err != nil { - re.ErrorContains(err, errs.NotLeaderErr) - } - localTS := tsoutil.ComposeTS(localPhysical, localLogical) - globalPhysical2, globalLogical2, err := cli.GetTS(context.TODO()) - if err != nil { - re.ErrorContains(err, errs.NotLeaderErr) - } - globalTS2 := tsoutil.ComposeTS(globalPhysical2, globalLogical2) - re.Less(lastTS, globalTS1) - re.Less(globalTS1, localTS) - re.Less(localTS, globalTS2) - lastTS = globalTS2 - } - re.Positive(lastTS) - }(dcLocation) - } - } - wg.Wait() -} - -// GetTSOAllocators defines the TSO allocators getter. -type TSOAllocatorsGetter interface{ GetTSOAllocators() *sync.Map } - -func getTSOAllocatorServingEndpointURLs(c TSOAllocatorsGetter) map[string]string { - allocatorLeaders := make(map[string]string) - c.GetTSOAllocators().Range(func(dcLocation, url any) bool { - allocatorLeaders[dcLocation.(string)] = url.(string) - return true - }) - return allocatorLeaders -} - func TestCustomTimeout(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) diff --git a/tests/integrations/go.mod b/tests/integrations/go.mod index 7638cb6e05b..87fa96ec1d3 100644 --- a/tests/integrations/go.mod +++ b/tests/integrations/go.mod @@ -125,8 +125,7 @@ require ( github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/oleiade/reflections v1.0.1 // indirect - github.com/opentracing/basictracer-go v1.1.0 - github.com/opentracing/opentracing-go v1.2.0 + github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pelletier/go-toml/v2 v2.2.2 // indirect github.com/petermattis/goid v0.0.0-20240813172612-4fcff4a6cae7 // indirect github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d // indirect diff --git a/tests/integrations/go.sum b/tests/integrations/go.sum index a57f20ce70b..ce37ebeea19 100644 --- a/tests/integrations/go.sum +++ b/tests/integrations/go.sum @@ -1344,8 +1344,6 @@ github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042 github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.20.1 h1:PA/3qinGoukvymdIDV8pii6tiZgC8kbmJO6Z5+b002Q= github.com/onsi/gomega v1.20.1/go.mod h1:DtrZpjmvpn2mPm4YWQa0/ALMDj9v4YxLgojwPeREyVo= -github.com/opentracing/basictracer-go v1.1.0 h1:Oa1fTSBvAl8pa3U+IJYqrKm0NALwH9OsgwOqDv4xJW0= -github.com/opentracing/basictracer-go v1.1.0/go.mod h1:V2HZueSJEp879yv285Aap1BS69fQMD+MNP1mRs6mBQc= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -1968,7 +1966,6 @@ golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190206041539-40960b6deb8e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= diff --git a/tests/integrations/mcs/discovery/register_test.go b/tests/integrations/mcs/discovery/register_test.go index 63a40a1fb5b..147e16530b7 100644 --- a/tests/integrations/mcs/discovery/register_test.go +++ b/tests/integrations/mcs/discovery/register_test.go @@ -156,8 +156,6 @@ func (suite *serverRegisterTestSuite) addServer(serviceName string) (bs.Server, switch serviceName { case constant.TSOServiceName: return tests.StartSingleTSOTestServer(suite.ctx, re, suite.backendEndpoints, tempurl.Alloc()) - case constant.ResourceManagerServiceName: - return tests.StartSingleResourceManagerTestServer(suite.ctx, re, suite.backendEndpoints, tempurl.Alloc()) default: return nil, nil } diff --git a/tests/integrations/mcs/resourcemanager/server_test.go b/tests/integrations/mcs/resourcemanager/server_test.go deleted file mode 100644 index 7ca83ad6bac..00000000000 --- a/tests/integrations/mcs/resourcemanager/server_test.go +++ /dev/null @@ -1,124 +0,0 @@ -// Copyright 2023 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 resourcemanager_test - -import ( - "context" - "encoding/json" - "io" - "net/http" - "strings" - "testing" - - rmpb "github.com/pingcap/kvproto/pkg/resource_manager" - "github.com/stretchr/testify/require" - "github.com/tikv/pd/client/utils/grpcutil" - bs "github.com/tikv/pd/pkg/basicserver" - "github.com/tikv/pd/pkg/utils/tempurl" - "github.com/tikv/pd/pkg/versioninfo" - "github.com/tikv/pd/tests" -) - -func TestResourceManagerServer(t *testing.T) { - re := require.New(t) - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cluster, err := tests.NewTestAPICluster(ctx, 1) - defer cluster.Destroy() - re.NoError(err) - - err = cluster.RunInitialServers() - re.NoError(err) - - leaderName := cluster.WaitLeader() - re.NotEmpty(leaderName) - leader := cluster.GetServer(leaderName) - - s, cleanup := tests.StartSingleResourceManagerTestServer(ctx, re, leader.GetAddr(), tempurl.Alloc()) - addr := s.GetAddr() - defer cleanup() - tests.WaitForPrimaryServing(re, map[string]bs.Server{addr: s}) - - // Test registered GRPC Service - cc, err := grpcutil.GetClientConn(ctx, addr, nil) - re.NoError(err) - defer cc.Close() - - c := rmpb.NewResourceManagerClient(cc) - _, err = c.GetResourceGroup(context.Background(), &rmpb.GetResourceGroupRequest{ - ResourceGroupName: "pingcap", - }) - re.ErrorContains(err, "resource group not found") - - // Test registered REST HTTP Handler - url := addr + "/resource-manager/api/v1/config" - { - resp, err := tests.TestDialClient.Get(url + "/groups") - re.NoError(err) - defer resp.Body.Close() - re.Equal(http.StatusOK, resp.StatusCode) - respString, err := io.ReadAll(resp.Body) - re.NoError(err) - re.JSONEq(`[{"name":"default","mode":1,"r_u_settings":{"r_u":{"settings":{"fill_rate":2147483647,"burst_limit":-1},"state":{"initialized":false}}},"priority":8}]`, string(respString)) - } - { - group := &rmpb.ResourceGroup{ - Name: "pingcap", - Mode: 1, - } - createJSON, err := json.Marshal(group) - re.NoError(err) - resp, err := tests.TestDialClient.Post(url+"/group", "application/json", strings.NewReader(string(createJSON))) - re.NoError(err) - defer resp.Body.Close() - re.Equal(http.StatusOK, resp.StatusCode) - } - { - resp, err := tests.TestDialClient.Get(url + "/group/pingcap") - re.NoError(err) - defer resp.Body.Close() - re.Equal(http.StatusOK, resp.StatusCode) - respString, err := io.ReadAll(resp.Body) - re.NoError(err) - re.JSONEq("{\"name\":\"pingcap\",\"mode\":1,\"r_u_settings\":{\"r_u\":{\"state\":{\"initialized\":false}}},\"priority\":0}", string(respString)) - } - - // Test metrics handler - { - resp, err := tests.TestDialClient.Get(addr + "/metrics") - re.NoError(err) - defer resp.Body.Close() - re.Equal(http.StatusOK, resp.StatusCode) - respBytes, err := io.ReadAll(resp.Body) - re.NoError(err) - re.Contains(string(respBytes), "pd_server_info") - } - - // Test status handler - { - resp, err := tests.TestDialClient.Get(addr + "/status") - re.NoError(err) - defer resp.Body.Close() - re.Equal(http.StatusOK, resp.StatusCode) - respBytes, err := io.ReadAll(resp.Body) - re.NoError(err) - var s versioninfo.Status - re.NoError(json.Unmarshal(respBytes, &s)) - re.Equal(versioninfo.PDBuildTS, s.BuildTS) - re.Equal(versioninfo.PDGitHash, s.GitHash) - re.Equal(versioninfo.PDReleaseVersion, s.Version) - } -} diff --git a/tests/integrations/mcs/scheduling/api_test.go b/tests/integrations/mcs/scheduling/api_test.go index 443bee2cd6a..f3e7f235018 100644 --- a/tests/integrations/mcs/scheduling/api_test.go +++ b/tests/integrations/mcs/scheduling/api_test.go @@ -508,6 +508,8 @@ func (suite *apiTestSuite) checkAdminRegionCacheForward(cluster *tests.TestClust func (suite *apiTestSuite) TestFollowerForward() { suite.env.RunTestBasedOnMode(suite.checkFollowerForward) + suite.TearDownSuite() + suite.SetupSuite() } func (suite *apiTestSuite) checkFollowerForward(cluster *tests.TestCluster) { @@ -517,20 +519,6 @@ func (suite *apiTestSuite) checkFollowerForward(cluster *tests.TestCluster) { defer cancel() follower, err := cluster.JoinAPIServer(ctx) re.NoError(err) - defer func() { - leader := cluster.GetLeaderServer() - cli := leader.GetEtcdClient() - testutil.Eventually(re, func() bool { - _, err = cli.MemberRemove(context.Background(), follower.GetServer().GetMember().ID()) - return err == nil - }) - testutil.Eventually(re, func() bool { - res, err := cli.MemberList(context.Background()) - return err == nil && len(res.Members) == 1 - }) - cluster.DeleteServer(follower.GetConfig().Name) - follower.Destroy() - }() re.NoError(follower.Run()) re.NotEmpty(cluster.WaitLeader()) diff --git a/tests/integrations/mcs/tso/proxy_test.go b/tests/integrations/mcs/tso/proxy_test.go index 9dfd232348d..92aede7d8e1 100644 --- a/tests/integrations/mcs/tso/proxy_test.go +++ b/tests/integrations/mcs/tso/proxy_test.go @@ -342,8 +342,8 @@ func (s *tsoProxyTestSuite) verifyTSOProxy( re.Equal(req.GetCount(), resp.GetCount()) ts := resp.GetTimestamp() count := int64(resp.GetCount()) - physical, largestLogic, suffixBits := ts.GetPhysical(), ts.GetLogical(), ts.GetSuffixBits() - firstLogical := tsoutil.AddLogical(largestLogic, -count+1, suffixBits) + physical, largestLogic := ts.GetPhysical(), ts.GetLogical() + firstLogical := largestLogic - count + 1 re.False(tsoutil.TSLessEqual(physical, firstLogical, lastPhysical, lastLogical)) } }(i) diff --git a/tests/integrations/tso/testutil.go b/tests/integrations/tso/testutil.go index ae81f0fbe2b..5eda641486f 100644 --- a/tests/integrations/tso/testutil.go +++ b/tests/integrations/tso/testutil.go @@ -35,6 +35,6 @@ func checkAndReturnTimestampResponse[T tsoResponse](re *require.Assertions, resp re.Equal(uint32(tsoCount), resp.GetCount()) timestamp := resp.GetTimestamp() re.Positive(timestamp.GetPhysical()) - re.GreaterOrEqual(uint32(timestamp.GetLogical())>>timestamp.GetSuffixBits(), uint32(tsoCount)) + re.GreaterOrEqual(uint32(timestamp.GetLogical()), uint32(tsoCount)) return timestamp } diff --git a/tests/server/api/api_test.go b/tests/server/api/api_test.go index 828213587d6..14df5ff8eea 100644 --- a/tests/server/api/api_test.go +++ b/tests/server/api/api_test.go @@ -938,6 +938,8 @@ func TestPreparingProgress(t *testing.T) { defer cancel() cluster, err := tests.NewTestCluster(ctx, 1, func(conf *config.Config, _ string) { conf.Replication.MaxReplicas = 1 + // prevent scheduling + conf.Schedule.RegionScheduleLimit = 0 }) re.NoError(err) defer cluster.Destroy() @@ -1008,28 +1010,15 @@ func TestPreparingProgress(t *testing.T) { for _, store := range stores[2:] { tests.MustPutStore(re, cluster, store) } - // no store preparing - output := sendRequest(re, leader.GetAddr()+"/pd/api/v1/stores/progress?action=preparing", http.MethodGet, http.StatusNotFound) - re.Contains(string(output), "no progress found for the action") - output = sendRequest(re, leader.GetAddr()+"/pd/api/v1/stores/progress?id=4", http.MethodGet, http.StatusNotFound) - re.Contains(string(output), "no progress found for the given store ID") if !leader.GetRaftCluster().IsPrepared() { testutil.Eventually(re, func() bool { if leader.GetRaftCluster().IsPrepared() { return true } - url := leader.GetAddr() + "/pd/api/v1/stores/progress?action=preparing" - req, _ := http.NewRequest(http.MethodGet, url, http.NoBody) - resp, err := tests.TestDialClient.Do(req) - re.NoError(err) - defer resp.Body.Close() - if resp.StatusCode != http.StatusNotFound { - return false - } - // is not prepared - output, err := io.ReadAll(resp.Body) - re.NoError(err) + + // no store preparing + output := sendRequest(re, leader.GetAddr()+"/pd/api/v1/stores/progress?action=preparing", http.MethodGet, http.StatusNotFound) re.Contains(string(output), "no progress found for the action") output = sendRequest(re, leader.GetAddr()+"/pd/api/v1/stores/progress?id=4", http.MethodGet, http.StatusNotFound) re.Contains(string(output), "no progress found for the given store ID") @@ -1092,7 +1081,7 @@ func TestPreparingProgress(t *testing.T) { return true }) - output = sendRequest(re, leader.GetAddr()+"/pd/api/v1/stores/progress?id=4", http.MethodGet, http.StatusOK) + output := sendRequest(re, leader.GetAddr()+"/pd/api/v1/stores/progress?id=4", http.MethodGet, http.StatusOK) re.NoError(json.Unmarshal(output, &p)) re.Equal("preparing", p.Action) re.Equal("0.05", fmt.Sprintf("%.2f", p.Progress)) diff --git a/tests/server/cluster/cluster_test.go b/tests/server/cluster/cluster_test.go index 0620b624c39..dac13fa21ba 100644 --- a/tests/server/cluster/cluster_test.go +++ b/tests/server/cluster/cluster_test.go @@ -589,7 +589,9 @@ func TestRaftClusterMultipleRestart(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - tc, err := tests.NewTestCluster(ctx, 1) + tc, err := tests.NewTestCluster(ctx, 1, func(conf *config.Config, _ string) { + conf.LeaderLease = 300 + }) defer tc.Destroy() re.NoError(err) diff --git a/tests/server/member/member_test.go b/tests/server/member/member_test.go index 13f7fffe083..8c6ca6f76bf 100644 --- a/tests/server/member/member_test.go +++ b/tests/server/member/member_test.go @@ -47,16 +47,7 @@ func TestMemberDelete(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-2", - "pd3": "dc-3", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) + cluster, err := tests.NewTestCluster(ctx, 3) defer cluster.Destroy() re.NoError(err) diff --git a/tests/server/tso/allocator_test.go b/tests/server/tso/allocator_test.go deleted file mode 100644 index 257cd3b6a34..00000000000 --- a/tests/server/tso/allocator_test.go +++ /dev/null @@ -1,227 +0,0 @@ -// Copyright 2020 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. - -//go:build tso_full_test || tso_function_test -// +build tso_full_test tso_function_test - -package tso_test - -import ( - "context" - "strconv" - "sync" - "testing" - "time" - - "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/slice" - "github.com/tikv/pd/pkg/tso" - "github.com/tikv/pd/pkg/utils/etcdutil" - "github.com/tikv/pd/pkg/utils/testutil" - "github.com/tikv/pd/server/config" - "github.com/tikv/pd/tests" -) - -func TestAllocatorLeader(t *testing.T) { - re := require.New(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - // There will be three Local TSO Allocator leaders elected - dcLocationConfig := map[string]string{ - "pd2": "dc-1", - "pd4": "dc-2", - "pd6": "leader", /* Test dc-location name is same as the special key */ - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(ctx, dcLocationNum*2, func(conf *config.Config, serverName string) { - if zoneLabel, ok := dcLocationConfig[serverName]; ok { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = zoneLabel - } - }) - re.NoError(err) - defer cluster.Destroy() - re.NoError(cluster.RunInitialServers()) - cluster.WaitAllLeaders(re, dcLocationConfig) - // To check whether we have enough Local TSO Allocator leaders - allAllocatorLeaders := make([]tso.Allocator, 0, dcLocationNum) - for _, server := range cluster.GetServers() { - // Filter out Global TSO Allocator and Local TSO Allocator followers - allocators := server.GetTSOAllocatorManager().GetAllocators( - tso.FilterDCLocation(tso.GlobalDCLocation), - tso.FilterUnavailableLeadership(), - tso.FilterUninitialized()) - // One PD server will have at most three initialized Local TSO Allocators, - // which also means three allocator leaders - re.LessOrEqual(len(allocators), dcLocationNum) - if len(allocators) == 0 { - continue - } - if len(allAllocatorLeaders) == 0 { - allAllocatorLeaders = append(allAllocatorLeaders, allocators...) - continue - } - for _, allocator := range allocators { - if slice.NoneOf(allAllocatorLeaders, func(i int) bool { return allAllocatorLeaders[i] == allocator }) { - allAllocatorLeaders = append(allAllocatorLeaders, allocator) - } - } - } - // At the end, we should have three initialized Local TSO Allocator, - // i.e., the Local TSO Allocator leaders for all dc-locations in testDCLocations - re.Len(allAllocatorLeaders, dcLocationNum) - allocatorLeaderMemberIDs := make([]uint64, 0, dcLocationNum) - for _, allocator := range allAllocatorLeaders { - allocatorLeader, _ := allocator.(*tso.LocalTSOAllocator) - allocatorLeaderMemberIDs = append(allocatorLeaderMemberIDs, allocatorLeader.GetMember().ID()) - } - for _, server := range cluster.GetServers() { - // Filter out Global TSO Allocator - allocators := server.GetTSOAllocatorManager().GetAllocators(tso.FilterDCLocation(tso.GlobalDCLocation)) - if _, ok := dcLocationConfig[server.GetServer().Name()]; !ok { - re.Empty(allocators) - continue - } - re.Len(allocators, dcLocationNum) - for _, allocator := range allocators { - allocatorFollower, _ := allocator.(*tso.LocalTSOAllocator) - allocatorFollowerMemberID := allocatorFollower.GetAllocatorLeader().GetMemberId() - re.True( - slice.AnyOf( - allocatorLeaderMemberIDs, - func(i int) bool { return allocatorLeaderMemberIDs[i] == allocatorFollowerMemberID }, - ), - ) - } - } -} - -func TestPriorityAndDifferentLocalTSO(t *testing.T) { - re := require.New(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-2", - "pd3": "dc-3", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - defer cluster.Destroy() - re.NoError(err) - re.NoError(cluster.RunInitialServers()) - - cluster.WaitAllLeaders(re, dcLocationConfig) - leaderServer := cluster.GetLeaderServer() - re.NotNil(leaderServer) - leaderServer.BootstrapCluster() - - // Wait for all nodes becoming healthy. - time.Sleep(time.Second * 5) - - // Join a new dc-location - pd4, err := cluster.Join(ctx, func(conf *config.Config, _ string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = "dc-4" - }) - re.NoError(err) - re.NoError(pd4.Run()) - dcLocationConfig["pd4"] = "dc-4" - cluster.CheckClusterDCLocation() - re.NotEqual("", cluster.WaitAllocatorLeader( - "dc-4", - tests.WithRetryTimes(90), tests.WithWaitInterval(time.Second), - )) - - // Scatter the Local TSO Allocators to different servers - waitAllocatorPriorityCheck(cluster) - cluster.WaitAllLeaders(re, dcLocationConfig) - - // Before the priority is checked, we may have allocators typology like this: - // pd1: dc-1, dc-2 and dc-3 allocator leader - // pd2: None - // pd3: None - // pd4: dc-4 allocator leader - // After the priority is checked, we should have allocators typology like this: - // pd1: dc-1 allocator leader - // pd2: dc-2 allocator leader - // pd3: dc-3 allocator leader - // pd4: dc-4 allocator leader - wg := sync.WaitGroup{} - wg.Add(len(dcLocationConfig)) - for serverName, dcLocation := range dcLocationConfig { - go func(name, dc string) { - defer wg.Done() - testutil.Eventually(re, func() bool { - return cluster.WaitAllocatorLeader(dc) == name - }, testutil.WithWaitFor(90*time.Second), testutil.WithTickInterval(time.Second)) - }(serverName, dcLocation) - } - wg.Wait() - - for serverName, server := range cluster.GetServers() { - tsoAllocatorManager := server.GetTSOAllocatorManager() - localAllocatorLeaders, err := tsoAllocatorManager.GetHoldingLocalAllocatorLeaders() - re.NoError(err) - for _, localAllocatorLeader := range localAllocatorLeaders { - testTSOSuffix(re, cluster, tsoAllocatorManager, localAllocatorLeader.GetDCLocation()) - } - if serverName == cluster.GetLeader() { - testTSOSuffix(re, cluster, tsoAllocatorManager, tso.GlobalDCLocation) - } - } -} - -func waitAllocatorPriorityCheck(cluster *tests.TestCluster) { - wg := sync.WaitGroup{} - for _, server := range cluster.GetServers() { - wg.Add(1) - go func(s *tests.TestServer) { - s.GetTSOAllocatorManager().PriorityChecker() - wg.Done() - }(server) - } - wg.Wait() -} - -func testTSOSuffix(re *require.Assertions, cluster *tests.TestCluster, am *tso.AllocatorManager, dcLocation string) { - suffixBits := am.GetSuffixBits() - re.Greater(suffixBits, 0) - var suffix int64 - // The suffix of a Global TSO will always be 0 - if dcLocation != tso.GlobalDCLocation { - suffixResp, err := etcdutil.EtcdKVGet( - cluster.GetEtcdClient(), - am.GetLocalTSOSuffixPath(dcLocation)) - re.NoError(err) - re.Len(suffixResp.Kvs, 1) - suffix, err = strconv.ParseInt(string(suffixResp.Kvs[0].Value), 10, 64) - re.NoError(err) - re.GreaterOrEqual(suffixBits, tso.CalSuffixBits(int32(suffix))) - } - allocator, err := am.GetAllocator(dcLocation) - re.NoError(err) - var tso pdpb.Timestamp - testutil.Eventually(re, func() bool { - tso, err = allocator.GenerateTSO(context.Background(), 1) - re.NoError(err) - return tso.GetPhysical() != 0 - }) - // Test whether the TSO has the right suffix - re.Equal(suffix, tso.Logical&((1<>timestamp.GetSuffixBits(), req.GetCount()) - return timestamp -} - -func testGetTimestamp(re *require.Assertions, ctx context.Context, pdCli pdpb.PDClient, req *pdpb.TsoRequest) *pdpb.Timestamp { - tsoClient, err := pdCli.Tso(ctx) - re.NoError(err) - defer tsoClient.CloseSend() - re.NoError(tsoClient.Send(req)) - resp, err := tsoClient.Recv() - re.NoError(err) - return checkAndReturnTimestampResponse(re, req, resp) -} - -func TestMain(m *testing.M) { - goleak.VerifyTestMain(m, testutil.LeakOptions...) -} diff --git a/tests/server/tso/consistency_test.go b/tests/server/tso/consistency_test.go deleted file mode 100644 index c7acc69fa60..00000000000 --- a/tests/server/tso/consistency_test.go +++ /dev/null @@ -1,344 +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. - -//go:build tso_full_test || tso_consistency_test -// +build tso_full_test tso_consistency_test - -package tso_test - -import ( - "context" - "sync" - "testing" - "time" - - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - "github.com/tikv/pd/pkg/tso" - "github.com/tikv/pd/pkg/utils/grpcutil" - "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/tikv/pd/pkg/utils/testutil" - "github.com/tikv/pd/pkg/utils/tsoutil" - "github.com/tikv/pd/server/config" - "github.com/tikv/pd/tests" -) - -type tsoConsistencyTestSuite struct { - suite.Suite - ctx context.Context - cancel context.CancelFunc - - leaderServer *tests.TestServer - dcClientMap map[string]pdpb.PDClient - - tsPoolMutex syncutil.Mutex - tsPool map[uint64]struct{} -} - -func TestTSOConsistencyTestSuite(t *testing.T) { - suite.Run(t, new(tsoConsistencyTestSuite)) -} - -func (suite *tsoConsistencyTestSuite) SetupSuite() { - suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.dcClientMap = make(map[string]pdpb.PDClient) - suite.tsPool = make(map[uint64]struct{}) -} - -func (suite *tsoConsistencyTestSuite) TearDownSuite() { - suite.cancel() -} - -// TestSynchronizedGlobalTSO is used to test the synchronized way of global TSO generation. -func (suite *tsoConsistencyTestSuite) TestSynchronizedGlobalTSO() { - re := suite.Require() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-2", - "pd3": "dc-3", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(suite.ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - defer cluster.Destroy() - re.NoError(err) - re.NoError(cluster.RunInitialServers()) - - cluster.WaitAllLeaders(re, dcLocationConfig) - - suite.leaderServer = cluster.GetLeaderServer() - re.NotNil(suite.leaderServer) - suite.dcClientMap[tso.GlobalDCLocation] = testutil.MustNewGrpcClient(re, suite.leaderServer.GetAddr()) - for _, dcLocation := range dcLocationConfig { - pdName := suite.leaderServer.GetAllocatorLeader(dcLocation).GetName() - suite.dcClientMap[dcLocation] = testutil.MustNewGrpcClient(re, cluster.GetServer(pdName).GetAddr()) - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - maxGlobalTSO := &pdpb.Timestamp{} - for range tsoRequestRound { - // Get some local TSOs first - oldLocalTSOs := make([]*pdpb.Timestamp, 0, dcLocationNum) - for _, dcLocation := range dcLocationConfig { - localTSO := suite.getTimestampByDC(ctx, re, cluster, dcLocation) - oldLocalTSOs = append(oldLocalTSOs, localTSO) - re.Equal(-1, tsoutil.CompareTimestamp(maxGlobalTSO, localTSO)) - } - // Get a global TSO then - globalTSO := suite.getTimestampByDC(ctx, re, cluster, tso.GlobalDCLocation) - for _, oldLocalTSO := range oldLocalTSOs { - re.Equal(1, tsoutil.CompareTimestamp(globalTSO, oldLocalTSO)) - } - if tsoutil.CompareTimestamp(maxGlobalTSO, globalTSO) < 0 { - maxGlobalTSO = globalTSO - } - // Get some local TSOs again - newLocalTSOs := make([]*pdpb.Timestamp, 0, dcLocationNum) - for _, dcLocation := range dcLocationConfig { - newLocalTSOs = append(newLocalTSOs, suite.getTimestampByDC(ctx, re, cluster, dcLocation)) - } - for _, newLocalTSO := range newLocalTSOs { - re.Equal(-1, tsoutil.CompareTimestamp(maxGlobalTSO, newLocalTSO)) - } - } -} - -func (suite *tsoConsistencyTestSuite) getTimestampByDC( - ctx context.Context, - re *require.Assertions, - cluster *tests.TestCluster, - dcLocation string, -) *pdpb.Timestamp { - req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(suite.leaderServer.GetClusterID()), - Count: tsoCount, - DcLocation: dcLocation, - } - pdClient, ok := suite.dcClientMap[dcLocation] - re.True(ok) - forwardedHost := cluster.GetServer(suite.leaderServer.GetAllocatorLeader(dcLocation).GetName()).GetAddr() - ctx = grpcutil.BuildForwardContext(ctx, forwardedHost) - tsoClient, err := pdClient.Tso(ctx) - re.NoError(err) - defer tsoClient.CloseSend() - re.NoError(tsoClient.Send(req)) - resp, err := tsoClient.Recv() - re.NoError(err) - return checkAndReturnTimestampResponse(re, req, resp) -} - -func (suite *tsoConsistencyTestSuite) TestSynchronizedGlobalTSOOverflow() { - re := suite.Require() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-2", - "pd3": "dc-3", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(suite.ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - defer cluster.Destroy() - re.NoError(err) - re.NoError(cluster.RunInitialServers()) - - cluster.WaitAllLeaders(re, dcLocationConfig) - - suite.leaderServer = cluster.GetLeaderServer() - re.NotNil(suite.leaderServer) - suite.dcClientMap[tso.GlobalDCLocation] = testutil.MustNewGrpcClient(re, suite.leaderServer.GetAddr()) - for _, dcLocation := range dcLocationConfig { - pdName := suite.leaderServer.GetAllocatorLeader(dcLocation).GetName() - suite.dcClientMap[dcLocation] = testutil.MustNewGrpcClient(re, cluster.GetServer(pdName).GetAddr()) - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/globalTSOOverflow", `return(true)`)) - suite.getTimestampByDC(ctx, re, cluster, tso.GlobalDCLocation) - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/globalTSOOverflow")) -} - -func (suite *tsoConsistencyTestSuite) TestLocalAllocatorLeaderChange() { - re := suite.Require() - re.NoError(failpoint.Enable("github.com/tikv/pd/server/mockLocalAllocatorLeaderChange", `return(true)`)) - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(suite.ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - defer cluster.Destroy() - re.NoError(err) - re.NoError(cluster.RunInitialServers()) - - cluster.WaitAllLeaders(re, dcLocationConfig) - - suite.leaderServer = cluster.GetLeaderServer() - re.NotNil(suite.leaderServer) - suite.dcClientMap[tso.GlobalDCLocation] = testutil.MustNewGrpcClient(re, suite.leaderServer.GetAddr()) - for _, dcLocation := range dcLocationConfig { - pdName := suite.leaderServer.GetAllocatorLeader(dcLocation).GetName() - suite.dcClientMap[dcLocation] = testutil.MustNewGrpcClient(re, cluster.GetServer(pdName).GetAddr()) - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - suite.getTimestampByDC(ctx, re, cluster, tso.GlobalDCLocation) - re.NoError(failpoint.Disable("github.com/tikv/pd/server/mockLocalAllocatorLeaderChange")) -} - -func (suite *tsoConsistencyTestSuite) TestLocalTSO() { - re := suite.Require() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-2", - "pd3": "dc-3", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(suite.ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - defer cluster.Destroy() - re.NoError(err) - re.NoError(cluster.RunInitialServers()) - - cluster.WaitAllLeaders(re, dcLocationConfig) - suite.testTSO(cluster, dcLocationConfig, nil) -} - -func (suite *tsoConsistencyTestSuite) checkTSOUnique(tso *pdpb.Timestamp) bool { - suite.tsPoolMutex.Lock() - defer suite.tsPoolMutex.Unlock() - ts := tsoutil.GenerateTS(tso) - if _, exist := suite.tsPool[ts]; exist { - return false - } - suite.tsPool[ts] = struct{}{} - return true -} - -func (suite *tsoConsistencyTestSuite) TestLocalTSOAfterMemberChanged() { - re := suite.Require() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-2", - "pd3": "dc-3", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(suite.ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - defer cluster.Destroy() - re.NoError(err) - re.NoError(cluster.RunInitialServers()) - - cluster.WaitAllLeaders(re, dcLocationConfig) - - leaderServer := cluster.GetLeaderServer() - leaderCli := testutil.MustNewGrpcClient(re, leaderServer.GetAddr()) - req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(cluster.GetCluster().GetId()), - Count: tsoCount, - DcLocation: tso.GlobalDCLocation, - } - ctx, cancel := context.WithCancel(context.Background()) - ctx = grpcutil.BuildForwardContext(ctx, leaderServer.GetAddr()) - previousTS := testGetTimestamp(re, ctx, leaderCli, req) - cancel() - - // Wait for all nodes becoming healthy. - time.Sleep(time.Second * 5) - - // Mock the situation that the system time of PD nodes in dc-4 is slower than others. - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/systemTimeSlow", `return(true)`)) - - // Join a new dc-location - pd4, err := cluster.Join(suite.ctx, func(conf *config.Config, _ string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = "dc-4" - }) - re.NoError(err) - re.NoError(pd4.Run()) - dcLocationConfig["pd4"] = "dc-4" - cluster.CheckClusterDCLocation() - re.NotEqual("", cluster.WaitAllocatorLeader( - "dc-4", - tests.WithRetryTimes(90), tests.WithWaitInterval(time.Second), - )) - suite.testTSO(cluster, dcLocationConfig, previousTS) - - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/systemTimeSlow")) -} - -func (suite *tsoConsistencyTestSuite) testTSO(cluster *tests.TestCluster, dcLocationConfig map[string]string, previousTS *pdpb.Timestamp) { - re := suite.Require() - leaderServer := cluster.GetLeaderServer() - dcClientMap := make(map[string]pdpb.PDClient) - for _, dcLocation := range dcLocationConfig { - pdName := leaderServer.GetAllocatorLeader(dcLocation).GetName() - dcClientMap[dcLocation] = testutil.MustNewGrpcClient(re, cluster.GetServer(pdName).GetAddr()) - } - - var wg sync.WaitGroup - wg.Add(tsoRequestConcurrencyNumber) - for range tsoRequestConcurrencyNumber { - go func() { - defer wg.Done() - lastList := make(map[string]*pdpb.Timestamp) - for _, dcLocation := range dcLocationConfig { - lastList[dcLocation] = &pdpb.Timestamp{ - Physical: 0, - Logical: 0, - } - } - for range tsoRequestRound { - for _, dcLocation := range dcLocationConfig { - req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(leaderServer.GetClusterID()), - Count: tsoCount, - DcLocation: dcLocation, - } - ctx, cancel := context.WithCancel(context.Background()) - ctx = grpcutil.BuildForwardContext(ctx, cluster.GetServer(leaderServer.GetAllocatorLeader(dcLocation).GetName()).GetAddr()) - ts := testGetTimestamp(re, ctx, dcClientMap[dcLocation], req) - cancel() - lastTS := lastList[dcLocation] - // Check whether the TSO fallbacks - re.Equal(1, tsoutil.CompareTimestamp(ts, lastTS)) - if previousTS != nil { - // Because we have a Global TSO synchronization, even though the system time - // of the PD nodes in dc-4 is slower, its TSO will still be big enough. - re.Equal(1, tsoutil.CompareTimestamp(ts, previousTS)) - } - lastList[dcLocation] = ts - // Check whether the TSO is not unique - re.True(suite.checkTSOUnique(ts)) - } - time.Sleep(10 * time.Millisecond) - } - }() - } - wg.Wait() -} diff --git a/tests/server/tso/global_tso_test.go b/tests/server/tso/global_tso_test.go deleted file mode 100644 index bf39c57e3d6..00000000000 --- a/tests/server/tso/global_tso_test.go +++ /dev/null @@ -1,181 +0,0 @@ -// Copyright 2020 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. - -//go:build tso_full_test || tso_function_test -// +build tso_full_test tso_function_test - -package tso_test - -import ( - "context" - "testing" - "time" - - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/tso" - "github.com/tikv/pd/pkg/utils/grpcutil" - "github.com/tikv/pd/pkg/utils/testutil" - "github.com/tikv/pd/pkg/utils/typeutil" - "github.com/tikv/pd/server/config" - "github.com/tikv/pd/tests" -) - -// There are three kinds of ways to generate a TSO: -// 1. Normal Global TSO, the normal way to get a global TSO from the PD leader, -// a.k.a the single Global TSO Allocator. -// 2. Normal Local TSO, the new way to get a local TSO may from any of PD servers, -// a.k.a the Local TSO Allocator leader. -// 3. Synchronized global TSO, the new way to get a global TSO from the PD leader, -// which will coordinate and synchronize a TSO with other Local TSO Allocator -// leaders. - -func TestRequestFollower(t *testing.T) { - re := require.New(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cluster, err := tests.NewTestCluster(ctx, 2) - re.NoError(err) - defer cluster.Destroy() - - re.NoError(cluster.RunInitialServers()) - re.NotEmpty(cluster.WaitLeader()) - - var followerServer *tests.TestServer - for _, s := range cluster.GetServers() { - if s.GetConfig().Name != cluster.GetLeader() { - followerServer = s - } - } - re.NotNil(followerServer) - - grpcPDClient := testutil.MustNewGrpcClient(re, followerServer.GetAddr()) - clusterID := followerServer.GetClusterID() - req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(clusterID), - Count: 1, - DcLocation: tso.GlobalDCLocation, - } - ctx = grpcutil.BuildForwardContext(ctx, followerServer.GetAddr()) - tsoClient, err := grpcPDClient.Tso(ctx) - re.NoError(err) - defer tsoClient.CloseSend() - - start := time.Now() - re.NoError(tsoClient.Send(req)) - _, err = tsoClient.Recv() - re.Error(err) - re.Contains(err.Error(), "generate timestamp failed") - - // Requesting follower should fail fast, or the unavailable time will be - // too long. - re.Less(time.Since(start), time.Second) -} - -// In some cases, when a TSO request arrives, the SyncTimestamp may not finish yet. -// This test is used to simulate this situation and verify that the retry mechanism. -func TestDelaySyncTimestamp(t *testing.T) { - re := require.New(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cluster, err := tests.NewTestCluster(ctx, 2) - re.NoError(err) - defer cluster.Destroy() - re.NoError(cluster.RunInitialServers()) - re.NotEmpty(cluster.WaitLeader()) - - var leaderServer, nextLeaderServer *tests.TestServer - leaderServer = cluster.GetLeaderServer() - re.NotNil(leaderServer) - leaderServer.BootstrapCluster() - for _, s := range cluster.GetServers() { - if s.GetConfig().Name != cluster.GetLeader() { - nextLeaderServer = s - } - } - re.NotNil(nextLeaderServer) - - grpcPDClient := testutil.MustNewGrpcClient(re, nextLeaderServer.GetAddr()) - clusterID := nextLeaderServer.GetClusterID() - req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(clusterID), - Count: 1, - DcLocation: tso.GlobalDCLocation, - } - - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/delaySyncTimestamp", `return(true)`)) - - // Make the old leader resign and wait for the new leader to get a lease - leaderServer.ResignLeader() - re.True(nextLeaderServer.WaitLeader()) - - ctx = grpcutil.BuildForwardContext(ctx, nextLeaderServer.GetAddr()) - tsoClient, err := grpcPDClient.Tso(ctx) - re.NoError(err) - defer tsoClient.CloseSend() - re.NoError(tsoClient.Send(req)) - resp, err := tsoClient.Recv() - re.NoError(err) - re.NotNil(checkAndReturnTimestampResponse(re, req, resp)) - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/delaySyncTimestamp")) -} - -func TestLogicalOverflow(t *testing.T) { - re := require.New(t) - - runCase := func(updateInterval time.Duration) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cluster, err := tests.NewTestCluster(ctx, 1, func(conf *config.Config, _ string) { - conf.TSOUpdatePhysicalInterval = typeutil.Duration{Duration: updateInterval} - }) - defer cluster.Destroy() - re.NoError(err) - re.NoError(cluster.RunInitialServers()) - re.NotEmpty(cluster.WaitLeader()) - - leaderServer := cluster.GetLeaderServer() - re.NotNil(leaderServer) - leaderServer.BootstrapCluster() - grpcPDClient := testutil.MustNewGrpcClient(re, leaderServer.GetAddr()) - clusterID := leaderServer.GetClusterID() - - tsoClient, err := grpcPDClient.Tso(ctx) - re.NoError(err) - defer tsoClient.CloseSend() - - begin := time.Now() - for i := range 3 { - req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(clusterID), - Count: 150000, - DcLocation: tso.GlobalDCLocation, - } - re.NoError(tsoClient.Send(req)) - _, err = tsoClient.Recv() - re.NoError(err) - if i == 1 { - // the 2nd request may (but not must) overflow, as max logical interval is 262144 - re.Less(time.Since(begin), updateInterval+50*time.Millisecond) // additional 50ms for gRPC latency - } - } - // the 3rd request must overflow - re.GreaterOrEqual(time.Since(begin), updateInterval) - } - - for _, updateInterval := range []int{1, 5, 30, 50} { - runCase(time.Duration(updateInterval) * time.Millisecond) - } -} diff --git a/tests/server/tso/manager_test.go b/tests/server/tso/manager_test.go deleted file mode 100644 index 1feb74e6643..00000000000 --- a/tests/server/tso/manager_test.go +++ /dev/null @@ -1,188 +0,0 @@ -// Copyright 2020 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. - -//go:build tso_full_test || tso_function_test -// +build tso_full_test tso_function_test - -package tso_test - -import ( - "context" - "strconv" - "testing" - "time" - - "github.com/pingcap/failpoint" - "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/tso" - "github.com/tikv/pd/pkg/utils/etcdutil" - "github.com/tikv/pd/pkg/utils/testutil" - "github.com/tikv/pd/server/config" - "github.com/tikv/pd/tests" - clientv3 "go.etcd.io/etcd/client/v3" -) - -// TestClusterDCLocations will write different dc-locations to each server -// and test whether we can get the whole dc-location config from each server. -func TestClusterDCLocations(t *testing.T) { - re := require.New(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - testCase := struct { - dcLocationNumber int - dcLocationConfig map[string]string - }{ - dcLocationNumber: 3, - dcLocationConfig: map[string]string{ - "pd1": "dc-1", - "pd2": "dc-1", - "pd3": "dc-2", - "pd4": "dc-2", - "pd5": "dc-3", - "pd6": "dc-3", - }, - } - serverNumber := len(testCase.dcLocationConfig) - cluster, err := tests.NewTestCluster(ctx, serverNumber, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = testCase.dcLocationConfig[serverName] - }) - defer cluster.Destroy() - re.NoError(err) - re.NoError(cluster.RunInitialServers()) - - cluster.WaitAllLeaders(re, testCase.dcLocationConfig) - serverNameMap := make(map[uint64]string) - for _, server := range cluster.GetServers() { - serverNameMap[server.GetServerID()] = server.GetServer().Name() - // To speed up the test, we force to do the check - server.GetTSOAllocatorManager().ClusterDCLocationChecker() - } - // Start to check every server's GetClusterDCLocations() result - for _, server := range cluster.GetServers() { - obtainedServerNumber := 0 - dcLocationMap := server.GetTSOAllocatorManager().GetClusterDCLocations() - re.NoError(err) - re.Len(dcLocationMap, testCase.dcLocationNumber) - for obtainedDCLocation, info := range dcLocationMap { - obtainedServerNumber += len(info.ServerIDs) - for _, serverID := range info.ServerIDs { - expectedDCLocation, exist := testCase.dcLocationConfig[serverNameMap[serverID]] - re.True(exist) - re.Equal(expectedDCLocation, obtainedDCLocation) - } - } - re.Equal(serverNumber, obtainedServerNumber) - } -} - -func TestLocalTSOSuffix(t *testing.T) { - re := require.New(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - testCase := struct { - dcLocations []string - dcLocationConfig map[string]string - }{ - dcLocations: []string{"dc-1", "dc-2", "dc-3"}, - dcLocationConfig: map[string]string{ - "pd1": "dc-1", - "pd2": "dc-1", - "pd3": "dc-2", - "pd4": "dc-2", - "pd5": "dc-3", - "pd6": "dc-3", - }, - } - serverNumber := len(testCase.dcLocationConfig) - cluster, err := tests.NewTestCluster(ctx, serverNumber, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = testCase.dcLocationConfig[serverName] - }) - defer cluster.Destroy() - re.NoError(err) - re.NoError(cluster.RunInitialServers()) - - cluster.WaitAllLeaders(re, testCase.dcLocationConfig) - - tsoAllocatorManager := cluster.GetServer("pd1").GetTSOAllocatorManager() - for _, dcLocation := range testCase.dcLocations { - suffixResp, err := etcdutil.EtcdKVGet( - cluster.GetEtcdClient(), - tsoAllocatorManager.GetLocalTSOSuffixPath(dcLocation)) - re.NoError(err) - re.Len(suffixResp.Kvs, 1) - // Test the increment of the suffix - allSuffixResp, err := etcdutil.EtcdKVGet( - cluster.GetEtcdClient(), - tsoAllocatorManager.GetLocalTSOSuffixPathPrefix(), - clientv3.WithPrefix(), - clientv3.WithSort(clientv3.SortByValue, clientv3.SortAscend)) - re.NoError(err) - re.Len(allSuffixResp.Kvs, len(testCase.dcLocations)) - var lastSuffixNum int64 - for _, kv := range allSuffixResp.Kvs { - suffixNum, err := strconv.ParseInt(string(kv.Value), 10, 64) - re.NoError(err) - re.Greater(suffixNum, lastSuffixNum) - lastSuffixNum = suffixNum - } - } -} - -func TestNextLeaderKey(t *testing.T) { - re := require.New(t) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - tso.PriorityCheck = 5 * time.Second - defer func() { - tso.PriorityCheck = time.Minute - }() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-1", - } - serverNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(ctx, serverNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - defer cluster.Destroy() - re.NoError(err) - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/injectNextLeaderKey", "return(true)")) - re.NoError(cluster.RunInitialServers()) - - cluster.WaitLeader(tests.WithWaitInterval(5*time.Second), tests.WithRetryTimes(3)) - // To speed up the test, we force to do the check - cluster.CheckClusterDCLocation() - originName := cluster.WaitAllocatorLeader("dc-1", tests.WithRetryTimes(5), tests.WithWaitInterval(5*time.Second)) - re.Equal("", originName) - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/injectNextLeaderKey")) - cluster.CheckClusterDCLocation() - originName = cluster.WaitAllocatorLeader("dc-1") - re.NotEqual("", originName) - for name, server := range cluster.GetServers() { - if name == originName { - continue - } - err := server.GetTSOAllocatorManager().TransferAllocatorForDCLocation("dc-1", server.GetServer().GetMember().ID()) - re.NoError(err) - testutil.Eventually(re, func() bool { - cluster.CheckClusterDCLocation() - currName := cluster.WaitAllocatorLeader("dc-1") - return currName == name - }, testutil.WithTickInterval(time.Second)) - return - } -} diff --git a/tests/server/tso/tso_test.go b/tests/server/tso/tso_test.go index fc2f5999840..cb6b87c83d3 100644 --- a/tests/server/tso/tso_test.go +++ b/tests/server/tso/tso_test.go @@ -1,4 +1,4 @@ -// Copyright 2021 TiKV Project Authors. +// 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. @@ -12,14 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -//go:build tso_full_test || tso_function_test -// +build tso_full_test tso_function_test - package tso_test import ( "context" "testing" + "time" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" @@ -27,116 +25,96 @@ import ( "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/testutil" + "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" + "go.uber.org/goleak" ) -func TestLoadTimestamp(t *testing.T) { +func TestMain(m *testing.M) { + goleak.VerifyTestMain(m, testutil.LeakOptions...) +} + +func TestRequestFollower(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-2", - "pd3": "dc-3", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - defer cluster.Destroy() + cluster, err := tests.NewTestCluster(ctx, 2) re.NoError(err) - re.NoError(cluster.RunInitialServers()) - - cluster.WaitAllLeaders(re, dcLocationConfig) - - lastTSMap := requestLocalTSOs(re, cluster, dcLocationConfig) - - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/systemTimeSlow", `return(true)`)) + defer cluster.Destroy() - // Reboot the cluster. - re.NoError(cluster.StopAll()) re.NoError(cluster.RunInitialServers()) + re.NotEmpty(cluster.WaitLeader()) - cluster.WaitAllLeaders(re, dcLocationConfig) + var followerServer *tests.TestServer + for _, s := range cluster.GetServers() { + if s.GetConfig().Name != cluster.GetLeader() { + followerServer = s + } + } + re.NotNil(followerServer) - // Re-request the Local TSOs. - newTSMap := requestLocalTSOs(re, cluster, dcLocationConfig) - for dcLocation, newTS := range newTSMap { - lastTS, ok := lastTSMap[dcLocation] - re.True(ok) - // The new physical time of TSO should be larger even if the system time is slow. - re.Greater(newTS.GetPhysical()-lastTS.GetPhysical(), int64(0)) + grpcPDClient := testutil.MustNewGrpcClient(re, followerServer.GetAddr()) + clusterID := followerServer.GetClusterID() + req := &pdpb.TsoRequest{ + Header: testutil.NewRequestHeader(clusterID), + Count: 1, + DcLocation: tso.GlobalDCLocation, } + ctx = grpcutil.BuildForwardContext(ctx, followerServer.GetAddr()) + tsoClient, err := grpcPDClient.Tso(ctx) + re.NoError(err) + defer tsoClient.CloseSend() - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/systemTimeSlow")) -} + start := time.Now() + re.NoError(tsoClient.Send(req)) + _, err = tsoClient.Recv() + re.Error(err) + re.Contains(err.Error(), "generate timestamp failed") -func requestLocalTSOs(re *require.Assertions, cluster *tests.TestCluster, dcLocationConfig map[string]string) map[string]*pdpb.Timestamp { - dcClientMap := make(map[string]pdpb.PDClient) - tsMap := make(map[string]*pdpb.Timestamp) - leaderServer := cluster.GetLeaderServer() - for _, dcLocation := range dcLocationConfig { - pdName := leaderServer.GetAllocatorLeader(dcLocation).GetName() - dcClientMap[dcLocation] = testutil.MustNewGrpcClient(re, cluster.GetServer(pdName).GetAddr()) - } - for _, dcLocation := range dcLocationConfig { - req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(leaderServer.GetClusterID()), - Count: tsoCount, - DcLocation: dcLocation, - } - ctx, cancel := context.WithCancel(context.Background()) - ctx = grpcutil.BuildForwardContext(ctx, cluster.GetServer(leaderServer.GetAllocatorLeader(dcLocation).GetName()).GetAddr()) - tsMap[dcLocation] = testGetTimestamp(re, ctx, dcClientMap[dcLocation], req) - cancel() - } - return tsMap + // Requesting follower should fail fast, or the unavailable time will be + // too long. + re.Less(time.Since(start), time.Second) } -func TestDisableLocalTSOAfterEnabling(t *testing.T) { +// In some cases, when a TSO request arrives, the SyncTimestamp may not finish yet. +// This test is used to simulate this situation and verify that the retry mechanism. +func TestDelaySyncTimestamp(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - dcLocationConfig := map[string]string{ - "pd1": "dc-1", - "pd2": "dc-2", - "pd3": "dc-3", - } - dcLocationNum := len(dcLocationConfig) - cluster, err := tests.NewTestCluster(ctx, dcLocationNum, func(conf *config.Config, serverName string) { - conf.EnableLocalTSO = true - conf.Labels[config.ZoneLabel] = dcLocationConfig[serverName] - }) - defer cluster.Destroy() + cluster, err := tests.NewTestCluster(ctx, 2) re.NoError(err) + defer cluster.Destroy() re.NoError(cluster.RunInitialServers()) + re.NotEmpty(cluster.WaitLeader()) - cluster.WaitAllLeaders(re, dcLocationConfig) - leaderServer := cluster.GetLeaderServer() + var leaderServer, nextLeaderServer *tests.TestServer + leaderServer = cluster.GetLeaderServer() + re.NotNil(leaderServer) leaderServer.BootstrapCluster() - requestLocalTSOs(re, cluster, dcLocationConfig) - - // Reboot the cluster. - re.NoError(cluster.StopAll()) - for _, server := range cluster.GetServers() { - server.SetEnableLocalTSO(false) + for _, s := range cluster.GetServers() { + if s.GetConfig().Name != cluster.GetLeader() { + nextLeaderServer = s + } } - re.NoError(cluster.RunInitialServers()) - re.NotEmpty(cluster.WaitLeader()) + re.NotNil(nextLeaderServer) - // Re-request the global TSOs. - leaderServer = cluster.GetLeaderServer() - grpcPDClient := testutil.MustNewGrpcClient(re, leaderServer.GetAddr()) - clusterID := leaderServer.GetClusterID() + grpcPDClient := testutil.MustNewGrpcClient(re, nextLeaderServer.GetAddr()) + clusterID := nextLeaderServer.GetClusterID() req := &pdpb.TsoRequest{ Header: testutil.NewRequestHeader(clusterID), Count: 1, DcLocation: tso.GlobalDCLocation, } - ctx = grpcutil.BuildForwardContext(ctx, leaderServer.GetAddr()) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/delaySyncTimestamp", `return(true)`)) + + // Make the old leader resign and wait for the new leader to get a lease + leaderServer.ResignLeader() + re.True(nextLeaderServer.WaitLeader()) + + ctx = grpcutil.BuildForwardContext(ctx, nextLeaderServer.GetAddr()) tsoClient, err := grpcPDClient.Tso(ctx) re.NoError(err) defer tsoClient.CloseSend() @@ -144,8 +122,60 @@ func TestDisableLocalTSOAfterEnabling(t *testing.T) { resp, err := tsoClient.Recv() re.NoError(err) re.NotNil(checkAndReturnTimestampResponse(re, req, resp)) - // Test whether the number of existing DCs is as expected. - dcLocations, err := leaderServer.GetTSOAllocatorManager().GetClusterDCLocationsFromEtcd() - re.NoError(err) - re.Equal(0, len(dcLocations)) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/delaySyncTimestamp")) +} + +func checkAndReturnTimestampResponse(re *require.Assertions, req *pdpb.TsoRequest, resp *pdpb.TsoResponse) *pdpb.Timestamp { + re.Equal(req.GetCount(), resp.GetCount()) + timestamp := resp.GetTimestamp() + re.Positive(timestamp.GetPhysical()) + re.GreaterOrEqual(uint32(timestamp.GetLogical())>>timestamp.GetSuffixBits(), req.GetCount()) + return timestamp +} +func TestLogicalOverflow(t *testing.T) { + re := require.New(t) + + runCase := func(updateInterval time.Duration) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + cluster, err := tests.NewTestCluster(ctx, 1, func(conf *config.Config, _ string) { + conf.TSOUpdatePhysicalInterval = typeutil.Duration{Duration: updateInterval} + }) + defer cluster.Destroy() + re.NoError(err) + re.NoError(cluster.RunInitialServers()) + re.NotEmpty(cluster.WaitLeader()) + + leaderServer := cluster.GetLeaderServer() + re.NotNil(leaderServer) + leaderServer.BootstrapCluster() + grpcPDClient := testutil.MustNewGrpcClient(re, leaderServer.GetAddr()) + clusterID := leaderServer.GetClusterID() + + tsoClient, err := grpcPDClient.Tso(ctx) + re.NoError(err) + defer tsoClient.CloseSend() + + begin := time.Now() + for i := range 3 { + req := &pdpb.TsoRequest{ + Header: testutil.NewRequestHeader(clusterID), + Count: 150000, + DcLocation: tso.GlobalDCLocation, + } + re.NoError(tsoClient.Send(req)) + _, err = tsoClient.Recv() + re.NoError(err) + if i == 1 { + // the 2nd request may (but not must) overflow, as max logical interval is 262144 + re.Less(time.Since(begin), updateInterval+50*time.Millisecond) // additional 50ms for gRPC latency + } + } + // the 3rd request must overflow + re.GreaterOrEqual(time.Since(begin), updateInterval) + } + + for _, updateInterval := range []int{1, 5, 30, 50} { + runCase(time.Duration(updateInterval) * time.Millisecond) + } } diff --git a/tests/testutil.go b/tests/testutil.go index b5ea0a9f53a..b56fd245bd3 100644 --- a/tests/testutil.go +++ b/tests/testutil.go @@ -36,7 +36,6 @@ import ( "github.com/stretchr/testify/require" bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/core" - rm "github.com/tikv/pd/pkg/mcs/resourcemanager/server" scheduling "github.com/tikv/pd/pkg/mcs/scheduling/server" sc "github.com/tikv/pd/pkg/mcs/scheduling/server/config" tso "github.com/tikv/pd/pkg/mcs/tso/server" @@ -107,24 +106,6 @@ func InitLogger(logConfig log.Config, logger *zap.Logger, logProps *log.ZapPrope return err } -// StartSingleResourceManagerTestServer creates and starts a resource manager server with default config for testing. -func StartSingleResourceManagerTestServer(ctx context.Context, re *require.Assertions, backendEndpoints, listenAddrs string) (*rm.Server, func()) { - cfg := rm.NewConfig() - cfg.BackendEndpoints = backendEndpoints - cfg.ListenAddr = listenAddrs - cfg.Name = cfg.ListenAddr - cfg, err := rm.GenerateConfig(cfg) - re.NoError(err) - - s, cleanup, err := rm.NewTestServer(ctx, re, cfg) - re.NoError(err) - testutil.Eventually(re, func() bool { - return !s.IsClosed() - }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) - - return s, cleanup -} - // StartSingleTSOTestServerWithoutCheck creates and starts a tso server with default config for testing. func StartSingleTSOTestServerWithoutCheck(ctx context.Context, re *require.Assertions, backendEndpoints, listenAddrs string) (*tso.Server, func(), error) { cfg := tso.NewConfig() diff --git a/tools/pd-tso-bench/README.md b/tools/pd-tso-bench/README.md index 5eb94390546..d1ec3736406 100644 --- a/tools/pd-tso-bench/README.md +++ b/tools/pd-tso-bench/README.md @@ -24,8 +24,6 @@ This section describes how to benchmark the GetTS performance. the number of pd clients involved in each benchmark (default 1) -count int the count number that the test will run (default 1) --dc string - which dc-location this bench will request (default "global") -duration duration how many seconds the test will last (default 1m0s) -interval duration diff --git a/tools/pd-tso-bench/main.go b/tools/pd-tso-bench/main.go index e4a99773d99..62ba79a7584 100644 --- a/tools/pd-tso-bench/main.go +++ b/tools/pd-tso-bench/main.go @@ -50,7 +50,6 @@ var ( concurrency = flag.Int("c", 1000, "concurrency") count = flag.Int("count", 1, "the count number that the test will run") duration = flag.Duration("duration", 60*time.Second, "how many seconds the test will last") - dcLocation = flag.String("dc", "global", "which dc-location this bench will request") verbose = flag.Bool("v", false, "output statistics info every interval and output metrics info at the end") interval = flag.Duration("interval", time.Second, "interval to output the statistics") caPath = flag.String("cacert", "", "path of file that contains list of trusted SSL CAs") @@ -115,7 +114,7 @@ func bench(mainCtx context.Context) { ctx, cancel := context.WithCancel(mainCtx) // To avoid the first time high latency. for idx, pdCli := range pdClients { - _, _, err := pdCli.GetLocalTS(ctx, *dcLocation) + _, _, err := pdCli.GetTS(ctx) if err != nil { log.Fatal("get first time tso failed", zap.Int("client-number", idx), zap.Error(err)) } @@ -396,7 +395,7 @@ func reqWorker(ctx context.Context, pdClients []pd.Client, clientIdx int, durCh } } } - _, _, err = pdCli.GetLocalTS(reqCtx, *dcLocation) + _, _, err = pdCli.GetTS(reqCtx) if errors.Cause(err) == context.Canceled { if ticker != nil { ticker.Stop() diff --git a/tools/pd-ut/ut.go b/tools/pd-ut/ut.go index df8bff526f2..7bb0cf17e9f 100644 --- a/tools/pd-ut/ut.go +++ b/tools/pd-ut/ut.go @@ -719,8 +719,8 @@ func generateBuildCache() error { return nil } fmt.Println("generate build cache") - // cd cmd/pd-server && go test -tags=tso_function_test,deadlock -exec-=true -vet=off -toolexec=go-compile-without-link - cmd := exec.Command("go", "test", "-exec=true", "-vet", "off", "--tags=tso_function_test,deadlock") + // cd cmd/pd-server && go test -tags=deadlock -exec-=true -vet=off -toolexec=go-compile-without-link + cmd := exec.Command("go", "test", "-exec=true", "-vet", "off", "--tags=deadlock") goCompileWithoutLink := fmt.Sprintf("-toolexec=%s", filepath.Join(workDir, "tools", "pd-ut", "go-compile-without-link.sh")) cmd.Dir = filepath.Join(workDir, "cmd", "pd-server") if strings.Contains(workDir, integrationsTestPath) { @@ -745,7 +745,7 @@ func buildTestBinaryMulti(pkgs []string) ([]byte, error) { return nil, withTrace(err) } - // go test --exec=xprog --tags=tso_function_test,deadlock -vet=off --count=0 $(pkgs) + // go test --exec=xprog --tags=deadlock -vet=off --count=0 $(pkgs) // workPath just like `/pd/tests/integrations` xprogPath := filepath.Join(workDir, "bin", "xprog") if strings.Contains(workDir, integrationsTestPath) { @@ -758,7 +758,7 @@ func buildTestBinaryMulti(pkgs []string) ([]byte, error) { // We use 2 * parallel for `go build` to make it faster. p := strconv.Itoa(parallel * 2) - cmd := exec.Command("go", "test", "-p", p, "--exec", xprogPath, "-vet", "off", "--tags=tso_function_test,deadlock") + cmd := exec.Command("go", "test", "-p", p, "--exec", xprogPath, "-vet", "off", "--tags=deadlock") if coverProfile != "" { coverPkg := strings.Join([]string{".", "..."}, string(filepath.Separator)) if strings.Contains(workDir, integrationsTestPath) { @@ -793,7 +793,7 @@ func buildTestBinaryMulti(pkgs []string) ([]byte, error) { func buildTestBinary(pkg string) error { //nolint:gosec - cmd := exec.Command("go", "test", "-c", "-vet", "off", "--tags=tso_function_test,deadlock", "-o", testFileName(pkg), "-v") + cmd := exec.Command("go", "test", "-c", "-vet", "off", "--tags=deadlock", "-o", testFileName(pkg), "-v") if coverProfile != "" { coverPkg := strings.Join([]string{".", "..."}, string(filepath.Separator)) cmd.Args = append(cmd.Args, "-cover", fmt.Sprintf("-coverpkg=%s", coverPkg))