diff --git a/.gitignore b/.gitignore index 93e6189a687..748d24872b6 100644 --- a/.gitignore +++ b/.gitignore @@ -24,3 +24,4 @@ report.xml coverage.xml coverage *.txt +go.work* diff --git a/client/client.go b/client/client.go index 2a75b0effc5..aea4100289d 100644 --- a/client/client.go +++ b/client/client.go @@ -251,6 +251,14 @@ func WithMaxErrorRetry(count int) ClientOption { } } +// WithAllowTSOFallback configures the client with `allowTSOFallback` option. +// NOTICE: This should only be used for testing. +func WithAllowTSOFallback() ClientOption { + return func(c *client) { + c.option.allowTSOFallback = true + } +} + var _ Client = (*client)(nil) // serviceModeKeeper is for service mode switching. @@ -263,14 +271,6 @@ type serviceModeKeeper struct { tsoSvcDiscovery ServiceDiscovery } -func (k *serviceModeKeeper) SetKeyspaceID(keyspaceID uint32) { - k.Lock() - defer k.Unlock() - if k.serviceMode == pdpb.ServiceMode_API_SVC_MODE { - k.tsoSvcDiscovery.SetKeyspaceID(keyspaceID) - } -} - func (k *serviceModeKeeper) close() { k.Lock() defer k.Unlock() @@ -378,7 +378,7 @@ func createClientWithKeyspace( c.pdSvcDiscovery = newPDServiceDiscovery( clientCtx, clientCancel, &c.wg, c.setServiceMode, - keyspaceID, c.svrUrls, c.tlsCfg, c.option) + nil, keyspaceID, c.svrUrls, c.tlsCfg, c.option) if err := c.setup(); err != nil { c.cancel() return nil, err @@ -490,28 +490,34 @@ func newClientWithKeyspaceName( opt(c) } + updateKeyspaceIDCb := func() error { + if err := c.initRetry(c.loadKeyspaceMeta, keyspaceName); err != nil { + return err + } + // c.keyspaceID is the source of truth for keyspace id. + c.pdSvcDiscovery.(*pdServiceDiscovery).SetKeyspaceID(c.keyspaceID) + return nil + } + // Create a PD service discovery with null keyspace id, then query the real id wth the keyspace name, // finally update the keyspace id to the PD service discovery for the following interactions. c.pdSvcDiscovery = newPDServiceDiscovery( - clientCtx, clientCancel, &c.wg, c.setServiceMode, nullKeyspaceID, c.svrUrls, c.tlsCfg, c.option) + clientCtx, clientCancel, &c.wg, c.setServiceMode, updateKeyspaceIDCb, nullKeyspaceID, c.svrUrls, c.tlsCfg, c.option) if err := c.setup(); err != nil { c.cancel() return nil, err } - if err := c.initRetry(c.loadKeyspaceMeta, keyspaceName); err != nil { - return nil, err - } - // We call "c.pdSvcDiscovery.SetKeyspaceID(c.keyspaceID)" after service mode already switching to API mode - // and tso service discovery already initialized, so here we need to set the tso_service_discovery's keyspace id too. - c.pdSvcDiscovery.SetKeyspaceID(c.keyspaceID) - c.serviceModeKeeper.SetKeyspaceID(c.keyspaceID) log.Info("[pd] create pd client with endpoints and keyspace", - zap.Strings("pd-address", svrAddrs), zap.String("keyspace-name", keyspaceName), zap.Uint32("keyspace-id", c.keyspaceID)) + zap.Strings("pd-address", svrAddrs), + zap.String("keyspace-name", keyspaceName), + zap.Uint32("keyspace-id", c.keyspaceID)) return c, nil } func (c *client) initRetry(f func(s string) error, str string) error { var err error + ticker := time.NewTicker(time.Second) + defer ticker.Stop() for i := 0; i < c.option.maxRetryTimes; i++ { if err = f(str); err == nil { return nil @@ -519,7 +525,7 @@ func (c *client) initRetry(f func(s string) error, str string) error { select { case <-c.ctx.Done(): return err - case <-time.After(time.Second): + case <-ticker.C: } } return errors.WithStack(err) diff --git a/client/option.go b/client/option.go index e2fcfcbbef4..65599bf2293 100644 --- a/client/option.go +++ b/client/option.go @@ -51,6 +51,7 @@ type option struct { timeout time.Duration maxRetryTimes int enableForwarding bool + allowTSOFallback bool // Dynamic options. dynamicOptions [dynamicOptionCount]atomic.Value diff --git a/client/pd_service_discovery.go b/client/pd_service_discovery.go index 9966b8c75cb..4499c9e17c0 100644 --- a/client/pd_service_discovery.go +++ b/client/pd_service_discovery.go @@ -59,8 +59,6 @@ type ServiceDiscovery interface { GetClusterID() uint64 // GetKeyspaceID returns the ID of the keyspace GetKeyspaceID() uint32 - // SetKeyspaceID sets the ID of the keyspace - SetKeyspaceID(keyspaceID uint32) // GetKeyspaceGroupID returns the ID of the keyspace group GetKeyspaceGroupID() uint32 // DiscoverServiceURLs discovers the microservice with the specified type and returns the server urls. @@ -99,6 +97,7 @@ type ServiceDiscovery interface { AddServiceAddrsSwitchedCallback(callbacks ...func()) } +type updateKeyspaceIDFunc func() error type tsoLocalServAddrsUpdatedFunc func(map[string]string) error type tsoGlobalServAddrUpdatedFunc func(string) error @@ -149,8 +148,9 @@ type pdServiceDiscovery struct { cancel context.CancelFunc closeOnce sync.Once - keyspaceID uint32 - tlsCfg *tlsutil.TLSConfig + updateKeyspaceIDCb updateKeyspaceIDFunc + keyspaceID uint32 + tlsCfg *tlsutil.TLSConfig // Client option. option *option } @@ -160,6 +160,7 @@ func newPDServiceDiscovery( ctx context.Context, cancel context.CancelFunc, wg *sync.WaitGroup, serviceModeUpdateCb func(pdpb.ServiceMode), + updateKeyspaceIDCb updateKeyspaceIDFunc, keyspaceID uint32, urls []string, tlsCfg *tlsutil.TLSConfig, option *option, ) *pdServiceDiscovery { @@ -169,6 +170,7 @@ func newPDServiceDiscovery( cancel: cancel, wg: wg, serviceModeUpdateCb: serviceModeUpdateCb, + updateKeyspaceIDCb: updateKeyspaceIDCb, keyspaceID: keyspaceID, tlsCfg: tlsCfg, option: option, @@ -192,6 +194,14 @@ func (c *pdServiceDiscovery) Init() error { } log.Info("[pd] init cluster id", zap.Uint64("cluster-id", c.clusterID)) + // We need to update the keyspace ID before we discover and update the service mode + // so that TSO in API mode can be initialized with the correct keyspace ID. + if c.updateKeyspaceIDCb != nil { + if err := c.updateKeyspaceIDCb(); err != nil { + return err + } + } + if err := c.checkServiceModeChanged(); err != nil { log.Warn("[pd] failed to check service mode and will check later", zap.Error(err)) } @@ -206,6 +216,8 @@ func (c *pdServiceDiscovery) Init() error { func (c *pdServiceDiscovery) initRetry(f func() error) error { var err error + ticker := time.NewTicker(time.Second) + defer ticker.Stop() for i := 0; i < c.option.maxRetryTimes; i++ { if err = f(); err == nil { return nil @@ -213,7 +225,7 @@ func (c *pdServiceDiscovery) initRetry(f func() error) error { select { case <-c.ctx.Done(): return err - case <-time.After(time.Second): + case <-ticker.C: } } return errors.WithStack(err) diff --git a/client/resource_manager_client.go b/client/resource_manager_client.go index b506611451d..97d065efbf5 100644 --- a/client/resource_manager_client.go +++ b/client/resource_manager_client.go @@ -349,6 +349,8 @@ func (c *client) tryResourceManagerConnect(ctx context.Context, connection *reso err error stream rmpb.ResourceManager_AcquireTokenBucketsClient ) + ticker := time.NewTicker(retryInterval) + defer ticker.Stop() for i := 0; i < maxRetryTimes; i++ { cctx, cancel := context.WithCancel(ctx) stream, err = c.resourceManagerClient().AcquireTokenBuckets(cctx) @@ -362,7 +364,7 @@ func (c *client) tryResourceManagerConnect(ctx context.Context, connection *reso select { case <-ctx.Done(): return err - case <-time.After(retryInterval): + case <-ticker.C: } } return err diff --git a/client/timerpool/pool.go b/client/timerpool/pool.go new file mode 100644 index 00000000000..28ffacfc629 --- /dev/null +++ b/client/timerpool/pool.go @@ -0,0 +1,43 @@ +// Copyright 2020 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Note: This file is copied from https://go-review.googlesource.com/c/go/+/276133 + +package timerpool + +import ( + "sync" + "time" +) + +// GlobalTimerPool is a global pool for reusing *time.Timer. +var GlobalTimerPool TimerPool + +// TimerPool is a wrapper of sync.Pool which caches *time.Timer for reuse. +type TimerPool struct { + pool sync.Pool +} + +// Get returns a timer with a given duration. +func (tp *TimerPool) Get(d time.Duration) *time.Timer { + if v := tp.pool.Get(); v != nil { + timer := v.(*time.Timer) + timer.Reset(d) + return timer + } + return time.NewTimer(d) +} + +// Put tries to call timer.Stop() before putting it back into pool, +// if the timer.Stop() returns false (it has either already expired or been stopped), +// have a shot at draining the channel with residual time if there is one. +func (tp *TimerPool) Put(timer *time.Timer) { + if !timer.Stop() { + select { + case <-timer.C: + default: + } + } + tp.pool.Put(timer) +} diff --git a/client/timerpool/pool_test.go b/client/timerpool/pool_test.go new file mode 100644 index 00000000000..d6dffc723a9 --- /dev/null +++ b/client/timerpool/pool_test.go @@ -0,0 +1,70 @@ +// Copyright 2020 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Note: This file is copied from https://go-review.googlesource.com/c/go/+/276133 + +package timerpool + +import ( + "testing" + "time" +) + +func TestTimerPool(t *testing.T) { + var tp TimerPool + + for i := 0; i < 100; i++ { + timer := tp.Get(20 * time.Millisecond) + + select { + case <-timer.C: + t.Errorf("timer expired too early") + continue + default: + } + + select { + case <-time.After(100 * time.Millisecond): + t.Errorf("timer didn't expire on time") + case <-timer.C: + } + + tp.Put(timer) + } +} + +const timeout = 10 * time.Millisecond + +func BenchmarkTimerUtilization(b *testing.B) { + b.Run("TimerWithPool", func(b *testing.B) { + for i := 0; i < b.N; i++ { + t := GlobalTimerPool.Get(timeout) + GlobalTimerPool.Put(t) + } + }) + b.Run("TimerWithoutPool", func(b *testing.B) { + for i := 0; i < b.N; i++ { + t := time.NewTimer(timeout) + t.Stop() + } + }) +} + +func BenchmarkTimerPoolParallel(b *testing.B) { + b.RunParallel(func(pb *testing.PB) { + for pb.Next() { + t := GlobalTimerPool.Get(timeout) + GlobalTimerPool.Put(t) + } + }) +} + +func BenchmarkTimerNativeParallel(b *testing.B) { + b.RunParallel(func(pb *testing.PB) { + for pb.Next() { + t := time.NewTimer(timeout) + t.Stop() + } + }) +} diff --git a/client/tso_dispatcher.go b/client/tso_dispatcher.go index 37bea8db9e5..c93c281ec04 100644 --- a/client/tso_dispatcher.go +++ b/client/tso_dispatcher.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/log" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/grpcutil" + "github.com/tikv/pd/client/timerpool" "github.com/tikv/pd/client/tsoutil" "go.uber.org/zap" "google.golang.org/grpc" @@ -139,11 +140,24 @@ func (c *tsoClient) updateTSODispatcher() { } type deadline struct { - timer <-chan time.Time + timer *time.Timer done chan struct{} cancel context.CancelFunc } +func newTSDeadline( + timeout time.Duration, + done chan struct{}, + cancel context.CancelFunc, +) *deadline { + timer := timerpool.GlobalTimerPool.Get(timeout) + return &deadline{ + timer: timer, + done: done, + cancel: cancel, + } +} + func (c *tsoClient) tsCancelLoop() { defer c.wg.Done() @@ -172,19 +186,21 @@ func (c *tsoClient) tsCancelLoop() { func (c *tsoClient) watchTSDeadline(ctx context.Context, dcLocation string) { if _, exist := c.tsDeadline.Load(dcLocation); !exist { - tsDeadlineCh := make(chan deadline, 1) + tsDeadlineCh := make(chan *deadline, 1) c.tsDeadline.Store(dcLocation, tsDeadlineCh) - go func(dc string, tsDeadlineCh <-chan deadline) { + go func(dc string, tsDeadlineCh <-chan *deadline) { for { select { case d := <-tsDeadlineCh: select { - case <-d.timer: + case <-d.timer.C: log.Error("[tso] tso request is canceled due to timeout", zap.String("dc-location", dc), errs.ZapError(errs.ErrClientGetTSOTimeout)) d.cancel() + timerpool.GlobalTimerPool.Put(d.timer) case <-d.done: - continue + timerpool.GlobalTimerPool.Put(d.timer) case <-ctx.Done(): + timerpool.GlobalTimerPool.Put(d.timer) return } case <-ctx.Done(): @@ -234,6 +250,8 @@ func (c *tsoClient) checkAllocator( }() cc, u := c.GetTSOAllocatorClientConnByDCLocation(dc) healthCli := healthpb.NewHealthClient(cc) + ticker := time.NewTicker(time.Second) + defer ticker.Stop() for { // the pd/allocator leader change, we need to re-establish the stream if u != url { @@ -259,7 +277,7 @@ func (c *tsoClient) checkAllocator( select { case <-dispatcherCtx.Done(): return - case <-time.After(time.Second): + case <-ticker.C: // To ensure we can get the latest allocator leader // and once the leader is changed, we can exit this function. _, u = c.GetTSOAllocatorClientConnByDCLocation(dc) @@ -366,6 +384,7 @@ func (c *tsoClient) handleDispatcher( // Loop through each batch of TSO requests and send them for processing. streamLoopTimer := time.NewTimer(c.option.timeout) + defer streamLoopTimer.Stop() tsoBatchLoop: for { select { @@ -389,6 +408,15 @@ tsoBatchLoop: if maxBatchWaitInterval >= 0 { tbc.adjustBestBatchSize() } + // Stop the timer if it's not stopped. + if !streamLoopTimer.Stop() { + select { + case <-streamLoopTimer.C: // try to drain from the channel + default: + } + } + // We need be careful here, see more details in the comments of Timer.Reset. + // https://pkg.go.dev/time@master#Timer.Reset streamLoopTimer.Reset(c.option.timeout) // Choose a stream to send the TSO gRPC request. streamChoosingLoop: @@ -403,16 +431,20 @@ tsoBatchLoop: if c.updateTSOConnectionCtxs(dispatcherCtx, dc, &connectionCtxs) { continue streamChoosingLoop } + timer := time.NewTimer(retryInterval) select { case <-dispatcherCtx.Done(): + timer.Stop() return case <-streamLoopTimer.C: err = errs.ErrClientCreateTSOStream.FastGenByArgs(errs.RetryTimeoutErr) log.Error("[tso] create tso stream error", zap.String("dc-location", dc), errs.ZapError(err)) c.svcDiscovery.ScheduleCheckMemberChanged() c.finishRequest(tbc.getCollectedRequests(), 0, 0, 0, errors.WithStack(err)) + timer.Stop() continue tsoBatchLoop - case <-time.After(retryInterval): + case <-timer.C: + timer.Stop() continue streamChoosingLoop } } @@ -429,11 +461,7 @@ tsoBatchLoop: } } done := make(chan struct{}) - dl := deadline{ - timer: time.After(c.option.timeout), - done: done, - cancel: cancel, - } + dl := newTSDeadline(c.option.timeout, done, cancel) tsDeadlineCh, ok := c.tsDeadline.Load(dc) for !ok || tsDeadlineCh == nil { c.scheduleCheckTSDeadline() @@ -443,7 +471,7 @@ tsoBatchLoop: select { case <-dispatcherCtx.Done(): return - case tsDeadlineCh.(chan deadline) <- dl: + case tsDeadlineCh.(chan *deadline) <- dl: } opts = extractSpanReference(tbc, opts[:0]) err = c.processRequests(stream, dc, tbc, opts) @@ -558,6 +586,8 @@ func (c *tsoClient) tryConnectToTSO( } // retry several times before falling back to the follower when the network problem happens + ticker := time.NewTicker(retryInterval) + defer ticker.Stop() for i := 0; i < maxRetryTimes; i++ { c.svcDiscovery.ScheduleCheckMemberChanged() cc, url = c.GetTSOAllocatorClientConnByDCLocation(dc) @@ -587,7 +617,7 @@ func (c *tsoClient) tryConnectToTSO( select { case <-dispatcherCtx.Done(): return err - case <-time.After(retryInterval): + case <-ticker.C: } } @@ -757,7 +787,22 @@ func (c *tsoClient) compareAndSwapTS( // all TSOs we get will be [6, 7, 8, 9, 10]. lastTSOInfo.logical stores the logical part of the largest ts returned // last time. if tsoutil.TSLessEqual(physical, firstLogical, lastTSOInfo.physical, lastTSOInfo.logical) { - log.Panic("[tso] timestamp fallback", + if !c.option.allowTSOFallback { + log.Panic("[tso] timestamp fallback", + zap.String("dc-location", dcLocation), + zap.Uint32("keyspace", c.svcDiscovery.GetKeyspaceID()), + zap.String("last-ts", fmt.Sprintf("(%d, %d)", lastTSOInfo.physical, lastTSOInfo.logical)), + zap.String("cur-ts", fmt.Sprintf("(%d, %d)", physical, firstLogical)), + zap.String("last-tso-server", lastTSOInfo.tsoServer), + zap.String("cur-tso-server", curTSOInfo.tsoServer), + zap.Uint32("last-keyspace-group-in-request", lastTSOInfo.reqKeyspaceGroupID), + zap.Uint32("cur-keyspace-group-in-request", curTSOInfo.reqKeyspaceGroupID), + zap.Uint32("last-keyspace-group-in-response", lastTSOInfo.respKeyspaceGroupID), + zap.Uint32("cur-keyspace-group-in-response", curTSOInfo.respKeyspaceGroupID), + zap.Time("last-response-received-at", lastTSOInfo.respReceivedAt), + zap.Time("cur-response-received-at", curTSOInfo.respReceivedAt)) + } + log.Error("[tso] timestamp fallback", zap.String("dc-location", dcLocation), zap.Uint32("keyspace", c.svcDiscovery.GetKeyspaceID()), zap.String("last-ts", fmt.Sprintf("(%d, %d)", lastTSOInfo.physical, lastTSOInfo.logical)), @@ -769,8 +814,7 @@ func (c *tsoClient) compareAndSwapTS( zap.Uint32("last-keyspace-group-in-response", lastTSOInfo.respKeyspaceGroupID), zap.Uint32("cur-keyspace-group-in-response", curTSOInfo.respKeyspaceGroupID), zap.Time("last-response-received-at", lastTSOInfo.respReceivedAt), - zap.Time("cur-response-received-at", curTSOInfo.respReceivedAt), - ) + zap.Time("cur-response-received-at", curTSOInfo.respReceivedAt)) } lastTSOInfo.tsoServer = curTSOInfo.tsoServer lastTSOInfo.reqKeyspaceGroupID = curTSOInfo.reqKeyspaceGroupID diff --git a/client/tso_service_discovery.go b/client/tso_service_discovery.go index cee079634e9..2aeb49e1523 100644 --- a/client/tso_service_discovery.go +++ b/client/tso_service_discovery.go @@ -209,6 +209,8 @@ func (c *tsoServiceDiscovery) retry( maxRetryTimes int, retryInterval time.Duration, f func() error, ) error { var err error + ticker := time.NewTicker(retryInterval) + defer ticker.Stop() for i := 0; i < maxRetryTimes; i++ { if err = f(); err == nil { return nil @@ -216,7 +218,7 @@ func (c *tsoServiceDiscovery) retry( select { case <-c.ctx.Done(): return err - case <-time.After(retryInterval): + case <-ticker.C: } } return errors.WithStack(err) @@ -245,11 +247,13 @@ func (c *tsoServiceDiscovery) startCheckMemberLoop() { ctx, cancel := context.WithCancel(c.ctx) defer cancel() + ticker := time.NewTicker(memberUpdateInterval) + defer ticker.Stop() for { select { case <-c.checkMembershipCh: - case <-time.After(memberUpdateInterval): + case <-ticker.C: case <-ctx.Done(): log.Info("[tso] exit check member loop") return @@ -273,11 +277,6 @@ func (c *tsoServiceDiscovery) GetKeyspaceID() uint32 { return c.keyspaceID.Load() } -// SetKeyspaceID sets the ID of the keyspace -func (c *tsoServiceDiscovery) SetKeyspaceID(keyspaceID uint32) { - c.keyspaceID.Store(keyspaceID) -} - // GetKeyspaceGroupID returns the ID of the keyspace group. If the keyspace group is unknown, // it returns the default keyspace group ID. func (c *tsoServiceDiscovery) GetKeyspaceGroupID() uint32 { @@ -425,12 +424,16 @@ func (c *tsoServiceDiscovery) updateMember() error { return err } + keyspaceID := c.GetKeyspaceID() var keyspaceGroup *tsopb.KeyspaceGroup if len(tsoServerAddr) > 0 { - keyspaceGroup, err = c.findGroupByKeyspaceID(c.GetKeyspaceID(), tsoServerAddr, updateMemberTimeout) + keyspaceGroup, err = c.findGroupByKeyspaceID(keyspaceID, tsoServerAddr, updateMemberTimeout) if err != nil { if c.tsoServerDiscovery.countFailure() { - log.Error("[tso] failed to find the keyspace group", errs.ZapError(err)) + log.Error("[tso] failed to find the keyspace group", + zap.Uint32("keyspace-id-in-request", keyspaceID), + zap.String("tso-server-addr", tsoServerAddr), + errs.ZapError(err)) } return err } @@ -444,6 +447,8 @@ func (c *tsoServiceDiscovery) updateMember() error { c.printFallbackLogOnce.Do(func() { log.Warn("[tso] no tso server address found,"+ " fallback to the legacy path to discover from etcd directly", + zap.Uint32("keyspace-id-in-request", keyspaceID), + zap.String("tso-server-addr", tsoServerAddr), zap.String("discovery-key", c.defaultDiscoveryKey)) }) addrs, err := c.discoverWithLegacyPath() @@ -464,6 +469,14 @@ func (c *tsoServiceDiscovery) updateMember() error { } } + oldGroupID := c.GetKeyspaceGroupID() + if oldGroupID != keyspaceGroup.Id { + log.Info("[tso] the keyspace group changed", + zap.Uint32("keyspace-id", keyspaceGroup.Id), + zap.Uint32("new-keyspace-group-id", keyspaceGroup.Id), + zap.Uint32("old-keyspace-group-id", oldGroupID)) + } + // Initialize the serving addresses from the returned keyspace group info. primaryAddr := "" secondaryAddrs := make([]string, 0) @@ -483,6 +496,8 @@ func (c *tsoServiceDiscovery) updateMember() error { if primarySwitched := !strings.EqualFold(primaryAddr, c.getPrimaryAddr()); primarySwitched { if _, err := c.GetOrCreateGRPCConn(primaryAddr); err != nil { log.Warn("[tso] failed to connect the next primary", + zap.Uint32("keyspace-id-in-request", keyspaceID), + zap.String("tso-server-addr", tsoServerAddr), zap.String("next-primary", primaryAddr), errs.ZapError(err)) return err } @@ -493,6 +508,8 @@ func (c *tsoServiceDiscovery) updateMember() error { c.keyspaceGroupSD.update(keyspaceGroup, primaryAddr, secondaryAddrs, addrs) if primarySwitched { log.Info("[tso] updated keyspace group service discovery info", + zap.Uint32("keyspace-id-in-request", keyspaceID), + zap.String("tso-server-addr", tsoServerAddr), zap.String("keyspace-group-service", keyspaceGroup.String())) if err := c.afterPrimarySwitched(oldPrimary, primaryAddr); err != nil { return err diff --git a/client/tso_stream.go b/client/tso_stream.go index 892512d8559..e3203818938 100644 --- a/client/tso_stream.go +++ b/client/tso_stream.go @@ -87,10 +87,12 @@ func (b *tsoTSOStreamBuilder) build( } func checkStreamTimeout(ctx context.Context, cancel context.CancelFunc, done chan struct{}, timeout time.Duration) { + timer := time.NewTimer(timeout) + defer timer.Stop() select { case <-done: return - case <-time.After(timeout): + case <-timer.C: cancel() case <-ctx.Done(): } diff --git a/cmd/pd-server/main.go b/cmd/pd-server/main.go index 91fac928ca3..c31afc22901 100644 --- a/cmd/pd-server/main.go +++ b/cmd/pd-server/main.go @@ -88,6 +88,8 @@ func NewTSOServiceCommand() *cobra.Command { 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", "info", "log level: debug, info, warn, error, fatal (default 'info')") + cmd.Flags().StringP("log-file", "", "", "log file path") return cmd } @@ -106,6 +108,8 @@ func NewResourceManagerServiceCommand() *cobra.Command { 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", "info", "log level: debug, info, warn, error, fatal (default 'info')") + cmd.Flags().StringP("log-file", "", "", "log file path") return cmd } diff --git a/pkg/election/lease.go b/pkg/election/lease.go index a0db045256f..1e3e66ddcce 100644 --- a/pkg/election/lease.go +++ b/pkg/election/lease.go @@ -109,6 +109,8 @@ func (l *lease) KeepAlive(ctx context.Context) { timeCh := l.keepAliveWorker(ctx, l.leaseTimeout/3) var maxExpire time.Time + timer := time.NewTimer(l.leaseTimeout) + defer timer.Stop() for { select { case t := <-timeCh: @@ -122,7 +124,17 @@ func (l *lease) KeepAlive(ctx context.Context) { l.expireTime.Store(t) } } - case <-time.After(l.leaseTimeout): + // Stop the timer if it's not stopped. + if !timer.Stop() { + select { + case <-timer.C: // try to drain from the channel + default: + } + } + // We need be careful here, see more details in the comments of Timer.Reset. + // https://pkg.go.dev/time@master#Timer.Reset + timer.Reset(l.leaseTimeout) + case <-timer.C: log.Info("lease timeout", zap.Time("expire", l.expireTime.Load().(time.Time)), zap.String("purpose", l.Purpose)) return case <-ctx.Done(): diff --git a/pkg/keyspace/keyspace.go b/pkg/keyspace/keyspace.go index 6e768ebe683..e5cbe1a1c42 100644 --- a/pkg/keyspace/keyspace.go +++ b/pkg/keyspace/keyspace.go @@ -48,11 +48,13 @@ const ( // UserKindKey is the key for user kind in keyspace config. UserKindKey = "user_kind" // TSOKeyspaceGroupIDKey is the key for tso keyspace group id in keyspace config. + // Note: Config[TSOKeyspaceGroupIDKey] is only used to judge whether there is keyspace group id. + // It will not update the keyspace group id when merging or splitting. TSOKeyspaceGroupIDKey = "tso_keyspace_group_id" - // maxEtcdTxnOps is the max value of operations in an etcd txn. The default limit of etcd txn op is 128. + // MaxEtcdTxnOps is the max value of operations in an etcd txn. The default limit of etcd txn op is 128. // We use 120 here to leave some space for other operations. // See: https://github.com/etcd-io/etcd/blob/d3e43d4de6f6d9575b489dd7850a85e37e0f6b6c/server/embed/config.go#L61 - maxEtcdTxnOps = 120 + MaxEtcdTxnOps = 120 ) // Config is the interface for keyspace config. @@ -361,6 +363,8 @@ func (manager *Manager) splitKeyspaceRegion(id uint32, waitRegionSplit bool) (er if region == nil || !bytes.Equal(region.GetStartKey(), txnRightBound) { continue } + // Note: we reset the ticker here to support updating configuration dynamically. + ticker.Reset(manager.config.GetCheckRegionSplitInterval()) case <-timer.C: log.Warn("[keyspace] wait region split timeout", zap.Uint32("keyspace-id", id), @@ -706,7 +710,7 @@ func (manager *Manager) PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID zap.Duration("cost", time.Since(start)), zap.Uint64("patrolled-keyspace-count", patrolledKeyspaceCount), zap.Uint64("assigned-keyspace-count", assignedKeyspaceCount), - zap.Int("batch-size", maxEtcdTxnOps), + zap.Int("batch-size", MaxEtcdTxnOps), zap.Uint32("start-keyspace-id", startKeyspaceID), zap.Uint32("end-keyspace-id", endKeyspaceID), zap.Uint32("current-start-id", currentStartID), @@ -730,7 +734,7 @@ func (manager *Manager) PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID if defaultKeyspaceGroup.IsMerging() { return ErrKeyspaceGroupInMerging(utils.DefaultKeyspaceGroupID) } - keyspaces, err := manager.store.LoadRangeKeyspace(txn, manager.nextPatrolStartID, maxEtcdTxnOps) + keyspaces, err := manager.store.LoadRangeKeyspace(txn, manager.nextPatrolStartID, MaxEtcdTxnOps) if err != nil { return err } @@ -740,9 +744,9 @@ func (manager *Manager) PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID currentStartID = keyspaces[0].GetId() nextStartID = keyspaces[keyspaceNum-1].GetId() + 1 } - // If there are less than `maxEtcdTxnOps` keyspaces or the next start ID reaches the end, + // If there are less than `MaxEtcdTxnOps` keyspaces or the next start ID reaches the end, // there is no need to patrol again. - moreToPatrol = keyspaceNum == maxEtcdTxnOps + moreToPatrol = keyspaceNum == MaxEtcdTxnOps var ( assigned = false keyspaceIDsToUnlock = make([]uint32, 0, keyspaceNum) @@ -781,7 +785,7 @@ func (manager *Manager) PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID err = manager.store.SaveKeyspaceMeta(txn, ks) if err != nil { log.Error("[keyspace] failed to save keyspace meta during patrol", - zap.Int("batch-size", maxEtcdTxnOps), + zap.Int("batch-size", MaxEtcdTxnOps), zap.Uint32("start-keyspace-id", startKeyspaceID), zap.Uint32("end-keyspace-id", endKeyspaceID), zap.Uint32("current-start-id", currentStartID), @@ -795,7 +799,7 @@ func (manager *Manager) PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID err = manager.kgm.store.SaveKeyspaceGroup(txn, defaultKeyspaceGroup) if err != nil { log.Error("[keyspace] failed to save default keyspace group meta during patrol", - zap.Int("batch-size", maxEtcdTxnOps), + zap.Int("batch-size", MaxEtcdTxnOps), zap.Uint32("start-keyspace-id", startKeyspaceID), zap.Uint32("end-keyspace-id", endKeyspaceID), zap.Uint32("current-start-id", currentStartID), @@ -808,7 +812,9 @@ func (manager *Manager) PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID if err != nil { return err } + manager.kgm.Lock() manager.kgm.groups[endpoint.StringUserKind(defaultKeyspaceGroup.UserKind)].Put(defaultKeyspaceGroup) + manager.kgm.Unlock() // If all keyspaces in the current batch are assigned, update the next start ID. manager.nextPatrolStartID = nextStartID } diff --git a/pkg/keyspace/keyspace_test.go b/pkg/keyspace/keyspace_test.go index b06921e48db..27e7de359ee 100644 --- a/pkg/keyspace/keyspace_test.go +++ b/pkg/keyspace/keyspace_test.go @@ -405,7 +405,7 @@ func (suite *keyspaceTestSuite) TestPatrolKeyspaceAssignment() { func (suite *keyspaceTestSuite) TestPatrolKeyspaceAssignmentInBatch() { re := suite.Require() // Create some keyspaces without any keyspace group. - for i := 1; i < maxEtcdTxnOps*2+1; i++ { + for i := 1; i < MaxEtcdTxnOps*2+1; i++ { now := time.Now().Unix() err := suite.manager.saveNewKeyspace(&keyspacepb.KeyspaceMeta{ Id: uint32(i), @@ -420,7 +420,7 @@ func (suite *keyspaceTestSuite) TestPatrolKeyspaceAssignmentInBatch() { defaultKeyspaceGroup, err := suite.manager.kgm.GetKeyspaceGroupByID(utils.DefaultKeyspaceGroupID) re.NoError(err) re.NotNil(defaultKeyspaceGroup) - for i := 1; i < maxEtcdTxnOps*2+1; i++ { + for i := 1; i < MaxEtcdTxnOps*2+1; i++ { re.NotContains(defaultKeyspaceGroup.Keyspaces, uint32(i)) } // Patrol the keyspace assignment. @@ -430,7 +430,7 @@ func (suite *keyspaceTestSuite) TestPatrolKeyspaceAssignmentInBatch() { defaultKeyspaceGroup, err = suite.manager.kgm.GetKeyspaceGroupByID(utils.DefaultKeyspaceGroupID) re.NoError(err) re.NotNil(defaultKeyspaceGroup) - for i := 1; i < maxEtcdTxnOps*2+1; i++ { + for i := 1; i < MaxEtcdTxnOps*2+1; i++ { re.Contains(defaultKeyspaceGroup.Keyspaces, uint32(i)) } } @@ -438,7 +438,7 @@ func (suite *keyspaceTestSuite) TestPatrolKeyspaceAssignmentInBatch() { func (suite *keyspaceTestSuite) TestPatrolKeyspaceAssignmentWithRange() { re := suite.Require() // Create some keyspaces without any keyspace group. - for i := 1; i < maxEtcdTxnOps*2+1; i++ { + for i := 1; i < MaxEtcdTxnOps*2+1; i++ { now := time.Now().Unix() err := suite.manager.saveNewKeyspace(&keyspacepb.KeyspaceMeta{ Id: uint32(i), @@ -453,14 +453,14 @@ func (suite *keyspaceTestSuite) TestPatrolKeyspaceAssignmentWithRange() { defaultKeyspaceGroup, err := suite.manager.kgm.GetKeyspaceGroupByID(utils.DefaultKeyspaceGroupID) re.NoError(err) re.NotNil(defaultKeyspaceGroup) - for i := 1; i < maxEtcdTxnOps*2+1; i++ { + for i := 1; i < MaxEtcdTxnOps*2+1; i++ { re.NotContains(defaultKeyspaceGroup.Keyspaces, uint32(i)) } - // Patrol the keyspace assignment with range [maxEtcdTxnOps/2, maxEtcdTxnOps/2+maxEtcdTxnOps+1] + // Patrol the keyspace assignment with range [MaxEtcdTxnOps/2, MaxEtcdTxnOps/2+MaxEtcdTxnOps+1] // to make sure the range crossing the boundary of etcd transaction operation limit. var ( - startKeyspaceID = uint32(maxEtcdTxnOps / 2) - endKeyspaceID = startKeyspaceID + maxEtcdTxnOps + 1 + startKeyspaceID = uint32(MaxEtcdTxnOps / 2) + endKeyspaceID = startKeyspaceID + MaxEtcdTxnOps + 1 ) err = suite.manager.PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID) re.NoError(err) @@ -468,7 +468,7 @@ func (suite *keyspaceTestSuite) TestPatrolKeyspaceAssignmentWithRange() { defaultKeyspaceGroup, err = suite.manager.kgm.GetKeyspaceGroupByID(utils.DefaultKeyspaceGroupID) re.NoError(err) re.NotNil(defaultKeyspaceGroup) - for i := 1; i < maxEtcdTxnOps*2+1; i++ { + for i := 1; i < MaxEtcdTxnOps*2+1; i++ { keyspaceID := uint32(i) if keyspaceID >= startKeyspaceID && keyspaceID <= endKeyspaceID { re.Contains(defaultKeyspaceGroup.Keyspaces, keyspaceID) diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index dd9319e806f..084380549fc 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/balancer" "github.com/tikv/pd/pkg/mcs/discovery" @@ -387,6 +388,20 @@ func (m *GroupManager) getKeyspaceConfigByKindLocked(userKind endpoint.UserKind) return config, nil } +// GetGroupByKeyspaceID returns the keyspace group ID for the given keyspace ID. +func (m *GroupManager) GetGroupByKeyspaceID(id uint32) (uint32, error) { + m.RLock() + defer m.RUnlock() + for _, groups := range m.groups { + for _, group := range groups.GetAll() { + if slice.Contains(group.Keyspaces, id) { + return group.ID, nil + } + } + } + return 0, ErrKeyspaceNotInAnyKeyspaceGroup +} + var failpointOnce sync.Once // UpdateKeyspaceForGroup updates the keyspace field for the keyspace group. @@ -887,7 +902,7 @@ func (m *GroupManager) MergeKeyspaceGroups(mergeTargetID uint32, mergeList []uin // - Load and delete the keyspace groups in the merge list. // - Load and update the target keyspace group. // So we pre-check the number of operations to avoid exceeding the maximum number of etcd transaction. - if (mergeListNum+1)*2 > maxEtcdTxnOps { + if (mergeListNum+1)*2 > MaxEtcdTxnOps { return ErrExceedMaxEtcdTxnOps } if slice.Contains(mergeList, utils.DefaultKeyspaceGroupID) { @@ -1010,3 +1025,128 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { zap.Reflect("merge-list", mergeList)) return nil } + +// MergeAllIntoDefaultKeyspaceGroup merges all other keyspace groups into the default keyspace group. +func (m *GroupManager) MergeAllIntoDefaultKeyspaceGroup() error { + defer logutil.LogPanic() + // Since we don't take the default keyspace group into account, + // the number of unmerged keyspace groups is -1. + unmergedGroupNum := -1 + // Calculate the total number of keyspace groups to merge. + for _, groups := range m.groups { + unmergedGroupNum += groups.Len() + } + mergedGroupNum := 0 + // Start to merge all keyspace groups into the default one. + for userKind, groups := range m.groups { + mergeNum := groups.Len() + log.Info("start to merge all keyspace groups into the default one", + zap.Stringer("user-kind", userKind), + zap.Int("merge-num", mergeNum), + zap.Int("merged-group-num", mergedGroupNum), + zap.Int("unmerged-group-num", unmergedGroupNum)) + if mergeNum == 0 { + continue + } + var ( + maxBatchSize = MaxEtcdTxnOps/2 - 1 + groupsToMerge = make([]uint32, 0, maxBatchSize) + ) + for idx, group := range groups.GetAll() { + if group.ID == utils.DefaultKeyspaceGroupID { + continue + } + groupsToMerge = append(groupsToMerge, group.ID) + if len(groupsToMerge) < maxBatchSize && idx < mergeNum-1 { + continue + } + log.Info("merge keyspace groups into the default one", + zap.Int("index", idx), + zap.Int("batch-size", len(groupsToMerge)), + zap.Int("merge-num", mergeNum), + zap.Int("merged-group-num", mergedGroupNum), + zap.Int("unmerged-group-num", unmergedGroupNum)) + // Reach the batch size, merge them into the default keyspace group. + if err := m.MergeKeyspaceGroups(utils.DefaultKeyspaceGroupID, groupsToMerge); err != nil { + log.Error("failed to merge all keyspace groups into the default one", + zap.Int("index", idx), + zap.Int("batch-size", len(groupsToMerge)), + zap.Int("merge-num", mergeNum), + zap.Int("merged-group-num", mergedGroupNum), + zap.Int("unmerged-group-num", unmergedGroupNum), + zap.Error(err)) + return err + } + // Wait for the merge to finish. + ctx, cancel := context.WithTimeout(m.ctx, time.Minute) + ticker := time.NewTicker(time.Second) + checkLoop: + for { + select { + case <-ctx.Done(): + log.Info("cancel merging all keyspace groups into the default one", + zap.Int("index", idx), + zap.Int("batch-size", len(groupsToMerge)), + zap.Int("merge-num", mergeNum), + zap.Int("merged-group-num", mergedGroupNum), + zap.Int("unmerged-group-num", unmergedGroupNum)) + cancel() + ticker.Stop() + return nil + case <-ticker.C: + kg, err := m.GetKeyspaceGroupByID(utils.DefaultKeyspaceGroupID) + if err != nil { + log.Error("failed to check the default keyspace group merge state", + zap.Int("index", idx), + zap.Int("batch-size", len(groupsToMerge)), + zap.Int("merge-num", mergeNum), + zap.Int("merged-group-num", mergedGroupNum), + zap.Int("unmerged-group-num", unmergedGroupNum), + zap.Error(err)) + cancel() + ticker.Stop() + return err + } + if !kg.IsMergeTarget() { + break checkLoop + } + } + } + cancel() + ticker.Stop() + mergedGroupNum += len(groupsToMerge) + unmergedGroupNum -= len(groupsToMerge) + groupsToMerge = groupsToMerge[:0] + } + } + log.Info("finish merging all keyspace groups into the default one", + zap.Int("merged-group-num", mergedGroupNum), + zap.Int("unmerged-group-num", unmergedGroupNum)) + return nil +} + +// GetKeyspaceGroupPrimaryByID returns the primary node of the keyspace group by ID. +func (m *GroupManager) GetKeyspaceGroupPrimaryByID(id uint32) (string, error) { + // check if the keyspace group exists + kg, err := m.GetKeyspaceGroupByID(id) + if err != nil { + return "", err + } + if kg == nil { + return "", ErrKeyspaceGroupNotExists(id) + } + + rootPath := endpoint.TSOSvcRootPath(m.clusterID) + primaryPath := endpoint.KeyspaceGroupPrimaryPath(rootPath, id) + leader := &tsopb.Participant{} + ok, _, err := etcdutil.GetProtoMsgWithModRev(m.client, primaryPath, leader) + if err != nil { + return "", err + } + if !ok { + return "", ErrKeyspaceGroupPrimaryNotFound + } + // The format of leader name is address-groupID. + contents := strings.Split(leader.GetName(), "-") + return contents[0], err +} diff --git a/pkg/keyspace/tso_keyspace_group_test.go b/pkg/keyspace/tso_keyspace_group_test.go index 4c2d989da8d..55e162fb970 100644 --- a/pkg/keyspace/tso_keyspace_group_test.go +++ b/pkg/keyspace/tso_keyspace_group_test.go @@ -448,7 +448,7 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupMerge() { err = suite.kgm.MergeKeyspaceGroups(4, []uint32{5}) re.ErrorContains(err, ErrKeyspaceGroupNotExists(5).Error()) // merge with the number of keyspace groups exceeds the limit - err = suite.kgm.MergeKeyspaceGroups(1, make([]uint32, maxEtcdTxnOps/2)) + err = suite.kgm.MergeKeyspaceGroups(1, make([]uint32, MaxEtcdTxnOps/2)) re.ErrorIs(err, ErrExceedMaxEtcdTxnOps) // merge the default keyspace group err = suite.kgm.MergeKeyspaceGroups(1, []uint32{utils.DefaultKeyspaceGroupID}) diff --git a/pkg/keyspace/util.go b/pkg/keyspace/util.go index d6844cf4492..4d7feb7667f 100644 --- a/pkg/keyspace/util.go +++ b/pkg/keyspace/util.go @@ -70,6 +70,8 @@ var ( } // ErrKeyspaceNotInKeyspaceGroup is used to indicate target keyspace is not in this keyspace group. ErrKeyspaceNotInKeyspaceGroup = errors.New("keyspace is not in this keyspace group") + // ErrKeyspaceNotInAnyKeyspaceGroup is used to indicate target keyspace is not in any keyspace group. + ErrKeyspaceNotInAnyKeyspaceGroup = errors.New("keyspace is not in any keyspace group") // ErrNodeNotInKeyspaceGroup is used to indicate the tso node is not in this keyspace group. ErrNodeNotInKeyspaceGroup = errors.New("the tso node is not in this keyspace group") // ErrKeyspaceGroupNotEnoughReplicas is used to indicate not enough replicas in the keyspace group. @@ -94,6 +96,9 @@ var ( } // Only keyspaces in the state specified by allowChangeConfig are allowed to change their config. allowChangeConfig = []keyspacepb.KeyspaceState{keyspacepb.KeyspaceState_ENABLED, keyspacepb.KeyspaceState_DISABLED} + + // ErrKeyspaceGroupPrimaryNotFound is used to indicate primary of target keyspace group does not exist. + ErrKeyspaceGroupPrimaryNotFound = errors.New("primary of keyspace group does not exist") ) // validateID check if keyspace falls within the acceptable range. diff --git a/pkg/mcs/discovery/discover.go b/pkg/mcs/discovery/discover.go index 6d939fde540..00e168114b0 100644 --- a/pkg/mcs/discovery/discover.go +++ b/pkg/mcs/discovery/discover.go @@ -21,7 +21,7 @@ import ( // Discover is used to get all the service instances of the specified service name. func Discover(cli *clientv3.Client, clusterID, serviceName string) ([]string, error) { - key := ServicePath(clusterID, serviceName) + "/" + key := ServicePath(clusterID, serviceName) endKey := clientv3.GetPrefixRangeEnd(key) withRange := clientv3.WithRange(endKey) diff --git a/pkg/mcs/discovery/key_path.go b/pkg/mcs/discovery/key_path.go index 4eb339dd5db..b7bf9d1cac3 100644 --- a/pkg/mcs/discovery/key_path.go +++ b/pkg/mcs/discovery/key_path.go @@ -17,24 +17,25 @@ package discovery import ( "strconv" "strings" + + "github.com/tikv/pd/pkg/mcs/utils" ) const ( - registryPrefix = "/ms" - registryKey = "registry" + registryKey = "registry" ) // RegistryPath returns the full path to store microservice addresses. func RegistryPath(clusterID, serviceName, serviceAddr string) string { - return strings.Join([]string{registryPrefix, clusterID, serviceName, registryKey, serviceAddr}, "/") + return strings.Join([]string{utils.MicroserviceRootPath, clusterID, serviceName, registryKey, serviceAddr}, "/") } // ServicePath returns the path to store microservice addresses. func ServicePath(clusterID, serviceName string) string { - return strings.Join([]string{registryPrefix, clusterID, serviceName, registryKey}, "/") + return strings.Join([]string{utils.MicroserviceRootPath, clusterID, serviceName, registryKey, ""}, "/") } // TSOPath returns the path to store TSO addresses. func TSOPath(clusterID uint64) string { - return ServicePath(strconv.FormatUint(clusterID, 10), "tso") + "/" + return ServicePath(strconv.FormatUint(clusterID, 10), "tso") } diff --git a/pkg/mcs/discovery/register.go b/pkg/mcs/discovery/register.go index 3e08d9b49cf..1e7800295de 100644 --- a/pkg/mcs/discovery/register.go +++ b/pkg/mcs/discovery/register.go @@ -86,6 +86,7 @@ func (sr *ServiceRegister) Register() error { select { case <-sr.ctx.Done(): log.Info("exit register process", zap.String("key", sr.key)) + t.Stop() return default: } @@ -94,11 +95,13 @@ func (sr *ServiceRegister) Register() error { resp, err := sr.cli.Grant(sr.ctx, sr.ttl) if err != nil { log.Error("grant lease failed", zap.String("key", sr.key), zap.Error(err)) + t.Stop() continue } if _, err := sr.cli.Put(sr.ctx, sr.key, sr.value, clientv3.WithLease(resp.ID)); err != nil { log.Error("put the key failed", zap.String("key", sr.key), zap.Error(err)) + t.Stop() continue } } diff --git a/pkg/mcs/resource_manager/server/config.go b/pkg/mcs/resource_manager/server/config.go index 35a18151b05..39ae3c86bed 100644 --- a/pkg/mcs/resource_manager/server/config.go +++ b/pkg/mcs/resource_manager/server/config.go @@ -64,6 +64,9 @@ type Config struct { Security configutil.SecurityConfig `toml:"security" json:"security"` + // WarningMsgs contains all warnings during parsing. + WarningMsgs []string + // RequestUnit is the configuration determines the coefficients of the RRU and WRU cost. // This configuration should be modified carefully. RequestUnit RequestUnitConfig @@ -145,11 +148,9 @@ func (c *Config) Parse(flagSet *pflag.FlagSet) error { // Adjust is used to adjust the resource manager configurations. func (c *Config) Adjust(meta *toml.MetaData, reloading bool) error { configMetaData := configutil.NewConfigMetadata(meta) - warningMsgs := make([]string, 0) if err := configMetaData.CheckUndecoded(); err != nil { - warningMsgs = append(warningMsgs, err.Error()) + c.WarningMsgs = append(c.WarningMsgs, err.Error()) } - configutil.PrintConfigCheckMsg(os.Stdout, warningMsgs) if c.Name == "" { hostname, err := os.Hostname() diff --git a/pkg/mcs/resource_manager/server/server.go b/pkg/mcs/resource_manager/server/server.go index 29633662652..90fdf9a5077 100644 --- a/pkg/mcs/resource_manager/server/server.go +++ b/pkg/mcs/resource_manager/server/server.go @@ -29,7 +29,9 @@ import ( "time" grpcprometheus "github.com/grpc-ecosystem/go-grpc-prometheus" + "github.com/pingcap/kvproto/pkg/diagnosticspb" "github.com/pingcap/log" + "github.com/pingcap/sysutil" "github.com/soheilhy/cmux" "github.com/spf13/cobra" "github.com/tikv/pd/pkg/errs" @@ -47,8 +49,11 @@ import ( // Server is the resource manager server, and it implements bs.Server. // nolint type Server struct { + diagnosticspb.DiagnosticsServer // Server state. 0 is not running, 1 is running. isRunning int64 + // Server start timestamp + startTimestamp int64 ctx context.Context serverLoopWg sync.WaitGroup @@ -185,9 +190,11 @@ func (s *Server) startGRPCServer(l net.Listener) { gs.GracefulStop() close(done) }() + timer := time.NewTimer(utils.DefaultGRPCGracefulStopTimeout) + defer timer.Stop() select { case <-done: - case <-time.After(utils.DefaultGRPCGracefulStopTimeout): + case <-timer.C: log.Info("stopping grpc gracefully is taking longer than expected and force stopping now", zap.Duration("default", utils.DefaultGRPCGracefulStopTimeout)) gs.Stop() } @@ -294,13 +301,15 @@ func (s *Server) startServer() error { return nil } -// NewServer creates a new resource manager server. -func NewServer(ctx context.Context, cfg *Config) *Server { - return &Server{ - name: cfg.Name, - ctx: ctx, - cfg: cfg, +// CreateServer creates the Server +func CreateServer(ctx context.Context, cfg *Config) *Server { + svr := &Server{ + DiagnosticsServer: sysutil.NewDiagnosticsServer(cfg.Log.File.Filename), + startTimestamp: time.Now().Unix(), + cfg: cfg, + ctx: ctx, } + return svr } // CreateServerWrapper encapsulates the configuration/log/metrics initialization and create the server @@ -334,15 +343,14 @@ func CreateServerWrapper(cmd *cobra.Command, args []string) { // Flushing any buffered log entries defer log.Sync() - versioninfo.Log("resource manager") - log.Info("resource manager config", zap.Reflect("config", cfg)) + versioninfo.Log("Resource Manager") + log.Info("Resource Manager config", zap.Reflect("config", cfg)) grpcprometheus.EnableHandlingTimeHistogram() - metricutil.Push(&cfg.Metric) ctx, cancel := context.WithCancel(context.Background()) - svr := NewServer(ctx, cfg) + svr := CreateServer(ctx, cfg) sc := make(chan os.Signal, 1) signal.Notify(sc, diff --git a/pkg/mcs/resource_manager/server/testutil.go b/pkg/mcs/resource_manager/server/testutil.go index 171e5c84e45..ca1ec673f2e 100644 --- a/pkg/mcs/resource_manager/server/testutil.go +++ b/pkg/mcs/resource_manager/server/testutil.go @@ -36,7 +36,7 @@ func NewTestServer(ctx context.Context, re *require.Assertions, cfg *Config) (*S // Flushing any buffered log entries defer log.Sync() - s := NewServer(ctx, cfg) + s := CreateServer(ctx, cfg) if err = s.Run(); err != nil { return nil, nil, err } diff --git a/pkg/mcs/tso/server/server.go b/pkg/mcs/tso/server/server.go index 0075da9ba25..d601e23448a 100644 --- a/pkg/mcs/tso/server/server.go +++ b/pkg/mcs/tso/server/server.go @@ -24,7 +24,6 @@ import ( "net/url" "os" "os/signal" - "path" "strconv" "strings" "sync" @@ -47,6 +46,7 @@ import ( "github.com/tikv/pd/pkg/mcs/discovery" mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/member" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/systimemon" "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/etcdutil" @@ -64,12 +64,6 @@ import ( ) const ( - // pdRootPath is the old path for storing the tso related root path. - pdRootPath = "/pd" - msServiceRootPath = "/ms" - // tsoSvcRootPathFormat defines the root path for all etcd paths used for different purposes. - // format: "/ms/{cluster_id}/tso". - tsoSvcRootPathFormat = msServiceRootPath + "/%d/" + mcsutils.TSOServiceName // maxRetryTimesWaitAPIService is the max retry times for initializing the cluster ID. maxRetryTimesWaitAPIService = 360 // retryIntervalWaitAPIService is the interval to retry. @@ -536,8 +530,8 @@ func (s *Server) startServer() (err error) { // Initialize the TSO service. s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx) - legacySvcRootPath := path.Join(pdRootPath, strconv.FormatUint(s.clusterID, 10)) - tsoSvcRootPath := fmt.Sprintf(tsoSvcRootPathFormat, s.clusterID) + legacySvcRootPath := endpoint.LegacyRootPath(s.clusterID) + tsoSvcRootPath := endpoint.TSOSvcRootPath(s.clusterID) s.serviceID = &discovery.ServiceRegistryEntry{ServiceAddr: s.cfg.AdvertiseListenAddr} s.keyspaceGroupManager = tso.NewKeyspaceGroupManager( s.serverLoopCtx, s.serviceID, s.etcdClient, s.httpClient, s.cfg.AdvertiseListenAddr, @@ -596,6 +590,8 @@ func (s *Server) waitAPIServiceReady() error { ready bool err error ) + ticker := time.NewTicker(retryIntervalWaitAPIService) + defer ticker.Stop() for i := 0; i < maxRetryTimesWaitAPIService; i++ { ready, err = s.isAPIServiceReady() if err == nil && ready { @@ -605,7 +601,7 @@ func (s *Server) waitAPIServiceReady() error { select { case <-s.ctx.Done(): return errors.New("context canceled while waiting api server ready") - case <-time.After(retryIntervalWaitAPIService): + case <-ticker.C: } } if err != nil { diff --git a/pkg/mcs/utils/constant.go b/pkg/mcs/utils/constant.go index c87cec16a64..26c0f369819 100644 --- a/pkg/mcs/utils/constant.go +++ b/pkg/mcs/utils/constant.go @@ -49,8 +49,8 @@ const ( // We also reserved 0 for the keyspace group for the same purpose. DefaultKeyspaceGroupID = uint32(0) - // MicroserviceKey is the key of microservice. - MicroserviceKey = "ms" + // MicroserviceRootPath is the root path of microservice in etcd. + MicroserviceRootPath = "/ms" // APIServiceName is the name of api server. APIServiceName = "api" // TSOServiceName is the name of tso server. @@ -59,6 +59,8 @@ const ( ResourceManagerServiceName = "resource_manager" // KeyspaceGroupsKey is the path component of keyspace groups. KeyspaceGroupsKey = "keyspace_groups" + // KeyspaceGroupsPrimaryKey is the path component of primary for keyspace groups. + KeyspaceGroupsPrimaryKey = "primary" // MaxKeyspaceGroupCount is the max count of keyspace groups. keyspace group in tso // is the sharding unit, i.e., by the definition here, the max count of the shards diff --git a/pkg/mcs/utils/util.go b/pkg/mcs/utils/util.go index 1d66fc08c1c..a89c6c72b76 100644 --- a/pkg/mcs/utils/util.go +++ b/pkg/mcs/utils/util.go @@ -37,6 +37,8 @@ const ( // InitClusterID initializes the cluster ID. func InitClusterID(ctx context.Context, client *clientv3.Client) (id uint64, err error) { + ticker := time.NewTicker(retryInterval) + defer ticker.Stop() for i := 0; i < maxRetryTimes; i++ { if clusterID, err := etcdutil.GetClusterID(client, clusterIDPath); err == nil && clusterID != 0 { return clusterID, nil @@ -44,7 +46,7 @@ func InitClusterID(ctx context.Context, client *clientv3.Client) (id uint64, err select { case <-ctx.Done(): return 0, err - case <-time.After(retryInterval): + case <-ticker.C: } } return 0, errors.Errorf("failed to init cluster ID after retrying %d times", maxRetryTimes) diff --git a/pkg/storage/endpoint/key_path.go b/pkg/storage/endpoint/key_path.go index 03a0d6d5106..2a72a87f396 100644 --- a/pkg/storage/endpoint/key_path.go +++ b/pkg/storage/endpoint/key_path.go @@ -25,6 +25,7 @@ import ( ) const ( + pdRootPath = "/pd" clusterPath = "raft" configPath = "config" serviceMiddlewarePath = "service_middleware" @@ -53,11 +54,14 @@ const ( resourceGroupStatesPath = "states" requestUnitConfigPath = "ru_config" // tso storage endpoint has prefix `tso` - tsoServiceKey = utils.TSOServiceName - timestampKey = "timestamp" + tsoServiceKey = utils.TSOServiceName + globalTSOAllocatorEtcdPrefix = "gta" + // TimestampKey is the key of timestamp oracle used for the suffix. + TimestampKey = "timestamp" - tsoKeyspaceGroupPrefix = tsoServiceKey + "/" + utils.KeyspaceGroupsKey - keyspaceGroupMembershipKey = "membership" + tsoKeyspaceGroupPrefix = tsoServiceKey + "/" + utils.KeyspaceGroupsKey + keyspaceGroupsMembershipKey = "membership" + keyspaceGroupsElectionKey = "election" // we use uint64 to represent ID, the max length of uint64 is 20. keyLen = 20 @@ -228,13 +232,13 @@ func EncodeKeyspaceID(spaceID uint32) string { // KeyspaceGroupIDPrefix returns the prefix of keyspace group id. // Path: tso/keyspace_groups/membership func KeyspaceGroupIDPrefix() string { - return path.Join(tsoKeyspaceGroupPrefix, keyspaceGroupMembershipKey) + return path.Join(tsoKeyspaceGroupPrefix, keyspaceGroupsMembershipKey) } // KeyspaceGroupIDPath returns the path to keyspace id from the given name. // Path: tso/keyspace_groups/membership/{id} func KeyspaceGroupIDPath(id uint32) string { - return path.Join(tsoKeyspaceGroupPrefix, keyspaceGroupMembershipKey, encodeKeyspaceGroupID(id)) + return path.Join(tsoKeyspaceGroupPrefix, keyspaceGroupsMembershipKey, encodeKeyspaceGroupID(id)) } // GetCompiledKeyspaceGroupIDRegexp returns the compiled regular expression for matching keyspace group id. @@ -243,6 +247,54 @@ 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(clusterID uint64) string { + return svcRootPath(clusterID, utils.ResourceManagerServiceName) +} + +// TSOSvcRootPath returns the root path of tso service. +// Path: /ms/{cluster_id}/tso +func TSOSvcRootPath(clusterID uint64) string { + return svcRootPath(clusterID, utils.TSOServiceName) +} + +func svcRootPath(clusterID uint64, svcName string) string { + c := strconv.FormatUint(clusterID, 10) + return path.Join(utils.MicroserviceRootPath, c, svcName) +} + +// LegacyRootPath returns the root path of legacy pd service. +// Path: /pd/{cluster_id} +func LegacyRootPath(clusterID uint64) string { + return path.Join(pdRootPath, strconv.FormatUint(clusterID, 10)) +} + +// KeyspaceGroupPrimaryPath returns the path of keyspace group primary. +// default keyspace group: "/ms/{cluster_id}/tso/00000/primary". +// non-default keyspace group: "/ms/{cluster_id}/tso/keyspace_groups/election/{group}/primary". +func KeyspaceGroupPrimaryPath(rootPath string, keyspaceGroupID uint32) string { + electionPath := KeyspaceGroupsElectionPath(rootPath, keyspaceGroupID) + return path.Join(electionPath, utils.KeyspaceGroupsPrimaryKey) +} + +// KeyspaceGroupsElectionPath returns the path of keyspace groups election. +// default keyspace group: "/ms/{cluster_id}/tso/00000". +// non-default keyspace group: "/ms/{cluster_id}/tso/keyspace_groups/election/{group}". +func KeyspaceGroupsElectionPath(rootPath string, keyspaceGroupID uint32) string { + if keyspaceGroupID == utils.DefaultKeyspaceGroupID { + return path.Join(rootPath, "00000") + } + return path.Join(rootPath, utils.KeyspaceGroupsKey, keyspaceGroupsElectionKey, fmt.Sprintf("%05d", keyspaceGroupID)) +} + +// GetCompiledNonDefaultIDRegexp returns the compiled regular expression for matching non-default keyspace group id. +func GetCompiledNonDefaultIDRegexp(clusterID uint64) *regexp.Regexp { + rootPath := TSOSvcRootPath(clusterID) + pattern := strings.Join([]string{rootPath, utils.KeyspaceGroupsKey, keyspaceGroupsElectionKey, `(\d{5})`, utils.KeyspaceGroupsPrimaryKey + `$`}, "/") + return regexp.MustCompile(pattern) +} + // encodeKeyspaceGroupID from uint32 to string. func encodeKeyspaceGroupID(groupID uint32) string { return fmt.Sprintf("%05d", groupID) @@ -261,3 +313,34 @@ func buildPath(withSuffix bool, str ...string) string { } return sb.String() } + +// KeyspaceGroupTSPath constructs the timestampOracle path prefix, which is: +// 1. for the default keyspace group: +// "" in /pd/{cluster_id}/timestamp +// 2. for the non-default keyspace groups: +// {group}/gta in /ms/{cluster_id}/tso/{group}/gta/timestamp +func KeyspaceGroupTSPath(groupID uint32) string { + if groupID == utils.DefaultKeyspaceGroupID { + return "" + } + return path.Join(fmt.Sprintf("%05d", groupID), globalTSOAllocatorEtcdPrefix) +} + +// TimestampPath returns the timestamp path for the given timestamp oracle path prefix. +func TimestampPath(tsPath string) string { + return path.Join(tsPath, TimestampKey) +} + +// FullTimestampPath returns the full timestamp path. +// 1. for the default keyspace group: +// /pd/{cluster_id}/timestamp +// 2. for the non-default keyspace groups: +// /ms/{cluster_id}/tso/{group}/gta/timestamp +func FullTimestampPath(clusterID uint64, groupID uint32) string { + rootPath := TSOSvcRootPath(clusterID) + tsPath := TimestampPath(KeyspaceGroupTSPath(groupID)) + if groupID == utils.DefaultKeyspaceGroupID { + rootPath = LegacyRootPath(clusterID) + } + return path.Join(rootPath, tsPath) +} diff --git a/pkg/storage/endpoint/tso.go b/pkg/storage/endpoint/tso.go index e67f5d33f4f..f0aa9d8cde0 100644 --- a/pkg/storage/endpoint/tso.go +++ b/pkg/storage/endpoint/tso.go @@ -31,6 +31,7 @@ import ( type TSOStorage interface { LoadTimestamp(prefix string) (time.Time, error) SaveTimestamp(key string, ts time.Time) error + DeleteTimestamp(key string) error } var _ TSOStorage = (*StorageEndpoint)(nil) @@ -51,7 +52,7 @@ func (se *StorageEndpoint) LoadTimestamp(prefix string) (time.Time, error) { maxTSWindow := typeutil.ZeroTime for i, key := range keys { key := strings.TrimSpace(key) - if !strings.HasSuffix(key, timestampKey) { + if !strings.HasSuffix(key, TimestampKey) { continue } tsWindow, err := typeutil.ParseTimestamp([]byte(values[i])) @@ -89,3 +90,10 @@ func (se *StorageEndpoint) SaveTimestamp(key string, ts time.Time) error { return txn.Save(key, string(data)) }) } + +// DeleteTimestamp deletes the timestamp from the storage. +func (se *StorageEndpoint) DeleteTimestamp(key string) error { + return se.RunInTxn(context.Background(), func(txn kv.Txn) error { + return txn.Remove(key) + }) +} diff --git a/pkg/storage/storage_tso_test.go b/pkg/storage/storage_tso_test.go index b8f31cc377c..1dbba289512 100644 --- a/pkg/storage/storage_tso_test.go +++ b/pkg/storage/storage_tso_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" @@ -42,9 +43,8 @@ func TestSaveLoadTimestamp(t *testing.T) { rootPath := path.Join("/pd", strconv.FormatUint(100, 10)) storage := NewStorageWithEtcdBackend(client, rootPath) - key := "timestamp" expectedTS := time.Now().Round(0) - err = storage.SaveTimestamp(key, expectedTS) + err = storage.SaveTimestamp(endpoint.TimestampKey, expectedTS) re.NoError(err) ts, err := storage.LoadTimestamp("") re.NoError(err) @@ -68,16 +68,15 @@ func TestGlobalLocalTimestamp(t *testing.T) { storage := NewStorageWithEtcdBackend(client, rootPath) ltaKey := "lta" - timestampKey := "timestamp" dc1LocationKey, dc2LocationKey := "dc1", "dc2" localTS1 := time.Now().Round(0) - l1 := path.Join(ltaKey, dc1LocationKey, timestampKey) - l2 := path.Join(ltaKey, dc2LocationKey, timestampKey) + l1 := path.Join(ltaKey, dc1LocationKey, endpoint.TimestampKey) + l2 := path.Join(ltaKey, dc2LocationKey, endpoint.TimestampKey) err = storage.SaveTimestamp(l1, localTS1) re.NoError(err) globalTS := time.Now().Round(0) - err = storage.SaveTimestamp(timestampKey, globalTS) + err = storage.SaveTimestamp(endpoint.TimestampKey, globalTS) re.NoError(err) localTS2 := time.Now().Round(0) err = storage.SaveTimestamp(l2, localTS2) @@ -108,14 +107,12 @@ func TestTimestampTxn(t *testing.T) { rootPath := path.Join("/pd", strconv.FormatUint(100, 10)) storage := NewStorageWithEtcdBackend(client, rootPath) - timestampKey := "timestamp" - globalTS1 := time.Now().Round(0) - err = storage.SaveTimestamp(timestampKey, globalTS1) + err = storage.SaveTimestamp(endpoint.TimestampKey, globalTS1) re.NoError(err) globalTS2 := globalTS1.Add(-time.Millisecond).Round(0) - err = storage.SaveTimestamp(timestampKey, globalTS2) + err = storage.SaveTimestamp(endpoint.TimestampKey, globalTS2) re.Error(err) ts, err := storage.LoadTimestamp("") diff --git a/pkg/tso/allocator_manager.go b/pkg/tso/allocator_manager.go index 754f69cb664..7834c2b275b 100644 --- a/pkg/tso/allocator_manager.go +++ b/pkg/tso/allocator_manager.go @@ -46,13 +46,12 @@ import ( const ( // GlobalDCLocation is the Global TSO Allocator's DC location label. - GlobalDCLocation = "global" - checkStep = time.Minute - patrolStep = time.Second - defaultAllocatorLeaderLease = 3 - globalTSOAllocatorEtcdPrefix = "gta" - localTSOAllocatorEtcdPrefix = "lta" - localTSOSuffixEtcdPrefix = "lts" + GlobalDCLocation = "global" + checkStep = time.Minute + patrolStep = time.Second + defaultAllocatorLeaderLease = 3 + localTSOAllocatorEtcdPrefix = "lta" + localTSOSuffixEtcdPrefix = "lts" ) var ( @@ -1404,16 +1403,3 @@ func (am *AllocatorManager) GetLeaderAddr() string { } return leaderAddrs[0] } - -// Construct the timestampOracle path prefix, which is: -// 1. for the default keyspace group: -// "" in /pd/{cluster_id}/timestamp -// 2. for the non-default keyspace groups: -// {group}/gta in /ms/{cluster_id}/tso/{group}/gta/timestamp -func (am *AllocatorManager) getKeyspaceGroupTSPath(groupID uint32) string { - tsPath := "" - if am.kgID != mcsutils.DefaultKeyspaceGroupID { - tsPath = path.Join(fmt.Sprintf("%05d", groupID), globalTSOAllocatorEtcdPrefix) - } - return tsPath -} diff --git a/pkg/tso/global_allocator.go b/pkg/tso/global_allocator.go index 2c715d0cc7c..e81c05f9707 100644 --- a/pkg/tso/global_allocator.go +++ b/pkg/tso/global_allocator.go @@ -28,6 +28,7 @@ import ( "github.com/tikv/pd/pkg/errs" mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/slice" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/typeutil" @@ -96,7 +97,7 @@ func NewGlobalTSOAllocator( member: am.member, timestampOracle: ×tampOracle{ client: am.member.GetLeadership().GetClient(), - tsPath: am.getKeyspaceGroupTSPath(am.kgID), + tsPath: endpoint.KeyspaceGroupTSPath(am.kgID), storage: am.storage, saveInterval: am.saveInterval, updatePhysicalInterval: am.updatePhysicalInterval, diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 0291bc5863d..12cd56aa58b 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -20,7 +20,6 @@ import ( "fmt" "math" "net/http" - "path" "regexp" "sort" "strings" @@ -51,9 +50,6 @@ import ( ) const ( - keyspaceGroupsElectionPath = mcsutils.KeyspaceGroupsKey + "/election" - // primaryKey is the key for keyspace group primary election. - primaryKey = "primary" // mergingCheckInterval is the interval for merging check to see if the keyspace groups // merging process could be moved forward. mergingCheckInterval = 5 * time.Second @@ -61,6 +57,7 @@ const ( // of the primaries on this TSO server/pod have changed. A goroutine will periodically check // do this check and re-distribute the primaries if necessary. defaultPrimaryPriorityCheckInterval = 10 * time.Second + groupPatrolInterval = time.Minute ) type state struct { @@ -74,13 +71,19 @@ type state struct { kgs [mcsutils.MaxKeyspaceGroupCountInUse]*endpoint.KeyspaceGroup // keyspaceLookupTable is a map from keyspace to the keyspace group to which it belongs. keyspaceLookupTable map[uint32]uint32 + // splittingGroups is the cache of splitting keyspace group related information. + splittingGroups map[uint32]struct{} + // deletedGroups is the cache of deleted keyspace group related information. + deletedGroups map[uint32]struct{} } func (s *state) initialize() { s.keyspaceLookupTable = make(map[uint32]uint32) + s.splittingGroups = make(map[uint32]struct{}) + s.deletedGroups = make(map[uint32]struct{}) } -func (s *state) deinitialize() { +func (s *state) deInitialize() { log.Info("closing all keyspace groups") s.Lock() @@ -112,6 +115,28 @@ func (s *state) getKeyspaceGroupMeta( return s.ams[groupID], s.kgs[groupID] } +// getSplittingGroups returns the IDs of the splitting keyspace groups. +func (s *state) getSplittingGroups() []uint32 { + s.RLock() + defer s.RUnlock() + groups := make([]uint32, 0, len(s.splittingGroups)) + for groupID := range s.splittingGroups { + groups = append(groups, groupID) + } + return groups +} + +// getDeletedGroups returns the IDs of the deleted keyspace groups. +func (s *state) getDeletedGroups() []uint32 { + s.RLock() + defer s.RUnlock() + groups := make([]uint32, 0, len(s.deletedGroups)) + for groupID := range s.deletedGroups { + groups = append(groups, groupID) + } + return groups +} + func (s *state) checkTSOSplit( targetGroupID uint32, ) (splitTargetAM, splitSourceAM *AllocatorManager, err error) { @@ -227,32 +252,6 @@ func (s *state) getNextPrimaryToReset( return nil, nil, 0, groupID } -// kgPrimaryPathBuilder builds the path for keyspace group primary election. -// default keyspace group: "/ms/{cluster_id}/tso/00000/primary". -// non-default keyspace group: "/ms/{cluster_id}/tso/keyspace_groups/election/{group}/primary". -type kgPrimaryPathBuilder struct { - // rootPath is "/ms/{cluster_id}/tso". - rootPath string - // defaultKeyspaceGroupIDPath is "/ms/{cluster_id}/tso/00000". - defaultKeyspaceGroupIDPath string -} - -// getKeyspaceGroupIDPath returns the keyspace group primary ID path. -// default keyspace group: "/ms/{cluster_id}/tso/00000". -// non-default keyspace group: "/ms/{cluster_id}/tso/keyspace_groups/election/{group}". -func (p *kgPrimaryPathBuilder) getKeyspaceGroupIDPath(keyspaceGroupID uint32) string { - if keyspaceGroupID == mcsutils.DefaultKeyspaceGroupID { - return p.defaultKeyspaceGroupIDPath - } - return path.Join(p.rootPath, keyspaceGroupsElectionPath, fmt.Sprintf("%05d", keyspaceGroupID)) -} - -// getCompiledNonDefaultIDRegexp returns the compiled regular expression for matching non-default keyspace group id. -func (p *kgPrimaryPathBuilder) getCompiledNonDefaultIDRegexp() *regexp.Regexp { - pattern := strings.Join([]string{p.rootPath, keyspaceGroupsElectionPath, `(\d{5})`, primaryKey + `$`}, "/") - return regexp.MustCompile(pattern) -} - // KeyspaceGroupManager manages the members of the keyspace groups assigned to this host. // The replicas campaign for the leaders which provide the tso service for the corresponding // keyspace groups. @@ -326,7 +325,6 @@ type KeyspaceGroupManager struct { // mergeCheckerCancelMap is the cancel function map for the merge checker of each keyspace group. mergeCheckerCancelMap sync.Map // GroupID -> context.CancelFunc - primaryPathBuilder *kgPrimaryPathBuilder primaryPriorityCheckInterval time.Duration // tsoNodes is the registered tso servers. @@ -377,10 +375,6 @@ func NewKeyspaceGroupManager( kgm.tsoSvcStorage = endpoint.NewStorageEndpoint( kv.NewEtcdKVBase(kgm.etcdClient, kgm.tsoSvcRootPath), nil) kgm.compiledKGMembershipIDRegexp = endpoint.GetCompiledKeyspaceGroupIDRegexp() - kgm.primaryPathBuilder = &kgPrimaryPathBuilder{ - rootPath: kgm.tsoSvcRootPath, - defaultKeyspaceGroupIDPath: path.Join(kgm.tsoSvcRootPath, "00000"), - } kgm.state.initialize() return kgm } @@ -398,8 +392,10 @@ func (kgm *KeyspaceGroupManager) Initialize() error { return errs.ErrLoadKeyspaceGroupsTerminated.Wrap(err) } - kgm.wg.Add(1) + kgm.wg.Add(3) go kgm.primaryPriorityCheckLoop() + go kgm.groupSplitPatroller() + go kgm.deletedGroupCleaner() return nil } @@ -415,7 +411,7 @@ func (kgm *KeyspaceGroupManager) Close() { // added/initialized after that. kgm.cancel() kgm.wg.Wait() - kgm.state.deinitialize() + kgm.state.deInitialize() log.Info("keyspace group manager closed") } @@ -568,6 +564,8 @@ func (kgm *KeyspaceGroupManager) primaryPriorityCheckLoop() { kgm.primaryPriorityCheckInterval = 200 * time.Millisecond }) + ticker := time.NewTicker(kgm.primaryPriorityCheckInterval) + defer ticker.Stop() ctx, cancel := context.WithCancel(kgm.ctx) defer cancel() groupID := 0 @@ -576,7 +574,7 @@ func (kgm *KeyspaceGroupManager) primaryPriorityCheckLoop() { case <-ctx.Done(): log.Info("exit primary priority check loop") return - case <-time.After(kgm.primaryPriorityCheckInterval): + case <-ticker.C: // Every primaryPriorityCheckInterval, we only reset the primary of one keyspace group member, kg, localPriority, nextGroupID := kgm.getNextPrimaryToReset(groupID, kgm.tsoServiceID.ServiceAddr) if member != nil { @@ -676,6 +674,7 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroup(group *endpoint.KeyspaceGro } // If the keyspace group is not initialized, initialize it. + // The format of leader name is address-groupID. uniqueName := fmt.Sprintf("%s-%05d", kgm.electionNamePrefix, group.ID) uniqueID := memberutil.GenerateUniqueID(uniqueName) log.Info("joining primary election", @@ -685,8 +684,8 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroup(group *endpoint.KeyspaceGro // Initialize the participant info to join the primary election. participant := member.NewParticipant(kgm.etcdClient) participant.InitInfo( - uniqueName, uniqueID, kgm.primaryPathBuilder.getKeyspaceGroupIDPath(group.ID), - primaryKey, "keyspace group primary election", kgm.cfg.GetAdvertiseListenAddr()) + uniqueName, uniqueID, endpoint.KeyspaceGroupsElectionPath(kgm.tsoSvcRootPath, group.ID), + mcsutils.KeyspaceGroupsPrimaryKey, "keyspace group primary election", kgm.cfg.GetAdvertiseListenAddr()) // If the keyspace group is in split, we should ensure that the primary elected by the new keyspace group // is always on the same TSO Server node as the primary of the old keyspace group, and this constraint cannot // be broken until the entire split process is completed. @@ -730,6 +729,10 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroup(group *endpoint.KeyspaceGro } kgm.kgs[group.ID] = group kgm.ams[group.ID] = am + // If the group is the split target, add it to the splitting group map. + if group.IsSplitTarget() { + kgm.splittingGroups[group.ID] = struct{}{} + } kgm.Unlock() } @@ -857,6 +860,7 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroupMembership( // Check if the split is completed. if oldGroup != nil && oldGroup.IsSplitTarget() && !newGroup.IsSplitting() { kgm.ams[groupID].GetMember().(*member.Participant).SetCampaignChecker(nil) + delete(kgm.splittingGroups, groupID) } kgm.kgs[groupID] = newGroup } @@ -903,10 +907,12 @@ func (kgm *KeyspaceGroupManager) deleteKeyspaceGroup(groupID uint32) { am.close() kgm.ams[groupID] = nil } + + kgm.deletedGroups[groupID] = struct{}{} } // exitElectionMembership exits the election membership of the given keyspace group by -// deinitializing the allocator manager, but still keeps the keyspace group info. +// de-initializing the allocator manager, but still keeps the keyspace group info. func (kgm *KeyspaceGroupManager) exitElectionMembership(group *endpoint.KeyspaceGroup) { log.Info("resign election membership", zap.Uint32("keyspace-group-id", group.ID)) @@ -1236,7 +1242,7 @@ func (kgm *KeyspaceGroupManager) mergingChecker(ctx context.Context, mergeTarget // Check if the keyspace group primaries in the merge map are all gone. if len(mergeMap) != 0 { for id := range mergeMap { - leaderPath := path.Join(kgm.primaryPathBuilder.getKeyspaceGroupIDPath(id), primaryKey) + leaderPath := endpoint.KeyspaceGroupPrimaryPath(kgm.tsoSvcRootPath, id) val, err := kgm.tsoSvcStorage.Load(leaderPath) if err != nil { log.Error("failed to check if the keyspace group primary in the merge list has gone", @@ -1256,12 +1262,17 @@ func (kgm *KeyspaceGroupManager) mergingChecker(ctx context.Context, mergeTarget if len(mergeMap) > 0 { continue } + log.Info("all the keyspace group primaries in the merge list are gone, "+ + "start to calculate the newly merged TSO", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("merge-target-id", mergeTargetID), + zap.Any("merge-list", mergeList)) // All the keyspace group primaries in the merge list are gone, - // update the newly merged TSO to make sure it is greater than the original ones. + // calculate the newly merged TSO to make sure it is greater than the original ones. var mergedTS time.Time for _, id := range mergeList { - ts, err := kgm.tsoSvcStorage.LoadTimestamp(am.getKeyspaceGroupTSPath(id)) - if err != nil || ts == typeutil.ZeroTime { + ts, err := kgm.tsoSvcStorage.LoadTimestamp(endpoint.KeyspaceGroupTSPath(id)) + if err != nil { log.Error("failed to load the keyspace group TSO", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), @@ -1269,38 +1280,44 @@ func (kgm *KeyspaceGroupManager) mergingChecker(ctx context.Context, mergeTarget zap.Uint32("merge-id", id), zap.Time("ts", ts), zap.Error(err)) - mergedTS = typeutil.ZeroTime break } if ts.After(mergedTS) { mergedTS = ts } } - if mergedTS == typeutil.ZeroTime { - continue - } - // Update the newly merged TSO. - // TODO: support the Local TSO Allocator. - allocator, err := am.GetAllocator(GlobalDCLocation) if err != nil { - log.Error("failed to get the allocator", - zap.String("member", kgm.tsoServiceID.ServiceAddr), - zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList), - zap.Error(err)) continue } - err = allocator.SetTSO( - tsoutil.GenerateTS(tsoutil.GenerateTimestamp(mergedTS, 1)), - true, true) - if err != nil { - log.Error("failed to update the newly merged TSO", + // Update the newly merged TSO if the merged TSO is not zero. + if mergedTS != typeutil.ZeroTime { + log.Info("start to set the newly merged TSO", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), zap.Any("merge-list", mergeList), - zap.Time("merged-ts", mergedTS), - zap.Error(err)) - continue + zap.Time("merged-ts", mergedTS)) + // TODO: support the Local TSO Allocator. + allocator, err := am.GetAllocator(GlobalDCLocation) + if err != nil { + log.Error("failed to get the allocator", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("merge-target-id", mergeTargetID), + zap.Any("merge-list", mergeList), + zap.Error(err)) + continue + } + err = allocator.SetTSO( + tsoutil.GenerateTS(tsoutil.GenerateTimestamp(mergedTS, 1)), + true, true) + if err != nil { + log.Error("failed to update the newly merged TSO", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("merge-target-id", mergeTargetID), + zap.Any("merge-list", mergeList), + zap.Time("merged-ts", mergedTS), + zap.Error(err)) + continue + } } // Finish the merge. err = kgm.finishMergeKeyspaceGroup(mergeTargetID) @@ -1320,3 +1337,103 @@ func (kgm *KeyspaceGroupManager) mergingChecker(ctx context.Context, mergeTarget return } } + +// groupSplitPatroller is used to patrol the groups that are in the on-going +// split state and to check if we could speed up the split process. +func (kgm *KeyspaceGroupManager) groupSplitPatroller() { + defer logutil.LogPanic() + defer kgm.wg.Done() + patrolInterval := groupPatrolInterval + failpoint.Inject("fastGroupSplitPatroller", func() { + patrolInterval = 200 * time.Millisecond + }) + ticker := time.NewTicker(patrolInterval) + defer ticker.Stop() + log.Info("group split patroller is started", + zap.Duration("patrol-interval", patrolInterval)) + for { + select { + case <-kgm.ctx.Done(): + log.Info("group split patroller exited") + return + case <-ticker.C: + } + for _, groupID := range kgm.getSplittingGroups() { + am, group := kgm.getKeyspaceGroupMeta(groupID) + if !am.IsLeader() { + continue + } + if len(group.Keyspaces) == 0 { + log.Warn("abnormal keyspace group with empty keyspace list", + zap.Uint32("keyspace-group-id", groupID)) + continue + } + log.Info("request tso for the splitting keyspace group", + zap.Uint32("keyspace-group-id", groupID), + zap.Uint32("keyspace-id", group.Keyspaces[0])) + // Request the TSO manually to speed up the split process. + _, _, err := kgm.HandleTSORequest(group.Keyspaces[0], groupID, GlobalDCLocation, 1) + if err != nil { + log.Warn("failed to request tso for the splitting keyspace group", + zap.Uint32("keyspace-group-id", groupID), + zap.Uint32("keyspace-id", group.Keyspaces[0]), + zap.Error(err)) + continue + } + } + } +} + +// deletedGroupCleaner is used to clean the deleted keyspace groups related data. +// For example, the TSO keys of the merged keyspace groups remain in the storage. +func (kgm *KeyspaceGroupManager) deletedGroupCleaner() { + defer logutil.LogPanic() + defer kgm.wg.Done() + patrolInterval := groupPatrolInterval + failpoint.Inject("fastDeletedGroupCleaner", func() { + patrolInterval = 200 * time.Millisecond + }) + ticker := time.NewTicker(patrolInterval) + defer ticker.Stop() + log.Info("deleted group cleaner is started", + zap.Duration("patrol-interval", patrolInterval)) + for { + select { + case <-kgm.ctx.Done(): + log.Info("deleted group cleaner exited") + return + case <-ticker.C: + } + for _, groupID := range kgm.getDeletedGroups() { + // Do not clean the default keyspace group data. + if groupID == mcsutils.DefaultKeyspaceGroupID { + continue + } + // Make sure the allocator and group meta are not in use anymore. + am, _ := kgm.getKeyspaceGroupMeta(groupID) + if am != nil { + log.Info("the keyspace group tso allocator has not been closed yet", + zap.Uint32("keyspace-group-id", groupID)) + continue + } + log.Info("delete the keyspace group tso key", + zap.Uint32("keyspace-group-id", groupID)) + // Clean up the remaining TSO keys. + // TODO: support the Local TSO Allocator clean up. + err := kgm.tsoSvcStorage.DeleteTimestamp( + endpoint.TimestampPath( + endpoint.KeyspaceGroupTSPath(groupID), + ), + ) + if err != nil { + log.Warn("failed to delete the keyspace group tso key", + zap.Uint32("keyspace-group-id", groupID), + zap.Error(err)) + continue + } + kgm.Lock() + delete(kgm.deletedGroups, groupID) + kgm.Unlock() + } + } +} diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index 9792ac54728..327081d1813 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -38,6 +38,7 @@ import ( "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/tsoutil" + "github.com/tikv/pd/pkg/utils/typeutil" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/goleak" @@ -91,6 +92,58 @@ func (suite *keyspaceGroupManagerTestSuite) createConfig() *TestServiceConfig { } } +func (suite *keyspaceGroupManagerTestSuite) TestDeletedGroupCleanup() { + re := suite.Require() + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastDeletedGroupCleaner", "return(true)")) + + // Start with the empty keyspace group assignment. + mgr := suite.newUniqueKeyspaceGroupManager(0) + re.NotNil(mgr) + defer mgr.Close() + err := mgr.Initialize() + re.NoError(err) + + rootPath := mgr.legacySvcRootPath + svcAddr := mgr.tsoServiceID.ServiceAddr + + // Add keyspace group 1. + suite.applyEtcdEvents(re, rootPath, []*etcdEvent{generateKeyspaceGroupPutEvent(1, []uint32{1}, []string{svcAddr})}) + // Check if the TSO key is created. + testutil.Eventually(re, func() bool { + ts, err := mgr.tsoSvcStorage.LoadTimestamp(endpoint.KeyspaceGroupTSPath(1)) + re.NoError(err) + return ts != typeutil.ZeroTime + }) + // Delete keyspace group 1. + suite.applyEtcdEvents(re, rootPath, []*etcdEvent{generateKeyspaceGroupDeleteEvent(1)}) + // Check if the TSO key is deleted. + testutil.Eventually(re, func() bool { + ts, err := mgr.tsoSvcStorage.LoadTimestamp(endpoint.KeyspaceGroupTSPath(1)) + re.NoError(err) + return ts == typeutil.ZeroTime + }) + // Check if the keyspace group is deleted completely. + mgr.RLock() + re.Nil(mgr.ams[1]) + re.Nil(mgr.kgs[1]) + re.NotContains(mgr.deletedGroups, 1) + mgr.RUnlock() + // Try to delete the default keyspace group. + suite.applyEtcdEvents(re, rootPath, []*etcdEvent{generateKeyspaceGroupDeleteEvent(mcsutils.DefaultKeyspaceGroupID)}) + // Default keyspace group should NOT be deleted. + mgr.RLock() + re.NotNil(mgr.ams[mcsutils.DefaultKeyspaceGroupID]) + re.NotNil(mgr.kgs[mcsutils.DefaultKeyspaceGroupID]) + re.NotContains(mgr.deletedGroups, mcsutils.DefaultKeyspaceGroupID) + mgr.RUnlock() + // Default keyspace group TSO key should NOT be deleted. + ts, err := mgr.legacySvcStorage.LoadTimestamp(endpoint.KeyspaceGroupTSPath(mcsutils.DefaultKeyspaceGroupID)) + re.NoError(err) + re.NotEmpty(ts) + + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastDeletedGroupCleaner")) +} + // TestNewKeyspaceGroupManager tests the initialization of KeyspaceGroupManager. // It should initialize the allocator manager with the desired configurations and parameters. func (suite *keyspaceGroupManagerTestSuite) TestNewKeyspaceGroupManager() { @@ -98,9 +151,9 @@ func (suite *keyspaceGroupManagerTestSuite) TestNewKeyspaceGroupManager() { tsoServiceID := &discovery.ServiceRegistryEntry{ServiceAddr: suite.cfg.AdvertiseListenAddr} guid := uuid.New().String() - tsoServiceKey := discovery.ServicePath(guid, "tso") + "/" + tsoServiceKey := discovery.ServicePath(guid, "tso") legacySvcRootPath := path.Join("/pd", guid) - tsoSvcRootPath := path.Join("/ms", guid, "tso") + tsoSvcRootPath := path.Join(mcsutils.MicroserviceRootPath, guid, "tso") electionNamePrefix := "tso-server-" + guid kgm := NewKeyspaceGroupManager( @@ -742,9 +795,9 @@ func (suite *keyspaceGroupManagerTestSuite) newKeyspaceGroupManager( cfg *TestServiceConfig, ) *KeyspaceGroupManager { tsoServiceID := &discovery.ServiceRegistryEntry{ServiceAddr: cfg.GetAdvertiseListenAddr()} - tsoServiceKey := discovery.ServicePath(uniqueStr, "tso") + "/" + tsoServiceKey := discovery.ServicePath(uniqueStr, "tso") legacySvcRootPath := path.Join("/pd", uniqueStr) - tsoSvcRootPath := path.Join("/ms", uniqueStr, "tso") + tsoSvcRootPath := path.Join(mcsutils.MicroserviceRootPath, uniqueStr, "tso") electionNamePrefix := "kgm-test-" + cfg.GetAdvertiseListenAddr() kgm := NewKeyspaceGroupManager( diff --git a/pkg/tso/tso.go b/pkg/tso/tso.go index 54f0cb927be..33da6e8d11f 100644 --- a/pkg/tso/tso.go +++ b/pkg/tso/tso.go @@ -16,7 +16,6 @@ package tso import ( "fmt" - "path" "sync/atomic" "time" @@ -34,7 +33,6 @@ import ( ) const ( - timestampKey = "timestamp" // UpdateTimestampGuard is the min timestamp interval. UpdateTimestampGuard = time.Millisecond // maxLogical is the max upper limit for logical time. @@ -142,7 +140,7 @@ func (t *timestampOracle) calibrateLogical(rawLogical int64, suffixBits int) int // GetTimestampPath returns the timestamp path in etcd. func (t *timestampOracle) GetTimestampPath() string { - return path.Join(t.tsPath, timestampKey) + return endpoint.TimestampPath(t.tsPath) } // SyncTimestamp is used to synchronize the timestamp. @@ -172,7 +170,9 @@ func (t *timestampOracle) SyncTimestamp(leadership *election.Leadership) error { zap.Time("last", last), zap.Time("next", next), errs.ZapError(errs.ErrIncorrectSystemTime)) next = last.Add(UpdateTimestampGuard) } - + failpoint.Inject("failedToSaveTimestamp", func() { + failpoint.Return(errs.ErrEtcdTxnInternal) + }) save := next.Add(t.saveInterval) if err = t.storage.SaveTimestamp(t.GetTimestampPath(), save); err != nil { tsoCounter.WithLabelValues("err_save_sync_ts", t.dcLocation).Inc() diff --git a/pkg/tso/util_test.go b/pkg/tso/util_test.go index 8b7c7a4ce0c..8e4971797d5 100644 --- a/pkg/tso/util_test.go +++ b/pkg/tso/util_test.go @@ -15,7 +15,6 @@ package tso import ( - "path" "testing" "github.com/stretchr/testify/require" @@ -75,13 +74,7 @@ func TestExtractKeyspaceGroupIDFromKeyspaceGroupMembershipPath(t *testing.T) { func TestExtractKeyspaceGroupIDFromKeyspaceGroupPrimaryPath(t *testing.T) { re := require.New(t) - tsoSvcRootPath := "/ms/111/tso" - primaryPathBuilder := &kgPrimaryPathBuilder{ - rootPath: tsoSvcRootPath, - defaultKeyspaceGroupIDPath: path.Join(tsoSvcRootPath, "00000"), - } - - compiledRegexp := primaryPathBuilder.getCompiledNonDefaultIDRegexp() + compiledRegexp := endpoint.GetCompiledNonDefaultIDRegexp(uint64(111)) rightCases := []struct { path string diff --git a/server/api/pprof.go b/server/api/pprof.go index b64278a21b7..900c48f8368 100644 --- a/server/api/pprof.go +++ b/server/api/pprof.go @@ -209,8 +209,10 @@ func (h *pprofHandler) PProfThreadcreate(w http.ResponseWriter, r *http.Request) } func sleepWithCtx(ctx context.Context, d time.Duration) { + timer := time.NewTimer(d) + defer timer.Stop() select { - case <-time.After(d): + case <-timer.C: case <-ctx.Done(): } } diff --git a/server/apiv2/handlers/keyspace.go b/server/apiv2/handlers/keyspace.go index d9c4251ffed..139a1c44157 100644 --- a/server/apiv2/handlers/keyspace.go +++ b/server/apiv2/handlers/keyspace.go @@ -114,6 +114,20 @@ func LoadKeyspace(c *gin.Context) { c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) return } + if value, ok := c.GetQuery("force_refresh_group_id"); ok && value == "true" { + groupManager := svr.GetKeyspaceGroupManager() + if groupManager == nil { + c.AbortWithStatusJSON(http.StatusInternalServerError, managerUninitializedErr) + return + } + // keyspace has been checked in LoadKeyspace, so no need to check again. + groupID, err := groupManager.GetGroupByKeyspaceID(meta.GetId()) + if err != nil { + c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) + return + } + meta.Config[keyspace.TSOKeyspaceGroupIDKey] = strconv.FormatUint(uint64(groupID), 10) + } c.IndentedJSON(http.StatusOK, &KeyspaceMeta{meta}) } diff --git a/server/apiv2/handlers/tso_keyspace_group.go b/server/apiv2/handlers/tso_keyspace_group.go index 5de8fd6a9cc..fc7ef680104 100644 --- a/server/apiv2/handlers/tso_keyspace_group.go +++ b/server/apiv2/handlers/tso_keyspace_group.go @@ -40,9 +40,9 @@ func RegisterTSOKeyspaceGroup(r *gin.RouterGroup) { router.GET("", GetKeyspaceGroups) router.GET("/:id", GetKeyspaceGroupByID) router.DELETE("/:id", DeleteKeyspaceGroupByID) + router.PATCH("/:id", SetNodesForKeyspaceGroup) // only to support set nodes + router.PATCH("/:id/*node", SetPriorityForKeyspaceGroup) // only to support set priority router.POST("/:id/alloc", AllocNodesForKeyspaceGroup) - router.POST("/:id/nodes", SetNodesForKeyspaceGroup) - router.POST("/:id/priority", SetPriorityForKeyspaceGroup) router.POST("/:id/split", SplitKeyspaceGroupByID) router.DELETE("/:id/split", FinishSplitKeyspaceByID) router.POST("/:id/merge", MergeKeyspaceGroups) @@ -135,6 +135,12 @@ func GetKeyspaceGroups(c *gin.Context) { c.IndentedJSON(http.StatusOK, kgs) } +// GetKeyspaceGroupPrimaryResponse defines the response for getting primary node of keyspace group. +type GetKeyspaceGroupPrimaryResponse struct { + ID uint32 `json:"id"` + Primary string `json:"primary"` +} + // GetKeyspaceGroupByID gets keyspace group by ID. func GetKeyspaceGroupByID(c *gin.Context) { id, err := validateKeyspaceGroupID(c) @@ -149,12 +155,26 @@ func GetKeyspaceGroupByID(c *gin.Context) { c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) return } + + fields := c.Query("fields") // split by comma if need to add more fields + if fields == "primary" { + primary, err := manager.GetKeyspaceGroupPrimaryByID(id) + if err != nil { + c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) + return + } + c.JSON(http.StatusOK, &GetKeyspaceGroupPrimaryResponse{ + ID: id, + Primary: primary, + }) + return + } + kg, err := manager.GetKeyspaceGroupByID(id) if err != nil { c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) return } - c.IndentedJSON(http.StatusOK, kg) } @@ -276,7 +296,8 @@ func FinishSplitKeyspaceByID(c *gin.Context) { // MergeKeyspaceGroupsParams defines the params for merging the keyspace groups. type MergeKeyspaceGroupsParams struct { - MergeList []uint32 `json:"merge-list"` + MergeList []uint32 `json:"merge-list"` + MergeAllIntoDefault bool `json:"merge-all-into-default"` } // MergeKeyspaceGroups merges the keyspace groups in the merge list into the target keyspace group. @@ -292,10 +313,14 @@ func MergeKeyspaceGroups(c *gin.Context) { c.AbortWithStatusJSON(http.StatusBadRequest, errs.ErrBindJSON.Wrap(err).GenWithStackByCause()) return } - if len(mergeParams.MergeList) == 0 { + if len(mergeParams.MergeList) == 0 && !mergeParams.MergeAllIntoDefault { c.AbortWithStatusJSON(http.StatusBadRequest, "invalid empty merge list") return } + if len(mergeParams.MergeList) > 0 && mergeParams.MergeAllIntoDefault { + c.AbortWithStatusJSON(http.StatusBadRequest, "non-empty merge list when merge all into default") + return + } for _, mergeID := range mergeParams.MergeList { if !isValid(mergeID) { c.AbortWithStatusJSON(http.StatusBadRequest, "invalid keyspace group id") @@ -310,7 +335,11 @@ func MergeKeyspaceGroups(c *gin.Context) { return } // Merge keyspace group. - err = groupManager.MergeKeyspaceGroups(id, mergeParams.MergeList) + if mergeParams.MergeAllIntoDefault { + err = groupManager.MergeAllIntoDefaultKeyspaceGroup() + } else { + err = groupManager.MergeKeyspaceGroups(id, mergeParams.MergeList) + } if err != nil { c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) return @@ -436,8 +465,7 @@ func SetNodesForKeyspaceGroup(c *gin.Context) { // SetPriorityForKeyspaceGroupParams defines the params for setting priority of tso node for the keyspace group. type SetPriorityForKeyspaceGroupParams struct { - Node string `json:"node"` - Priority int `json:"priority"` + Priority int `json:"priority"` } // SetPriorityForKeyspaceGroup sets priority of tso node for the keyspace group. @@ -447,6 +475,11 @@ func SetPriorityForKeyspaceGroup(c *gin.Context) { c.AbortWithStatusJSON(http.StatusBadRequest, "invalid keyspace group id") return } + node, err := parseNodeAddress(c) + if err != nil { + c.AbortWithStatusJSON(http.StatusBadRequest, "invalid node address") + return + } svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { @@ -468,12 +501,12 @@ func SetPriorityForKeyspaceGroup(c *gin.Context) { // check if node exists members := kg.Members if slice.NoneOf(members, func(i int) bool { - return members[i].Address == setParams.Node + return members[i].Address == node }) { c.AbortWithStatusJSON(http.StatusBadRequest, "tso node does not exist in the keyspace group") } // set priority - err = manager.SetPriorityForKeyspaceGroup(id, setParams.Node, setParams.Priority) + err = manager.SetPriorityForKeyspaceGroup(id, node, setParams.Priority) if err != nil { c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) return @@ -492,6 +525,19 @@ func validateKeyspaceGroupID(c *gin.Context) (uint32, error) { return uint32(id), nil } +func parseNodeAddress(c *gin.Context) (string, error) { + node := c.Param("node") + if node == "" { + return "", errors.New("invalid node address") + } + // In pd-ctl, we use url.PathEscape to escape the node address and replace the % to \%. + // But in the gin framework, it will unescape the node address automatically. + // So we need to replace the \/ to /. + node = strings.ReplaceAll(node, "\\/", "/") + node = strings.TrimPrefix(node, "/") + return node, nil +} + func isValid(id uint32) bool { return id >= utils.DefaultKeyspaceGroupID && id <= utils.MaxKeyspaceGroupCountInUse } diff --git a/server/cluster/coordinator.go b/server/cluster/coordinator.go index 52b0e725411..79243e3e3e9 100644 --- a/server/cluster/coordinator.go +++ b/server/cluster/coordinator.go @@ -118,8 +118,8 @@ func (c *coordinator) patrolRegions() { defer logutil.LogPanic() defer c.wg.Done() - timer := time.NewTimer(c.cluster.GetOpts().GetPatrolRegionInterval()) - defer timer.Stop() + ticker := time.NewTicker(c.cluster.GetOpts().GetPatrolRegionInterval()) + defer ticker.Stop() log.Info("coordinator starts patrol regions") start := time.Now() @@ -129,8 +129,9 @@ func (c *coordinator) patrolRegions() { ) for { select { - case <-timer.C: - timer.Reset(c.cluster.GetOpts().GetPatrolRegionInterval()) + case <-ticker.C: + // Note: we reset the ticker here to support updating configuration dynamically. + ticker.Reset(c.cluster.GetOpts().GetPatrolRegionInterval()) case <-c.ctx.Done(): log.Info("patrol regions has been stopped") return @@ -796,12 +797,11 @@ func (c *coordinator) runScheduler(s *scheduleController) { defer c.wg.Done() defer s.Cleanup(c.cluster) - timer := time.NewTimer(s.GetInterval()) - defer timer.Stop() + ticker := time.NewTicker(s.GetInterval()) + defer ticker.Stop() for { select { - case <-timer.C: - timer.Reset(s.GetInterval()) + case <-ticker.C: diagnosable := s.diagnosticRecorder.isAllowed() if !s.AllowSchedule(diagnosable) { continue @@ -810,7 +810,8 @@ func (c *coordinator) runScheduler(s *scheduleController) { added := c.opController.AddWaitingOperator(op...) log.Debug("add operator", zap.Int("added", added), zap.Int("total", len(op)), zap.String("scheduler", s.GetName())) } - + // Note: we reset the ticker here to support updating configuration dynamically. + ticker.Reset(s.GetInterval()) case <-s.Ctx().Done(): log.Info("scheduler has been stopped", zap.String("scheduler-name", s.GetName()), diff --git a/server/grpc_service.go b/server/grpc_service.go index e6076770031..f0db576589c 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -958,13 +958,15 @@ func (b *bucketHeartbeatServer) Send(bucket *pdpb.ReportBucketsResponse) error { defer logutil.LogPanic() done <- b.stream.SendAndClose(bucket) }() + timer := time.NewTimer(heartbeatSendTimeout) + defer timer.Stop() select { case err := <-done: if err != nil { atomic.StoreInt32(&b.closed, 1) } return err - case <-time.After(heartbeatSendTimeout): + case <-timer.C: atomic.StoreInt32(&b.closed, 1) return ErrSendHeartbeatTimeout } @@ -998,13 +1000,15 @@ func (s *heartbeatServer) Send(m *pdpb.RegionHeartbeatResponse) error { defer logutil.LogPanic() done <- s.stream.Send(m) }() + timer := time.NewTimer(heartbeatSendTimeout) + defer timer.Stop() select { case err := <-done: if err != nil { atomic.StoreInt32(&s.closed, 1) } return errors.WithStack(err) - case <-time.After(heartbeatSendTimeout): + case <-timer.C: atomic.StoreInt32(&s.closed, 1) return ErrSendHeartbeatTimeout } diff --git a/server/replication/replication_mode.go b/server/replication/replication_mode.go index 60877f31d54..1ae23f15d56 100644 --- a/server/replication/replication_mode.go +++ b/server/replication/replication_mode.go @@ -375,14 +375,18 @@ const ( // Run starts the background job. func (m *ModeManager) Run(ctx context.Context) { // Wait for a while when just start, in case tikv do not connect in time. + timer := time.NewTimer(idleTimeout) + defer timer.Stop() select { - case <-time.After(idleTimeout): + case <-timer.C: case <-ctx.Done(): return } + ticker := time.NewTicker(tickInterval) + defer ticker.Stop() for { select { - case <-time.After(tickInterval): + case <-ticker.C: case <-ctx.Done(): return } diff --git a/server/server.go b/server/server.go index 91bff25ca32..39fd6c30c41 100644 --- a/server/server.go +++ b/server/server.go @@ -530,6 +530,9 @@ func (s *Server) Run() error { if err := s.startEtcd(s.ctx); err != nil { return err } + failpoint.Inject("delayStartServer", func() { + time.Sleep(2 * time.Second) + }) if err := s.startServer(s.ctx); err != nil { return err } @@ -587,9 +590,11 @@ func (s *Server) serverMetricsLoop() { ctx, cancel := context.WithCancel(s.serverLoopCtx) defer cancel() + ticker := time.NewTicker(serverMetricsInterval) + defer ticker.Stop() for { select { - case <-time.After(serverMetricsInterval): + case <-ticker.C: s.collectEtcdStateMetrics() case <-ctx.Done(): log.Info("server is closed, exit metrics loop") @@ -1606,10 +1611,14 @@ func (s *Server) etcdLeaderLoop() { ctx, cancel := context.WithCancel(s.serverLoopCtx) defer cancel() + ticker := time.NewTicker(s.cfg.LeaderPriorityCheckInterval.Duration) + defer ticker.Stop() for { select { - case <-time.After(s.cfg.LeaderPriorityCheckInterval.Duration): + case <-ticker.C: s.member.CheckPriority(ctx) + // Note: we reset the ticker here to support updating configuration dynamically. + ticker.Reset(s.cfg.LeaderPriorityCheckInterval.Duration) case <-ctx.Done(): log.Info("server is closed, exit etcd leader loop") return @@ -1749,6 +1758,8 @@ func (s *Server) UnmarkSnapshotRecovering(ctx context.Context) error { // GetServicePrimaryAddr returns the primary address for a given service. // Note: This function will only return primary address without judging if it's alive. func (s *Server) GetServicePrimaryAddr(ctx context.Context, serviceName string) (string, bool) { + ticker := time.NewTicker(retryIntervalGetServicePrimary) + defer ticker.Stop() for i := 0; i < maxRetryTimesGetServicePrimary; i++ { if v, ok := s.servicePrimaryMap.Load(serviceName); ok { return v.(string), true @@ -1758,7 +1769,7 @@ func (s *Server) GetServicePrimaryAddr(ctx context.Context, serviceName string) return "", false case <-ctx.Done(): return "", false - case <-time.After(retryIntervalGetServicePrimary): + case <-ticker.C: } } return "", false @@ -1770,13 +1781,10 @@ func (s *Server) SetServicePrimaryAddr(serviceName, addr string) { s.servicePrimaryMap.Store(serviceName, addr) } -func (s *Server) servicePrimaryKey(serviceName string) string { - return fmt.Sprintf("/ms/%d/%s/%s/%s", s.clusterID, serviceName, fmt.Sprintf("%05d", 0), "primary") -} - func (s *Server) initTSOPrimaryWatcher() { serviceName := mcs.TSOServiceName - tsoServicePrimaryKey := s.servicePrimaryKey(serviceName) + tsoRootPath := endpoint.TSOSvcRootPath(s.clusterID) + tsoServicePrimaryKey := endpoint.KeyspaceGroupPrimaryPath(tsoRootPath, mcs.DefaultKeyspaceGroupID) putFn := func(kv *mvccpb.KeyValue) error { primary := &tsopb.Participant{} // TODO: use Generics if err := proto.Unmarshal(kv.Value, primary); err != nil { diff --git a/tests/cluster.go b/tests/cluster.go index 37b24fa7dd1..12c41bb5843 100644 --- a/tests/cluster.go +++ b/tests/cluster.go @@ -33,7 +33,6 @@ import ( "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/id" "github.com/tikv/pd/pkg/keyspace" - tsoserver "github.com/tikv/pd/pkg/mcs/tso/server" "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/swaggerserver" "github.com/tikv/pd/pkg/tso" @@ -85,38 +84,6 @@ func NewTestAPIServer(ctx context.Context, cfg *config.Config) (*TestServer, err return createTestServer(ctx, cfg, []string{utils.APIServiceName}) } -// StartSingleTSOTestServer creates and starts a tso server with default config for testing. -func StartSingleTSOTestServer(ctx context.Context, re *require.Assertions, backendEndpoints, listenAddrs string) (*tsoserver.Server, func(), error) { - cfg := tsoserver.NewConfig() - cfg.BackendEndpoints = backendEndpoints - cfg.ListenAddr = listenAddrs - cfg, err := tsoserver.GenerateConfig(cfg) - re.NoError(err) - // Setup the logger. - err = logutil.SetupLogger(cfg.Log, &cfg.Logger, &cfg.LogProps, cfg.Security.RedactInfoLog) - if err != nil { - return nil, nil, err - } - zapLogOnce.Do(func() { - log.ReplaceGlobals(cfg.Logger, cfg.LogProps) - }) - re.NoError(err) - return NewTSOTestServer(ctx, cfg) -} - -// NewTSOTestServer creates a tso server with given config for testing. -func NewTSOTestServer(ctx context.Context, cfg *tsoserver.Config) (*tsoserver.Server, testutil.CleanupFunc, error) { - s := tsoserver.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 -} - func createTestServer(ctx context.Context, cfg *config.Config, services []string) (*TestServer, error) { err := logutil.SetupLogger(cfg.Log, &cfg.Logger, &cfg.LogProps, cfg.Security.RedactInfoLog) if err != nil { @@ -485,8 +452,8 @@ func NewTestAPICluster(ctx context.Context, initialServerCount int, opts ...Conf func createTestCluster(ctx context.Context, initialServerCount int, isAPIServiceMode bool, opts ...ConfigOption) (*TestCluster, error) { config := newClusterConfig(initialServerCount) servers := make(map[string]*TestServer) - for _, conf := range config.InitialServers { - serverConf, err := conf.Generate(opts...) + for _, cfg := range config.InitialServers { + serverConf, err := cfg.Generate(opts...) if err != nil { return nil, err } @@ -499,7 +466,7 @@ func createTestCluster(ctx context.Context, initialServerCount int, isAPIService if err != nil { return nil, err } - servers[conf.Name] = s + servers[cfg.Name] = s } return &TestCluster{ config: config, @@ -513,6 +480,67 @@ func createTestCluster(ctx context.Context, initialServerCount int, isAPIService }, nil } +// RestartTestAPICluster restarts the API test cluster. +func RestartTestAPICluster(ctx context.Context, cluster *TestCluster) (*TestCluster, error) { + return restartTestCluster(ctx, cluster, true) +} + +func restartTestCluster( + ctx context.Context, cluster *TestCluster, isAPIServiceMode bool, +) (newTestCluster *TestCluster, err error) { + newTestCluster = &TestCluster{ + config: cluster.config, + servers: make(map[string]*TestServer, len(cluster.servers)), + tsPool: struct { + sync.Mutex + pool map[uint64]struct{} + }{ + pool: make(map[uint64]struct{}), + }, + } + + var serverMap sync.Map + var errorMap sync.Map + wg := sync.WaitGroup{} + for serverName, server := range newTestCluster.servers { + serverCfg := server.GetConfig() + wg.Add(1) + go func(serverName string, server *TestServer) { + defer wg.Done() + server.Destroy() + var ( + newServer *TestServer + serverErr error + ) + if isAPIServiceMode { + newServer, serverErr = NewTestAPIServer(ctx, serverCfg) + } else { + newServer, serverErr = NewTestServer(ctx, serverCfg) + } + serverMap.Store(serverName, newServer) + errorMap.Store(serverName, serverErr) + }(serverName, server) + } + wg.Wait() + + errorMap.Range(func(key, value interface{}) bool { + if value != nil { + err = value.(error) + return false + } + serverName := key.(string) + newServer, _ := serverMap.Load(serverName) + newTestCluster.servers[serverName] = newServer.(*TestServer) + return true + }) + + if err != nil { + return nil, errors.New("failed to restart cluster. " + err.Error()) + } + + return newTestCluster, nil +} + // RunServer starts to run TestServer. func (c *TestCluster) RunServer(server *TestServer) <-chan error { resC := make(chan error) @@ -597,9 +625,11 @@ func (c *TestCluster) WaitLeader(ops ...WaitOption) string { counter := make(map[string]int) running := 0 for _, s := range c.servers { + s.RLock() if s.state == Running { running++ } + s.RUnlock() n := s.GetLeader().GetName() if n != "" { counter[n]++ diff --git a/tests/integrations/mcs/discovery/register_test.go b/tests/integrations/mcs/discovery/register_test.go index 27a5a937e4c..2cdcccedddd 100644 --- a/tests/integrations/mcs/discovery/register_test.go +++ b/tests/integrations/mcs/discovery/register_test.go @@ -27,7 +27,6 @@ import ( "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "github.com/tikv/pd/tests/integrations/mcs" "go.uber.org/goleak" ) @@ -97,7 +96,7 @@ func (suite *serverRegisterTestSuite) checkServerRegister(serviceName string) { re.Equal(addr, returnedEntry.ServiceAddr) // test primary when only one server - expectedPrimary := mcs.WaitForPrimaryServing(suite.Require(), map[string]bs.Server{addr: s}) + expectedPrimary := tests.WaitForPrimaryServing(suite.Require(), map[string]bs.Server{addr: s}) primary, exist := suite.pdLeader.GetServer().GetServicePrimaryAddr(suite.ctx, serviceName) re.True(exist) re.Equal(primary, expectedPrimary) @@ -131,7 +130,7 @@ func (suite *serverRegisterTestSuite) checkServerPrimaryChange(serviceName strin serverMap[s.GetAddr()] = s } - expectedPrimary := mcs.WaitForPrimaryServing(suite.Require(), serverMap) + expectedPrimary := tests.WaitForPrimaryServing(suite.Require(), serverMap) primary, exist = suite.pdLeader.GetServer().GetServicePrimaryAddr(suite.ctx, serviceName) re.True(exist) re.Equal(expectedPrimary, primary) @@ -139,7 +138,7 @@ func (suite *serverRegisterTestSuite) checkServerPrimaryChange(serviceName strin serverMap[primary].Close() delete(serverMap, primary) - expectedPrimary = mcs.WaitForPrimaryServing(suite.Require(), serverMap) + expectedPrimary = tests.WaitForPrimaryServing(suite.Require(), serverMap) // test API server discovery client := suite.pdLeader.GetEtcdClient() endpoints, err := discovery.Discover(client, suite.clusterID, serviceName) @@ -156,9 +155,9 @@ func (suite *serverRegisterTestSuite) addServer(serviceName string) (bs.Server, re := suite.Require() switch serviceName { case utils.TSOServiceName: - return mcs.StartSingleTSOTestServer(suite.ctx, re, suite.backendEndpoints, tempurl.Alloc()) + return tests.StartSingleTSOTestServer(suite.ctx, re, suite.backendEndpoints, tempurl.Alloc()) case utils.ResourceManagerServiceName: - return mcs.StartSingleResourceManagerTestServer(suite.ctx, re, suite.backendEndpoints, tempurl.Alloc()) + return tests.StartSingleResourceManagerTestServer(suite.ctx, re, suite.backendEndpoints, tempurl.Alloc()) default: return nil, nil } diff --git a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go index 41bcba0e90b..59aabb260ae 100644 --- a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go +++ b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go @@ -33,7 +33,6 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/apiv2/handlers" "github.com/tikv/pd/tests" - "github.com/tikv/pd/tests/integrations/mcs" ) const ( @@ -86,11 +85,11 @@ func (suite *keyspaceGroupTestSuite) TestAllocNodesUpdate() { // add three nodes. nodes := make(map[string]bs.Server) for i := 0; i < utils.DefaultKeyspaceGroupReplicaCount+1; i++ { - s, cleanup := mcs.StartSingleTSOTestServer(suite.ctx, suite.Require(), suite.backendEndpoints, tempurl.Alloc()) + s, cleanup := tests.StartSingleTSOTestServer(suite.ctx, suite.Require(), suite.backendEndpoints, tempurl.Alloc()) defer cleanup() nodes[s.GetAddr()] = s } - mcs.WaitForPrimaryServing(suite.Require(), nodes) + tests.WaitForPrimaryServing(suite.Require(), nodes) // create a keyspace group. kgs := &handlers.CreateKeyspaceGroupParams{KeyspaceGroups: []*endpoint.KeyspaceGroup{ @@ -135,11 +134,11 @@ func (suite *keyspaceGroupTestSuite) TestAllocNodesUpdate() { func (suite *keyspaceGroupTestSuite) TestAllocReplica() { nodes := make(map[string]bs.Server) for i := 0; i < utils.DefaultKeyspaceGroupReplicaCount; i++ { - s, cleanup := mcs.StartSingleTSOTestServer(suite.ctx, suite.Require(), suite.backendEndpoints, tempurl.Alloc()) + s, cleanup := tests.StartSingleTSOTestServer(suite.ctx, suite.Require(), suite.backendEndpoints, tempurl.Alloc()) defer cleanup() nodes[s.GetAddr()] = s } - mcs.WaitForPrimaryServing(suite.Require(), nodes) + tests.WaitForPrimaryServing(suite.Require(), nodes) // miss replica. id := 1 @@ -188,10 +187,10 @@ func (suite *keyspaceGroupTestSuite) TestAllocReplica() { suite.Equal(http.StatusBadRequest, code) // the keyspace group is exist, the new replica is more than the old replica. - s2, cleanup2 := mcs.StartSingleTSOTestServer(suite.ctx, suite.Require(), suite.backendEndpoints, tempurl.Alloc()) + s2, cleanup2 := tests.StartSingleTSOTestServer(suite.ctx, suite.Require(), suite.backendEndpoints, tempurl.Alloc()) defer cleanup2() nodes[s2.GetAddr()] = s2 - mcs.WaitForPrimaryServing(suite.Require(), nodes) + tests.WaitForPrimaryServing(suite.Require(), nodes) params = &handlers.AllocNodesForKeyspaceGroupParams{ Replica: utils.DefaultKeyspaceGroupReplicaCount + 1, } @@ -228,12 +227,12 @@ func (suite *keyspaceGroupTestSuite) TestSetNodes() { nodes := make(map[string]bs.Server) nodesList := []string{} for i := 0; i < utils.DefaultKeyspaceGroupReplicaCount; i++ { - s, cleanup := mcs.StartSingleTSOTestServer(suite.ctx, suite.Require(), suite.backendEndpoints, tempurl.Alloc()) + s, cleanup := tests.StartSingleTSOTestServer(suite.ctx, suite.Require(), suite.backendEndpoints, tempurl.Alloc()) defer cleanup() nodes[s.GetAddr()] = s nodesList = append(nodesList, s.GetAddr()) } - mcs.WaitForPrimaryServing(suite.Require(), nodes) + tests.WaitForPrimaryServing(suite.Require(), nodes) // the keyspace group is not exist. id := 1 @@ -288,11 +287,11 @@ func (suite *keyspaceGroupTestSuite) TestSetNodes() { func (suite *keyspaceGroupTestSuite) TestDefaultKeyspaceGroup() { nodes := make(map[string]bs.Server) for i := 0; i < utils.DefaultKeyspaceGroupReplicaCount; i++ { - s, cleanup := mcs.StartSingleTSOTestServer(suite.ctx, suite.Require(), suite.backendEndpoints, tempurl.Alloc()) + s, cleanup := tests.StartSingleTSOTestServer(suite.ctx, suite.Require(), suite.backendEndpoints, tempurl.Alloc()) defer cleanup() nodes[s.GetAddr()] = s } - mcs.WaitForPrimaryServing(suite.Require(), nodes) + tests.WaitForPrimaryServing(suite.Require(), nodes) // the default keyspace group is exist. var kg *endpoint.KeyspaceGroup @@ -354,7 +353,7 @@ func (suite *keyspaceGroupTestSuite) tryGetKeyspaceGroup(id uint32) (*endpoint.K func (suite *keyspaceGroupTestSuite) trySetNodesForKeyspaceGroup(id int, request *handlers.SetNodesForKeyspaceGroupParams) (*endpoint.KeyspaceGroup, int) { data, err := json.Marshal(request) suite.NoError(err) - httpReq, err := http.NewRequest(http.MethodPost, suite.server.GetAddr()+keyspaceGroupsPrefix+fmt.Sprintf("/%d/nodes", id), bytes.NewBuffer(data)) + httpReq, err := http.NewRequest(http.MethodPatch, suite.server.GetAddr()+keyspaceGroupsPrefix+fmt.Sprintf("/%d", id), bytes.NewBuffer(data)) suite.NoError(err) resp, err := suite.dialClient.Do(httpReq) suite.NoError(err) diff --git a/tests/integrations/mcs/resource_manager/server_test.go b/tests/integrations/mcs/resource_manager/server_test.go index 5d94c2eb970..de625838183 100644 --- a/tests/integrations/mcs/resource_manager/server_test.go +++ b/tests/integrations/mcs/resource_manager/server_test.go @@ -27,7 +27,6 @@ import ( "github.com/tikv/pd/client/grpcutil" "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/tests" - "github.com/tikv/pd/tests/integrations/mcs" ) func TestResourceManagerServer(t *testing.T) { @@ -45,7 +44,7 @@ func TestResourceManagerServer(t *testing.T) { leaderName := cluster.WaitLeader() leader := cluster.GetServer(leaderName) - s, cleanup := mcs.StartSingleResourceManagerTestServer(ctx, re, leader.GetAddr(), tempurl.Alloc()) + s, cleanup := tests.StartSingleResourceManagerTestServer(ctx, re, leader.GetAddr(), tempurl.Alloc()) addr := s.GetAddr() defer cleanup() diff --git a/tests/integrations/mcs/testutil.go b/tests/integrations/mcs/testutil.go index 9a3108067c9..bbedd65209d 100644 --- a/tests/integrations/mcs/testutil.go +++ b/tests/integrations/mcs/testutil.go @@ -16,38 +16,14 @@ package mcs import ( "context" - "os" "sync" - "time" - "github.com/pingcap/log" "github.com/stretchr/testify/require" pd "github.com/tikv/pd/client" - bs "github.com/tikv/pd/pkg/basicserver" - rm "github.com/tikv/pd/pkg/mcs/resource_manager/server" - tso "github.com/tikv/pd/pkg/mcs/tso/server" - "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/tsoutil" ) -var once sync.Once - -// InitLogger initializes the logger for test. -func InitLogger(cfg *tso.Config) (err error) { - once.Do(func() { - // Setup the logger. - err = logutil.SetupLogger(cfg.Log, &cfg.Logger, &cfg.LogProps, cfg.Security.RedactInfoLog) - if err != nil { - return - } - log.ReplaceGlobals(cfg.Logger, cfg.LogProps) - // Flushing any buffered log entries. - log.Sync() - }) - return err -} - // SetupClientWithAPIContext creates a TSO client with api context name for test. func SetupClientWithAPIContext( ctx context.Context, re *require.Assertions, apiCtx pd.APIContext, endpoints []string, opts ...pd.ClientOption, @@ -67,76 +43,6 @@ func SetupClientWithKeyspaceID( return cli } -// 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, 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() - cfg.BackendEndpoints = backendEndpoints - cfg.ListenAddr = listenAddrs - cfg, err := tso.GenerateConfig(cfg) - re.NoError(err) - // Setup the logger. - err = InitLogger(cfg) - re.NoError(err) - return NewTSOTestServer(ctx, cfg) -} - -// StartSingleTSOTestServer creates and starts a tso server with default config for testing. -func StartSingleTSOTestServer(ctx context.Context, re *require.Assertions, backendEndpoints, listenAddrs string) (*tso.Server, func()) { - s, cleanup, err := StartSingleTSOTestServerWithoutCheck(ctx, re, backendEndpoints, listenAddrs) - re.NoError(err) - testutil.Eventually(re, func() bool { - return !s.IsClosed() - }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) - - return s, cleanup -} - -// NewTSOTestServer creates a tso server with given config for testing. -func NewTSOTestServer(ctx context.Context, cfg *tso.Config) (*tso.Server, testutil.CleanupFunc, error) { - s := tso.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 -} - -// WaitForPrimaryServing waits for one of servers being elected to be the primary/leader -func WaitForPrimaryServing(re *require.Assertions, serverMap map[string]bs.Server) string { - var primary string - testutil.Eventually(re, func() bool { - for name, s := range serverMap { - if s.IsServing() { - primary = name - return true - } - } - return false - }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) - - return primary -} - // WaitForTSOServiceAvailable waits for the pd client being served by the tso server side func WaitForTSOServiceAvailable( ctx context.Context, re *require.Assertions, client pd.Client, diff --git a/tests/integrations/mcs/tso/api_test.go b/tests/integrations/mcs/tso/api_test.go index e5dbcd7998c..c8a28d9e02f 100644 --- a/tests/integrations/mcs/tso/api_test.go +++ b/tests/integrations/mcs/tso/api_test.go @@ -15,19 +15,23 @@ package tso import ( + "bytes" "context" "encoding/json" "io" "net/http" "testing" + "time" + "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" tso "github.com/tikv/pd/pkg/mcs/tso/server" apis "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" mcsutils "github.com/tikv/pd/pkg/mcs/utils" + "github.com/tikv/pd/pkg/storage/endpoint" + "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "github.com/tikv/pd/tests/integrations/mcs" ) const ( @@ -46,7 +50,7 @@ type tsoAPITestSuite struct { ctx context.Context cancel context.CancelFunc pdCluster *tests.TestCluster - tsoCluster *mcs.TestTSOCluster + tsoCluster *tests.TestTSOCluster } func TestTSOAPI(t *testing.T) { @@ -65,7 +69,7 @@ func (suite *tsoAPITestSuite) SetupTest() { leaderName := suite.pdCluster.WaitLeader() pdLeaderServer := suite.pdCluster.GetServer(leaderName) re.NoError(pdLeaderServer.BootstrapCluster()) - suite.tsoCluster, err = mcs.NewTestTSOCluster(suite.ctx, 1, pdLeaderServer.GetAddr()) + suite.tsoCluster, err = tests.NewTestTSOCluster(suite.ctx, 1, pdLeaderServer.GetAddr()) re.NoError(err) } @@ -104,3 +108,67 @@ func mustGetKeyspaceGroupMembers(re *require.Assertions, server *tso.Server) map re.NoError(json.Unmarshal(data, &resp)) return resp } + +func TestTSOServerStartFirst(t *testing.T) { + re := require.New(t) + re.NoError(failpoint.Enable("github.com/tikv/pd/server/delayStartServer", `return(true)`)) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + apiCluster, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, serverName string) { + conf.Keyspace.PreAlloc = []string{"k1", "k2"} + }) + defer apiCluster.Destroy() + re.NoError(err) + addr := apiCluster.GetConfig().GetClientURL() + ch := make(chan struct{}) + defer close(ch) + clusterCh := make(chan *tests.TestTSOCluster) + defer close(clusterCh) + go func() { + tsoCluster, err := tests.NewTestTSOCluster(ctx, 2, addr) + re.NoError(err) + primary := tsoCluster.WaitForDefaultPrimaryServing(re) + re.NotNil(primary) + clusterCh <- tsoCluster + ch <- struct{}{} + }() + err = apiCluster.RunInitialServers() + re.NoError(err) + leaderName := apiCluster.WaitLeader() + pdLeaderServer := apiCluster.GetServer(leaderName) + re.NoError(pdLeaderServer.BootstrapCluster()) + re.NoError(err) + tsoCluster := <-clusterCh + defer tsoCluster.Destroy() + <-ch + + time.Sleep(time.Second * 1) + input := make(map[string]interface{}) + input["new-id"] = 1 + input["keyspaces"] = []uint32{2} + jsonBody, err := json.Marshal(input) + re.NoError(err) + httpReq, err := http.NewRequest(http.MethodPost, addr+"/pd/api/v2/tso/keyspace-groups/0/split", bytes.NewBuffer(jsonBody)) + re.NoError(err) + httpResp, err := dialClient.Do(httpReq) + re.NoError(err) + defer httpResp.Body.Close() + re.Equal(http.StatusOK, httpResp.StatusCode) + + httpReq, err = http.NewRequest(http.MethodGet, addr+"/pd/api/v2/tso/keyspace-groups/0", nil) + re.NoError(err) + httpResp, err = dialClient.Do(httpReq) + re.NoError(err) + data, err := io.ReadAll(httpResp.Body) + re.NoError(err) + defer httpResp.Body.Close() + re.Equal(http.StatusOK, httpResp.StatusCode) + + var group endpoint.KeyspaceGroup + re.NoError(json.Unmarshal(data, &group)) + re.Len(group.Keyspaces, 2) + re.Len(group.Members, 2) + + re.NoError(failpoint.Disable("github.com/tikv/pd/server/delayStartServer")) +} diff --git a/tests/integrations/mcs/tso/keyspace_group_manager_test.go b/tests/integrations/mcs/tso/keyspace_group_manager_test.go index 98c6b90ca28..4af10fa7107 100644 --- a/tests/integrations/mcs/tso/keyspace_group_manager_test.go +++ b/tests/integrations/mcs/tso/keyspace_group_manager_test.go @@ -16,8 +16,8 @@ package tso import ( "context" - "fmt" - "strconv" + "math/rand" + "net/http" "strings" "sync" "testing" @@ -30,6 +30,7 @@ import ( pd "github.com/tikv/pd/client" "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/keyspace" mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/member" "github.com/tikv/pd/pkg/storage/endpoint" @@ -54,7 +55,7 @@ type tsoKeyspaceGroupManagerTestSuite struct { // pdLeaderServer is the leader server of the PD cluster. pdLeaderServer *tests.TestServer // tsoCluster is the TSO service cluster. - tsoCluster *mcs.TestTSOCluster + tsoCluster *tests.TestTSOCluster } func TestTSOKeyspaceGroupManager(t *testing.T) { @@ -63,6 +64,7 @@ func TestTSOKeyspaceGroupManager(t *testing.T) { func (suite *tsoKeyspaceGroupManagerTestSuite) SetupSuite() { re := suite.Require() + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller", `return(true)`)) var err error suite.ctx, suite.cancel = context.WithCancel(context.Background()) @@ -73,7 +75,7 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) SetupSuite() { leaderName := suite.cluster.WaitLeader() suite.pdLeaderServer = suite.cluster.GetServer(leaderName) re.NoError(suite.pdLeaderServer.BootstrapCluster()) - suite.tsoCluster, err = mcs.NewTestTSOCluster(suite.ctx, 2, suite.pdLeaderServer.GetAddr()) + suite.tsoCluster, err = tests.NewTestTSOCluster(suite.ctx, 2, suite.pdLeaderServer.GetAddr()) re.NoError(err) } @@ -81,6 +83,7 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TearDownSuite() { suite.cancel() suite.tsoCluster.Destroy() suite.cluster.Destroy() + suite.Require().NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller")) } func (suite *tsoKeyspaceGroupManagerTestSuite) TearDownTest() { @@ -198,20 +201,10 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestKeyspacesServedByNonDefaultKe // Make sure every keyspace group is using the right timestamp path // for loading/saving timestamp from/to etcd and the right primary path // for primary election. - var ( - timestampPath string - primaryPath string - ) - clusterID := strconv.FormatUint(suite.pdLeaderServer.GetClusterID(), 10) - if param.keyspaceGroupID == mcsutils.DefaultKeyspaceGroupID { - timestampPath = fmt.Sprintf("/pd/%s/timestamp", clusterID) - primaryPath = fmt.Sprintf("/ms/%s/tso/00000/primary", clusterID) - } else { - timestampPath = fmt.Sprintf("/ms/%s/tso/%05d/gta/timestamp", - clusterID, param.keyspaceGroupID) - primaryPath = fmt.Sprintf("/ms/%s/tso/%s/election/%05d/primary", - clusterID, mcsutils.KeyspaceGroupsKey, param.keyspaceGroupID) - } + clusterID := suite.pdLeaderServer.GetClusterID() + rootPath := endpoint.TSOSvcRootPath(clusterID) + primaryPath := endpoint.KeyspaceGroupPrimaryPath(rootPath, param.keyspaceGroupID) + timestampPath := endpoint.FullTimestampPath(clusterID, param.keyspaceGroupID) re.Equal(timestampPath, am.GetTimestampPath(tsopkg.GlobalDCLocation)) re.Equal(primaryPath, am.GetMember().GetLeaderPath()) @@ -276,17 +269,15 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplit() { NewID: 2, Keyspaces: []uint32{222, 333}, }) - kg2 := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 2) - re.Equal(uint32(2), kg2.ID) - re.Equal([]uint32{222, 333}, kg2.Keyspaces) - re.True(kg2.IsSplitTarget()) - // Check the split TSO from keyspace group 2. - var splitTS pdpb.Timestamp + // Wait for the split to complete automatically even there is no TSO request from the outside. testutil.Eventually(re, func() bool { - splitTS, err = suite.requestTSO(re, 222, 2) - return err == nil && tsoutil.CompareTimestamp(&splitTS, &pdpb.Timestamp{}) > 0 + kg2 := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 2) + re.Equal(uint32(2), kg2.ID) + re.Equal([]uint32{222, 333}, kg2.Keyspaces) + return !kg2.IsSplitting() }) - splitTS, err = suite.requestTSO(re, 222, 2) + // Check the split TSO from keyspace group 2 now. + splitTS, err := suite.requestTSO(re, 222, 2) re.NoError(err) re.Greater(tsoutil.CompareTimestamp(&splitTS, &ts), 0) } @@ -356,8 +347,6 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplitElection return len(member1.GetLeaderListenUrls()) > 0 && len(member2.GetLeaderListenUrls()) > 0 }) re.Equal(member1.GetLeaderListenUrls(), member2.GetLeaderListenUrls()) - // Finish the split. - handlersutil.MustFinishSplitKeyspaceGroup(re, suite.pdLeaderServer, 2) // Wait for the keyspace groups to finish the split. waitFinishSplit(re, suite.pdLeaderServer, 1, 2, []uint32{111}, []uint32{222, 333}) } @@ -369,13 +358,19 @@ func waitFinishSplit( splitSourceKeyspaces, splitTargetKeyspaces []uint32, ) { testutil.Eventually(re, func() bool { - kg := handlersutil.MustLoadKeyspaceGroupByID(re, server, splitTargetID) + kg, code := handlersutil.TryLoadKeyspaceGroupByID(re, server, splitTargetID) + if code != http.StatusOK { + return false + } re.Equal(splitTargetID, kg.ID) re.Equal(splitTargetKeyspaces, kg.Keyspaces) return !kg.IsSplitTarget() }) testutil.Eventually(re, func() bool { - kg := handlersutil.MustLoadKeyspaceGroupByID(re, server, splitSourceID) + kg, code := handlersutil.TryLoadKeyspaceGroupByID(re, server, splitSourceID) + if code != http.StatusOK { + return false + } re.Equal(splitSourceID, kg.ID) re.Equal(splitSourceKeyspaces, kg.Keyspaces) return !kg.IsSplitSource() @@ -401,15 +396,33 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplitClient() re.Equal(uint32(1), kg1.ID) re.Equal([]uint32{444, 555, 666}, kg1.Keyspaces) re.False(kg1.IsSplitting()) - // Make sure the leader of the keyspace group 1 is elected. - member, err := suite.tsoCluster.WaitForPrimaryServing(re, 555, 1).GetMember(555, 1) + // Request the TSO for keyspace 555 concurrently via client. + cancel := suite.dispatchClient(re, 555, 1) + // Split the keyspace group 1 to 2. + handlersutil.MustSplitKeyspaceGroup(re, suite.pdLeaderServer, 1, &handlers.SplitKeyspaceGroupByIDParams{ + NewID: 2, + Keyspaces: []uint32{555, 666}, + }) + // Wait for the keyspace groups to finish the split. + waitFinishSplit(re, suite.pdLeaderServer, 1, 2, []uint32{444}, []uint32{555, 666}) + // Stop the client. + cancel() + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/systemTimeSlow")) +} + +func (suite *tsoKeyspaceGroupManagerTestSuite) dispatchClient( + re *require.Assertions, keyspaceID, keyspaceGroupID uint32, +) context.CancelFunc { + // Make sure the leader of the keyspace group is elected. + member, err := suite.tsoCluster. + WaitForPrimaryServing(re, keyspaceID, keyspaceGroupID). + GetMember(keyspaceID, keyspaceGroupID) re.NoError(err) re.NotNil(member) - // Prepare the client for keyspace 555. - tsoClient, err := pd.NewClientWithKeyspace(suite.ctx, 555, []string{suite.pdLeaderServer.GetAddr()}, pd.SecurityOption{}) + // Prepare the client for keyspace. + tsoClient, err := pd.NewClientWithKeyspace(suite.ctx, keyspaceID, []string{suite.pdLeaderServer.GetAddr()}, pd.SecurityOption{}) re.NoError(err) re.NotNil(tsoClient) - // Request the TSO for keyspace 555 concurrently. var ( wg sync.WaitGroup ctx, cancel = context.WithCancel(suite.ctx) @@ -421,8 +434,6 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplitClient() for { select { case <-ctx.Done(): - // Make sure at least one TSO request is successful. - re.NotEmpty(lastPhysical) return default: } @@ -433,7 +444,8 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplitClient() if strings.Contains(errMsg, "context canceled") || strings.Contains(errMsg, "not leader") || strings.Contains(errMsg, "not served") || - strings.Contains(errMsg, "ErrKeyspaceNotAssigned") { + strings.Contains(errMsg, "ErrKeyspaceNotAssigned") || + strings.Contains(errMsg, "ErrKeyspaceGroupIsMerging") { continue } re.FailNow(errMsg) @@ -446,32 +458,22 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplitClient() lastPhysical, lastLogical = physical, logical } }() - // Split the keyspace group 1 to 2. - handlersutil.MustSplitKeyspaceGroup(re, suite.pdLeaderServer, 1, &handlers.SplitKeyspaceGroupByIDParams{ - NewID: 2, - Keyspaces: []uint32{555, 666}, - }) - // Wait for the keyspace groups to finish the split. - waitFinishSplit(re, suite.pdLeaderServer, 1, 2, []uint32{444}, []uint32{555, 666}) - // Stop the client. - cancel() - wg.Wait() - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/systemTimeSlow")) + return func() { + // Wait for a while to make sure the client has sent more TSO requests. + time.Sleep(time.Second) + // Cancel the context to stop the client. + cancel() + // Wait for the client to stop. + wg.Wait() + } } func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupMembers() { re := suite.Require() re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/skipSplitRegion", "return(true)")) re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) - kg := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 0) - re.Equal(uint32(0), kg.ID) - re.Equal([]uint32{0}, kg.Keyspaces) - re.False(kg.IsSplitting()) // wait for finishing alloc nodes - testutil.Eventually(re, func() bool { - kg = handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 0) - return len(kg.Members) == 2 - }) + waitFinishAllocNodes(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID) testConfig := map[string]string{ "config": "1", "tso_keyspace_group_id": "0", @@ -481,20 +483,25 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupMembers() { Name: "test_keyspace", Config: testConfig, }) - kg = handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 0) - testutil.Eventually(re, func() bool { - kg = handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 0) - return len(kg.Members) == 2 - }) + waitFinishAllocNodes(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID) re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/skipSplitRegion")) re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) } +func waitFinishAllocNodes(re *require.Assertions, server *tests.TestServer, groupID uint32) { + testutil.Eventually(re, func() bool { + kg := handlersutil.MustLoadKeyspaceGroupByID(re, server, groupID) + re.Equal(groupID, kg.ID) + return len(kg.Members) == mcsutils.DefaultKeyspaceGroupReplicaCount + }) +} + func TestTwiceSplitKeyspaceGroup(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller", `return(true)`)) // Init api server config but not start. tc, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { @@ -505,21 +512,6 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { re.NoError(err) pdAddr := tc.GetConfig().GetClientURL() - // Start pd client and wait pd server start. - var clients sync.Map - go func() { - apiCtx := pd.NewAPIContextV2("keyspace_b") // its keyspace id is 2. - cli, err := pd.NewClientWithAPIContext(ctx, apiCtx, []string{pdAddr}, pd.SecurityOption{}) - re.NoError(err) - clients.Store("keyspace_b", cli) - }() - go func() { - apiCtx := pd.NewAPIContextV2("keyspace_a") // its keyspace id is 1. - cli, err := pd.NewClientWithAPIContext(ctx, apiCtx, []string{pdAddr}, pd.SecurityOption{}) - re.NoError(err) - clients.Store("keyspace_a", cli) - }() - // Start api server and tso server. err = tc.RunInitialServers() re.NoError(err) @@ -528,25 +520,11 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { leaderServer := tc.GetServer(tc.GetLeader()) re.NoError(leaderServer.BootstrapCluster()) - tsoCluster, err := mcs.NewTestTSOCluster(ctx, 2, pdAddr) + tsoCluster, err := tests.NewTestTSOCluster(ctx, 2, pdAddr) re.NoError(err) defer tsoCluster.Destroy() tsoCluster.WaitForDefaultPrimaryServing(re) - // Wait pd clients are ready. - testutil.Eventually(re, func() bool { - count := 0 - clients.Range(func(_, _ interface{}) bool { - count++ - return true - }) - return count == 2 - }) - clientA, ok := clients.Load("keyspace_a") - re.True(ok) - clientB, ok := clients.Load("keyspace_b") - re.True(ok) - // First split keyspace group 0 to 1 with keyspace 2. kgm := leaderServer.GetServer().GetKeyspaceGroupManager() re.NotNil(kgm) @@ -555,13 +533,7 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { return err == nil }) - // Trigger checkTSOSplit to ensure the split is finished. - testutil.Eventually(re, func() bool { - _, _, err = clientB.(pd.Client).GetTS(ctx) - return err == nil - }) waitFinishSplit(re, leaderServer, 0, 1, []uint32{mcsutils.DefaultKeyspaceID, 1}, []uint32{2}) - clientB.(pd.Client).Close() // Then split keyspace group 0 to 2 with keyspace 1. testutil.Eventually(re, func() bool { @@ -569,13 +541,7 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { return err == nil }) - // Trigger checkTSOSplit to ensure the split is finished. - testutil.Eventually(re, func() bool { - _, _, err = clientA.(pd.Client).GetTS(ctx) - return err == nil - }) waitFinishSplit(re, leaderServer, 0, 2, []uint32{mcsutils.DefaultKeyspaceID}, []uint32{1}) - clientA.(pd.Client).Close() // Check the keyspace group 0 is split to 1 and 2. kg0 := handlersutil.MustLoadKeyspaceGroupByID(re, leaderServer, 0) @@ -588,6 +554,7 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { re.False(kg1.IsSplitting()) re.False(kg2.IsSplitting()) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller")) re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) } @@ -663,66 +630,171 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupMergeClient() re.Equal(uint32(1), kg1.ID) re.Equal([]uint32{111, 222, 333}, kg1.Keyspaces) re.False(kg1.IsMerging()) - // Make sure the leader of the keyspace group 1 is elected. - member, err := suite.tsoCluster.WaitForPrimaryServing(re, 222, 1).GetMember(222, 1) - re.NoError(err) - re.NotNil(member) - // Prepare the client for keyspace 222. - tsoClient, err := pd.NewClientWithKeyspace(suite.ctx, 222, []string{suite.pdLeaderServer.GetAddr()}, pd.SecurityOption{}) - re.NoError(err) - re.NotNil(tsoClient) - // Request the TSO for keyspace 222 concurrently. - var ( - wg sync.WaitGroup - ctx, cancel = context.WithCancel(suite.ctx) - lastPhysical, lastLogical int64 - ) - wg.Add(1) - go func() { - defer wg.Done() - for { - select { - case <-ctx.Done(): - // Make sure at least one TSO request is successful. - re.NotEmpty(lastPhysical) - return - default: - } - physical, logical, err := tsoClient.GetTS(ctx) - if err != nil { - errMsg := err.Error() - // Ignore the errors caused by the merge and context cancellation. - if strings.Contains(errMsg, "context canceled") || - strings.Contains(errMsg, "not leader") || - strings.Contains(errMsg, "not served") || - strings.Contains(errMsg, "ErrKeyspaceNotAssigned") || - strings.Contains(errMsg, "ErrKeyspaceGroupIsMerging") { - continue - } - re.FailNow(errMsg) - } - if physical == lastPhysical { - re.Greater(logical, lastLogical) - } else { - re.Greater(physical, lastPhysical) - } - lastPhysical, lastLogical = physical, logical - } - }() + // Request the TSO for keyspace 222 concurrently via client. + cancel := suite.dispatchClient(re, 222, 1) // Merge the keyspace group 1 to the default keyspace group. handlersutil.MustMergeKeyspaceGroup(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID, &handlers.MergeKeyspaceGroupsParams{ MergeList: []uint32{1}, }) // Wait for the default keyspace group to finish the merge. + waitFinishMerge(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID, []uint32{111, 222, 333}) + // Stop the client. + cancel() +} + +func waitFinishMerge( + re *require.Assertions, + server *tests.TestServer, + mergeTargetID uint32, + keyspaces []uint32, +) { testutil.Eventually(re, func() bool { - kg := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID) + kg := handlersutil.MustLoadKeyspaceGroupByID(re, server, mergeTargetID) re.Equal(mcsutils.DefaultKeyspaceGroupID, kg.ID) - for _, keyspaceID := range []uint32{111, 222, 333} { + for _, keyspaceID := range keyspaces { re.Contains(kg.Keyspaces, keyspaceID) } return !kg.IsMergeTarget() }) +} + +func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupMergeBeforeInitTSO() { + re := suite.Require() + // Make sure the TSO of keyspace group 1 won't be initialized before it's merged. + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/failedToSaveTimestamp", `return(true)`)) + // Request the TSO for the default keyspace concurrently via client. + cancel := suite.dispatchClient(re, mcsutils.DefaultKeyspaceID, mcsutils.DefaultKeyspaceGroupID) + // Create the keyspace group 1 with keyspaces [111, 222, 333]. + handlersutil.MustCreateKeyspaceGroup(re, suite.pdLeaderServer, &handlers.CreateKeyspaceGroupParams{ + KeyspaceGroups: []*endpoint.KeyspaceGroup{ + { + ID: 1, + UserKind: endpoint.Standard.String(), + Members: suite.tsoCluster.GetKeyspaceGroupMember(), + Keyspaces: []uint32{111, 222, 333}, + }, + }, + }) + // Merge the keyspace group 1 to the default keyspace group. + handlersutil.MustMergeKeyspaceGroup(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID, &handlers.MergeKeyspaceGroupsParams{ + MergeList: []uint32{1}, + }) + // Wait for the default keyspace group to finish the merge. + waitFinishMerge(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID, []uint32{111, 222, 333}) // Stop the client. cancel() - wg.Wait() + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/failedToSaveTimestamp")) +} + +// See https://github.com/tikv/pd/issues/6748 +func TestGetTSOImmediately(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastPrimaryPriorityCheck", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller", `return(true)`)) + + // Init api server config but not start. + tc, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { + conf.Keyspace.PreAlloc = []string{ + "keyspace_a", "keyspace_b", + } + }) + re.NoError(err) + pdAddr := tc.GetConfig().GetClientURL() + + // Start api server and tso server. + err = tc.RunInitialServers() + re.NoError(err) + defer tc.Destroy() + tc.WaitLeader() + leaderServer := tc.GetServer(tc.GetLeader()) + re.NoError(leaderServer.BootstrapCluster()) + + tsoCluster, err := tests.NewTestTSOCluster(ctx, 2, pdAddr) + re.NoError(err) + defer tsoCluster.Destroy() + tsoCluster.WaitForDefaultPrimaryServing(re) + + // First split keyspace group 0 to 1 with keyspace 2. + kgm := leaderServer.GetServer().GetKeyspaceGroupManager() + re.NotNil(kgm) + testutil.Eventually(re, func() bool { + err = kgm.SplitKeyspaceGroupByID(0, 1, []uint32{2}) + return err == nil + }) + + waitFinishSplit(re, leaderServer, 0, 1, []uint32{mcsutils.DefaultKeyspaceID, 1}, []uint32{2}) + + kg0 := handlersutil.MustLoadKeyspaceGroupByID(re, leaderServer, 0) + kg1 := handlersutil.MustLoadKeyspaceGroupByID(re, leaderServer, 1) + re.Equal([]uint32{0, 1}, kg0.Keyspaces) + re.Equal([]uint32{2}, kg1.Keyspaces) + re.False(kg0.IsSplitting()) + re.False(kg1.IsSplitting()) + + // Let group 0 and group 1 have different primary node. + kgm.SetPriorityForKeyspaceGroup(0, kg0.Members[0].Address, 100) + kgm.SetPriorityForKeyspaceGroup(1, kg1.Members[1].Address, 100) + testutil.Eventually(re, func() bool { + p0, _ := kgm.GetKeyspaceGroupPrimaryByID(0) + p1, _ := kgm.GetKeyspaceGroupPrimaryByID(1) + return p0 == kg0.Members[0].Address && p1 == kg1.Members[1].Address + }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) + + apiCtx := pd.NewAPIContextV2("keyspace_b") // its keyspace id is 2. + cli, err := pd.NewClientWithAPIContext(ctx, apiCtx, []string{pdAddr}, pd.SecurityOption{}) + re.NoError(err) + _, _, err = cli.GetTS(ctx) + re.NoError(err) + cli.Close() + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastPrimaryPriorityCheck")) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller")) +} + +func (suite *tsoKeyspaceGroupManagerTestSuite) TestKeyspaceGroupMergeIntoDefault() { + re := suite.Require() + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) + + var ( + keyspaceGroupNum = keyspace.MaxEtcdTxnOps + keyspaceGroups = make([]*endpoint.KeyspaceGroup, 0, keyspaceGroupNum) + keyspaces = make([]uint32, 0, keyspaceGroupNum) + ) + for i := 1; i <= keyspaceGroupNum; i++ { + keyspaceGroups = append(keyspaceGroups, &endpoint.KeyspaceGroup{ + ID: uint32(i), + UserKind: endpoint.UserKind(rand.Intn(int(endpoint.UserKindCount))).String(), + Keyspaces: []uint32{uint32(i)}, + }) + keyspaces = append(keyspaces, uint32(i)) + if len(keyspaceGroups) < keyspace.MaxEtcdTxnOps/2 && i != keyspaceGroupNum { + continue + } + handlersutil.MustCreateKeyspaceGroup(re, suite.pdLeaderServer, &handlers.CreateKeyspaceGroupParams{ + KeyspaceGroups: keyspaceGroups, + }) + keyspaceGroups = keyspaceGroups[:0] + } + // Check if all the keyspace groups are created. + groups := handlersutil.MustLoadKeyspaceGroups(re, suite.pdLeaderServer, "0", "0") + re.Len(groups, keyspaceGroupNum+1) + // Wait for all the keyspace groups to be served. + svr := suite.tsoCluster.WaitForDefaultPrimaryServing(re) + re.NotNil(svr) + svr = suite.tsoCluster.WaitForPrimaryServing(re, uint32(keyspaceGroupNum), uint32(keyspaceGroupNum)) + re.NotNil(svr) + // Merge all the keyspace groups into the default keyspace group. + handlersutil.MustMergeKeyspaceGroup(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID, &handlers.MergeKeyspaceGroupsParams{ + MergeAllIntoDefault: true, + }) + // Wait for all the keyspace groups to be merged. + waitFinishMerge(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID, keyspaces) + // Check if all the keyspace groups are merged. + groups = handlersutil.MustLoadKeyspaceGroups(re, suite.pdLeaderServer, "0", "0") + re.Len(groups, 1) + + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) } diff --git a/tests/integrations/mcs/tso/proxy_test.go b/tests/integrations/mcs/tso/proxy_test.go index d873665c258..45646dfa48e 100644 --- a/tests/integrations/mcs/tso/proxy_test.go +++ b/tests/integrations/mcs/tso/proxy_test.go @@ -32,7 +32,6 @@ import ( "github.com/tikv/pd/client/tsoutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "github.com/tikv/pd/tests/integrations/mcs" "go.uber.org/zap" "google.golang.org/grpc" ) @@ -44,7 +43,7 @@ type tsoProxyTestSuite struct { apiCluster *tests.TestCluster apiLeader *tests.TestServer backendEndpoints string - tsoCluster *mcs.TestTSOCluster + tsoCluster *tests.TestTSOCluster defaultReq *pdpb.TsoRequest streams []pdpb.PD_TsoClient cleanupFuncs []testutil.CleanupFunc @@ -70,7 +69,7 @@ func (s *tsoProxyTestSuite) SetupSuite() { s.NoError(s.apiLeader.BootstrapCluster()) // Create a TSO cluster with 2 servers - s.tsoCluster, err = mcs.NewTestTSOCluster(s.ctx, 2, s.backendEndpoints) + s.tsoCluster, err = tests.NewTestTSOCluster(s.ctx, 2, s.backendEndpoints) re.NoError(err) s.tsoCluster.WaitForDefaultPrimaryServing(re) diff --git a/tests/integrations/mcs/tso/server_test.go b/tests/integrations/mcs/tso/server_test.go index 4dce8313034..49eab333066 100644 --- a/tests/integrations/mcs/tso/server_test.go +++ b/tests/integrations/mcs/tso/server_test.go @@ -95,7 +95,7 @@ func (suite *tsoServerTestSuite) TestTSOServerStartAndStopNormally() { }() re := suite.Require() - s, cleanup := mcs.StartSingleTSOTestServer(suite.ctx, re, suite.backendEndpoints, tempurl.Alloc()) + s, cleanup := tests.StartSingleTSOTestServer(suite.ctx, re, suite.backendEndpoints, tempurl.Alloc()) defer cleanup() testutil.Eventually(re, func() bool { @@ -135,7 +135,11 @@ func (suite *tsoServerTestSuite) TestParticipantStartWithAdvertiseListenAddr() { cfg, err := tso.GenerateConfig(cfg) re.NoError(err) - s, cleanup, err := mcs.NewTSOTestServer(suite.ctx, cfg) + // Setup the logger. + err = tests.InitLogger(cfg) + re.NoError(err) + + s, cleanup, err := tests.NewTSOTestServer(suite.ctx, cfg) re.NoError(err) defer cleanup() testutil.Eventually(re, func() bool { @@ -182,7 +186,7 @@ func checkTSOPath(re *require.Assertions, isAPIServiceMode bool) { re.Equal(1, getEtcdTimestampKeyNum(re, client)) } - _, cleanup := mcs.StartSingleTSOTestServer(ctx, re, backendEndpoints, tempurl.Alloc()) + _, cleanup := tests.StartSingleTSOTestServer(ctx, re, backendEndpoints, tempurl.Alloc()) defer cleanup() cli := mcs.SetupClientWithAPIContext(ctx, re, pd.NewAPIContextV2(""), []string{backendEndpoints}) @@ -200,7 +204,7 @@ func getEtcdTimestampKeyNum(re *require.Assertions, client *clientv3.Client) int var count int for _, kv := range resp.Kvs { key := strings.TrimSpace(string(kv.Key)) - if !strings.HasSuffix(key, "timestamp") { + if !strings.HasSuffix(key, endpoint.TimestampKey) { continue } count++ @@ -232,7 +236,7 @@ func TestWaitAPIServiceReady(t *testing.T) { cluster, backendEndpoints := startCluster(false /*isAPIServiceMode*/) sctx, scancel := context.WithTimeout(ctx, time.Second*10) defer scancel() - s, _, err := mcs.StartSingleTSOTestServerWithoutCheck(sctx, re, backendEndpoints, tempurl.Alloc()) + s, _, err := tests.StartSingleTSOTestServerWithoutCheck(sctx, re, backendEndpoints, tempurl.Alloc()) re.Error(err) re.Nil(s) cluster.Destroy() @@ -241,7 +245,7 @@ func TestWaitAPIServiceReady(t *testing.T) { cluster, backendEndpoints = startCluster(true /*isAPIServiceMode*/) sctx, scancel = context.WithTimeout(ctx, time.Second*10) defer scancel() - s, cleanup, err := mcs.StartSingleTSOTestServerWithoutCheck(sctx, re, backendEndpoints, tempurl.Alloc()) + s, cleanup, err := tests.StartSingleTSOTestServerWithoutCheck(sctx, re, backendEndpoints, tempurl.Alloc()) re.NoError(err) defer cluster.Destroy() defer cleanup() @@ -314,7 +318,7 @@ func (suite *APIServerForwardTestSuite) TearDownTest() { func (suite *APIServerForwardTestSuite) TestForwardTSORelated() { // Unable to use the tso-related interface without tso server suite.checkUnavailableTSO() - tc, err := mcs.NewTestTSOCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestTSOCluster(suite.ctx, 1, suite.backendEndpoints) suite.NoError(err) defer tc.Destroy() tc.WaitForDefaultPrimaryServing(suite.Require()) @@ -324,7 +328,7 @@ func (suite *APIServerForwardTestSuite) TestForwardTSORelated() { func (suite *APIServerForwardTestSuite) TestForwardTSOWhenPrimaryChanged() { re := suite.Require() - tc, err := mcs.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) + tc, err := tests.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) re.NoError(err) defer tc.Destroy() tc.WaitForDefaultPrimaryServing(re) @@ -363,7 +367,7 @@ func (suite *APIServerForwardTestSuite) TestResignTSOPrimaryForward() { // TODO: test random kill primary with 3 nodes re := suite.Require() - tc, err := mcs.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) + tc, err := tests.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) re.NoError(err) defer tc.Destroy() tc.WaitForDefaultPrimaryServing(re) @@ -387,7 +391,7 @@ func (suite *APIServerForwardTestSuite) TestResignTSOPrimaryForward() { func (suite *APIServerForwardTestSuite) TestResignAPIPrimaryForward() { re := suite.Require() - tc, err := mcs.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) + tc, err := tests.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) re.NoError(err) defer tc.Destroy() tc.WaitForDefaultPrimaryServing(re) @@ -431,7 +435,7 @@ func (suite *APIServerForwardTestSuite) TestForwardTSOUnexpectedToFollower3() { func (suite *APIServerForwardTestSuite) checkForwardTSOUnexpectedToFollower(checkTSO func()) { re := suite.Require() - tc, err := mcs.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) + tc, err := tests.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) re.NoError(err) tc.WaitForDefaultPrimaryServing(re) @@ -512,7 +516,7 @@ type CommonTestSuite struct { ctx context.Context cancel context.CancelFunc cluster *tests.TestCluster - tsoCluster *mcs.TestTSOCluster + tsoCluster *tests.TestTSOCluster pdLeader *tests.TestServer // tsoDefaultPrimaryServer is the primary server of the default keyspace group tsoDefaultPrimaryServer *tso.Server @@ -538,7 +542,7 @@ func (suite *CommonTestSuite) SetupSuite() { suite.backendEndpoints = suite.pdLeader.GetAddr() suite.NoError(suite.pdLeader.BootstrapCluster()) - suite.tsoCluster, err = mcs.NewTestTSOCluster(suite.ctx, 1, suite.backendEndpoints) + suite.tsoCluster, err = tests.NewTestTSOCluster(suite.ctx, 1, suite.backendEndpoints) suite.NoError(err) suite.tsoCluster.WaitForDefaultPrimaryServing(re) suite.tsoDefaultPrimaryServer = suite.tsoCluster.GetPrimaryServer(utils.DefaultKeyspaceID, utils.DefaultKeyspaceGroupID) diff --git a/tests/integrations/tso/client_test.go b/tests/integrations/tso/client_test.go index 9a21c0f921a..83afe7234dd 100644 --- a/tests/integrations/tso/client_test.go +++ b/tests/integrations/tso/client_test.go @@ -54,7 +54,7 @@ type tsoClientTestSuite struct { // pdLeaderServer is the leader server of the PD cluster. pdLeaderServer *tests.TestServer // The TSO service in microservice mode. - tsoCluster *mcs.TestTSOCluster + tsoCluster *tests.TestTSOCluster keyspaceGroups []struct { keyspaceGroupID uint32 @@ -108,7 +108,7 @@ func (suite *tsoClientTestSuite) SetupSuite() { suite.clients = make([]pd.Client, 0) suite.clients = append(suite.clients, client) } else { - suite.tsoCluster, err = mcs.NewTestTSOCluster(suite.ctx, 3, suite.backendEndpoints) + suite.tsoCluster, err = tests.NewTestTSOCluster(suite.ctx, 3, suite.backendEndpoints) re.NoError(err) suite.keyspaceGroups = []struct { @@ -430,13 +430,13 @@ func TestMixedTSODeployment(t *testing.T) { err = apiSvr.Run() re.NoError(err) - s, cleanup := mcs.StartSingleTSOTestServer(ctx, re, backendEndpoints, tempurl.Alloc()) + s, cleanup := tests.StartSingleTSOTestServer(ctx, re, backendEndpoints, tempurl.Alloc()) defer cleanup() - mcs.WaitForPrimaryServing(re, map[string]bs.Server{s.GetAddr(): s}) + tests.WaitForPrimaryServing(re, map[string]bs.Server{s.GetAddr(): s}) ctx1, cancel1 := context.WithCancel(context.Background()) var wg sync.WaitGroup - checkTSO(ctx1, re, &wg, backendEndpoints) + checkTSO(ctx1, re, &wg, backendEndpoints, pd.WithAllowTSOFallback() /* It's expected that the timestamp fallback happens here */) wg.Add(1) go func() { defer wg.Done() @@ -451,12 +451,62 @@ func TestMixedTSODeployment(t *testing.T) { wg.Wait() } -func checkTSO(ctx context.Context, re *require.Assertions, wg *sync.WaitGroup, backendEndpoints string) { +// TestUpgradingAPIandTSOClusters tests the scenario that after we restart the API cluster +// then restart the TSO cluster, the TSO service can still serve TSO requests normally. +func TestUpgradingAPIandTSOClusters(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + + // Create an API cluster which has 3 servers + apiCluster, err := tests.NewTestAPICluster(ctx, 3) + re.NoError(err) + err = apiCluster.RunInitialServers() + re.NoError(err) + leaderName := apiCluster.WaitLeader() + pdLeader := apiCluster.GetServer(leaderName) + backendEndpoints := pdLeader.GetAddr() + + // Create a pd client in PD mode to let the API leader to forward requests to the TSO cluster. + re.NoError(failpoint.Enable("github.com/tikv/pd/client/usePDServiceMode", "return(true)")) + pdClient, err := pd.NewClientWithContext(context.Background(), + []string{backendEndpoints}, pd.SecurityOption{}, pd.WithMaxErrorRetry(1)) + re.NoError(err) + + // Create a TSO cluster which has 2 servers + tsoCluster, err := tests.NewTestTSOCluster(ctx, 2, backendEndpoints) + re.NoError(err) + tsoCluster.WaitForDefaultPrimaryServing(re) + // The TSO service should be eventually healthy + mcs.WaitForTSOServiceAvailable(ctx, re, pdClient) + + // Restart the API cluster + apiCluster, err = tests.RestartTestAPICluster(ctx, apiCluster) + re.NoError(err) + // The TSO service should be eventually healthy + mcs.WaitForTSOServiceAvailable(ctx, re, pdClient) + + // Restart the TSO cluster + tsoCluster, err = tests.RestartTestTSOCluster(ctx, tsoCluster) + re.NoError(err) + // The TSO service should be eventually healthy + mcs.WaitForTSOServiceAvailable(ctx, re, pdClient) + + tsoCluster.Destroy() + apiCluster.Destroy() + cancel() + re.NoError(failpoint.Disable("github.com/tikv/pd/client/usePDServiceMode")) +} + +func checkTSO( + ctx context.Context, re *require.Assertions, wg *sync.WaitGroup, + backendEndpoints string, opts ...pd.ClientOption, +) { wg.Add(tsoRequestConcurrencyNumber) for i := 0; i < tsoRequestConcurrencyNumber; i++ { go func() { defer wg.Done() - cli := mcs.SetupClientWithAPIContext(ctx, re, pd.NewAPIContextV1(), strings.Split(backendEndpoints, ",")) + cli := mcs.SetupClientWithAPIContext(ctx, re, pd.NewAPIContextV1(), strings.Split(backendEndpoints, ","), opts...) + defer cli.Close() var ts, lastTS uint64 for { select { diff --git a/tests/integrations/tso/consistency_test.go b/tests/integrations/tso/consistency_test.go index 70000d2abd4..1d35e8bf5e2 100644 --- a/tests/integrations/tso/consistency_test.go +++ b/tests/integrations/tso/consistency_test.go @@ -30,7 +30,6 @@ import ( pd "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/tests" - "github.com/tikv/pd/tests/integrations/mcs" "google.golang.org/grpc" ) @@ -85,7 +84,7 @@ func (suite *tsoConsistencyTestSuite) SetupSuite() { if suite.legacy { suite.pdClient = pd.MustNewGrpcClient(re, backendEndpoints) } else { - suite.tsoServer, suite.tsoServerCleanup = mcs.StartSingleTSOTestServer(suite.ctx, re, backendEndpoints, tempurl.Alloc()) + suite.tsoServer, suite.tsoServerCleanup = tests.StartSingleTSOTestServer(suite.ctx, re, backendEndpoints, tempurl.Alloc()) suite.tsoClientConn, suite.tsoClient = tso.MustNewGrpcClient(re, suite.tsoServer.GetAddr()) } } diff --git a/tests/integrations/tso/server_test.go b/tests/integrations/tso/server_test.go index 96fc5d334b0..518335442f4 100644 --- a/tests/integrations/tso/server_test.go +++ b/tests/integrations/tso/server_test.go @@ -28,7 +28,6 @@ import ( "github.com/tikv/pd/pkg/utils/tempurl" pd "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "github.com/tikv/pd/tests/integrations/mcs" "google.golang.org/grpc" ) @@ -83,7 +82,7 @@ func (suite *tsoServerTestSuite) SetupSuite() { if suite.legacy { suite.pdClient = pd.MustNewGrpcClient(re, backendEndpoints) } else { - suite.tsoServer, suite.tsoServerCleanup = mcs.StartSingleTSOTestServer(suite.ctx, re, backendEndpoints, tempurl.Alloc()) + suite.tsoServer, suite.tsoServerCleanup = tests.StartSingleTSOTestServer(suite.ctx, re, backendEndpoints, tempurl.Alloc()) suite.tsoClientConn, suite.tsoClient = tso.MustNewGrpcClient(re, suite.tsoServer.GetAddr()) } } diff --git a/tests/pdctl/keyspace/keyspace_group_test.go b/tests/pdctl/keyspace/keyspace_group_test.go index c982dc140af..1d0c8132c13 100644 --- a/tests/pdctl/keyspace/keyspace_group_test.go +++ b/tests/pdctl/keyspace/keyspace_group_test.go @@ -26,7 +26,6 @@ import ( "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/storage/endpoint" - "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/apiv2/handlers" "github.com/tikv/pd/server/config" @@ -108,12 +107,9 @@ func TestSplitKeyspaceGroup(t *testing.T) { re.NoError(err) pdAddr := tc.GetConfig().GetClientURL() - _, tsoServerCleanup1, err := tests.StartSingleTSOTestServer(ctx, re, pdAddr, tempurl.Alloc()) - defer tsoServerCleanup1() - re.NoError(err) - _, tsoServerCleanup2, err := tests.StartSingleTSOTestServer(ctx, re, pdAddr, tempurl.Alloc()) - defer tsoServerCleanup2() + ttc, err := tests.NewTestTSOCluster(ctx, 2, pdAddr) re.NoError(err) + defer ttc.Destroy() cmd := pdctlCmd.GetRootCmd() tc.WaitLeader() @@ -204,12 +200,10 @@ func TestSetNodeAndPriorityKeyspaceGroup(t *testing.T) { re.NoError(err) pdAddr := tc.GetConfig().GetClientURL() - s1, tsoServerCleanup1, err := tests.StartSingleTSOTestServer(ctx, re, pdAddr, tempurl.Alloc()) - defer tsoServerCleanup1() - re.NoError(err) - s2, tsoServerCleanup2, err := tests.StartSingleTSOTestServer(ctx, re, pdAddr, tempurl.Alloc()) - defer tsoServerCleanup2() + ttc, err := tests.NewTestTSOCluster(ctx, 2, pdAddr) re.NoError(err) + defer ttc.Destroy() + tsoAddrs := ttc.GetAddrs() cmd := pdctlCmd.GetRootCmd() tc.WaitLeader() @@ -219,7 +213,7 @@ func TestSetNodeAndPriorityKeyspaceGroup(t *testing.T) { // set-node keyspace group. defaultKeyspaceGroupID := fmt.Sprintf("%d", utils.DefaultKeyspaceGroupID) testutil.Eventually(re, func() bool { - args := []string{"-u", pdAddr, "keyspace-group", "set-node", defaultKeyspaceGroupID, s1.GetAddr(), s2.GetAddr()} + args := []string{"-u", pdAddr, "keyspace-group", "set-node", defaultKeyspaceGroupID, tsoAddrs[0], tsoAddrs[1]} output, err := pdctl.ExecuteCommand(cmd, args...) re.NoError(err) return strings.Contains(string(output), "Success") @@ -228,7 +222,7 @@ func TestSetNodeAndPriorityKeyspaceGroup(t *testing.T) { // set-priority keyspace group. checkPriority := func(p int) { testutil.Eventually(re, func() bool { - args := []string{"-u", pdAddr, "keyspace-group", "set-priority", defaultKeyspaceGroupID, s1.GetAddr()} + args := []string{"-u", pdAddr, "keyspace-group", "set-priority", defaultKeyspaceGroupID, tsoAddrs[0]} if p >= 0 { args = append(args, strconv.Itoa(p)) } else { @@ -249,8 +243,8 @@ func TestSetNodeAndPriorityKeyspaceGroup(t *testing.T) { re.Equal(utils.DefaultKeyspaceGroupID, keyspaceGroup.ID) re.Len(keyspaceGroup.Members, 2) for _, member := range keyspaceGroup.Members { - re.Contains([]string{s1.GetAddr(), s2.GetAddr()}, member.Address) - if member.Address == s1.GetAddr() { + re.Contains(tsoAddrs, member.Address) + if member.Address == tsoAddrs[0] { re.Equal(p, member.Priority) } else { re.Equal(0, member.Priority) @@ -262,7 +256,7 @@ func TestSetNodeAndPriorityKeyspaceGroup(t *testing.T) { checkPriority(-200) // params error for set-node. - args := []string{"-u", pdAddr, "keyspace-group", "set-node", defaultKeyspaceGroupID, s1.GetAddr()} + args := []string{"-u", pdAddr, "keyspace-group", "set-node", defaultKeyspaceGroupID, tsoAddrs[0]} output, err := pdctl.ExecuteCommand(cmd, args...) re.NoError(err) re.Contains(string(output), "invalid num of nodes") @@ -270,7 +264,7 @@ func TestSetNodeAndPriorityKeyspaceGroup(t *testing.T) { output, err = pdctl.ExecuteCommand(cmd, args...) re.NoError(err) re.Contains(string(output), "Failed to parse the tso node address") - args = []string{"-u", pdAddr, "keyspace-group", "set-node", defaultKeyspaceGroupID, s1.GetAddr(), "http://pingcap.com"} + args = []string{"-u", pdAddr, "keyspace-group", "set-node", defaultKeyspaceGroupID, tsoAddrs[0], "http://pingcap.com"} output, err = pdctl.ExecuteCommand(cmd, args...) re.NoError(err) re.Contains(string(output), "node does not exist") @@ -284,7 +278,7 @@ func TestSetNodeAndPriorityKeyspaceGroup(t *testing.T) { output, err = pdctl.ExecuteCommand(cmd, args...) re.NoError(err) re.Contains(string(output), "node does not exist") - args = []string{"-u", pdAddr, "keyspace-group", "set-priority", defaultKeyspaceGroupID, s1.GetAddr(), "xxx"} + args = []string{"-u", pdAddr, "keyspace-group", "set-priority", defaultKeyspaceGroupID, tsoAddrs[0], "xxx"} output, err = pdctl.ExecuteCommand(cmd, args...) re.NoError(err) re.Contains(string(output), "Failed to parse the priority") @@ -309,12 +303,9 @@ func TestMergeKeyspaceGroup(t *testing.T) { re.NoError(err) pdAddr := tc.GetConfig().GetClientURL() - _, tsoServerCleanup1, err := tests.StartSingleTSOTestServer(ctx, re, pdAddr, tempurl.Alloc()) - defer tsoServerCleanup1() - re.NoError(err) - _, tsoServerCleanup2, err := tests.StartSingleTSOTestServer(ctx, re, pdAddr, tempurl.Alloc()) - defer tsoServerCleanup2() + ttc, err := tests.NewTestTSOCluster(ctx, 2, pdAddr) re.NoError(err) + defer ttc.Destroy() cmd := pdctlCmd.GetRootCmd() tc.WaitLeader() @@ -355,6 +346,59 @@ func TestMergeKeyspaceGroup(t *testing.T) { re.Len(keyspaceGroup.Keyspaces, 130) re.Nil(keyspaceGroup.MergeState) + // split keyspace group multiple times. + for i := 1; i <= 10; i++ { + splitTargetID := fmt.Sprintf("%d", i) + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group", "split", "0", splitTargetID, splitTargetID} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + return strings.Contains(string(output), "Success") + }) + args := []string{"-u", pdAddr, "keyspace-group", "finish-split", splitTargetID} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Success") + } + + // merge keyspace group with `all` flag. + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group", "merge", "0", "--all"} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + return strings.Contains(string(output), "Success") + }) + + args = []string{"-u", pdAddr, "keyspace-group", "finish-merge", "0"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Success") + args = []string{"-u", pdAddr, "keyspace-group", "0"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + err = json.Unmarshal(output, &keyspaceGroup) + re.NoError(err) + re.Len(keyspaceGroup.Keyspaces, 130) + re.Nil(keyspaceGroup.MergeState) + + // merge keyspace group with wrong args. + args = []string{"-u", pdAddr, "keyspace-group", "merge"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Must specify the source keyspace group ID(s) or the merge all flag") + args = []string{"-u", pdAddr, "keyspace-group", "merge", "0"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Must specify the source keyspace group ID(s) or the merge all flag") + args = []string{"-u", pdAddr, "keyspace-group", "merge", "0", "1", "--all"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Must specify the source keyspace group ID(s) or the merge all flag") + args = []string{"-u", pdAddr, "keyspace-group", "merge", "1", "--all"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Unable to merge all keyspace groups into a non-default keyspace group") + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) re.NoError(failpoint.Disable("github.com/tikv/pd/server/delayStartServerLoop")) } @@ -377,12 +421,9 @@ func TestKeyspaceGroupState(t *testing.T) { re.NoError(err) pdAddr := tc.GetConfig().GetClientURL() - _, tsoServerCleanup1, err := tests.StartSingleTSOTestServer(ctx, re, pdAddr, tempurl.Alloc()) - defer tsoServerCleanup1() - re.NoError(err) - _, tsoServerCleanup2, err := tests.StartSingleTSOTestServer(ctx, re, pdAddr, tempurl.Alloc()) - defer tsoServerCleanup2() + ttc, err := tests.NewTestTSOCluster(ctx, 2, pdAddr) re.NoError(err) + defer ttc.Destroy() cmd := pdctlCmd.GetRootCmd() tc.WaitLeader() @@ -450,3 +491,95 @@ func TestKeyspaceGroupState(t *testing.T) { re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) re.NoError(failpoint.Disable("github.com/tikv/pd/server/delayStartServerLoop")) } + +func TestShowKeyspaceGroupPrimary(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/server/delayStartServerLoop", `return(true)`)) + keyspaces := make([]string, 0) + for i := 0; i < 10; i++ { + keyspaces = append(keyspaces, fmt.Sprintf("keyspace_%d", i)) + } + tc, err := tests.NewTestAPICluster(ctx, 3, func(conf *config.Config, serverName string) { + conf.Keyspace.PreAlloc = keyspaces + }) + re.NoError(err) + err = tc.RunInitialServers() + re.NoError(err) + pdAddr := tc.GetConfig().GetClientURL() + + ttc, err := tests.NewTestTSOCluster(ctx, 2, pdAddr) + re.NoError(err) + defer ttc.Destroy() + tsoAddrs := ttc.GetAddrs() + cmd := pdctlCmd.GetRootCmd() + + tc.WaitLeader() + leaderServer := tc.GetServer(tc.GetLeader()) + re.NoError(leaderServer.BootstrapCluster()) + defaultKeyspaceGroupID := fmt.Sprintf("%d", utils.DefaultKeyspaceGroupID) + + // check keyspace group 0 information. + var keyspaceGroup endpoint.KeyspaceGroup + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group"} + output, err := pdctl.ExecuteCommand(cmd, append(args, defaultKeyspaceGroupID)...) + re.NoError(err) + + err = json.Unmarshal(output, &keyspaceGroup) + re.NoError(err) + re.Equal(utils.DefaultKeyspaceGroupID, keyspaceGroup.ID) + return len(keyspaceGroup.Members) == 2 + }) + for _, member := range keyspaceGroup.Members { + re.Contains(tsoAddrs, member.Address) + } + + // get primary for keyspace group 0. + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group", "primary", defaultKeyspaceGroupID} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + var resp handlers.GetKeyspaceGroupPrimaryResponse + json.Unmarshal(output, &resp) + return tsoAddrs[0] == resp.Primary || tsoAddrs[1] == resp.Primary + }) + + // split keyspace group. + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group", "split", "0", "1", "2"} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + return strings.Contains(string(output), "Success") + }) + + // check keyspace group 1 information. + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group"} + output, err := pdctl.ExecuteCommand(cmd, append(args, "1")...) + re.NoError(err) + err = json.Unmarshal(output, &keyspaceGroup) + re.NoErrorf(err, "output: %s", string(output)) + return len(keyspaceGroup.Members) == 2 + }) + for _, member := range keyspaceGroup.Members { + re.Contains(tsoAddrs, member.Address) + } + + // get primary for keyspace group 1. + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group", "primary", "1"} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + var resp handlers.GetKeyspaceGroupPrimaryResponse + json.Unmarshal(output, &resp) + return tsoAddrs[0] == resp.Primary || tsoAddrs[1] == resp.Primary + }) + + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller")) + re.NoError(failpoint.Disable("github.com/tikv/pd/server/delayStartServerLoop")) +} diff --git a/tests/pdctl/keyspace/keyspace_test.go b/tests/pdctl/keyspace/keyspace_test.go new file mode 100644 index 00000000000..a0bab4114df --- /dev/null +++ b/tests/pdctl/keyspace/keyspace_test.go @@ -0,0 +1,103 @@ +// 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 keyspace_test + +import ( + "context" + "encoding/json" + "fmt" + "strings" + "testing" + + "github.com/pingcap/failpoint" + "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/keyspace" + "github.com/tikv/pd/pkg/mcs/utils" + "github.com/tikv/pd/pkg/utils/testutil" + api "github.com/tikv/pd/server/apiv2/handlers" + "github.com/tikv/pd/server/config" + "github.com/tikv/pd/tests" + "github.com/tikv/pd/tests/pdctl" + pdctlCmd "github.com/tikv/pd/tools/pd-ctl/pdctl" +) + +func TestKeyspace(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/server/delayStartServerLoop", `return(true)`)) + keyspaces := make([]string, 0) + for i := 1; i < 10; i++ { + keyspaces = append(keyspaces, fmt.Sprintf("keyspace_%d", i)) + } + tc, err := tests.NewTestAPICluster(ctx, 3, func(conf *config.Config, serverName string) { + conf.Keyspace.PreAlloc = keyspaces + }) + re.NoError(err) + err = tc.RunInitialServers() + re.NoError(err) + pdAddr := tc.GetConfig().GetClientURL() + + ttc, err := tests.NewTestTSOCluster(ctx, 2, pdAddr) + re.NoError(err) + defer ttc.Destroy() + cmd := pdctlCmd.GetRootCmd() + + tc.WaitLeader() + leaderServer := tc.GetServer(tc.GetLeader()) + re.NoError(leaderServer.BootstrapCluster()) + defaultKeyspaceGroupID := fmt.Sprintf("%d", utils.DefaultKeyspaceGroupID) + + var k api.KeyspaceMeta + keyspaceName := "keyspace_1" + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace", keyspaceName} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + re.NoError(json.Unmarshal(output, &k)) + return k.GetName() == keyspaceName + }) + re.Equal(uint32(1), k.GetId()) + re.Equal(defaultKeyspaceGroupID, k.Config[keyspace.TSOKeyspaceGroupIDKey]) + + // split keyspace group. + newGroupID := "2" + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group", "split", "0", newGroupID, "1"} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + return strings.Contains(string(output), "Success") + }) + + // check keyspace group in keyspace whether changed. + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace", keyspaceName} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + re.NoError(json.Unmarshal(output, &k)) + return newGroupID == k.Config[keyspace.TSOKeyspaceGroupIDKey] + }) + + // test error name + args := []string{"-u", pdAddr, "keyspace", "error_name"} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + re.Contains(string(output), "Fail") + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller")) + re.NoError(failpoint.Disable("github.com/tikv/pd/server/delayStartServerLoop")) +} diff --git a/tests/server/apiv2/handlers/testutil.go b/tests/server/apiv2/handlers/testutil.go index 900cd84b829..aca29ebeb52 100644 --- a/tests/server/apiv2/handlers/testutil.go +++ b/tests/server/apiv2/handlers/testutil.go @@ -168,6 +168,13 @@ func tryCreateKeyspaceGroup(re *require.Assertions, server *tests.TestServer, re // MustLoadKeyspaceGroupByID loads the keyspace group by ID with HTTP API. func MustLoadKeyspaceGroupByID(re *require.Assertions, server *tests.TestServer, id uint32) *endpoint.KeyspaceGroup { + kg, code := TryLoadKeyspaceGroupByID(re, server, id) + re.Equal(http.StatusOK, code) + return kg +} + +// TryLoadKeyspaceGroupByID loads the keyspace group by ID with HTTP API. +func TryLoadKeyspaceGroupByID(re *require.Assertions, server *tests.TestServer, id uint32) (*endpoint.KeyspaceGroup, int) { httpReq, err := http.NewRequest(http.MethodGet, server.GetAddr()+keyspaceGroupsPrefix+fmt.Sprintf("/%d", id), nil) re.NoError(err) resp, err := dialClient.Do(httpReq) @@ -175,10 +182,13 @@ func MustLoadKeyspaceGroupByID(re *require.Assertions, server *tests.TestServer, defer resp.Body.Close() data, err := io.ReadAll(resp.Body) re.NoError(err) - re.Equal(http.StatusOK, resp.StatusCode, string(data)) + if resp.StatusCode != http.StatusOK { + return nil, resp.StatusCode + } + var kg endpoint.KeyspaceGroup re.NoError(json.Unmarshal(data, &kg)) - return &kg + return &kg, resp.StatusCode } // MustCreateKeyspaceGroup creates a keyspace group with HTTP API. diff --git a/tests/testutil.go b/tests/testutil.go new file mode 100644 index 00000000000..1657d7bfc3d --- /dev/null +++ b/tests/testutil.go @@ -0,0 +1,117 @@ +// 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 tests + +import ( + "context" + "os" + "sync" + "time" + + "github.com/pingcap/log" + "github.com/stretchr/testify/require" + bs "github.com/tikv/pd/pkg/basicserver" + rm "github.com/tikv/pd/pkg/mcs/resource_manager/server" + tso "github.com/tikv/pd/pkg/mcs/tso/server" + "github.com/tikv/pd/pkg/utils/logutil" + "github.com/tikv/pd/pkg/utils/testutil" +) + +var once sync.Once + +// InitLogger initializes the logger for test. +func InitLogger(cfg *tso.Config) (err error) { + once.Do(func() { + // Setup the logger. + err = logutil.SetupLogger(cfg.Log, &cfg.Logger, &cfg.LogProps, cfg.Security.RedactInfoLog) + if err != nil { + return + } + log.ReplaceGlobals(cfg.Logger, cfg.LogProps) + // Flushing any buffered log entries. + log.Sync() + }) + 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, 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() + cfg.BackendEndpoints = backendEndpoints + cfg.ListenAddr = listenAddrs + cfg, err := tso.GenerateConfig(cfg) + re.NoError(err) + // Setup the logger. + err = InitLogger(cfg) + re.NoError(err) + return NewTSOTestServer(ctx, cfg) +} + +// StartSingleTSOTestServer creates and starts a tso server with default config for testing. +func StartSingleTSOTestServer(ctx context.Context, re *require.Assertions, backendEndpoints, listenAddrs string) (*tso.Server, func()) { + s, cleanup, err := StartSingleTSOTestServerWithoutCheck(ctx, re, backendEndpoints, listenAddrs) + re.NoError(err) + testutil.Eventually(re, func() bool { + return !s.IsClosed() + }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) + + return s, cleanup +} + +// NewTSOTestServer creates a tso server with given config for testing. +func NewTSOTestServer(ctx context.Context, cfg *tso.Config) (*tso.Server, testutil.CleanupFunc, error) { + s := tso.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 +} + +// WaitForPrimaryServing waits for one of servers being elected to be the primary/leader +func WaitForPrimaryServing(re *require.Assertions, serverMap map[string]bs.Server) string { + var primary string + testutil.Eventually(re, func() bool { + for name, s := range serverMap { + if s.IsServing() { + primary = name + return true + } + } + return false + }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) + + return primary +} diff --git a/tests/integrations/mcs/cluster.go b/tests/tso_cluster.go similarity index 74% rename from tests/integrations/mcs/cluster.go rename to tests/tso_cluster.go index 961f26728c6..dc23c24d779 100644 --- a/tests/integrations/mcs/cluster.go +++ b/tests/tso_cluster.go @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -package mcs +package tests import ( "context" "fmt" + "sync" "time" "github.com/stretchr/testify/require" @@ -53,6 +54,56 @@ func NewTestTSOCluster(ctx context.Context, initialServerCount int, backendEndpo return tc, nil } +// RestartTestTSOCluster restarts the TSO test cluster. +func RestartTestTSOCluster( + ctx context.Context, cluster *TestTSOCluster, +) (newCluster *TestTSOCluster, err error) { + newCluster = &TestTSOCluster{ + ctx: ctx, + backendEndpoints: cluster.backendEndpoints, + servers: make(map[string]*tso.Server, len(cluster.servers)), + cleanupFuncs: make(map[string]testutil.CleanupFunc, len(cluster.servers)), + } + var ( + serverMap sync.Map + cleanupMap sync.Map + errorMap sync.Map + ) + wg := sync.WaitGroup{} + for addr, cleanup := range cluster.cleanupFuncs { + wg.Add(1) + go func(addr string, clean testutil.CleanupFunc) { + defer wg.Done() + clean() + serverCfg := cluster.servers[addr].GetConfig() + newServer, newCleanup, err := NewTSOTestServer(newCluster.ctx, serverCfg) + serverMap.Store(addr, newServer) + cleanupMap.Store(addr, newCleanup) + errorMap.Store(addr, err) + }(addr, cleanup) + } + wg.Wait() + + errorMap.Range(func(key, value interface{}) bool { + if value != nil { + err = value.(error) + return false + } + addr := key.(string) + newServer, _ := serverMap.Load(addr) + newCleanup, _ := cleanupMap.Load(addr) + newCluster.servers[addr] = newServer.(*tso.Server) + newCluster.cleanupFuncs[addr] = newCleanup.(testutil.CleanupFunc) + return true + }) + + if err != nil { + return nil, fmt.Errorf("failed to restart the cluster." + err.Error()) + } + + return newCluster, nil +} + // AddServer adds a new TSO server to the test cluster. func (tc *TestTSOCluster) AddServer(addr string) error { cfg := tso.NewConfig() @@ -157,3 +208,12 @@ func (tc *TestTSOCluster) GetKeyspaceGroupMember() (members []endpoint.KeyspaceG } return } + +// GetAddrs returns all TSO server addresses. +func (tc *TestTSOCluster) GetAddrs() []string { + addrs := make([]string, 0, len(tc.servers)) + for _, server := range tc.servers { + addrs = append(addrs, server.GetAddr()) + } + return addrs +} diff --git a/tools/pd-backup/pdbackup/backup.go b/tools/pd-backup/pdbackup/backup.go index a645138b68b..f0b17e48267 100644 --- a/tools/pd-backup/pdbackup/backup.go +++ b/tools/pd-backup/pdbackup/backup.go @@ -25,6 +25,7 @@ import ( "path" "strconv" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" @@ -74,8 +75,7 @@ func GetBackupInfo(client *clientv3.Client, pdAddr string) (*BackupInfo, error) backInfo.AllocIDMax = allocIDMax - timestampPath := path.Join(rootPath, "timestamp") - resp, err = etcdutil.EtcdKVGet(client, timestampPath) + resp, err = etcdutil.EtcdKVGet(client, endpoint.TimestampPath(rootPath)) if err != nil { return nil, err } diff --git a/tools/pd-backup/pdbackup/backup_test.go b/tools/pd-backup/pdbackup/backup_test.go index c747ace21de..ccbefbe4fe8 100644 --- a/tools/pd-backup/pdbackup/backup_test.go +++ b/tools/pd-backup/pdbackup/backup_test.go @@ -16,6 +16,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" @@ -133,10 +134,9 @@ func (s *backupTestSuite) BeforeTest(suiteName, testName string) { var ( rootPath = path.Join(pdRootPath, strconv.FormatUint(clusterID, 10)) - timestampPath = path.Join(rootPath, "timestamp") allocTimestampMaxBytes = typeutil.Uint64ToBytes(allocTimestampMax) ) - _, err = s.etcdClient.Put(ctx, timestampPath, string(allocTimestampMaxBytes)) + _, err = s.etcdClient.Put(ctx, endpoint.TimestampPath(rootPath), string(allocTimestampMaxBytes)) s.NoError(err) var ( diff --git a/tools/pd-ctl/pdctl/command/global.go b/tools/pd-ctl/pdctl/command/global.go index 8d888b60b1f..85fe63ac8be 100644 --- a/tools/pd-ctl/pdctl/command/global.go +++ b/tools/pd-ctl/pdctl/command/global.go @@ -165,7 +165,7 @@ func getEndpoints(cmd *cobra.Command) []string { return strings.Split(addrs, ",") } -func postJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { +func requestJSON(cmd *cobra.Command, method, prefix string, input map[string]interface{}) { data, err := json.Marshal(input) if err != nil { cmd.Println(err) @@ -175,19 +175,31 @@ func postJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { endpoints := getEndpoints(cmd) err = tryURLs(cmd, endpoints, func(endpoint string) error { var msg []byte - var r *http.Response + var req *http.Request + var resp *http.Response url := endpoint + "/" + prefix - r, err = dialClient.Post(url, "application/json", bytes.NewBuffer(data)) + switch method { + case http.MethodPost, http.MethodPut, http.MethodPatch, http.MethodDelete, http.MethodGet: + req, err = http.NewRequest(method, url, bytes.NewBuffer(data)) + if err != nil { + return err + } + req.Header.Set("Content-Type", "application/json") + resp, err = dialClient.Do(req) + default: + err := errors.Errorf("method %s not supported", method) + return err + } if err != nil { return err } - defer r.Body.Close() - if r.StatusCode != http.StatusOK { - msg, err = io.ReadAll(r.Body) + defer resp.Body.Close() + if resp.StatusCode != http.StatusOK { + msg, err = io.ReadAll(resp.Body) if err != nil { return err } - return errors.Errorf("[%d] %s", r.StatusCode, msg) + return errors.Errorf("[%d] %s", resp.StatusCode, msg) } return nil }) @@ -198,6 +210,14 @@ func postJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { cmd.Println("Success!") } +func postJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { + requestJSON(cmd, http.MethodPost, prefix, input) +} + +func patchJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { + requestJSON(cmd, http.MethodPatch, prefix, input) +} + // do send a request to server. Default is Get. func do(endpoint, prefix, method string, resp *string, customHeader http.Header, b *bodyOption) error { var err error diff --git a/tools/pd-ctl/pdctl/command/keyspace_command.go b/tools/pd-ctl/pdctl/command/keyspace_command.go new file mode 100644 index 00000000000..a68e2f05a80 --- /dev/null +++ b/tools/pd-ctl/pdctl/command/keyspace_command.go @@ -0,0 +1,48 @@ +// 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 command + +import ( + "fmt" + "net/http" + + "github.com/spf13/cobra" +) + +const keyspacePrefix = "pd/api/v2/keyspaces" + +// NewKeyspaceCommand returns a keyspace subcommand of rootCmd. +func NewKeyspaceCommand() *cobra.Command { + cmd := &cobra.Command{ + Use: "keyspace [command] [flags]", + Short: "show keyspace information", + Run: showKeyspaceCommandFunc, + } + return cmd +} + +func showKeyspaceCommandFunc(cmd *cobra.Command, args []string) { + if len(args) != 1 { + cmd.Usage() + return + } + + resp, err := doRequest(cmd, fmt.Sprintf("%s/%s?force_refresh_group_id=true", keyspacePrefix, args[0]), http.MethodGet, http.Header{}) + if err != nil { + cmd.Printf("Failed to get the keyspace information: %s\n", err) + return + } + cmd.Println(resp) +} diff --git a/tools/pd-ctl/pdctl/command/keyspace_group_command.go b/tools/pd-ctl/pdctl/command/keyspace_group_command.go index a4be612a301..6dea6d78dd6 100644 --- a/tools/pd-ctl/pdctl/command/keyspace_group_command.go +++ b/tools/pd-ctl/pdctl/command/keyspace_group_command.go @@ -23,6 +23,7 @@ import ( "strings" "github.com/spf13/cobra" + mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/storage/endpoint" ) @@ -42,6 +43,7 @@ func NewKeyspaceGroupCommand() *cobra.Command { cmd.AddCommand(newFinishMergeKeyspaceGroupCommand()) cmd.AddCommand(newSetNodesKeyspaceGroupCommand()) cmd.AddCommand(newSetPriorityKeyspaceGroupCommand()) + cmd.AddCommand(newShowKeyspaceGroupPrimaryCommand()) cmd.Flags().String("state", "", "state filter") return cmd } @@ -80,6 +82,7 @@ func newMergeKeyspaceGroupCommand() *cobra.Command { Short: "merge the keyspace group with the given IDs into the target one", Run: mergeKeyspaceGroupCommandFunc, } + r.Flags().Bool("all", false, "merge all keyspace groups into the default one") return r } @@ -111,6 +114,15 @@ func newSetPriorityKeyspaceGroupCommand() *cobra.Command { return r } +func newShowKeyspaceGroupPrimaryCommand() *cobra.Command { + r := &cobra.Command{ + Use: "primary ", + Short: "show th primary of tso nodes for keyspace group with the given ID.", + Run: showKeyspaceGroupPrimaryCommandFunc, + } + return r +} + func showKeyspaceGroupsCommandFunc(cmd *cobra.Command, args []string) { prefix := keyspaceGroupsPrefix if len(args) > 1 { @@ -237,27 +249,52 @@ func finishSplitKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { } func mergeKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { - if len(args) < 2 { - cmd.Usage() - return - } - _, err := strconv.ParseUint(args[0], 10, 32) + var ( + targetGroupID uint32 + params = map[string]interface{}{} + argNum = len(args) + ) + mergeAll, err := cmd.Flags().GetBool("all") if err != nil { - cmd.Printf("Failed to parse the target keyspace group ID: %s\n", err) + cmd.Printf("Failed to get the merge all flag: %s\n", err) return } - groups := make([]uint32, 0, len(args)-1) - for _, arg := range args[1:] { - id, err := strconv.ParseUint(arg, 10, 32) + if argNum == 1 && mergeAll { + target, err := strconv.ParseUint(args[0], 10, 32) if err != nil { - cmd.Printf("Failed to parse the keyspace ID: %s\n", err) + cmd.Printf("Failed to parse the target keyspace group ID: %s\n", err) + return + } + targetGroupID = uint32(target) + if targetGroupID != mcsutils.DefaultKeyspaceGroupID { + cmd.Println("Unable to merge all keyspace groups into a non-default keyspace group") + return + } + params["merge-all-into-default"] = true + } else if argNum >= 2 && !mergeAll { + target, err := strconv.ParseUint(args[0], 10, 32) + if err != nil { + cmd.Printf("Failed to parse the target keyspace group ID: %s\n", err) return } - groups = append(groups, uint32(id)) + targetGroupID = uint32(target) + groups := make([]uint32, 0, len(args)-1) + for _, arg := range args[1:] { + id, err := strconv.ParseUint(arg, 10, 32) + if err != nil { + cmd.Printf("Failed to parse the keyspace ID: %s\n", err) + return + } + groups = append(groups, uint32(id)) + } + params["merge-list"] = groups + } else { + cmd.Println("Must specify the source keyspace group ID(s) or the merge all flag") + cmd.Usage() + return } - postJSON(cmd, fmt.Sprintf("%s/%s/merge", keyspaceGroupsPrefix, args[0]), map[string]interface{}{ - "merge-list": groups, - }) + // TODO: implement the retry mechanism under merge all flag. + postJSON(cmd, fmt.Sprintf("%s/%d/merge", keyspaceGroupsPrefix, targetGroupID), params) } func finishMergeKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { @@ -288,17 +325,17 @@ func setNodesKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { cmd.Printf("Failed to parse the keyspace group ID: %s\n", err) return } - addresses := make([]string, 0, len(args)-1) + nodes := make([]string, 0, len(args)-1) for _, arg := range args[1:] { u, err := url.ParseRequestURI(arg) if u == nil || err != nil { cmd.Printf("Failed to parse the tso node address: %s\n", err) return } - addresses = append(addresses, arg) + nodes = append(nodes, arg) } - postJSON(cmd, fmt.Sprintf("%s/%s/nodes", keyspaceGroupsPrefix, args[0]), map[string]interface{}{ - "Nodes": addresses, + patchJSON(cmd, fmt.Sprintf("%s/%s", keyspaceGroupsPrefix, args[0]), map[string]interface{}{ + "Nodes": nodes, }) } @@ -313,25 +350,48 @@ func setPriorityKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { return } - address := args[1] - u, err := url.ParseRequestURI(address) + node := args[1] + u, err := url.ParseRequestURI(node) if u == nil || err != nil { cmd.Printf("Failed to parse the tso node address: %s\n", err) return } + // Escape the node address to avoid the error of parsing the url + // But the url.PathEscape will escape the '/' to '%2F', which % will cause the error of parsing the url + // So we need to replace the % to \% + node = url.PathEscape(node) + node = strings.ReplaceAll(node, "%", "\\%") + priority, err := strconv.ParseInt(args[2], 10, 32) if err != nil { cmd.Printf("Failed to parse the priority: %s\n", err) return } - postJSON(cmd, fmt.Sprintf("%s/%s/priority", keyspaceGroupsPrefix, args[0]), map[string]interface{}{ - "Node": address, + patchJSON(cmd, fmt.Sprintf("%s/%s/%s", keyspaceGroupsPrefix, args[0], node), map[string]interface{}{ "Priority": priority, }) } +func showKeyspaceGroupPrimaryCommandFunc(cmd *cobra.Command, args []string) { + if len(args) < 1 { + cmd.Usage() + return + } + _, err := strconv.ParseUint(args[0], 10, 32) + if err != nil { + cmd.Printf("Failed to parse the keyspace group ID: %s\n", err) + return + } + r, err := doRequest(cmd, fmt.Sprintf("%s/%s?fields=primary", keyspaceGroupsPrefix, args[0]), http.MethodGet, http.Header{}) + if err != nil { + cmd.Printf("Failed to get the keyspace group primary information: %s\n", err) + return + } + cmd.Println(r) +} + func convertToKeyspaceGroup(content string) string { kg := endpoint.KeyspaceGroup{} err := json.Unmarshal([]byte(content), &kg) diff --git a/tools/pd-ctl/pdctl/ctl.go b/tools/pd-ctl/pdctl/ctl.go index 8fc3a454d7a..86494c046eb 100644 --- a/tools/pd-ctl/pdctl/ctl.go +++ b/tools/pd-ctl/pdctl/ctl.go @@ -66,6 +66,7 @@ func GetRootCmd() *cobra.Command { command.NewCompletionCommand(), command.NewUnsafeCommand(), command.NewKeyspaceGroupCommand(), + command.NewKeyspaceCommand(), ) rootCmd.Flags().ParseErrorsWhitelist.UnknownFlags = true diff --git a/tools/pd-simulator/simulator/client.go b/tools/pd-simulator/simulator/client.go index e71ead920eb..aeffd309333 100644 --- a/tools/pd-simulator/simulator/client.go +++ b/tools/pd-simulator/simulator/client.go @@ -149,21 +149,22 @@ func (c *client) createHeartbeatStream() (pdpb.PD_RegionHeartbeatClient, context cancel context.CancelFunc ctx context.Context ) + ticker := time.NewTicker(time.Second) + defer ticker.Stop() for { ctx, cancel = context.WithCancel(c.ctx) stream, err = c.pdClient().RegionHeartbeat(ctx) - if err != nil { - simutil.Logger.Error("create region heartbeat stream error", zap.String("tag", c.tag), zap.Error(err)) - cancel() - select { - case <-time.After(time.Second): - continue - case <-c.ctx.Done(): - simutil.Logger.Info("cancel create stream loop") - return nil, ctx, cancel - } + if err == nil { + break + } + simutil.Logger.Error("create region heartbeat stream error", zap.String("tag", c.tag), zap.Error(err)) + cancel() + select { + case <-c.ctx.Done(): + simutil.Logger.Info("cancel create stream loop") + return nil, ctx, cancel + case <-ticker.C: } - break } return stream, ctx, cancel } diff --git a/tools/pd-tso-bench/main.go b/tools/pd-tso-bench/main.go index e82a9dcabba..236e78c7808 100644 --- a/tools/pd-tso-bench/main.go +++ b/tools/pd-tso-bench/main.go @@ -19,6 +19,7 @@ import ( "flag" "fmt" "io" + "math/rand" "net/http" "net/http/httptest" "os" @@ -43,20 +44,25 @@ const ( ) var ( - pdAddrs = flag.String("pd", "127.0.0.1:2379", "pd address") - clientNumber = flag.Int("client", 1, "the number of pd clients involved in each benchmark") - concurrency = flag.Int("c", 1000, "concurrency") - count = flag.Int("count", 1, "the count number that the test will run") - duration = flag.Duration("duration", 60*time.Second, "how many seconds the test will last") - dcLocation = flag.String("dc", "global", "which dc-location this bench will request") - verbose = flag.Bool("v", false, "output statistics info every interval and output metrics info at the end") - interval = flag.Duration("interval", time.Second, "interval to output the statistics") - caPath = flag.String("cacert", "", "path of file that contains list of trusted SSL CAs") - certPath = flag.String("cert", "", "path of file that contains X509 certificate in PEM format") - keyPath = flag.String("key", "", "path of file that contains X509 key in PEM format") - maxBatchWaitInterval = flag.Duration("batch-interval", 0, "the max batch wait interval") - enableTSOFollowerProxy = flag.Bool("enable-tso-follower-proxy", false, "whether enable the TSO Follower Proxy") - wg sync.WaitGroup + pdAddrs = flag.String("pd", "127.0.0.1:2379", "pd address") + clientNumber = flag.Int("client", 1, "the number of pd clients involved in each benchmark") + concurrency = flag.Int("c", 1000, "concurrency") + count = flag.Int("count", 1, "the count number that the test will run") + duration = flag.Duration("duration", 60*time.Second, "how many seconds the test will last") + dcLocation = flag.String("dc", "global", "which dc-location this bench will request") + verbose = flag.Bool("v", false, "output statistics info every interval and output metrics info at the end") + interval = flag.Duration("interval", time.Second, "interval to output the statistics") + caPath = flag.String("cacert", "", "path of file that contains list of trusted SSL CAs") + certPath = flag.String("cert", "", "path of file that contains X509 certificate in PEM format") + keyPath = flag.String("key", "", "path of file that contains X509 key in PEM format") + maxBatchWaitInterval = flag.Duration("batch-interval", 0, "the max batch wait interval") + enableTSOFollowerProxy = flag.Bool("enable-tso-follower-proxy", false, "whether enable the TSO Follower Proxy") + enableFaultInjection = flag.Bool("enable-fault-injection", false, "whether enable fault injection") + faultInjectionRate = flag.Float64("fault-injection-rate", 0.01, "the failure rate [0.0001, 1]. 0.01 means 1% failure rate") + maxTSOSendIntervalMilliseconds = flag.Int("max-send-interval-ms", 0, "max tso send interval in milliseconds, 60s by default") + keyspaceID = flag.Uint("keyspace-id", 0, "the id of the keyspace to access") + keyspaceName = flag.String("keyspace-name", "", "the name of the keyspace to access") + wg sync.WaitGroup ) var promServer *httptest.Server @@ -97,26 +103,7 @@ func bench(mainCtx context.Context) { fmt.Printf("Create %d client(s) for benchmark\n", *clientNumber) pdClients := make([]pd.Client, *clientNumber) for idx := range pdClients { - var ( - pdCli pd.Client - err error - ) - - opt := pd.WithGRPCDialOptions( - grpc.WithKeepaliveParams(keepalive.ClientParameters{ - Time: keepaliveTime, - Timeout: keepaliveTimeout, - }), - ) - - pdCli, err = pd.NewClientWithContext(mainCtx, []string{*pdAddrs}, pd.SecurityOption{ - CAPath: *caPath, - CertPath: *certPath, - KeyPath: *keyPath, - }, opt) - - pdCli.UpdateOption(pd.MaxTSOBatchWaitInterval, *maxBatchWaitInterval) - pdCli.UpdateOption(pd.EnableTSOFollowerProxy, *enableTSOFollowerProxy) + pdCli, err := createPDClient(mainCtx) if err != nil { log.Fatal(fmt.Sprintf("create pd client #%d failed: %v", idx, err)) } @@ -134,10 +121,18 @@ func bench(mainCtx context.Context) { durCh := make(chan time.Duration, 2*(*concurrency)*(*clientNumber)) - wg.Add((*concurrency) * (*clientNumber)) - for _, pdCli := range pdClients { - for i := 0; i < *concurrency; i++ { - go reqWorker(ctx, pdCli, durCh) + if *enableFaultInjection { + fmt.Printf("Enable fault injection, failure rate: %f\n", *faultInjectionRate) + wg.Add(*clientNumber) + for i := 0; i < *clientNumber; i++ { + go reqWorker(ctx, pdClients, i, durCh) + } + } else { + wg.Add((*concurrency) * (*clientNumber)) + for i := 0; i < *clientNumber; i++ { + for j := 0; j < *concurrency; j++ { + go reqWorker(ctx, pdClients, i, durCh) + } } } @@ -352,22 +347,51 @@ func (s *stats) calculate(count int) float64 { return float64(count) * 100 / float64(s.count) } -func reqWorker(ctx context.Context, pdCli pd.Client, durCh chan time.Duration) { +func reqWorker(ctx context.Context, pdClients []pd.Client, clientIdx int, durCh chan time.Duration) { defer wg.Done() reqCtx, cancel := context.WithCancel(ctx) defer cancel() + var ( + err error + maxRetryTime int = 120 + sleepIntervalOnFailure time.Duration = 1000 * time.Millisecond + totalSleepBeforeGetTS time.Duration + ) + pdCli := pdClients[clientIdx] for { + if pdCli == nil || (*enableFaultInjection && shouldInjectFault()) { + if pdCli != nil { + pdCli.Close() + } + pdCli, err = createPDClient(ctx) + if err != nil { + log.Error(fmt.Sprintf("re-create pd client #%d failed: %v", clientIdx, err)) + select { + case <-reqCtx.Done(): + case <-time.After(100 * time.Millisecond): + } + continue + } + pdClients[clientIdx] = pdCli + } + + totalSleepBeforeGetTS = 0 start := time.Now() - var ( - i int32 - err error - maxRetryTime int32 = 50 - sleepIntervalOnFailure time.Duration = 100 * time.Millisecond - ) + i := 0 for ; i < maxRetryTime; i++ { + if *maxTSOSendIntervalMilliseconds > 0 { + sleepBeforeGetTS := time.Duration(rand.Intn(*maxTSOSendIntervalMilliseconds)) * time.Millisecond + ticker := time.NewTicker(sleepBeforeGetTS) + defer ticker.Stop() + select { + case <-reqCtx.Done(): + case <-ticker.C: + totalSleepBeforeGetTS += sleepBeforeGetTS + } + } _, _, err = pdCli.GetLocalTS(reqCtx, *dcLocation) if errors.Cause(err) == context.Canceled { return @@ -381,7 +405,7 @@ func reqWorker(ctx context.Context, pdCli pd.Client, durCh chan time.Duration) { if err != nil { log.Fatal(fmt.Sprintf("%v", err)) } - dur := time.Since(start) - time.Duration(i)*sleepIntervalOnFailure + dur := time.Since(start) - time.Duration(i)*sleepIntervalOnFailure - totalSleepBeforeGetTS select { case <-reqCtx.Done(): @@ -390,3 +414,44 @@ func reqWorker(ctx context.Context, pdCli pd.Client, durCh chan time.Duration) { } } } + +func createPDClient(ctx context.Context) (pd.Client, error) { + var ( + pdCli pd.Client + err error + ) + + opts := make([]pd.ClientOption, 0) + opts = append(opts, pd.WithGRPCDialOptions( + grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: keepaliveTime, + Timeout: keepaliveTimeout, + }), + )) + + if len(*keyspaceName) > 0 { + apiCtx := pd.NewAPIContextV2(*keyspaceName) + pdCli, err = pd.NewClientWithAPIContext(ctx, apiCtx, []string{*pdAddrs}, pd.SecurityOption{ + CAPath: *caPath, + CertPath: *certPath, + KeyPath: *keyPath, + }, opts...) + } else { + pdCli, err = pd.NewClientWithKeyspace(ctx, uint32(*keyspaceID), []string{*pdAddrs}, pd.SecurityOption{ + CAPath: *caPath, + CertPath: *certPath, + KeyPath: *keyPath, + }, opts...) + } + if err != nil { + return nil, err + } + + pdCli.UpdateOption(pd.MaxTSOBatchWaitInterval, *maxBatchWaitInterval) + pdCli.UpdateOption(pd.EnableTSOFollowerProxy, *enableTSOFollowerProxy) + return pdCli, err +} + +func shouldInjectFault() bool { + return rand.Intn(10000) < int(*faultInjectionRate*10000) +}