diff --git a/client/grpcutil/grpcutil.go b/client/grpcutil/grpcutil.go index 59b7224f29e..125f1125721 100644 --- a/client/grpcutil/grpcutil.go +++ b/client/grpcutil/grpcutil.go @@ -78,6 +78,7 @@ func BuildForwardContext(ctx context.Context, addr string) context.Context { func GetOrCreateGRPCConn(ctx context.Context, clientConns *sync.Map, addr string, tlsCfg *tlsutil.TLSConfig, opt ...grpc.DialOption) (*grpc.ClientConn, error) { conn, ok := clientConns.Load(addr) if ok { + // TODO: check the connection state. return conn.(*grpc.ClientConn), nil } tlsConfig, err := tlsCfg.ToTLSConfig() diff --git a/client/tso_client.go b/client/tso_client.go index d4dfaa03a91..35d9388c72b 100644 --- a/client/tso_client.go +++ b/client/tso_client.go @@ -83,8 +83,8 @@ type tsoClient struct { tsoDispatcher sync.Map // Same as map[string]chan *tsoRequest // dc-location -> deadline tsDeadline sync.Map // Same as map[string]chan deadline - // dc-location -> *lastTSO - lastTSMap sync.Map // Same as map[string]*lastTSO + // dc-location -> *tsoInfo while the tsoInfo is the last TSO info + lastTSOInfoMap sync.Map // Same as map[string]*tsoInfo checkTSDeadlineCh chan struct{} checkTSODispatcherCh chan struct{} @@ -209,7 +209,7 @@ func (c *tsoClient) updateTSOLocalServAddrs(allocatorMap map[string]string) erro return err } c.tsoAllocators.Store(dcLocation, addr) - log.Info("[tso] switch dc tso allocator serving address", + log.Info("[tso] switch dc tso local allocator serving address", zap.String("dc-location", dcLocation), zap.String("new-address", addr), zap.String("old-address", oldAddr)) @@ -227,7 +227,7 @@ func (c *tsoClient) updateTSOLocalServAddrs(allocatorMap map[string]string) erro func (c *tsoClient) updateTSOGlobalServAddr(addr string) error { c.tsoAllocators.Store(globalDCLocation, addr) - log.Info("[tso] switch dc tso allocator serving address", + log.Info("[tso] switch dc tso global allocator serving address", zap.String("dc-location", globalDCLocation), zap.String("new-address", addr)) c.scheduleCheckTSODispatcher() diff --git a/client/tso_dispatcher.go b/client/tso_dispatcher.go index 30e7e670e09..37bea8db9e5 100644 --- a/client/tso_dispatcher.go +++ b/client/tso_dispatcher.go @@ -40,9 +40,13 @@ type tsoDispatcher struct { tsoBatchController *tsoBatchController } -type lastTSO struct { - physical int64 - logical int64 +type tsoInfo struct { + tsoServer string + reqKeyspaceGroupID uint32 + respKeyspaceGroupID uint32 + respReceivedAt time.Time + physical int64 + logical int64 } const ( @@ -708,8 +712,9 @@ func (c *tsoClient) processRequests( requests := tbc.getCollectedRequests() count := int64(len(requests)) - physical, logical, suffixBits, err := stream.processRequests( - c.svcDiscovery.GetClusterID(), c.svcDiscovery.GetKeyspaceID(), c.svcDiscovery.GetKeyspaceGroupID(), + reqKeyspaceGroupID := c.svcDiscovery.GetKeyspaceGroupID() + respKeyspaceGroupID, physical, logical, suffixBits, err := stream.processRequests( + c.svcDiscovery.GetClusterID(), c.svcDiscovery.GetKeyspaceID(), reqKeyspaceGroupID, dcLocation, requests, tbc.batchStartTime) if err != nil { c.finishRequest(requests, 0, 0, 0, err) @@ -717,36 +722,62 @@ func (c *tsoClient) processRequests( } // `logical` is the largest ts's logical part here, we need to do the subtracting before we finish each TSO request. firstLogical := tsoutil.AddLogical(logical, -count+1, suffixBits) - c.compareAndSwapTS(dcLocation, physical, firstLogical, suffixBits, count) + curTSOInfo := &tsoInfo{ + tsoServer: stream.getServerAddr(), + reqKeyspaceGroupID: reqKeyspaceGroupID, + respKeyspaceGroupID: respKeyspaceGroupID, + respReceivedAt: time.Now(), + physical: physical, + logical: tsoutil.AddLogical(firstLogical, count-1, suffixBits), + } + c.compareAndSwapTS(dcLocation, curTSOInfo, physical, firstLogical) c.finishRequest(requests, physical, firstLogical, suffixBits, nil) return nil } -func (c *tsoClient) compareAndSwapTS(dcLocation string, physical, firstLogical int64, suffixBits uint32, count int64) { - largestLogical := tsoutil.AddLogical(firstLogical, count-1, suffixBits) - lastTSOInterface, loaded := c.lastTSMap.LoadOrStore(dcLocation, &lastTSO{ - physical: physical, - // Save the largest logical part here - logical: largestLogical, - }) +func (c *tsoClient) compareAndSwapTS( + dcLocation string, + curTSOInfo *tsoInfo, + physical, firstLogical int64, +) { + val, loaded := c.lastTSOInfoMap.LoadOrStore(dcLocation, curTSOInfo) if !loaded { return } - lastTSOPointer := lastTSOInterface.(*lastTSO) - lastPhysical := lastTSOPointer.physical - lastLogical := lastTSOPointer.logical + lastTSOInfo := val.(*tsoInfo) + if lastTSOInfo.respKeyspaceGroupID != curTSOInfo.respKeyspaceGroupID { + log.Info("[tso] keyspace group changed", + zap.String("dc-location", dcLocation), + zap.Uint32("old-group-id", lastTSOInfo.respKeyspaceGroupID), + zap.Uint32("new-group-id", curTSOInfo.respKeyspaceGroupID)) + } + // The TSO we get is a range like [largestLogical-count+1, largestLogical], so we save the last TSO's largest logical // to compare with the new TSO's first logical. For example, if we have a TSO resp with logical 10, count 5, then - // all TSOs we get will be [6, 7, 8, 9, 10]. - if tsoutil.TSLessEqual(physical, firstLogical, lastPhysical, lastLogical) { - panic(errors.Errorf( - "%s timestamp fallback, new ts (%d, %d) <= the last one (%d, %d). keyspace: %d, keyspace group: %d", - dcLocation, physical, firstLogical, lastPhysical, lastLogical, - c.svcDiscovery.GetKeyspaceID(), c.svcDiscovery.GetKeyspaceGroupID())) + // 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", + 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), + ) } - lastTSOPointer.physical = physical - // Same as above, we save the largest logical part here. - lastTSOPointer.logical = largestLogical + lastTSOInfo.tsoServer = curTSOInfo.tsoServer + lastTSOInfo.reqKeyspaceGroupID = curTSOInfo.reqKeyspaceGroupID + lastTSOInfo.respKeyspaceGroupID = curTSOInfo.respKeyspaceGroupID + lastTSOInfo.respReceivedAt = curTSOInfo.respReceivedAt + lastTSOInfo.physical = curTSOInfo.physical + lastTSOInfo.logical = curTSOInfo.logical } func (c *tsoClient) finishRequest(requests []*tsoRequest, physical, firstLogical int64, suffixBits uint32, err error) { diff --git a/client/tso_service_discovery.go b/client/tso_service_discovery.go index e3c67a92fa6..cee079634e9 100644 --- a/client/tso_service_discovery.go +++ b/client/tso_service_discovery.go @@ -20,6 +20,7 @@ import ( "reflect" "strings" "sync" + "sync/atomic" "time" "github.com/gogo/protobuf/proto" @@ -121,7 +122,7 @@ type tsoServiceDiscovery struct { metacli MetaStorageClient apiSvcDiscovery ServiceDiscovery clusterID uint64 - keyspaceID uint32 + keyspaceID atomic.Uint32 // defaultDiscoveryKey is the etcd path used for discovering the serving endpoints of // the default keyspace group @@ -165,12 +166,12 @@ func newTSOServiceDiscovery( cancel: cancel, metacli: metacli, apiSvcDiscovery: apiSvcDiscovery, - keyspaceID: keyspaceID, clusterID: clusterID, tlsCfg: tlsCfg, option: option, checkMembershipCh: make(chan struct{}, 1), } + c.keyspaceID.Store(keyspaceID) c.keyspaceGroupSD = &keyspaceGroupSvcDiscovery{ primaryAddr: "", secondaryAddrs: make([]string, 0), @@ -269,12 +270,12 @@ func (c *tsoServiceDiscovery) GetClusterID() uint64 { // GetKeyspaceID returns the ID of the keyspace func (c *tsoServiceDiscovery) GetKeyspaceID() uint32 { - return c.keyspaceID + return c.keyspaceID.Load() } // SetKeyspaceID sets the ID of the keyspace func (c *tsoServiceDiscovery) SetKeyspaceID(keyspaceID uint32) { - c.keyspaceID = keyspaceID + c.keyspaceID.Store(keyspaceID) } // GetKeyspaceGroupID returns the ID of the keyspace group. If the keyspace group is unknown, @@ -426,7 +427,7 @@ func (c *tsoServiceDiscovery) updateMember() error { var keyspaceGroup *tsopb.KeyspaceGroup if len(tsoServerAddr) > 0 { - keyspaceGroup, err = c.findGroupByKeyspaceID(c.keyspaceID, tsoServerAddr, updateMemberTimeout) + keyspaceGroup, err = c.findGroupByKeyspaceID(c.GetKeyspaceID(), tsoServerAddr, updateMemberTimeout) if err != nil { if c.tsoServerDiscovery.countFailure() { log.Error("[tso] failed to find the keyspace group", errs.ZapError(err)) diff --git a/client/tso_stream.go b/client/tso_stream.go index 5b658279cac..892512d8559 100644 --- a/client/tso_stream.go +++ b/client/tso_stream.go @@ -35,13 +35,13 @@ type tsoStreamBuilderFactory interface { type pdTSOStreamBuilderFactory struct{} func (f *pdTSOStreamBuilderFactory) makeBuilder(cc *grpc.ClientConn) tsoStreamBuilder { - return &pdTSOStreamBuilder{client: pdpb.NewPDClient(cc)} + return &pdTSOStreamBuilder{client: pdpb.NewPDClient(cc), serverAddr: cc.Target()} } type tsoTSOStreamBuilderFactory struct{} func (f *tsoTSOStreamBuilderFactory) makeBuilder(cc *grpc.ClientConn) tsoStreamBuilder { - return &tsoTSOStreamBuilder{client: tsopb.NewTSOClient(cc)} + return &tsoTSOStreamBuilder{client: tsopb.NewTSOClient(cc), serverAddr: cc.Target()} } // TSO Stream Builder @@ -51,7 +51,8 @@ type tsoStreamBuilder interface { } type pdTSOStreamBuilder struct { - client pdpb.PDClient + serverAddr string + client pdpb.PDClient } func (b *pdTSOStreamBuilder) build(ctx context.Context, cancel context.CancelFunc, timeout time.Duration) (tsoStream, error) { @@ -61,13 +62,14 @@ func (b *pdTSOStreamBuilder) build(ctx context.Context, cancel context.CancelFun stream, err := b.client.Tso(ctx) done <- struct{}{} if err == nil { - return &pdTSOStream{stream: stream}, nil + return &pdTSOStream{stream: stream, serverAddr: b.serverAddr}, nil } return nil, err } type tsoTSOStreamBuilder struct { - client tsopb.TSOClient + serverAddr string + client tsopb.TSOClient } func (b *tsoTSOStreamBuilder) build( @@ -79,7 +81,7 @@ func (b *tsoTSOStreamBuilder) build( stream, err := b.client.Tso(ctx) done <- struct{}{} if err == nil { - return &tsoTSOStream{stream: stream}, nil + return &tsoTSOStream{stream: stream, serverAddr: b.serverAddr}, nil } return nil, err } @@ -98,20 +100,26 @@ func checkStreamTimeout(ctx context.Context, cancel context.CancelFunc, done cha // TSO Stream type tsoStream interface { + getServerAddr() string // processRequests processes TSO requests in streaming mode to get timestamps processRequests( clusterID uint64, keyspaceID, keyspaceGroupID uint32, dcLocation string, requests []*tsoRequest, batchStartTime time.Time, - ) (physical, logical int64, suffixBits uint32, err error) + ) (respKeyspaceGroupID uint32, physical, logical int64, suffixBits uint32, err error) } type pdTSOStream struct { - stream pdpb.PD_TsoClient + serverAddr string + stream pdpb.PD_TsoClient +} + +func (s *pdTSOStream) getServerAddr() string { + return s.serverAddr } func (s *pdTSOStream) processRequests( clusterID uint64, _, _ uint32, dcLocation string, requests []*tsoRequest, batchStartTime time.Time, -) (physical, logical int64, suffixBits uint32, err error) { +) (respKeyspaceGroupID uint32, physical, logical int64, suffixBits uint32, err error) { start := time.Now() count := int64(len(requests)) req := &pdpb.TsoRequest{ @@ -149,18 +157,24 @@ func (s *pdTSOStream) processRequests( } ts := resp.GetTimestamp() + respKeyspaceGroupID = defaultKeySpaceGroupID physical, logical, suffixBits = ts.GetPhysical(), ts.GetLogical(), ts.GetSuffixBits() return } type tsoTSOStream struct { - stream tsopb.TSO_TsoClient + serverAddr string + stream tsopb.TSO_TsoClient +} + +func (s *tsoTSOStream) getServerAddr() string { + return s.serverAddr } func (s *tsoTSOStream) processRequests( clusterID uint64, keyspaceID, keyspaceGroupID uint32, dcLocation string, requests []*tsoRequest, batchStartTime time.Time, -) (physical, logical int64, suffixBits uint32, err error) { +) (respKeyspaceGroupID uint32, physical, logical int64, suffixBits uint32, err error) { start := time.Now() count := int64(len(requests)) req := &tsopb.TsoRequest{ @@ -200,6 +214,7 @@ func (s *tsoTSOStream) processRequests( } ts := resp.GetTimestamp() + respKeyspaceGroupID = resp.GetHeader().GetKeyspaceGroupId() physical, logical, suffixBits = ts.GetPhysical(), ts.GetLogical(), ts.GetSuffixBits() return } diff --git a/codecov.yml b/codecov.yml index dc58a648479..44a3aac24b7 100644 --- a/codecov.yml +++ b/codecov.yml @@ -10,3 +10,20 @@ coverage: # basic target: auto threshold: 3% + +comment: + layout: "header, diff, flags" + behavior: default + require_changes: false + +flag_management: + default_rules: # the rules that will be followed for any flag added, generally + carryforward: true + statuses: + - type: project + target: 85% + - type: patch + target: 85% + +ignore: + - tests/** # integration test cases or tools. diff --git a/errors.toml b/errors.toml index c19b67d8b2f..ed3cd32d52a 100644 --- a/errors.toml +++ b/errors.toml @@ -766,6 +766,11 @@ error = ''' the keyspace group id is invalid, %s ''' +["PD:tso:ErrKeyspaceGroupIsMerging"] +error = ''' +the keyspace group %d is merging +''' + ["PD:tso:ErrKeyspaceGroupNotInitialized"] error = ''' the keyspace group %d isn't initialized diff --git a/go.mod b/go.mod index 5cc8b55950c..0dccd506aad 100644 --- a/go.mod +++ b/go.mod @@ -33,7 +33,7 @@ require ( github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 - github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5 + github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 github.com/prometheus/client_golang v1.11.1 github.com/prometheus/common v0.26.0 github.com/sasha-s/go-deadlock v0.2.0 diff --git a/go.sum b/go.sum index ac034473b3b..2962b6ca897 100644 --- a/go.sum +++ b/go.sum @@ -431,8 +431,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5 h1:adV4kUWI7v+v/6joR7lfjFngHhS4eiqwr4g3dHCjHtA= -github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= +github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 h1:/jqjj+ydlqjp144LAlHDfHtr7eWJyaNIIXX5viv0RZo= +github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index 2db8638a12a..0fda57fa7c9 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -55,6 +55,7 @@ var ( ErrKeyspaceGroupNotInitialized = errors.Normalize("the keyspace group %d isn't initialized", errors.RFCCodeText("PD:tso:ErrKeyspaceGroupNotInitialized")) ErrKeyspaceNotAssigned = errors.Normalize("the keyspace %d isn't assigned to any keyspace group", errors.RFCCodeText("PD:tso:ErrKeyspaceNotAssigned")) ErrGetMinTS = errors.Normalize("get min ts failed, %s", errors.RFCCodeText("PD:tso:ErrGetMinTS")) + ErrKeyspaceGroupIsMerging = errors.Normalize("the keyspace group %d is merging", errors.RFCCodeText("PD:tso:ErrKeyspaceGroupIsMerging")) ) // member errors diff --git a/pkg/keyspace/keyspace.go b/pkg/keyspace/keyspace.go index 5c5ffc3ffd0..df7eb653828 100644 --- a/pkg/keyspace/keyspace.go +++ b/pkg/keyspace/keyspace.go @@ -511,9 +511,9 @@ func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.Key // Changing the state of default keyspace is not allowed. if name == utils.DefaultKeyspaceName { log.Warn("[keyspace] failed to update keyspace config", - zap.Error(errModifyDefault), + zap.Error(ErrModifyDefaultKeyspace), ) - return nil, errModifyDefault + return nil, ErrModifyDefaultKeyspace } var meta *keyspacepb.KeyspaceMeta err := manager.store.RunInTxn(manager.ctx, func(txn kv.Txn) error { @@ -563,9 +563,9 @@ func (manager *Manager) UpdateKeyspaceStateByID(id uint32, newState keyspacepb.K // Changing the state of default keyspace is not allowed. if id == utils.DefaultKeyspaceID { log.Warn("[keyspace] failed to update keyspace config", - zap.Error(errModifyDefault), + zap.Error(ErrModifyDefaultKeyspace), ) - return nil, errModifyDefault + return nil, ErrModifyDefaultKeyspace } var meta *keyspacepb.KeyspaceMeta var err error @@ -652,7 +652,16 @@ func (manager *Manager) allocID() (uint32, error) { } // PatrolKeyspaceAssignment is used to patrol all keyspaces and assign them to the keyspace groups. -func (manager *Manager) PatrolKeyspaceAssignment() error { +func (manager *Manager) PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID uint32) error { + if startKeyspaceID > manager.nextPatrolStartID { + manager.nextPatrolStartID = startKeyspaceID + } + if endKeyspaceID != 0 && endKeyspaceID < manager.nextPatrolStartID { + log.Info("[keyspace] end keyspace id is smaller than the next patrol start id, skip patrol", + zap.Uint32("end-keyspace-id", endKeyspaceID), + zap.Uint32("next-patrol-start-id", manager.nextPatrolStartID)) + return nil + } var ( // Some statistics info. start = time.Now() @@ -671,6 +680,8 @@ func (manager *Manager) PatrolKeyspaceAssignment() error { zap.Uint64("patrolled-keyspace-count", patrolledKeyspaceCount), zap.Uint64("assigned-keyspace-count", assignedKeyspaceCount), zap.Int("batch-size", maxEtcdTxnOps), + zap.Uint32("start-keyspace-id", startKeyspaceID), + zap.Uint32("end-keyspace-id", endKeyspaceID), zap.Uint32("current-start-id", currentStartID), zap.Uint32("next-start-id", nextStartID), ) @@ -687,10 +698,10 @@ func (manager *Manager) PatrolKeyspaceAssignment() error { return errors.Errorf("default keyspace group %d not found", utils.DefaultKeyspaceGroupID) } if defaultKeyspaceGroup.IsSplitting() { - return ErrKeyspaceGroupInSplit + return ErrKeyspaceGroupInSplit(utils.DefaultKeyspaceGroupID) } if defaultKeyspaceGroup.IsMerging() { - return ErrKeyspaceGroupInMerging + return ErrKeyspaceGroupInMerging(utils.DefaultKeyspaceGroupID) } keyspaces, err := manager.store.LoadRangeKeyspace(txn, manager.nextPatrolStartID, maxEtcdTxnOps) if err != nil { @@ -702,8 +713,8 @@ func (manager *Manager) PatrolKeyspaceAssignment() error { currentStartID = keyspaces[0].GetId() nextStartID = keyspaces[keyspaceNum-1].GetId() + 1 } - // If there are less than `maxEtcdTxnOps` keyspaces, - // we have reached the end of the keyspace list. + // 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 var ( assigned = false @@ -718,6 +729,10 @@ func (manager *Manager) PatrolKeyspaceAssignment() error { if ks == nil { continue } + if endKeyspaceID != 0 && ks.Id > endKeyspaceID { + moreToPatrol = false + break + } patrolledKeyspaceCount++ manager.metaLock.Lock(ks.Id) if ks.Config == nil { @@ -740,6 +755,8 @@ func (manager *Manager) PatrolKeyspaceAssignment() error { if err != nil { log.Error("[keyspace] failed to save keyspace meta during patrol", zap.Int("batch-size", maxEtcdTxnOps), + zap.Uint32("start-keyspace-id", startKeyspaceID), + zap.Uint32("end-keyspace-id", endKeyspaceID), zap.Uint32("current-start-id", currentStartID), zap.Uint32("next-start-id", nextStartID), zap.Uint32("keyspace-id", ks.Id), zap.Error(err)) @@ -752,6 +769,8 @@ func (manager *Manager) PatrolKeyspaceAssignment() error { if err != nil { log.Error("[keyspace] failed to save default keyspace group meta during patrol", zap.Int("batch-size", maxEtcdTxnOps), + zap.Uint32("start-keyspace-id", startKeyspaceID), + zap.Uint32("end-keyspace-id", endKeyspaceID), zap.Uint32("current-start-id", currentStartID), zap.Uint32("next-start-id", nextStartID), zap.Error(err)) return err diff --git a/pkg/keyspace/keyspace_test.go b/pkg/keyspace/keyspace_test.go index 45c4bc90be2..b06921e48db 100644 --- a/pkg/keyspace/keyspace_test.go +++ b/pkg/keyspace/keyspace_test.go @@ -393,7 +393,7 @@ func (suite *keyspaceTestSuite) TestPatrolKeyspaceAssignment() { re.NotNil(defaultKeyspaceGroup) re.NotContains(defaultKeyspaceGroup.Keyspaces, uint32(111)) // Patrol the keyspace assignment. - err = suite.manager.PatrolKeyspaceAssignment() + err = suite.manager.PatrolKeyspaceAssignment(0, 0) re.NoError(err) // Check if the keyspace is attached to the default group. defaultKeyspaceGroup, err = suite.manager.kgm.GetKeyspaceGroupByID(utils.DefaultKeyspaceGroupID) @@ -424,7 +424,7 @@ func (suite *keyspaceTestSuite) TestPatrolKeyspaceAssignmentInBatch() { re.NotContains(defaultKeyspaceGroup.Keyspaces, uint32(i)) } // Patrol the keyspace assignment. - err = suite.manager.PatrolKeyspaceAssignment() + err = suite.manager.PatrolKeyspaceAssignment(0, 0) re.NoError(err) // Check if all the keyspaces are attached to the default group. defaultKeyspaceGroup, err = suite.manager.kgm.GetKeyspaceGroupByID(utils.DefaultKeyspaceGroupID) @@ -435,6 +435,49 @@ 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++ { + now := time.Now().Unix() + err := suite.manager.saveNewKeyspace(&keyspacepb.KeyspaceMeta{ + Id: uint32(i), + Name: strconv.Itoa(i), + State: keyspacepb.KeyspaceState_ENABLED, + CreatedAt: now, + StateChangedAt: now, + }) + re.NoError(err) + } + // Check if all the keyspaces are not attached to the default group. + defaultKeyspaceGroup, err := suite.manager.kgm.GetKeyspaceGroupByID(utils.DefaultKeyspaceGroupID) + re.NoError(err) + re.NotNil(defaultKeyspaceGroup) + 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] + // to make sure the range crossing the boundary of etcd transaction operation limit. + var ( + startKeyspaceID = uint32(maxEtcdTxnOps / 2) + endKeyspaceID = startKeyspaceID + maxEtcdTxnOps + 1 + ) + err = suite.manager.PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID) + re.NoError(err) + // Check if only the keyspaces within the range are attached to the default group. + defaultKeyspaceGroup, err = suite.manager.kgm.GetKeyspaceGroupByID(utils.DefaultKeyspaceGroupID) + re.NoError(err) + re.NotNil(defaultKeyspaceGroup) + for i := 1; i < maxEtcdTxnOps*2+1; i++ { + keyspaceID := uint32(i) + if keyspaceID >= startKeyspaceID && keyspaceID <= endKeyspaceID { + re.Contains(defaultKeyspaceGroup.Keyspaces, keyspaceID) + } else { + re.NotContains(defaultKeyspaceGroup.Keyspaces, keyspaceID) + } + } +} + // Benchmark the keyspace assignment patrol. func BenchmarkPatrolKeyspaceAssignment1000(b *testing.B) { benchmarkPatrolKeyspaceAssignmentN(1000, b) @@ -471,7 +514,7 @@ func benchmarkPatrolKeyspaceAssignmentN( // Benchmark the keyspace assignment patrol. b.ResetTimer() for i := 0; i < b.N; i++ { - err := suite.manager.PatrolKeyspaceAssignment() + err := suite.manager.PatrolKeyspaceAssignment(0, 0) re.NoError(err) } b.StopTimer() diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index c5e78d6b797..dd9319e806f 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -310,7 +310,7 @@ func (m *GroupManager) DeleteKeyspaceGroupByID(id uint32) (*endpoint.KeyspaceGro return nil } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit + return ErrKeyspaceGroupInSplit(id) } return m.store.DeleteKeyspaceGroup(txn, id) }); err != nil { @@ -339,10 +339,10 @@ func (m *GroupManager) saveKeyspaceGroups(keyspaceGroups []*endpoint.KeyspaceGro return ErrKeyspaceGroupExists } if oldKG.IsSplitting() && overwrite { - return ErrKeyspaceGroupInSplit + return ErrKeyspaceGroupInSplit(keyspaceGroup.ID) } if oldKG.IsMerging() && overwrite { - return ErrKeyspaceGroupInMerging + return ErrKeyspaceGroupInMerging(keyspaceGroup.ID) } newKG := &endpoint.KeyspaceGroup{ ID: keyspaceGroup.ID, @@ -414,13 +414,13 @@ func (m *GroupManager) UpdateKeyspaceForGroup(userKind endpoint.UserKind, groupI func (m *GroupManager) updateKeyspaceForGroupLocked(userKind endpoint.UserKind, groupID uint64, keyspaceID uint32, mutation int) error { kg := m.groups[userKind].Get(uint32(groupID)) if kg == nil { - return errors.Errorf("keyspace group %d not found", groupID) + return ErrKeyspaceGroupNotExists(uint32(groupID)) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit + return ErrKeyspaceGroupInSplit(uint32(groupID)) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging + return ErrKeyspaceGroupInMerging(uint32(groupID)) } changed := false @@ -473,11 +473,14 @@ func (m *GroupManager) UpdateKeyspaceGroup(oldGroupID, newGroupID string, oldUse if newKG == nil { return errors.Errorf("keyspace group %s not found in %s group", newGroupID, newUserKind) } - if oldKG.IsSplitting() || newKG.IsSplitting() { - return ErrKeyspaceGroupInSplit - } - if oldKG.IsMerging() || newKG.IsMerging() { - return ErrKeyspaceGroupInMerging + if oldKG.IsSplitting() { + return ErrKeyspaceGroupInSplit(uint32(oldID)) + } else if newKG.IsSplitting() { + return ErrKeyspaceGroupInSplit(uint32(newID)) + } else if oldKG.IsMerging() { + return ErrKeyspaceGroupInMerging(uint32(oldID)) + } else if newKG.IsMerging() { + return ErrKeyspaceGroupInMerging(uint32(newID)) } var updateOld, updateNew bool @@ -509,7 +512,10 @@ func (m *GroupManager) UpdateKeyspaceGroup(oldGroupID, newGroupID string, oldUse // SplitKeyspaceGroupByID splits the keyspace group by ID into a new keyspace group with the given new ID. // And the keyspaces in the old keyspace group will be moved to the new keyspace group. -func (m *GroupManager) SplitKeyspaceGroupByID(splitSourceID, splitTargetID uint32, keyspaces []uint32) error { +func (m *GroupManager) SplitKeyspaceGroupByID( + splitSourceID, splitTargetID uint32, + keyspaces []uint32, keyspaceIDRange ...uint32, +) error { var splitSourceKg, splitTargetKg *endpoint.KeyspaceGroup m.Lock() defer m.Unlock() @@ -520,15 +526,25 @@ func (m *GroupManager) SplitKeyspaceGroupByID(splitSourceID, splitTargetID uint3 return err } if splitSourceKg == nil { - return ErrKeyspaceGroupNotExists + return ErrKeyspaceGroupNotExists(splitSourceID) } // A keyspace group can not take part in multiple split processes. if splitSourceKg.IsSplitting() { - return ErrKeyspaceGroupInSplit + return ErrKeyspaceGroupInSplit(splitSourceID) } // A keyspace group can not be split when it is in merging. if splitSourceKg.IsMerging() { - return ErrKeyspaceGroupInMerging + return ErrKeyspaceGroupInMerging(splitSourceID) + } + // Build the new keyspace groups for split source and target. + var startKeyspaceID, endKeyspaceID uint32 + if len(keyspaceIDRange) >= 2 { + startKeyspaceID, endKeyspaceID = keyspaceIDRange[0], keyspaceIDRange[1] + } + splitSourceKeyspaces, splitTargetKeyspaces, err := buildSplitKeyspaces( + splitSourceKg.Keyspaces, keyspaces, startKeyspaceID, endKeyspaceID) + if err != nil { + return err } // Check if the source keyspace group has enough replicas. if len(splitSourceKg.Members) < utils.DefaultKeyspaceGroupReplicaCount { @@ -542,34 +558,8 @@ func (m *GroupManager) SplitKeyspaceGroupByID(splitSourceID, splitTargetID uint3 if splitTargetKg != nil { return ErrKeyspaceGroupExists } - keyspaceNum := len(keyspaces) - sourceKeyspaceNum := len(splitSourceKg.Keyspaces) - // Check if the keyspaces are all in the old keyspace group. - if keyspaceNum == 0 || keyspaceNum > sourceKeyspaceNum { - return ErrKeyspaceNotInKeyspaceGroup - } - var ( - oldKeyspaceMap = make(map[uint32]struct{}, sourceKeyspaceNum) - newKeyspaceMap = make(map[uint32]struct{}, keyspaceNum) - ) - for _, keyspace := range splitSourceKg.Keyspaces { - oldKeyspaceMap[keyspace] = struct{}{} - } - for _, keyspace := range keyspaces { - if _, ok := oldKeyspaceMap[keyspace]; !ok { - return ErrKeyspaceNotInKeyspaceGroup - } - newKeyspaceMap[keyspace] = struct{}{} - } - // Get the split keyspace group for the old keyspace group. - splitKeyspaces := make([]uint32, 0, sourceKeyspaceNum-keyspaceNum) - for _, keyspace := range splitSourceKg.Keyspaces { - if _, ok := newKeyspaceMap[keyspace]; !ok { - splitKeyspaces = append(splitKeyspaces, keyspace) - } - } // Update the old keyspace group. - splitSourceKg.Keyspaces = splitKeyspaces + splitSourceKg.Keyspaces = splitSourceKeyspaces splitSourceKg.SplitState = &endpoint.SplitState{ SplitSource: splitSourceKg.ID, } @@ -581,7 +571,7 @@ func (m *GroupManager) SplitKeyspaceGroupByID(splitSourceID, splitTargetID uint3 // Keep the same user kind and members as the old keyspace group. UserKind: splitSourceKg.UserKind, Members: splitSourceKg.Members, - Keyspaces: keyspaces, + Keyspaces: splitTargetKeyspaces, SplitState: &endpoint.SplitState{ SplitSource: splitSourceKg.ID, }, @@ -597,6 +587,82 @@ func (m *GroupManager) SplitKeyspaceGroupByID(splitSourceID, splitTargetID uint3 return nil } +func buildSplitKeyspaces( + // `old` is the original keyspace list which will be split out, + // `new` is the keyspace list which will be split from the old keyspace list. + old, new []uint32, + startKeyspaceID, endKeyspaceID uint32, +) ([]uint32, []uint32, error) { + oldNum, newNum := len(old), len(new) + // Split according to the new keyspace list. + if newNum != 0 { + if newNum > oldNum { + return nil, nil, ErrKeyspaceNotInKeyspaceGroup + } + var ( + oldKeyspaceMap = make(map[uint32]struct{}, oldNum) + newKeyspaceMap = make(map[uint32]struct{}, newNum) + ) + for _, keyspace := range old { + oldKeyspaceMap[keyspace] = struct{}{} + } + for _, keyspace := range new { + if keyspace == utils.DefaultKeyspaceID { + return nil, nil, ErrModifyDefaultKeyspace + } + if _, ok := oldKeyspaceMap[keyspace]; !ok { + return nil, nil, ErrKeyspaceNotInKeyspaceGroup + } + newKeyspaceMap[keyspace] = struct{}{} + } + // Get the split keyspace list for the old keyspace group. + oldSplit := make([]uint32, 0, oldNum-newNum) + for _, keyspace := range old { + if _, ok := newKeyspaceMap[keyspace]; !ok { + oldSplit = append(oldSplit, keyspace) + } + } + // If newNum != len(newKeyspaceMap), it means the provided new keyspace list contains + // duplicate keyspaces, and we need to dedup them (https://github.com/tikv/pd/issues/6687); + // otherwise, we can just return the old split and new keyspace list. + if newNum == len(newKeyspaceMap) { + return oldSplit, new, nil + } + newSplit := make([]uint32, 0, len(newKeyspaceMap)) + for keyspace := range newKeyspaceMap { + newSplit = append(newSplit, keyspace) + } + return oldSplit, newSplit, nil + } + // Split according to the start and end keyspace ID. + if startKeyspaceID == 0 && endKeyspaceID == 0 { + return nil, nil, ErrKeyspaceNotInKeyspaceGroup + } + var ( + newSplit = make([]uint32, 0, oldNum) + newKeyspaceMap = make(map[uint32]struct{}, newNum) + ) + for _, keyspace := range old { + if keyspace == utils.DefaultKeyspaceID { + // The source keyspace group must be the default keyspace group and we always keep the default + // keyspace in the default keyspace group. + continue + } + if startKeyspaceID <= keyspace && keyspace <= endKeyspaceID { + newSplit = append(newSplit, keyspace) + newKeyspaceMap[keyspace] = struct{}{} + } + } + // Get the split keyspace list for the old keyspace group. + oldSplit := make([]uint32, 0, oldNum-len(newSplit)) + for _, keyspace := range old { + if _, ok := newKeyspaceMap[keyspace]; !ok { + oldSplit = append(oldSplit, keyspace) + } + } + return oldSplit, newSplit, nil +} + // FinishSplitKeyspaceByID finishes the split keyspace group by the split target ID. func (m *GroupManager) FinishSplitKeyspaceByID(splitTargetID uint32) error { var splitTargetKg, splitSourceKg *endpoint.KeyspaceGroup @@ -609,11 +675,11 @@ func (m *GroupManager) FinishSplitKeyspaceByID(splitTargetID uint32) error { return err } if splitTargetKg == nil { - return ErrKeyspaceGroupNotExists + return ErrKeyspaceGroupNotExists(splitTargetID) } // Check if it's in the split state. if !splitTargetKg.IsSplitTarget() { - return ErrKeyspaceGroupNotInSplit + return ErrKeyspaceGroupNotInSplit(splitTargetID) } // Load the split source keyspace group then. splitSourceKg, err = m.store.LoadKeyspaceGroup(txn, splitTargetKg.SplitSource()) @@ -621,10 +687,10 @@ func (m *GroupManager) FinishSplitKeyspaceByID(splitTargetID uint32) error { return err } if splitSourceKg == nil { - return ErrKeyspaceGroupNotExists + return ErrKeyspaceGroupNotExists(splitTargetKg.SplitSource()) } if !splitSourceKg.IsSplitSource() { - return ErrKeyspaceGroupNotInSplit + return ErrKeyspaceGroupNotInSplit(splitTargetKg.SplitSource()) } splitTargetKg.SplitState = nil splitSourceKg.SplitState = nil @@ -639,6 +705,7 @@ func (m *GroupManager) FinishSplitKeyspaceByID(splitTargetID uint32) error { // Update the keyspace group cache. m.groups[endpoint.StringUserKind(splitTargetKg.UserKind)].Put(splitTargetKg) m.groups[endpoint.StringUserKind(splitSourceKg.UserKind)].Put(splitSourceKg) + log.Info("finish split keyspace group", zap.Uint32("split-source-id", splitSourceKg.ID), zap.Uint32("split-target-id", splitTargetID)) return nil } @@ -668,13 +735,13 @@ func (m *GroupManager) AllocNodesForKeyspaceGroup(id uint32, desiredReplicaCount return err } if kg == nil { - return ErrKeyspaceGroupNotExists + return ErrKeyspaceGroupNotExists(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit + return ErrKeyspaceGroupInSplit(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging + return ErrKeyspaceGroupInMerging(id) } exists := make(map[string]struct{}) for _, member := range kg.Members { @@ -714,7 +781,9 @@ func (m *GroupManager) AllocNodesForKeyspaceGroup(id uint32, desiredReplicaCount return nil, err } m.groups[endpoint.StringUserKind(kg.UserKind)].Put(kg) - log.Info("alloc nodes for keyspace group", zap.Uint32("keyspace-group-id", id), zap.Reflect("nodes", nodes)) + log.Info("alloc nodes for keyspace group", + zap.Uint32("keyspace-group-id", id), + zap.Reflect("nodes", nodes)) return nodes, nil } @@ -730,13 +799,13 @@ func (m *GroupManager) SetNodesForKeyspaceGroup(id uint32, nodes []string) error return err } if kg == nil { - return ErrKeyspaceGroupNotExists + return ErrKeyspaceGroupNotExists(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit + return ErrKeyspaceGroupInSplit(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging + return ErrKeyspaceGroupInMerging(id) } members := make([]endpoint.KeyspaceGroupMember, 0, len(nodes)) for _, node := range nodes { @@ -767,13 +836,13 @@ func (m *GroupManager) SetPriorityForKeyspaceGroup(id uint32, node string, prior return err } if kg == nil { - return ErrKeyspaceGroupNotExists + return ErrKeyspaceGroupNotExists(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit + return ErrKeyspaceGroupInSplit(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging + return ErrKeyspaceGroupInMerging(id) } inKeyspaceGroup := false members := make([]endpoint.KeyspaceGroupMember, 0, len(kg.Members)) @@ -838,32 +907,29 @@ func (m *GroupManager) MergeKeyspaceGroups(mergeTargetID uint32, mergeList []uin return err } if kg == nil { - return ErrKeyspaceGroupNotExists + return ErrKeyspaceGroupNotExists(kgID) } // A keyspace group can not be merged if it's in splitting. if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit + return ErrKeyspaceGroupInSplit(kgID) } // A keyspace group can not be split when it is in merging. if kg.IsMerging() { - return ErrKeyspaceGroupInMerging + return ErrKeyspaceGroupInMerging(kgID) } groups[kgID] = kg } + // Build the new keyspaces for the merge target keyspace group. mergeTargetKg = groups[mergeTargetID] keyspaces := make(map[uint32]struct{}) for _, keyspace := range mergeTargetKg.Keyspaces { keyspaces[keyspace] = struct{}{} } - // Delete the keyspace groups in merge list and move the keyspaces in it to the target keyspace group. for _, kgID := range mergeList { kg := groups[kgID] for _, keyspace := range kg.Keyspaces { keyspaces[keyspace] = struct{}{} } - if err := m.store.DeleteKeyspaceGroup(txn, kg.ID); err != nil { - return err - } } mergedKeyspaces := make([]uint32, 0, len(keyspaces)) for keyspace := range keyspaces { @@ -877,7 +943,17 @@ func (m *GroupManager) MergeKeyspaceGroups(mergeTargetID uint32, mergeList []uin mergeTargetKg.MergeState = &endpoint.MergeState{ MergeList: mergeList, } - return m.store.SaveKeyspaceGroup(txn, mergeTargetKg) + err = m.store.SaveKeyspaceGroup(txn, mergeTargetKg) + if err != nil { + return err + } + // Delete the keyspace groups in merge list and move the keyspaces in it to the target keyspace group. + for _, kgID := range mergeList { + if err := m.store.DeleteKeyspaceGroup(txn, kgID); err != nil { + return err + } + } + return nil }); err != nil { return err } @@ -892,7 +968,10 @@ func (m *GroupManager) MergeKeyspaceGroups(mergeTargetID uint32, mergeList []uin // FinishMergeKeyspaceByID finishes the merging keyspace group by the merge target ID. func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { - var mergeTargetKg *endpoint.KeyspaceGroup + var ( + mergeTargetKg *endpoint.KeyspaceGroup + mergeList []uint32 + ) m.Lock() defer m.Unlock() if err := m.store.RunInTxn(m.ctx, func(txn kv.Txn) (err error) { @@ -902,11 +981,11 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { return err } if mergeTargetKg == nil { - return ErrKeyspaceGroupNotExists + return ErrKeyspaceGroupNotExists(mergeTargetID) } // Check if it's in the merging state. if !mergeTargetKg.IsMergeTarget() { - return ErrKeyspaceGroupNotInMerging + return ErrKeyspaceGroupNotInMerging(mergeTargetID) } // Make sure all merging keyspace groups are deleted. for _, kgID := range mergeTargetKg.MergeState.MergeList { @@ -915,9 +994,10 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { return err } if kg != nil { - return ErrKeyspaceGroupNotInMerging + return ErrKeyspaceGroupNotInMerging(kgID) } } + mergeList = mergeTargetKg.MergeState.MergeList mergeTargetKg.MergeState = nil return m.store.SaveKeyspaceGroup(txn, mergeTargetKg) }); err != nil { @@ -925,5 +1005,8 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { } // Update the keyspace group cache. m.groups[endpoint.StringUserKind(mergeTargetKg.UserKind)].Put(mergeTargetKg) + log.Info("finish merge keyspace group", + zap.Uint32("merge-target-id", mergeTargetKg.ID), + zap.Reflect("merge-list", mergeList)) return nil } diff --git a/pkg/keyspace/tso_keyspace_group_test.go b/pkg/keyspace/tso_keyspace_group_test.go index 60f2793b8bb..e8a40a839c8 100644 --- a/pkg/keyspace/tso_keyspace_group_test.go +++ b/pkg/keyspace/tso_keyspace_group_test.go @@ -20,6 +20,7 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/mock/mockcluster" @@ -237,8 +238,9 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { keyspaceGroups := []*endpoint.KeyspaceGroup{ { - ID: uint32(1), - UserKind: endpoint.Basic.String(), + ID: uint32(1), + UserKind: endpoint.Basic.String(), + Keyspaces: []uint32{444}, }, { ID: uint32(2), @@ -249,8 +251,11 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { } err := suite.kgm.CreateKeyspaceGroups(keyspaceGroups) re.NoError(err) + // split the default keyspace + err = suite.kgm.SplitKeyspaceGroupByID(0, 4, []uint32{utils.DefaultKeyspaceID}) + re.ErrorIs(err, ErrModifyDefaultKeyspace) // split the keyspace group 1 to 4 - err = suite.kgm.SplitKeyspaceGroupByID(1, 4, []uint32{333}) + err = suite.kgm.SplitKeyspaceGroupByID(1, 4, []uint32{444}) re.ErrorIs(err, ErrKeyspaceGroupNotEnoughReplicas) // split the keyspace group 2 to 4 without giving any keyspace err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{}) @@ -275,25 +280,25 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { // finish the split of the keyspace group 2 err = suite.kgm.FinishSplitKeyspaceByID(2) - re.ErrorIs(err, ErrKeyspaceGroupNotInSplit) + re.ErrorContains(err, ErrKeyspaceGroupNotInSplit(2).Error()) // finish the split of a non-existing keyspace group err = suite.kgm.FinishSplitKeyspaceByID(5) - re.ErrorIs(err, ErrKeyspaceGroupNotExists) + re.ErrorContains(err, ErrKeyspaceGroupNotExists(5).Error()) // split the in-split keyspace group err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{333}) - re.ErrorIs(err, ErrKeyspaceGroupInSplit) + re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) // remove the in-split keyspace group kg2, err = suite.kgm.DeleteKeyspaceGroupByID(2) re.Nil(kg2) - re.ErrorIs(err, ErrKeyspaceGroupInSplit) + re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) kg4, err = suite.kgm.DeleteKeyspaceGroupByID(4) re.Nil(kg4) - re.ErrorIs(err, ErrKeyspaceGroupInSplit) + re.ErrorContains(err, ErrKeyspaceGroupInSplit(4).Error()) // update the in-split keyspace group err = suite.kg.kgm.UpdateKeyspaceForGroup(endpoint.Standard, "2", 444, opAdd) - re.ErrorIs(err, ErrKeyspaceGroupInSplit) + re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) err = suite.kg.kgm.UpdateKeyspaceForGroup(endpoint.Standard, "4", 444, opAdd) - re.ErrorIs(err, ErrKeyspaceGroupInSplit) + re.ErrorContains(err, ErrKeyspaceGroupInSplit(4).Error()) // finish the split of keyspace group 4 err = suite.kgm.FinishSplitKeyspaceByID(4) @@ -313,15 +318,66 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { // split a non-existing keyspace group err = suite.kgm.SplitKeyspaceGroupByID(3, 5, nil) - re.ErrorIs(err, ErrKeyspaceGroupNotExists) + re.ErrorContains(err, ErrKeyspaceGroupNotExists(3).Error()) // split into an existing keyspace group - err = suite.kgm.SplitKeyspaceGroupByID(2, 4, nil) + err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{111}) re.ErrorIs(err, ErrKeyspaceGroupExists) // split with the wrong keyspaces. err = suite.kgm.SplitKeyspaceGroupByID(2, 5, []uint32{111, 222, 444}) re.ErrorIs(err, ErrKeyspaceNotInKeyspaceGroup) } +func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplitRange() { + re := suite.Require() + + keyspaceGroups := []*endpoint.KeyspaceGroup{ + { + ID: uint32(1), + UserKind: endpoint.Basic.String(), + }, + { + ID: uint32(2), + UserKind: endpoint.Standard.String(), + Keyspaces: []uint32{111, 333, 444, 555, 666}, + Members: make([]endpoint.KeyspaceGroupMember, utils.DefaultKeyspaceGroupReplicaCount), + }, + } + err := suite.kgm.CreateKeyspaceGroups(keyspaceGroups) + re.NoError(err) + // split the keyspace group 2 to 4 with keyspace range [222, 555] + err = suite.kgm.SplitKeyspaceGroupByID(2, 4, nil, 222, 555) + re.NoError(err) + kg2, err := suite.kgm.GetKeyspaceGroupByID(2) + re.NoError(err) + re.Equal(uint32(2), kg2.ID) + re.Equal([]uint32{111, 666}, kg2.Keyspaces) + re.True(kg2.IsSplitSource()) + re.Equal(kg2.ID, kg2.SplitSource()) + kg4, err := suite.kgm.GetKeyspaceGroupByID(4) + re.NoError(err) + re.Equal(uint32(4), kg4.ID) + re.Equal([]uint32{333, 444, 555}, kg4.Keyspaces) + re.True(kg4.IsSplitTarget()) + re.Equal(kg2.ID, kg4.SplitSource()) + re.Equal(kg2.UserKind, kg4.UserKind) + re.Equal(kg2.Members, kg4.Members) + // finish the split of keyspace group 4 + err = suite.kgm.FinishSplitKeyspaceByID(4) + re.NoError(err) + kg2, err = suite.kgm.GetKeyspaceGroupByID(2) + re.NoError(err) + re.Equal(uint32(2), kg2.ID) + re.Equal([]uint32{111, 666}, kg2.Keyspaces) + re.False(kg2.IsSplitting()) + kg4, err = suite.kgm.GetKeyspaceGroupByID(4) + re.NoError(err) + re.Equal(uint32(4), kg4.ID) + re.Equal([]uint32{333, 444, 555}, kg4.Keyspaces) + re.False(kg4.IsSplitting()) + re.Equal(kg2.UserKind, kg4.UserKind) + re.Equal(kg2.Members, kg4.Members) +} + func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupMerge() { re := suite.Require() @@ -390,7 +446,7 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupMerge() { // merge a non-existing keyspace group err = suite.kgm.MergeKeyspaceGroups(4, []uint32{5}) - re.ErrorIs(err, ErrKeyspaceGroupNotExists) + 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)) re.ErrorIs(err, ErrExceedMaxEtcdTxnOps) @@ -398,3 +454,110 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupMerge() { err = suite.kgm.MergeKeyspaceGroups(1, []uint32{utils.DefaultKeyspaceGroupID}) re.ErrorIs(err, ErrModifyDefaultKeyspaceGroup) } + +func TestBuildSplitKeyspaces(t *testing.T) { + re := require.New(t) + testCases := []struct { + old []uint32 + new []uint32 + startKeyspaceID uint32 + endKeyspaceID uint32 + expectedOld []uint32 + expectedNew []uint32 + err error + }{ + { + old: []uint32{1, 2, 3, 4, 5}, + new: []uint32{1, 2, 3, 4, 5}, + expectedOld: []uint32{}, + expectedNew: []uint32{1, 2, 3, 4, 5}, + }, + { + old: []uint32{1, 2, 3, 4, 5}, + new: []uint32{1}, + expectedOld: []uint32{2, 3, 4, 5}, + expectedNew: []uint32{1}, + }, + { + old: []uint32{1, 2, 3, 4, 5}, + new: []uint32{6}, + err: ErrKeyspaceNotInKeyspaceGroup, + }, + { + old: []uint32{1, 2}, + new: []uint32{2, 2}, + expectedOld: []uint32{1}, + expectedNew: []uint32{2}, + }, + { + old: []uint32{0, 1, 2, 3, 4, 5}, + startKeyspaceID: 2, + endKeyspaceID: 4, + expectedOld: []uint32{0, 1, 5}, + expectedNew: []uint32{2, 3, 4}, + }, + { + old: []uint32{0, 1, 2, 3, 4, 5}, + startKeyspaceID: 0, + endKeyspaceID: 4, + expectedOld: []uint32{0, 5}, + expectedNew: []uint32{1, 2, 3, 4}, + }, + { + old: []uint32{1, 2, 3, 4, 5}, + startKeyspaceID: 2, + endKeyspaceID: 4, + expectedOld: []uint32{1, 5}, + expectedNew: []uint32{2, 3, 4}, + }, + { + old: []uint32{1, 2, 3, 4, 5}, + startKeyspaceID: 5, + endKeyspaceID: 6, + expectedOld: []uint32{1, 2, 3, 4}, + expectedNew: []uint32{5}, + }, + { + old: []uint32{1, 2, 3, 4, 5}, + startKeyspaceID: 2, + endKeyspaceID: 6, + expectedOld: []uint32{1}, + expectedNew: []uint32{2, 3, 4, 5}, + }, + { + old: []uint32{1, 2, 3, 4, 5}, + startKeyspaceID: 1, + endKeyspaceID: 1, + expectedOld: []uint32{2, 3, 4, 5}, + expectedNew: []uint32{1}, + }, + { + old: []uint32{1, 2, 3, 4, 5}, + startKeyspaceID: 0, + endKeyspaceID: 6, + expectedOld: []uint32{}, + expectedNew: []uint32{1, 2, 3, 4, 5}, + }, + { + old: []uint32{1, 2, 3, 4, 5}, + startKeyspaceID: 7, + endKeyspaceID: 10, + expectedOld: []uint32{1, 2, 3, 4, 5}, + expectedNew: []uint32{}, + }, + { + old: []uint32{1, 2, 3, 4, 5}, + err: ErrKeyspaceNotInKeyspaceGroup, + }, + } + for idx, testCase := range testCases { + old, new, err := buildSplitKeyspaces(testCase.old, testCase.new, testCase.startKeyspaceID, testCase.endKeyspaceID) + if testCase.err != nil { + re.ErrorIs(testCase.err, err, "test case %d", idx) + } else { + re.NoError(err, "test case %d", idx) + re.Equal(testCase.expectedOld, old, "test case %d", idx) + re.Equal(testCase.expectedNew, new, "test case %d", idx) + } + } +} diff --git a/pkg/keyspace/util.go b/pkg/keyspace/util.go index d2ebe31fdc3..240306f8124 100644 --- a/pkg/keyspace/util.go +++ b/pkg/keyspace/util.go @@ -49,15 +49,25 @@ var ( // ErrKeyspaceGroupExists indicates target keyspace group already exists. ErrKeyspaceGroupExists = errors.New("keyspace group already exists") // ErrKeyspaceGroupNotExists is used to indicate target keyspace group does not exist. - ErrKeyspaceGroupNotExists = errors.New("keyspace group does not exist") + ErrKeyspaceGroupNotExists = func(groupID uint32) error { + return errors.Errorf("keyspace group %v does not exist", groupID) + } // ErrKeyspaceGroupInSplit is used to indicate target keyspace group is in split state. - ErrKeyspaceGroupInSplit = errors.New("keyspace group is in split state") + ErrKeyspaceGroupInSplit = func(groupID uint32) error { + return errors.Errorf("keyspace group %v is in split state", groupID) + } // ErrKeyspaceGroupNotInSplit is used to indicate target keyspace group is not in split state. - ErrKeyspaceGroupNotInSplit = errors.New("keyspace group is not in split state") + ErrKeyspaceGroupNotInSplit = func(groupID uint32) error { + return errors.Errorf("keyspace group %v is not in split state", groupID) + } // ErrKeyspaceGroupInMerging is used to indicate target keyspace group is in merging state. - ErrKeyspaceGroupInMerging = errors.New("keyspace group is in merging state") + ErrKeyspaceGroupInMerging = func(groupID uint32) error { + return errors.Errorf("keyspace group %v is in merging state", groupID) + } // ErrKeyspaceGroupNotInMerging is used to indicate target keyspace group is not in merging state. - ErrKeyspaceGroupNotInMerging = errors.New("keyspace group is not in merging state") + ErrKeyspaceGroupNotInMerging = func(groupID uint32) error { + return errors.Errorf("keyspace group %v is not in merging state", groupID) + } // ErrKeyspaceNotInKeyspaceGroup is used to indicate target keyspace is not in this keyspace group. ErrKeyspaceNotInKeyspaceGroup = errors.New("keyspace is not in this keyspace group") // ErrNodeNotInKeyspaceGroup is used to indicate the tso node is not in this keyspace group. @@ -70,8 +80,9 @@ var ( ErrNoAvailableNode = errors.New("no available node") // ErrExceedMaxEtcdTxnOps is used to indicate the number of etcd txn operations exceeds the limit. ErrExceedMaxEtcdTxnOps = errors.New("exceed max etcd txn operations") - errModifyDefault = errors.New("cannot modify default keyspace's state") - errIllegalOperation = errors.New("unknown operation") + // ErrModifyDefaultKeyspace is used to indicate that default keyspace cannot be modified. + ErrModifyDefaultKeyspace = errors.New("cannot modify default keyspace's state") + errIllegalOperation = errors.New("unknown operation") // stateTransitionTable lists all allowed next state for the given current state. // Note that transit from any state to itself is allowed for idempotence. diff --git a/pkg/mcs/discovery/discover.go b/pkg/mcs/discovery/discover.go index d3c06ad2cc8..6d939fde540 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 := discoveryPath(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 0e53b21c9fe..4eb339dd5db 100644 --- a/pkg/mcs/discovery/key_path.go +++ b/pkg/mcs/discovery/key_path.go @@ -24,15 +24,17 @@ const ( registryKey = "registry" ) -func registryPath(clusterID, serviceName, serviceAddr string) string { +// 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}, "/") } -func discoveryPath(clusterID, serviceName string) string { +// ServicePath returns the path to store microservice addresses. +func ServicePath(clusterID, serviceName string) string { return strings.Join([]string{registryPrefix, clusterID, serviceName, registryKey}, "/") } // TSOPath returns the path to store TSO addresses. func TSOPath(clusterID uint64) string { - return discoveryPath(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 617c1520b8d..3e08d9b49cf 100644 --- a/pkg/mcs/discovery/register.go +++ b/pkg/mcs/discovery/register.go @@ -41,7 +41,7 @@ type ServiceRegister struct { // NewServiceRegister creates a new ServiceRegister. func NewServiceRegister(ctx context.Context, cli *clientv3.Client, clusterID, serviceName, serviceAddr, serializedValue string, ttl int64) *ServiceRegister { cctx, cancel := context.WithCancel(ctx) - serviceKey := registryPath(clusterID, serviceName, serviceAddr) + serviceKey := RegistryPath(clusterID, serviceName, serviceAddr) return &ServiceRegister{ ctx: cctx, cancel: cancel, diff --git a/pkg/mcs/tso/server/server.go b/pkg/mcs/tso/server/server.go index 1b4b5c36374..a0904f4dc7b 100644 --- a/pkg/mcs/tso/server/server.go +++ b/pkg/mcs/tso/server/server.go @@ -539,7 +539,8 @@ func (s *Server) startServer() (err error) { tsoSvcRootPath := fmt.Sprintf(tsoSvcRootPathFormat, 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, legacySvcRootPath, tsoSvcRootPath, s.cfg) + s.serverLoopCtx, s.serviceID, s.etcdClient, s.httpClient, s.cfg.AdvertiseListenAddr, + discovery.TSOPath(s.clusterID), legacySvcRootPath, tsoSvcRootPath, s.cfg) if err := s.keyspaceGroupManager.Initialize(); err != nil { return err } diff --git a/pkg/mcs/utils/constant.go b/pkg/mcs/utils/constant.go index 21a4a655afe..c87cec16a64 100644 --- a/pkg/mcs/utils/constant.go +++ b/pkg/mcs/utils/constant.go @@ -76,5 +76,8 @@ const ( DefaultKeyspaceGroupReplicaCount = 2 // DefaultKeyspaceGroupReplicaPriority is the default priority of a keyspace group replica. + // It's used in keyspace group primary weighted-election to balance primaries' distribution. + // Among multiple replicas of a keyspace group, the higher the priority, the more likely + // the replica is to be elected as primary. DefaultKeyspaceGroupReplicaPriority = 0 ) diff --git a/pkg/replication/replication_mode.go b/pkg/replication/replication_mode.go index 1a94abb09fe..03ed336fd9f 100644 --- a/pkg/replication/replication_mode.go +++ b/pkg/replication/replication_mode.go @@ -62,7 +62,7 @@ type FileReplicater interface { } const drStatusFile = "DR_STATE" -const persistFileTimeout = time.Second * 10 +const persistFileTimeout = time.Second * 3 // ModeManager is used to control how raft logs are synchronized between // different tikv nodes. diff --git a/pkg/schedule/checker/merge_checker_test.go b/pkg/schedule/checker/merge_checker_test.go index f6f2dd69868..9c38d677619 100644 --- a/pkg/schedule/checker/merge_checker_test.go +++ b/pkg/schedule/checker/merge_checker_test.go @@ -484,7 +484,7 @@ func (suite *mergeCheckerTestSuite) TestStoreLimitWithMerge() { suite.NotNil(ops) suite.True(oc.AddOperator(ops...)) for _, op := range ops { - oc.RemoveOperator(op) + oc.RemoveOperator(op, operator.ExceedStoreLimit) } } regions[2] = regions[2].Clone( @@ -498,7 +498,7 @@ func (suite *mergeCheckerTestSuite) TestStoreLimitWithMerge() { suite.NotNil(ops) suite.True(oc.AddOperator(ops...)) for _, op := range ops { - oc.RemoveOperator(op) + oc.RemoveOperator(op, operator.ExceedStoreLimit) } } { diff --git a/pkg/schedule/checker/rule_checker.go b/pkg/schedule/checker/rule_checker.go index d405941300e..95e63414966 100644 --- a/pkg/schedule/checker/rule_checker.go +++ b/pkg/schedule/checker/rule_checker.go @@ -398,12 +398,13 @@ func (c *RuleChecker) fixBetterLocation(region *core.RegionInfo, rf *placement.R isWitness := rf.Rule.IsWitness && c.isWitnessEnabled() // If the peer to be moved is a witness, since no snapshot is needed, we also reuse the fast failover logic. strategy := c.strategy(region, rf.Rule, isWitness) - regionStores := c.cluster.GetRegionStores(region) - oldStore := strategy.SelectStoreToRemove(regionStores) + ruleStores := c.getRuleFitStores(rf) + oldStore := strategy.SelectStoreToRemove(ruleStores) if oldStore == 0 { return nil, nil } var coLocationStores []*core.StoreInfo + regionStores := c.cluster.GetRegionStores(region) for _, s := range regionStores { if placement.MatchLabelConstraints(s, rf.Rule.LabelConstraints) { coLocationStores = append(coLocationStores, s) diff --git a/pkg/schedule/checker/rule_checker_test.go b/pkg/schedule/checker/rule_checker_test.go index ebe1d1aadaf..f01378249a6 100644 --- a/pkg/schedule/checker/rule_checker_test.go +++ b/pkg/schedule/checker/rule_checker_test.go @@ -1391,3 +1391,35 @@ func (suite *ruleCheckerTestSuite) TestLocationLabels() { suite.NotNil(op) suite.Equal("move-to-better-location", op.Desc()) } + +func (suite *ruleCheckerTestSuite) TestTiFlashLocationLabels() { + suite.cluster.SetEnableUseJointConsensus(true) + suite.cluster.AddLabelsStore(1, 1, map[string]string{"zone": "z1", "rack": "r1", "host": "h1"}) + suite.cluster.AddLabelsStore(2, 1, map[string]string{"zone": "z1", "rack": "r1", "host": "h1"}) + suite.cluster.AddLabelsStore(3, 1, map[string]string{"zone": "z1", "rack": "r2", "host": "h1"}) + suite.cluster.AddLabelsStore(4, 1, map[string]string{"zone": "z1", "rack": "r2", "host": "h1"}) + suite.cluster.AddLabelsStore(5, 1, map[string]string{"zone": "z2", "rack": "r3", "host": "h2"}) + suite.cluster.AddLabelsStore(6, 1, map[string]string{"zone": "z2", "rack": "r3", "host": "h2"}) + suite.cluster.AddLabelsStore(7, 1, map[string]string{"engine": "tiflash"}) + suite.cluster.AddRegionWithLearner(1, 1, []uint64{3, 5}, []uint64{7}) + + rule1 := &placement.Rule{ + GroupID: "tiflash", + ID: "test1", + Role: placement.Learner, + Count: 1, + LabelConstraints: []placement.LabelConstraint{ + { + Key: "engine", + Op: placement.In, + Values: []string{"tiflash"}, + }, + }, + } + suite.ruleManager.SetRule(rule1) + rule := suite.ruleManager.GetRule("pd", "default") + rule.LocationLabels = []string{"zone", "rack", "host"} + suite.ruleManager.SetRule(rule) + op := suite.rc.Check(suite.cluster.GetRegion(1)) + suite.Nil(op) +} diff --git a/pkg/schedule/coordinator.go b/pkg/schedule/coordinator.go index 07e519dac7a..419b6a7adae 100644 --- a/pkg/schedule/coordinator.go +++ b/pkg/schedule/coordinator.go @@ -519,11 +519,28 @@ func (c *Coordinator) GetHotRegionsByType(typ statistics.RWType) *statistics.Sto default: } // update params `IsLearner` and `LastUpdateTime` - for _, stores := range []statistics.StoreHotPeersStat{infos.AsLeader, infos.AsPeer} { - for _, store := range stores { - for _, hotPeer := range store.Stats { - region := c.cluster.GetRegion(hotPeer.RegionID) - hotPeer.UpdateHotPeerStatShow(region) + s := []statistics.StoreHotPeersStat{infos.AsLeader, infos.AsPeer} + for i, stores := range s { + for j, store := range stores { + for k := range store.Stats { + h := &s[i][j].Stats[k] + region := c.cluster.GetRegion(h.RegionID) + if region != nil { + h.IsLearner = core.IsLearner(region.GetPeer(h.StoreID)) + } + switch typ { + case statistics.Write: + if region != nil { + h.LastUpdateTime = time.Unix(int64(region.GetInterval().GetEndTimestamp()), 0) + } + case statistics.Read: + store := c.cluster.GetStore(h.StoreID) + if store != nil { + ts := store.GetMeta().GetLastHeartbeat() + h.LastUpdateTime = time.Unix(ts/1e9, ts%1e9) + } + default: + } } } } diff --git a/pkg/schedule/operator/operator.go b/pkg/schedule/operator/operator.go index 4a9f9e8cbbd..02a2f1a8a5f 100644 --- a/pkg/schedule/operator/operator.go +++ b/pkg/schedule/operator/operator.go @@ -32,6 +32,39 @@ const ( // OperatorExpireTime is the duration that when an operator is not started // after it, the operator will be considered expired. OperatorExpireTime = 3 * time.Second + cancelReason = "cancel-reason" +) + +// CancelReasonType is the type of cancel reason. +type CancelReasonType string + +var ( + // RegionNotFound is the cancel reason when the region is not found. + RegionNotFound CancelReasonType = "region not found" + // EpochNotMatch is the cancel reason when the region epoch is not match. + EpochNotMatch CancelReasonType = "epoch not match" + // AlreadyExist is the cancel reason when the operator is running. + AlreadyExist CancelReasonType = "already exist" + // AdminStop is the cancel reason when the operator is stopped by adminer. + AdminStop CancelReasonType = "admin stop" + // NotInRunningState is the cancel reason when the operator is not in running state. + NotInRunningState CancelReasonType = "not in running state" + // Succeed is the cancel reason when the operator is finished successfully. + Succeed CancelReasonType = "succeed" + // Timeout is the cancel reason when the operator is timeout. + Timeout CancelReasonType = "timeout" + // Expired is the cancel reason when the operator is expired. + Expired CancelReasonType = "expired" + // NotInCreateStatus is the cancel reason when the operator is not in create status. + NotInCreateStatus CancelReasonType = "not in create status" + // StaleStatus is the cancel reason when the operator is in a stale status. + StaleStatus CancelReasonType = "stale status" + // ExceedStoreLimit is the cancel reason when the operator exceeds the store limit. + ExceedStoreLimit CancelReasonType = "exceed store limit" + // ExceedWaitLimit is the cancel reason when the operator exceeds the waiting queue limit. + ExceedWaitLimit CancelReasonType = "exceed wait limit" + // Unknown is the cancel reason when the operator is cancelled by an unknown reason. + Unknown CancelReasonType = "unknown" ) // Operator contains execution steps generated by scheduler. @@ -227,7 +260,10 @@ func (o *Operator) CheckSuccess() bool { } // Cancel marks the operator canceled. -func (o *Operator) Cancel() bool { +func (o *Operator) Cancel(reason CancelReasonType) bool { + if _, ok := o.AdditionalInfos[cancelReason]; !ok { + o.AdditionalInfos[cancelReason] = string(reason) + } return o.status.To(CANCELED) } diff --git a/pkg/schedule/operator/operator_controller.go b/pkg/schedule/operator/operator_controller.go index cbda045130a..b1e40a35e58 100644 --- a/pkg/schedule/operator/operator_controller.go +++ b/pkg/schedule/operator/operator_controller.go @@ -125,7 +125,7 @@ func (oc *Controller) Dispatch(region *core.RegionInfo, source string, recordOpS if op.ContainNonWitnessStep() { recordOpStepWithTTL(op.RegionID()) } - if oc.RemoveOperator(op) { + if oc.RemoveOperator(op, Succeed) { operatorWaitCounter.WithLabelValues(op.Desc(), "promote-success").Inc() oc.PromoteWaitingOperator() } @@ -134,7 +134,7 @@ func (oc *Controller) Dispatch(region *core.RegionInfo, source string, recordOpS oc.pushFastOperator(op) } case TIMEOUT: - if oc.RemoveOperator(op) { + if oc.RemoveOperator(op, Timeout) { operatorCounter.WithLabelValues(op.Desc(), "promote-timeout").Inc() oc.PromoteWaitingOperator() } @@ -150,7 +150,7 @@ func (oc *Controller) Dispatch(region *core.RegionInfo, source string, recordOpS failpoint.Inject("unexpectedOperator", func() { panic(op) }) - _ = op.Cancel() + _ = op.Cancel(NotInRunningState) oc.buryOperator(op) operatorWaitCounter.WithLabelValues(op.Desc(), "promote-unexpected").Inc() oc.PromoteWaitingOperator() @@ -162,7 +162,8 @@ func (oc *Controller) Dispatch(region *core.RegionInfo, source string, recordOpS func (oc *Controller) checkStaleOperator(op *Operator, step OpStep, region *core.RegionInfo) bool { err := step.CheckInProgress(oc.cluster, oc.config, region) if err != nil { - if oc.RemoveOperator(op, zap.String("reason", err.Error())) { + log.Info("operator is stale", zap.Uint64("region-id", op.RegionID()), errs.ZapError(err)) + if oc.RemoveOperator(op, StaleStatus) { operatorCounter.WithLabelValues(op.Desc(), "stale").Inc() operatorWaitCounter.WithLabelValues(op.Desc(), "promote-stale").Inc() oc.PromoteWaitingOperator() @@ -177,11 +178,13 @@ func (oc *Controller) checkStaleOperator(op *Operator, step OpStep, region *core latest := region.GetRegionEpoch() changes := latest.GetConfVer() - origin.GetConfVer() if changes > op.ConfVerChanged(region) { + log.Info("operator is stale", + zap.Uint64("region-id", op.RegionID()), + zap.Uint64("diff", changes), + zap.Reflect("latest-epoch", region.GetRegionEpoch())) if oc.RemoveOperator( op, - zap.String("reason", "stale operator, confver does not meet expectations"), - zap.Reflect("latest-epoch", region.GetRegionEpoch()), - zap.Uint64("diff", changes), + EpochNotMatch, ) { operatorCounter.WithLabelValues(op.Desc(), "stale").Inc() operatorWaitCounter.WithLabelValues(op.Desc(), "promote-stale").Inc() @@ -220,7 +223,7 @@ func (oc *Controller) pollNeedDispatchRegion() (r *core.RegionInfo, next bool) { r = oc.cluster.GetRegion(regionID) if r == nil { _ = oc.removeOperatorLocked(op) - if op.Cancel() { + if op.Cancel(RegionNotFound) { log.Warn("remove operator because region disappeared", zap.Uint64("region-id", op.RegionID()), zap.Stringer("operator", op)) @@ -285,14 +288,14 @@ func (oc *Controller) AddWaitingOperator(ops ...*Operator) int { } isMerge = true } - if !oc.checkAddOperator(false, op) { - _ = op.Cancel() + if pass, reason := oc.checkAddOperator(false, op); !pass { + _ = op.Cancel(reason) oc.buryOperator(op) if isMerge { // Merge operation have two operators, cancel them all i++ next := ops[i] - _ = next.Cancel() + _ = next.Cancel(reason) oc.buryOperator(next) } continue @@ -327,9 +330,16 @@ func (oc *Controller) AddOperator(ops ...*Operator) bool { // note: checkAddOperator uses false param for `isPromoting`. // This is used to keep check logic before fixing issue #4946, // but maybe user want to add operator when waiting queue is busy - if oc.exceedStoreLimitLocked(ops...) || !oc.checkAddOperator(false, ops...) { + if oc.exceedStoreLimitLocked(ops...) { for _, op := range ops { - _ = op.Cancel() + _ = op.Cancel(ExceedStoreLimit) + oc.buryOperator(op) + } + return false + } + if pass, reason := oc.checkAddOperator(false, ops...); !pass { + for _, op := range ops { + _ = op.Cancel(reason) oc.buryOperator(op) } return false @@ -354,11 +364,20 @@ func (oc *Controller) PromoteWaitingOperator() { return } operatorWaitCounter.WithLabelValues(ops[0].Desc(), "get").Inc() + if oc.exceedStoreLimitLocked(ops...) { + for _, op := range ops { + operatorWaitCounter.WithLabelValues(op.Desc(), "promote-canceled").Inc() + _ = op.Cancel(ExceedStoreLimit) + oc.buryOperator(op) + } + oc.wopStatus.ops[ops[0].Desc()]-- + continue + } - if oc.exceedStoreLimitLocked(ops...) || !oc.checkAddOperator(true, ops...) { + if pass, reason := oc.checkAddOperator(true, ops...); !pass { for _, op := range ops { operatorWaitCounter.WithLabelValues(op.Desc(), "promote-canceled").Inc() - _ = op.Cancel() + _ = op.Cancel(reason) oc.buryOperator(op) } oc.wopStatus.ops[ops[0].Desc()]-- @@ -382,14 +401,14 @@ func (oc *Controller) PromoteWaitingOperator() { // - The region already has a higher priority or same priority // - Exceed the max number of waiting operators // - At least one operator is expired. -func (oc *Controller) checkAddOperator(isPromoting bool, ops ...*Operator) bool { +func (oc *Controller) checkAddOperator(isPromoting bool, ops ...*Operator) (bool, CancelReasonType) { for _, op := range ops { region := oc.cluster.GetRegion(op.RegionID()) if region == nil { log.Debug("region not found, cancel add operator", zap.Uint64("region-id", op.RegionID())) operatorWaitCounter.WithLabelValues(op.Desc(), "not-found").Inc() - return false + return false, RegionNotFound } if region.GetRegionEpoch().GetVersion() != op.RegionEpoch().GetVersion() || region.GetRegionEpoch().GetConfVer() != op.RegionEpoch().GetConfVer() { @@ -398,14 +417,14 @@ func (oc *Controller) checkAddOperator(isPromoting bool, ops ...*Operator) bool zap.Reflect("old", region.GetRegionEpoch()), zap.Reflect("new", op.RegionEpoch())) operatorWaitCounter.WithLabelValues(op.Desc(), "epoch-not-match").Inc() - return false + return false, EpochNotMatch } if old := oc.operators[op.RegionID()]; old != nil && !isHigherPriorityOperator(op, old) { log.Debug("already have operator, cancel add operator", zap.Uint64("region-id", op.RegionID()), zap.Reflect("old", old)) operatorWaitCounter.WithLabelValues(op.Desc(), "already-have").Inc() - return false + return false, AlreadyExist } if op.Status() != CREATED { log.Error("trying to add operator with unexpected status", @@ -416,26 +435,26 @@ func (oc *Controller) checkAddOperator(isPromoting bool, ops ...*Operator) bool panic(op) }) operatorWaitCounter.WithLabelValues(op.Desc(), "unexpected-status").Inc() - return false + return false, NotInCreateStatus } if !isPromoting && oc.wopStatus.ops[op.Desc()] >= oc.config.GetSchedulerMaxWaitingOperator() { log.Debug("exceed max return false", zap.Uint64("waiting", oc.wopStatus.ops[op.Desc()]), zap.String("desc", op.Desc()), zap.Uint64("max", oc.config.GetSchedulerMaxWaitingOperator())) operatorWaitCounter.WithLabelValues(op.Desc(), "exceed-max").Inc() - return false + return false, ExceedWaitLimit } if op.SchedulerKind() == OpAdmin || op.IsLeaveJointStateOperator() { continue } } - expired := false + var reason CancelReasonType for _, op := range ops { if op.CheckExpired() { - expired = true + reason = Expired operatorWaitCounter.WithLabelValues(op.Desc(), "expired").Inc() } } - return !expired + return reason != Expired, reason } func isHigherPriorityOperator(new, old *Operator) bool { @@ -521,18 +540,24 @@ func (oc *Controller) ack(op *Operator) { } // RemoveOperator removes an operator from the running operators. -func (oc *Controller) RemoveOperator(op *Operator, extraFields ...zap.Field) bool { +func (oc *Controller) RemoveOperator(op *Operator, reasons ...CancelReasonType) bool { oc.Lock() removed := oc.removeOperatorLocked(op) oc.Unlock() + var cancelReason CancelReasonType + if len(reasons) > 0 { + cancelReason = reasons[0] + } else { + cancelReason = Unknown + } if removed { - if op.Cancel() { + if op.Cancel(cancelReason) { log.Info("operator removed", zap.Uint64("region-id", op.RegionID()), zap.Duration("takes", op.RunningTime()), zap.Reflect("operator", op)) } - oc.buryOperator(op, extraFields...) + oc.buryOperator(op) } return removed } @@ -555,7 +580,7 @@ func (oc *Controller) removeOperatorLocked(op *Operator) bool { return false } -func (oc *Controller) buryOperator(op *Operator, extraFields ...zap.Field) { +func (oc *Controller) buryOperator(op *Operator) { st := op.Status() if !IsEndStatus(st) { @@ -567,7 +592,7 @@ func (oc *Controller) buryOperator(op *Operator, extraFields ...zap.Field) { panic(op) }) operatorCounter.WithLabelValues(op.Desc(), "unexpected").Inc() - _ = op.Cancel() + _ = op.Cancel(Unknown) } switch st { @@ -603,15 +628,11 @@ func (oc *Controller) buryOperator(op *Operator, extraFields ...zap.Field) { zap.String("additional-info", op.GetAdditionalInfo())) operatorCounter.WithLabelValues(op.Desc(), "timeout").Inc() case CANCELED: - fields := []zap.Field{ + log.Info("operator canceled", zap.Uint64("region-id", op.RegionID()), zap.Duration("takes", op.RunningTime()), zap.Reflect("operator", op), zap.String("additional-info", op.GetAdditionalInfo()), - } - fields = append(fields, extraFields...) - log.Info("operator canceled", - fields..., ) operatorCounter.WithLabelValues(op.Desc(), "cancel").Inc() } diff --git a/pkg/schedule/operator/operator_controller_test.go b/pkg/schedule/operator/operator_controller_test.go index eb2d69db944..112e5a11f9c 100644 --- a/pkg/schedule/operator/operator_controller_test.go +++ b/pkg/schedule/operator/operator_controller_test.go @@ -248,7 +248,7 @@ func (suite *operatorControllerTestSuite) TestCheckAddUnexpectedStatus() { // finished op canceled op := NewTestOperator(1, &metapb.RegionEpoch{}, OpRegion, TransferLeader{ToStore: 2}) suite.True(oc.checkAddOperator(false, op)) - suite.True(op.Cancel()) + suite.True(op.Cancel(AdminStop)) suite.False(oc.checkAddOperator(false, op)) } { diff --git a/pkg/schedule/scatter/region_scatterer.go b/pkg/schedule/scatter/region_scatterer.go index 54fc291b363..de90228f7f6 100644 --- a/pkg/schedule/scatter/region_scatterer.go +++ b/pkg/schedule/scatter/region_scatterer.go @@ -264,7 +264,7 @@ func (r *RegionScatterer) scatterRegions(regions map[uint64]*core.RegionInfo, fa } failpoint.Inject("scatterHbStreamsDrain", func() { r.opController.GetHBStreams().Drain(1) - r.opController.RemoveOperator(op) + r.opController.RemoveOperator(op, operator.AdminStop) }) } delete(failures, region.GetID()) diff --git a/pkg/schedule/schedulers/hot_region_test.go b/pkg/schedule/schedulers/hot_region_test.go index 362758d4ebf..e4cf6b121f8 100644 --- a/pkg/schedule/schedulers/hot_region_test.go +++ b/pkg/schedule/schedulers/hot_region_test.go @@ -145,7 +145,7 @@ func checkGCPendingOpInfos(re *require.Assertions, enablePlacementRules bool) { } justDoneOpInfluence := func(region *core.RegionInfo, ty opType) *pendingInfluence { infl := notDoneOpInfluence(region, ty) - infl.op.Cancel() + infl.op.Cancel(operator.AdminStop) return infl } shouldRemoveOpInfluence := func(region *core.RegionInfo, ty opType) *pendingInfluence { @@ -957,7 +957,7 @@ func checkHotWriteRegionScheduleWithPendingInfluence(re *require.Assertions, dim operatorutil.CheckTransferPeerWithLeaderTransfer(re, op, operator.OpHotRegion, 1, 4) cnt++ if cnt == 3 { - re.True(op.Cancel()) + re.True(op.Cancel(operator.AdminStop)) } default: re.FailNow("wrong op: " + op.String()) @@ -1367,14 +1367,14 @@ func checkHotReadRegionScheduleWithPendingInfluence(re *require.Assertions, dim op2 := ops[0] operatorutil.CheckTransferPeer(re, op2, operator.OpHotRegion, 1, 4) // After move-peer, store byte/key rate (min, max): (6.1, 7.1) | 6.1 | 6 | (5, 6) - re.True(op2.Cancel()) + re.True(op2.Cancel(operator.AdminStop)) ops, _ = hb.Schedule(tc, false) op2 = ops[0] operatorutil.CheckTransferPeer(re, op2, operator.OpHotRegion, 1, 4) // After move-peer, store byte/key rate (min, max): (6.1, 7.1) | 6.1 | (6, 6.5) | (5, 5.5) - re.True(op1.Cancel()) + re.True(op1.Cancel(operator.AdminStop)) // store byte/key rate (min, max): (6.6, 7.1) | 6.1 | 6 | (5, 5.5) ops, _ = hb.Schedule(tc, false) diff --git a/pkg/schedule/schedulers/split_bucket.go b/pkg/schedule/schedulers/split_bucket.go index fef20974ef0..7452d2ceafa 100644 --- a/pkg/schedule/schedulers/split_bucket.go +++ b/pkg/schedule/schedulers/split_bucket.go @@ -23,7 +23,6 @@ import ( "github.com/gorilla/mux" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/tikv/pd/pkg/core" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/plan" @@ -251,8 +250,6 @@ func (s *splitBucketScheduler) splitBucket(plan *splitBucketPlan) []*operator.Op return nil } splitBucketNewOperatorCounter.Inc() - op.AdditionalInfos["region-start-key"] = core.HexRegionKeyStr(region.GetStartKey()) - op.AdditionalInfos["region-end-key"] = core.HexRegionKeyStr(region.GetEndKey()) op.AdditionalInfos["hot-degree"] = strconv.FormatInt(int64(splitBucket.HotDegree), 10) return []*operator.Operator{op} } diff --git a/pkg/statistics/hot_peer_cache.go b/pkg/statistics/hot_peer_cache.go index 478a9f506d1..16c64b752e0 100644 --- a/pkg/statistics/hot_peer_cache.go +++ b/pkg/statistics/hot_peer_cache.go @@ -234,8 +234,8 @@ func (f *hotPeerCache) checkPeerFlow(peer *core.PeerInfo, region *core.RegionInf actionType: Update, stores: make([]uint64, len(peers)), } - for _, peer := range peers { - newItem.stores = append(newItem.stores, peer.GetStoreId()) + for i, peer := range peers { + newItem.stores[i] = peer.GetStoreId() } if oldItem == nil { diff --git a/pkg/statistics/hot_regions_stat.go b/pkg/statistics/hot_regions_stat.go index d606a0d8bb4..d30a153492b 100644 --- a/pkg/statistics/hot_regions_stat.go +++ b/pkg/statistics/hot_regions_stat.go @@ -14,11 +14,7 @@ package statistics -import ( - "time" - - "github.com/tikv/pd/pkg/core" -) +import "time" // HotPeersStat records all hot regions statistics type HotPeersStat struct { @@ -44,14 +40,5 @@ type HotPeerStatShow struct { KeyRate float64 `json:"flow_keys"` QueryRate float64 `json:"flow_query"` AntiCount int `json:"anti_count"` - LastUpdateTime time.Time `json:"last_update_time"` -} - -// UpdateHotPeerStatShow updates the region information, such as `IsLearner` and `LastUpdateTime`. -func (h *HotPeerStatShow) UpdateHotPeerStatShow(region *core.RegionInfo) { - if region == nil { - return - } - h.IsLearner = core.IsLearner(region.GetPeer(h.StoreID)) - h.LastUpdateTime = time.Unix(int64(region.GetInterval().GetEndTimestamp()), 0) + LastUpdateTime time.Time `json:"last_update_time,omitempty"` } diff --git a/pkg/timerpool/pool.go b/pkg/timerpool/pool.go new file mode 100644 index 00000000000..28ffacfc629 --- /dev/null +++ b/pkg/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/pkg/timerpool/pool_test.go b/pkg/timerpool/pool_test.go new file mode 100644 index 00000000000..d6dffc723a9 --- /dev/null +++ b/pkg/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/pkg/tso/allocator_manager.go b/pkg/tso/allocator_manager.go index d6a378704d8..6e67ccf6951 100644 --- a/pkg/tso/allocator_manager.go +++ b/pkg/tso/allocator_manager.go @@ -1406,3 +1406,16 @@ 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 284d7dc316a..2c715d0cc7c 100644 --- a/pkg/tso/global_allocator.go +++ b/pkg/tso/global_allocator.go @@ -18,7 +18,6 @@ import ( "context" "errors" "fmt" - "path" "sync" "sync/atomic" "time" @@ -89,16 +88,6 @@ func NewGlobalTSOAllocator( am *AllocatorManager, startGlobalLeaderLoop bool, ) Allocator { - // 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 - tsPath := "" - if am.kgID != mcsutils.DefaultKeyspaceGroupID { - tsPath = path.Join(fmt.Sprintf("%05d", am.kgID), globalTSOAllocatorEtcdPrefix) - } - ctx, cancel := context.WithCancel(ctx) gta := &GlobalTSOAllocator{ ctx: ctx, @@ -107,8 +96,7 @@ func NewGlobalTSOAllocator( member: am.member, timestampOracle: ×tampOracle{ client: am.member.GetLeadership().GetClient(), - rootPath: am.rootPath, - tsPath: tsPath, + tsPath: am.getKeyspaceGroupTSPath(am.kgID), storage: am.storage, saveInterval: am.saveInterval, updatePhysicalInterval: am.updatePhysicalInterval, diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 62a6986422c..0291bc5863d 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -18,6 +18,7 @@ import ( "context" "encoding/json" "fmt" + "math" "net/http" "path" "regexp" @@ -27,6 +28,7 @@ import ( "time" perrors "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/election" @@ -34,6 +36,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/slice" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/apiutil" @@ -41,6 +44,7 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/memberutil" "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/zap" @@ -50,6 +54,13 @@ 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 + // defaultPrimaryPriorityCheckInterval is the default interval for checking if the priorities + // 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 ) type state struct { @@ -101,6 +112,39 @@ func (s *state) getKeyspaceGroupMeta( return s.ams[groupID], s.kgs[groupID] } +func (s *state) checkTSOSplit( + targetGroupID uint32, +) (splitTargetAM, splitSourceAM *AllocatorManager, err error) { + s.RLock() + defer s.RUnlock() + splitTargetAM, splitTargetGroup := s.ams[targetGroupID], s.kgs[targetGroupID] + // Only the split target keyspace group needs to check the TSO split. + if !splitTargetGroup.IsSplitTarget() { + return nil, nil, nil // it isn't in the split state + } + sourceGroupID := splitTargetGroup.SplitSource() + splitSourceAM, splitSourceGroup := s.ams[sourceGroupID], s.kgs[sourceGroupID] + if splitSourceAM == nil || splitSourceGroup == nil { + log.Error("the split source keyspace group is not initialized", + zap.Uint32("source", sourceGroupID)) + return nil, nil, errs.ErrKeyspaceGroupNotInitialized.FastGenByArgs(sourceGroupID) + } + return splitTargetAM, splitSourceAM, nil +} + +// Reject any request if the keyspace group is in merging state, +// we need to wait for the merging checker to finish the TSO merging. +func (s *state) checkTSOMerge( + groupID uint32, +) error { + s.RLock() + defer s.RUnlock() + if s.kgs[groupID] == nil || !s.kgs[groupID].IsMerging() { + return nil + } + return errs.ErrKeyspaceGroupIsMerging.FastGenByArgs(groupID) +} + // getKeyspaceGroupMetaWithCheck returns the keyspace group meta of the given keyspace. // It also checks if the keyspace is served by the given keyspace group. If not, it returns the meta // of the keyspace group to which the keyspace currently belongs and returns NotServed (by the given @@ -148,6 +192,41 @@ func (s *state) getKeyspaceGroupMetaWithCheck( mcsutils.DefaultKeyspaceGroupID, nil } +func (s *state) getNextPrimaryToReset( + groupID int, localAddress string, +) (member ElectionMember, kg *endpoint.KeyspaceGroup, localPriority, nextGroupID int) { + s.RLock() + defer s.RUnlock() + + // Both s.ams and s.kgs are arrays with the fixed size defined by the const value MaxKeyspaceGroupCountInUse. + groupSize := int(mcsutils.MaxKeyspaceGroupCountInUse) + groupID %= groupSize + for j := 0; j < groupSize; groupID, j = (groupID+1)%groupSize, j+1 { + am := s.ams[groupID] + kg := s.kgs[groupID] + if am != nil && kg != nil && am.GetMember().IsLeader() { + maxPriority := math.MinInt32 + localPriority := math.MaxInt32 + for _, member := range kg.Members { + if member.Priority > maxPriority { + maxPriority = member.Priority + } + if member.Address == localAddress { + localPriority = member.Priority + } + } + + if localPriority < maxPriority { + // return here and reset the primary outside of the critical section + // as resetting the primary may take some time. + return am.GetMember(), kg, localPriority, (groupID + 1) % groupSize + } + } + } + + 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". @@ -193,6 +272,10 @@ type KeyspaceGroupManager struct { // which participate in the election of its keyspace group's primary, in the format of // "electionNamePrefix:keyspace-group-id" electionNamePrefix string + // tsoServiceKey is the path for storing the registered tso servers. + // Key: /ms/{cluster_id}/tso/registry/{tsoServerAddress} + // Value: discover.ServiceRegistryEntry + tsoServiceKey string // legacySvcRootPath defines the legacy root path for all etcd paths which derives from // the PD/API service. It's in the format of "/pd/{cluster_id}". // The main paths for different usages include: @@ -233,14 +316,26 @@ type KeyspaceGroupManager struct { loadKeyspaceGroupsBatchSize int64 loadFromEtcdMaxRetryTimes int - // compiledKGMembershipIDRegexp is the compiled regular expression for matching keyspace group id in the - // keyspace group membership path. + // compiledKGMembershipIDRegexp is the compiled regular expression for matching keyspace group id + // in the keyspace group membership path. compiledKGMembershipIDRegexp *regexp.Regexp // groupUpdateRetryList is the list of keyspace groups which failed to update and need to retry. groupUpdateRetryList map[uint32]*endpoint.KeyspaceGroup groupWatcher *etcdutil.LoopWatcher - primaryPathBuilder *kgPrimaryPathBuilder + // 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. + tsoNodes sync.Map // store as map[string]struct{} + // serviceRegistryMap stores the mapping from the service registry key to the service address. + // Note: it is only used in tsoNodesWatcher. + serviceRegistryMap map[string]string + // tsoNodesWatcher is the watcher for the registered tso servers. + tsoNodesWatcher *etcdutil.LoopWatcher } // NewKeyspaceGroupManager creates a new Keyspace Group Manager. @@ -250,6 +345,7 @@ func NewKeyspaceGroupManager( etcdClient *clientv3.Client, httpClient *http.Client, electionNamePrefix string, + tsoServiceKey string, legacySvcRootPath string, tsoSvcRootPath string, cfg ServiceConfig, @@ -262,16 +358,19 @@ func NewKeyspaceGroupManager( ctx, cancel := context.WithCancel(ctx) kgm := &KeyspaceGroupManager{ - ctx: ctx, - cancel: cancel, - tsoServiceID: tsoServiceID, - etcdClient: etcdClient, - httpClient: httpClient, - electionNamePrefix: electionNamePrefix, - legacySvcRootPath: legacySvcRootPath, - tsoSvcRootPath: tsoSvcRootPath, - cfg: cfg, - groupUpdateRetryList: make(map[uint32]*endpoint.KeyspaceGroup), + ctx: ctx, + cancel: cancel, + tsoServiceID: tsoServiceID, + etcdClient: etcdClient, + httpClient: httpClient, + electionNamePrefix: electionNamePrefix, + tsoServiceKey: tsoServiceKey, + legacySvcRootPath: legacySvcRootPath, + tsoSvcRootPath: tsoSvcRootPath, + primaryPriorityCheckInterval: defaultPrimaryPriorityCheckInterval, + cfg: cfg, + groupUpdateRetryList: make(map[uint32]*endpoint.KeyspaceGroup), + serviceRegistryMap: make(map[string]string), } kgm.legacySvcStorage = endpoint.NewStorageEndpoint( kv.NewEtcdKVBase(kgm.etcdClient, kgm.legacySvcRootPath), nil) @@ -288,6 +387,100 @@ func NewKeyspaceGroupManager( // Initialize this KeyspaceGroupManager func (kgm *KeyspaceGroupManager) Initialize() error { + if err := kgm.InitializeTSOServerWatchLoop(); err != nil { + log.Error("failed to initialize tso server watch loop", zap.Error(err)) + kgm.Close() // Close the manager to clean up the allocated resources. + return errs.ErrLoadKeyspaceGroupsTerminated.Wrap(err) + } + if err := kgm.InitializeGroupWatchLoop(); err != nil { + log.Error("failed to initialize group watch loop", zap.Error(err)) + kgm.Close() // Close the manager to clean up the loaded keyspace groups. + return errs.ErrLoadKeyspaceGroupsTerminated.Wrap(err) + } + + kgm.wg.Add(1) + go kgm.primaryPriorityCheckLoop() + + return nil +} + +// Close this KeyspaceGroupManager +func (kgm *KeyspaceGroupManager) Close() { + log.Info("closing keyspace group manager") + + // Note: don't change the order. We need to cancel all service loops in the keyspace group manager + // before closing all keyspace groups. It's to prevent concurrent addition/removal of keyspace groups + // during critical periods such as service shutdown and online keyspace group, while the former requires + // snapshot isolation to ensure all keyspace groups are properly closed and no new keyspace group is + // added/initialized after that. + kgm.cancel() + kgm.wg.Wait() + kgm.state.deinitialize() + + log.Info("keyspace group manager closed") +} + +// GetServiceConfig returns the service config. +func (kgm *KeyspaceGroupManager) GetServiceConfig() ServiceConfig { + return kgm.cfg +} + +// InitializeTSOServerWatchLoop initializes the watch loop monitoring the path for storing the +// registered tso servers. +// Key: /ms/{cluster_id}/tso/registry/{tsoServerAddress} +// Value: discover.ServiceRegistryEntry +func (kgm *KeyspaceGroupManager) InitializeTSOServerWatchLoop() error { + tsoServiceEndKey := clientv3.GetPrefixRangeEnd(kgm.tsoServiceKey) + "/" + + putFn := func(kv *mvccpb.KeyValue) error { + s := &discovery.ServiceRegistryEntry{} + if err := json.Unmarshal(kv.Value, s); err != nil { + log.Warn("failed to unmarshal service registry entry", + zap.String("event-kv-key", string(kv.Key)), zap.Error(err)) + return err + } + kgm.tsoNodes.Store(s.ServiceAddr, struct{}{}) + kgm.serviceRegistryMap[string(kv.Key)] = s.ServiceAddr + return nil + } + deleteFn := func(kv *mvccpb.KeyValue) error { + key := string(kv.Key) + if serviceAddr, ok := kgm.serviceRegistryMap[key]; ok { + delete(kgm.serviceRegistryMap, key) + kgm.tsoNodes.Delete(serviceAddr) + return nil + } + return perrors.Errorf("failed to find the service address for key %s", key) + } + + kgm.tsoNodesWatcher = etcdutil.NewLoopWatcher( + kgm.ctx, + &kgm.wg, + kgm.etcdClient, + "tso-nodes-watcher", + kgm.tsoServiceKey, + putFn, + deleteFn, + func() error { return nil }, + clientv3.WithRange(tsoServiceEndKey), + ) + + kgm.wg.Add(1) + go kgm.tsoNodesWatcher.StartWatchLoop() + + if err := kgm.tsoNodesWatcher.WaitLoad(); err != nil { + log.Error("failed to load the registered tso servers", errs.ZapError(err)) + return err + } + + return nil +} + +// InitializeGroupWatchLoop initializes the watch loop monitoring the path for storing keyspace group +// membership/distribution metadata. +// Key: /pd/{cluster_id}/tso/keyspace_groups/membership/{group} +// Value: endpoint.KeyspaceGroup +func (kgm *KeyspaceGroupManager) InitializeGroupWatchLoop() error { rootPath := kgm.legacySvcRootPath startKey := strings.Join([]string{rootPath, endpoint.KeyspaceGroupIDPath(mcsutils.DefaultKeyspaceGroupID)}, "/") endKey := strings.Join( @@ -367,29 +560,72 @@ func (kgm *KeyspaceGroupManager) Initialize() error { return nil } -// Close this KeyspaceGroupManager -func (kgm *KeyspaceGroupManager) Close() { - log.Info("closing keyspace group manager") +func (kgm *KeyspaceGroupManager) primaryPriorityCheckLoop() { + defer logutil.LogPanic() + defer kgm.wg.Done() - // Note: don't change the order. We need to cancel all service loops in the keyspace group manager - // before closing all keyspace groups. It's to prevent concurrent addition/removal of keyspace groups - // during critical periods such as service shutdown and online keyspace group, while the former requires - // snapshot isolation to ensure all keyspace groups are properly closed and no new keyspace group is - // added/initialized after that. - kgm.cancel() - kgm.wg.Wait() - kgm.state.deinitialize() + failpoint.Inject("fastPrimaryPriorityCheck", func() { + kgm.primaryPriorityCheckInterval = 200 * time.Millisecond + }) - log.Info("keyspace group manager closed") + ctx, cancel := context.WithCancel(kgm.ctx) + defer cancel() + groupID := 0 + for { + select { + case <-ctx.Done(): + log.Info("exit primary priority check loop") + return + case <-time.After(kgm.primaryPriorityCheckInterval): + // Every primaryPriorityCheckInterval, we only reset the primary of one keyspace group + member, kg, localPriority, nextGroupID := kgm.getNextPrimaryToReset(groupID, kgm.tsoServiceID.ServiceAddr) + if member != nil { + aliveTSONodes := make(map[string]struct{}) + kgm.tsoNodes.Range(func(key, _ interface{}) bool { + aliveTSONodes[key.(string)] = struct{}{} + return true + }) + if len(aliveTSONodes) == 0 { + log.Warn("no alive tso node", zap.String("local-address", kgm.tsoServiceID.ServiceAddr)) + continue + } + // If there is a alive member with higher priority, reset the leader. + resetLeader := false + for _, member := range kg.Members { + if member.Priority <= localPriority { + continue + } + if _, ok := aliveTSONodes[member.Address]; ok { + resetLeader = true + break + } + } + if resetLeader { + select { + case <-ctx.Done(): + default: + member.ResetLeader() + log.Info("reset primary", + zap.String("local-address", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("keyspace-group-id", kg.ID), + zap.Int("local-priority", localPriority)) + } + } else { + log.Warn("no need to reset primary as the replicas with higher priority are offline", + zap.String("local-address", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("keyspace-group-id", kg.ID), + zap.Int("local-priority", localPriority)) + } + } + groupID = nextGroupID + } + } } func (kgm *KeyspaceGroupManager) isAssignedToMe(group *endpoint.KeyspaceGroup) bool { - for _, member := range group.Members { - if member.Address == kgm.tsoServiceID.ServiceAddr { - return true - } - } - return false + return slice.AnyOf(group.Members, func(i int) bool { + return group.Members[i].Address == kgm.tsoServiceID.ServiceAddr + }) } // updateKeyspaceGroup applies the given keyspace group. If the keyspace group is just assigned to @@ -416,9 +652,25 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroup(group *endpoint.KeyspaceGro return } + oldAM, oldGroup := kgm.getKeyspaceGroupMeta(group.ID) + // If this host owns a replica of the keyspace group which is the merge target, + // it should run the merging checker when the merge state first time changes. + if !oldGroup.IsMergeTarget() && group.IsMergeTarget() { + ctx, cancel := context.WithCancel(kgm.ctx) + kgm.mergeCheckerCancelMap.Store(group.ID, cancel) + kgm.wg.Add(1) + go kgm.mergingChecker(ctx, group.ID, group.MergeState.MergeList) + } + // If the merge state has been finished, cancel its merging checker. + if oldGroup.IsMergeTarget() && !group.IsMergeTarget() { + if cancel, loaded := kgm.mergeCheckerCancelMap.LoadAndDelete(group.ID); loaded && cancel != nil { + cancel.(context.CancelFunc)() + } + } + // If this host is already assigned a replica of this keyspace group, i.e., the election member // is already initialized, just update the meta. - if oldAM, oldGroup := kgm.getKeyspaceGroupMeta(group.ID); oldAM != nil { + if oldAM != nil { kgm.updateKeyspaceGroupMembership(oldGroup, group, true) return } @@ -738,6 +990,10 @@ func (kgm *KeyspaceGroupManager) HandleTSORequest( if err != nil { return pdpb.Timestamp{}, curKeyspaceGroupID, err } + err = kgm.state.checkTSOMerge(curKeyspaceGroupID) + if err != nil { + return pdpb.Timestamp{}, curKeyspaceGroupID, err + } ts, err = am.HandleRequest(dcLocation, count) return ts, curKeyspaceGroupID, err } @@ -809,19 +1065,11 @@ func (kgm *KeyspaceGroupManager) checkTSOSplit( keyspaceGroupID uint32, dcLocation string, ) error { - splitAM, splitGroup := kgm.getKeyspaceGroupMeta(keyspaceGroupID) - // Only the split target keyspace group needs to check the TSO split. - if !splitGroup.IsSplitTarget() { - return nil - } - splitSource := splitGroup.SplitSource() - splitSourceAM, splitSourceGroup := kgm.getKeyspaceGroupMeta(splitSource) - if splitSourceAM == nil || splitSourceGroup == nil { - log.Error("the split source keyspace group is not initialized", - zap.Uint32("source", splitSource)) - return errs.ErrKeyspaceGroupNotInitialized.FastGenByArgs(splitSource) + splitTargetAM, splitSourceAM, err := kgm.state.checkTSOSplit(keyspaceGroupID) + if err != nil || splitTargetAM == nil { + return err } - splitAllocator, err := splitAM.GetAllocator(dcLocation) + splitTargetAllocator, err := splitTargetAM.GetAllocator(dcLocation) if err != nil { return err } @@ -829,7 +1077,7 @@ func (kgm *KeyspaceGroupManager) checkTSOSplit( if err != nil { return err } - splitTSO, err := splitAllocator.GenerateTSO(1) + splitTargetTSO, err := splitTargetAllocator.GenerateTSO(1) if err != nil { return err } @@ -837,23 +1085,31 @@ func (kgm *KeyspaceGroupManager) checkTSOSplit( if err != nil { return err } - if tsoutil.CompareTimestamp(&splitSourceTSO, &splitTSO) <= 0 { - log.Debug("the split source TSO is not greater than the newly split TSO", + // If the split source TSO is not greater than the newly split TSO, we don't need to do anything. + if tsoutil.CompareTimestamp(&splitSourceTSO, &splitTargetTSO) <= 0 { + log.Info("the split source tso is less than the newly split tso", zap.Int64("split-source-tso-physical", splitSourceTSO.Physical), zap.Int64("split-source-tso-logical", splitSourceTSO.Logical), - zap.Int64("split-tso-physical", splitTSO.Physical), - zap.Int64("split-tso-logical", splitTSO.Logical), - ) - return nil + zap.Int64("split-tso-physical", splitTargetTSO.Physical), + zap.Int64("split-tso-logical", splitTargetTSO.Logical)) + // Finish the split state directly. + return kgm.finishSplitKeyspaceGroup(keyspaceGroupID) } // If the split source TSO is greater than the newly split TSO, we need to update the split // TSO to make sure the following TSO will be greater than the split keyspaces ever had // in the past. - splitSourceTSO.Physical += 1 - err = splitAllocator.SetTSO(tsoutil.GenerateTS(&splitSourceTSO), true, true) + err = splitTargetAllocator.SetTSO(tsoutil.GenerateTS(&pdpb.Timestamp{ + Physical: splitSourceTSO.Physical + 1, + Logical: splitSourceTSO.Logical, + }), true, true) if err != nil { return err } + log.Info("the split source tso is greater than the newly split tso", + zap.Int64("split-source-tso-physical", splitSourceTSO.Physical), + zap.Int64("split-source-tso-logical", splitSourceTSO.Logical), + zap.Int64("split-tso-physical", splitTargetTSO.Physical), + zap.Int64("split-tso-logical", splitTargetTSO.Logical)) // Finish the split state. return kgm.finishSplitKeyspaceGroup(keyspaceGroupID) } @@ -885,8 +1141,182 @@ func (kgm *KeyspaceGroupManager) finishSplitKeyspaceGroup(id uint32) error { zap.Int("status-code", statusCode)) return errs.ErrSendRequest.FastGenByArgs() } - // Pre-update the split keyspace group split state in memory. - splitGroup.SplitState = nil - kgm.kgs[id] = splitGroup + // Pre-update the split keyspace group's split state in memory. + // Note: to avoid data race with state read APIs, we always replace the group in memory as a whole. + // For now, we only have scenarios to update split state/merge state, and the other fields are always + // loaded from etcd without any modification, so we can simply copy the group and replace the state. + newSplitGroup := *splitGroup + newSplitGroup.SplitState = nil + kgm.kgs[id] = &newSplitGroup + return nil +} + +func (kgm *KeyspaceGroupManager) finishMergeKeyspaceGroup(id uint32) error { + kgm.Lock() + defer kgm.Unlock() + // Check if the keyspace group is in the merging state. + mergeTarget := kgm.kgs[id] + if !mergeTarget.IsMergeTarget() { + return nil + } + // Check if the HTTP client is initialized. + if kgm.httpClient == nil { + return nil + } + statusCode, err := apiutil.DoDelete( + kgm.httpClient, + kgm.cfg.GeBackendEndpoints()+keyspaceGroupsAPIPrefix+fmt.Sprintf("/%d/merge", id)) + if err != nil { + return err + } + if statusCode != http.StatusOK { + log.Warn("failed to finish merging keyspace group", + zap.Uint32("keyspace-group-id", id), + zap.Int("status-code", statusCode)) + return errs.ErrSendRequest.FastGenByArgs() + } + + // Pre-update the merge target keyspace group's merge state in memory. + // Note: to avoid data race with state read APIs, we always replace the group in memory as a whole. + // For now, we only have scenarios to update split state/merge state, and the other fields are always + // loaded from etcd without any modification, so we can simply copy the group and replace the state. + newTargetGroup := *mergeTarget + newTargetGroup.MergeState = nil + kgm.kgs[id] = &newTargetGroup return nil } + +// mergingChecker is used to check if the keyspace group is in merge state, and if so, it will +// make sure the newly merged TSO keep consistent with the original ones. +func (kgm *KeyspaceGroupManager) mergingChecker(ctx context.Context, mergeTargetID uint32, mergeList []uint32) { + log.Info("start to merge the keyspace group", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("merge-target-id", mergeTargetID), + zap.Any("merge-list", mergeList)) + defer logutil.LogPanic() + defer kgm.wg.Done() + + checkTicker := time.NewTicker(mergingCheckInterval) + defer checkTicker.Stop() + // Prepare the merge map. + mergeMap := make(map[uint32]struct{}, len(mergeList)) + for _, id := range mergeList { + mergeMap[id] = struct{}{} + } + + for { + select { + case <-ctx.Done(): + log.Info("merging checker is closed", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("merge-target-id", mergeTargetID), + zap.Any("merge-list", mergeList)) + return + case <-checkTicker.C: + } + // Check if current TSO node is the merge target TSO primary node. + am, err := kgm.GetAllocatorManager(mergeTargetID) + if err != nil { + log.Warn("unable to get the merge target allocator manager", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("keyspace-group-id", mergeTargetID), + zap.Any("merge-list", mergeList), + zap.Error(err)) + continue + } + // If the current TSO node is not the merge target TSO primary node, + // we still need to keep this loop running to avoid unexpected primary changes. + if !am.IsLeader() { + log.Debug("current tso node is not the merge target primary", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("merge-target-id", mergeTargetID), + zap.Any("merge-list", mergeList)) + continue + } + // 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) + 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", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("merge-target-id", mergeTargetID), + zap.Any("merge-list", mergeList), + zap.Uint32("merge-id", id), + zap.Any("remaining", mergeMap), + zap.Error(err)) + continue + } + if len(val) == 0 { + delete(mergeMap, id) + } + } + } + if len(mergeMap) > 0 { + continue + } + // 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. + var mergedTS time.Time + for _, id := range mergeList { + ts, err := kgm.tsoSvcStorage.LoadTimestamp(am.getKeyspaceGroupTSPath(id)) + if err != nil || ts == typeutil.ZeroTime { + log.Error("failed to load the keyspace group TSO", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("merge-target-id", mergeTargetID), + zap.Any("merge-list", mergeList), + 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", + 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) + if err != nil { + log.Error("failed to finish the merge", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("merge-target-id", mergeTargetID), + zap.Any("merge-list", mergeList), + zap.Error(err)) + continue + } + log.Info("finished merging keyspace group", + zap.String("member", kgm.tsoServiceID.ServiceAddr), + zap.Uint32("merge-target-id", mergeTargetID), + zap.Any("merge-list", mergeList), + zap.Time("merged-ts", mergedTS)) + return + } +} diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index 1e7d072ade3..2e03418bae7 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -22,7 +22,6 @@ import ( "path" "reflect" "sort" - "strconv" "strings" "sync" "testing" @@ -36,8 +35,9 @@ import ( "github.com/tikv/pd/pkg/mcs/discovery" mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/storage/endpoint" - "github.com/tikv/pd/pkg/utils/memberutil" + "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" + "github.com/tikv/pd/pkg/utils/tsoutil" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/goleak" @@ -51,6 +51,7 @@ type keyspaceGroupManagerTestSuite struct { suite.Suite ctx context.Context cancel context.CancelFunc + ClusterID uint64 backendEndpoints string etcdClient *clientv3.Client clean func() @@ -64,13 +65,23 @@ func TestKeyspaceGroupManagerTestSuite(t *testing.T) { func (suite *keyspaceGroupManagerTestSuite) SetupSuite() { t := suite.T() suite.ctx, suite.cancel = context.WithCancel(context.Background()) + suite.ClusterID = rand.Uint64() suite.backendEndpoints, suite.etcdClient, suite.clean = startEmbeddedEtcd(t) + suite.cfg = suite.createConfig() +} + +func (suite *keyspaceGroupManagerTestSuite) TearDownSuite() { + suite.clean() + suite.cancel() +} - suite.cfg = &TestServiceConfig{ - Name: "tso-test-name", +func (suite *keyspaceGroupManagerTestSuite) createConfig() *TestServiceConfig { + addr := tempurl.Alloc() + return &TestServiceConfig{ + Name: "tso-test-name-default", BackendEndpoints: suite.backendEndpoints, - ListenAddr: "http://127.0.0.1:3379", - AdvertiseListenAddr: "http://127.0.0.1:3379", + ListenAddr: addr, + AdvertiseListenAddr: addr, LeaderLease: mcsutils.DefaultLeaderLease, LocalTSOEnabled: false, TSOUpdatePhysicalInterval: 50 * time.Millisecond, @@ -80,11 +91,6 @@ func (suite *keyspaceGroupManagerTestSuite) SetupSuite() { } } -func (suite *keyspaceGroupManagerTestSuite) TearDownSuite() { - suite.clean() - suite.cancel() -} - // TestNewKeyspaceGroupManager tests the initialization of KeyspaceGroupManager. // It should initialize the allocator manager with the desired configurations and parameters. func (suite *keyspaceGroupManagerTestSuite) TestNewKeyspaceGroupManager() { @@ -92,11 +98,15 @@ func (suite *keyspaceGroupManagerTestSuite) TestNewKeyspaceGroupManager() { tsoServiceID := &discovery.ServiceRegistryEntry{ServiceAddr: suite.cfg.AdvertiseListenAddr} guid := uuid.New().String() + tsoServiceKey := discovery.ServicePath(guid, "tso") + "/" legacySvcRootPath := path.Join("/pd", guid) tsoSvcRootPath := path.Join("/ms", guid, "tso") electionNamePrefix := "tso-server-" + guid - kgm := suite.newKeyspaceGroupManager(tsoServiceID, electionNamePrefix, legacySvcRootPath, tsoSvcRootPath) + kgm := NewKeyspaceGroupManager( + suite.ctx, tsoServiceID, suite.etcdClient, nil, electionNamePrefix, + tsoServiceKey, legacySvcRootPath, tsoSvcRootPath, suite.cfg) + defer kgm.Close() err := kgm.Initialize() re.NoError(err) @@ -116,8 +126,6 @@ func (suite *keyspaceGroupManagerTestSuite) TestNewKeyspaceGroupManager() { re.Equal(legacySvcRootPath, am.rootPath) re.Equal(time.Duration(mcsutils.DefaultLeaderLease)*time.Second, am.saveInterval) re.Equal(time.Duration(50)*time.Millisecond, am.updatePhysicalInterval) - - kgm.Close() } // TestLoadKeyspaceGroupsAssignment tests the loading of the keyspace group assignment. @@ -174,8 +182,8 @@ func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsTimeout() { defer mgr.Close() addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, uint32(0), []uint32{0}) + suite.ctx, suite.etcdClient, uint32(0), mgr.legacySvcRootPath, + []string{mgr.tsoServiceID.ServiceAddr}, []int{0}, []uint32{0}) // Set the timeout to 1 second and inject the delayLoad to return 3 seconds to let // the loading sleep 3 seconds. @@ -197,8 +205,8 @@ func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsSucceedWithTem defer mgr.Close() addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, uint32(0), []uint32{0}) + suite.ctx, suite.etcdClient, uint32(0), mgr.legacySvcRootPath, + []string{mgr.tsoServiceID.ServiceAddr}, []int{0}, []uint32{0}) // Set the max retry times to 3 and inject the loadTemporaryFail to return 2 to let // loading from etcd fail 2 times but the whole initialization still succeeds. @@ -219,8 +227,8 @@ func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsFailed() { defer mgr.Close() addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, uint32(0), []uint32{0}) + suite.ctx, suite.etcdClient, uint32(0), mgr.legacySvcRootPath, + []string{mgr.tsoServiceID.ServiceAddr}, []int{0}, []uint32{0}) // Set the max retry times to 3 and inject the loadTemporaryFail to return 3 to let // loading from etcd fail 3 times which should cause the whole initialization to fail. @@ -388,9 +396,8 @@ func (suite *keyspaceGroupManagerTestSuite) TestGetKeyspaceGroupMetaWithCheck() // Create keyspace group 0 which contains keyspace 0, 1, 2. addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, - uint32(0), []uint32{0, 1, 2}) + suite.ctx, suite.etcdClient, uint32(0), mgr.legacySvcRootPath, + []string{mgr.tsoServiceID.ServiceAddr}, []int{0}, []uint32{0, 1, 2}) err = mgr.Initialize() re.NoError(err) @@ -461,14 +468,12 @@ func (suite *keyspaceGroupManagerTestSuite) TestDefaultMembershipRestriction() { // Create keyspace group 0 which contains keyspace 0, 1, 2. addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, - mcsutils.DefaultKeyspaceGroupID, []uint32{mcsutils.DefaultKeyspaceID, 1, 2}) + suite.ctx, suite.etcdClient, mcsutils.DefaultKeyspaceGroupID, rootPath, + []string{svcAddr}, []int{0}, []uint32{mcsutils.DefaultKeyspaceID, 1, 2}) // Create keyspace group 3 which contains keyspace 3, 4. addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, - uint32(3), []uint32{3, 4}) + suite.ctx, suite.etcdClient, uint32(3), mgr.legacySvcRootPath, + []string{mgr.tsoServiceID.ServiceAddr}, []int{0}, []uint32{3, 4}) err = mgr.Initialize() re.NoError(err) @@ -536,14 +541,12 @@ func (suite *keyspaceGroupManagerTestSuite) TestKeyspaceMovementConsistency() { // Create keyspace group 0 which contains keyspace 0, 1, 2. addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, - mcsutils.DefaultKeyspaceGroupID, []uint32{mcsutils.DefaultKeyspaceID, 10, 20}) + suite.ctx, suite.etcdClient, mcsutils.DefaultKeyspaceGroupID, + rootPath, []string{svcAddr}, []int{0}, []uint32{mcsutils.DefaultKeyspaceID, 10, 20}) // Create keyspace group 1 which contains keyspace 3, 4. addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, - uint32(1), []uint32{11, 21}) + suite.ctx, suite.etcdClient, uint32(1), rootPath, + []string{svcAddr}, []int{0}, []uint32{11, 21}) err = mgr.Initialize() re.NoError(err) @@ -591,9 +594,8 @@ func (suite *keyspaceGroupManagerTestSuite) TestHandleTSORequestWithWrongMembers // Create keyspace group 0 which contains keyspace 0, 1, 2. addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, - uint32(0), []uint32{0, 1, 2}) + suite.ctx, suite.etcdClient, uint32(0), mgr.legacySvcRootPath, + []string{mgr.tsoServiceID.ServiceAddr}, []int{0}, []uint32{0, 1, 2}) err := mgr.Initialize() re.NoError(err) @@ -681,15 +683,6 @@ func (suite *keyspaceGroupManagerTestSuite) applyEtcdEvents( } } -func (suite *keyspaceGroupManagerTestSuite) newKeyspaceGroupManager( - tsoServiceID *discovery.ServiceRegistryEntry, - electionNamePrefix, legacySvcRootPath, tsoSvcRootPath string, -) *KeyspaceGroupManager { - return NewKeyspaceGroupManager( - suite.ctx, tsoServiceID, suite.etcdClient, nil, electionNamePrefix, - legacySvcRootPath, tsoSvcRootPath, suite.cfg) -} - // runTestLoadMultipleKeyspaceGroupsAssignment tests the loading of multiple keyspace group assignment. func (suite *keyspaceGroupManagerTestSuite) runTestLoadKeyspaceGroupsAssignment( re *require.Assertions, @@ -727,10 +720,16 @@ func (suite *keyspaceGroupManagerTestSuite) runTestLoadKeyspaceGroupsAssignment( expectedGroupIDs = append(expectedGroupIDs, uint32(j)) mux.Unlock() } + + svcAddrs := make([]string, 0) + if assignToMe { + svcAddrs = append(svcAddrs, mgr.tsoServiceID.ServiceAddr) + } else { + svcAddrs = append(svcAddrs, uuid.NewString()) + } addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, - assignToMe, mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, - uint32(j), []uint32{uint32(j)}) + suite.ctx, suite.etcdClient, uint32(j), mgr.legacySvcRootPath, + svcAddrs, []int{0}, []uint32{uint32(j)}) } }(i) } @@ -756,19 +755,27 @@ func (suite *keyspaceGroupManagerTestSuite) runTestLoadKeyspaceGroupsAssignment( func (suite *keyspaceGroupManagerTestSuite) newUniqueKeyspaceGroupManager( loadKeyspaceGroupsBatchSize int64, // set to 0 to use the default value ) *KeyspaceGroupManager { - tsoServiceID := &discovery.ServiceRegistryEntry{ServiceAddr: suite.cfg.AdvertiseListenAddr} - uniqueID := memberutil.GenerateUniqueID(uuid.New().String()) - uniqueStr := strconv.FormatUint(uniqueID, 10) + return suite.newKeyspaceGroupManager(loadKeyspaceGroupsBatchSize, uuid.New().String(), suite.cfg) +} + +func (suite *keyspaceGroupManagerTestSuite) newKeyspaceGroupManager( + loadKeyspaceGroupsBatchSize int64, // set to 0 to use the default value + uniqueStr string, + cfg *TestServiceConfig, +) *KeyspaceGroupManager { + tsoServiceID := &discovery.ServiceRegistryEntry{ServiceAddr: cfg.GetAdvertiseListenAddr()} + tsoServiceKey := discovery.ServicePath(uniqueStr, "tso") + "/" legacySvcRootPath := path.Join("/pd", uniqueStr) tsoSvcRootPath := path.Join("/ms", uniqueStr, "tso") - electionNamePrefix := "kgm-test-" + uniqueStr - - keyspaceGroupManager := suite.newKeyspaceGroupManager(tsoServiceID, electionNamePrefix, legacySvcRootPath, tsoSvcRootPath) + electionNamePrefix := "kgm-test-" + cfg.GetAdvertiseListenAddr() + kgm := NewKeyspaceGroupManager( + suite.ctx, tsoServiceID, suite.etcdClient, nil, electionNamePrefix, + tsoServiceKey, legacySvcRootPath, tsoSvcRootPath, cfg) if loadKeyspaceGroupsBatchSize != 0 { - keyspaceGroupManager.loadKeyspaceGroupsBatchSize = loadKeyspaceGroupsBatchSize + kgm.loadKeyspaceGroupsBatchSize = loadKeyspaceGroupsBatchSize } - return keyspaceGroupManager + return kgm } // putKeyspaceGroupToEtcd puts a keyspace group to etcd. @@ -805,19 +812,21 @@ func deleteKeyspaceGroupInEtcd( // addKeyspaceGroupAssignment adds a keyspace group assignment to etcd. func addKeyspaceGroupAssignment( - ctx context.Context, etcdClient *clientv3.Client, - assignToMe bool, rootPath, svcAddr string, - groupID uint32, keyspaces []uint32, + ctx context.Context, + etcdClient *clientv3.Client, + groupID uint32, + rootPath string, + svcAddrs []string, + priorites []int, + keyspaces []uint32, ) error { - var location string - if assignToMe { - location = svcAddr - } else { - location = uuid.NewString() + members := make([]endpoint.KeyspaceGroupMember, len(svcAddrs)) + for i, svcAddr := range svcAddrs { + members[i] = endpoint.KeyspaceGroupMember{Address: svcAddr, Priority: priorites[i]} } group := &endpoint.KeyspaceGroup{ ID: groupID, - Members: []endpoint.KeyspaceGroupMember{{Address: location}}, + Members: members, Keyspaces: keyspaces, } @@ -990,3 +999,185 @@ func (suite *keyspaceGroupManagerTestSuite) TestGroupSplitUpdateRetry() { return reflect.DeepEqual(expectedGroupIDs, assignedGroupIDs) }) } + +// TestPrimaryPriorityChange tests the case that the primary priority of a keyspace group changes +// and the locations of the primaries should be updated accordingly. +func (suite *keyspaceGroupManagerTestSuite) TestPrimaryPriorityChange() { + re := suite.Require() + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastPrimaryPriorityCheck", `return(true)`)) + defer func() { + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastPrimaryPriorityCheck")) + }() + + var err error + defaultPriority := mcsutils.DefaultKeyspaceGroupReplicaPriority + uniqueStr := uuid.New().String() + rootPath := path.Join("/pd", uniqueStr) + cfg1 := suite.createConfig() + cfg2 := suite.createConfig() + svcAddr1 := cfg1.GetAdvertiseListenAddr() + svcAddr2 := cfg2.GetAdvertiseListenAddr() + + // Register TSO server 1 + err = suite.registerTSOServer(re, uniqueStr, svcAddr1, cfg1) + re.NoError(err) + defer func() { + re.NoError(suite.deregisterTSOServer(uniqueStr, svcAddr1)) + }() + + // Create three keyspace groups on two TSO servers with default replica priority. + ids := []uint32{0, mcsutils.MaxKeyspaceGroupCountInUse / 2, mcsutils.MaxKeyspaceGroupCountInUse - 1} + for _, id := range ids { + addKeyspaceGroupAssignment( + suite.ctx, suite.etcdClient, id, rootPath, + []string{svcAddr1, svcAddr2}, []int{defaultPriority, defaultPriority}, []uint32{id}) + } + + // Create the first TSO server which loads all three keyspace groups created above. + // All primaries should be on the first TSO server. + mgr1 := suite.newKeyspaceGroupManager(1, uniqueStr, cfg1) + re.NotNil(mgr1) + defer mgr1.Close() + err = mgr1.Initialize() + re.NoError(err) + // Wait until all keyspace groups are ready for serving tso requests. + waitForPrimariesServing(re, []*KeyspaceGroupManager{mgr1, mgr1, mgr1}, ids) + + // We increase the priority of the TSO server 2 which hasn't started yet. The primaries + // on the TSO server 1 shouldn't move. + for _, id := range ids { + addKeyspaceGroupAssignment( + suite.ctx, suite.etcdClient, id, rootPath, + []string{svcAddr1, svcAddr2}, []int{defaultPriority, defaultPriority + 1}, []uint32{id}) + } + + // And the primaries on TSO Server 1 should continue to serve TSO requests without any failures. + for i := 0; i < 100; i++ { + for _, id := range ids { + _, keyspaceGroupBelongTo, err := mgr1.HandleTSORequest(id, id, GlobalDCLocation, 1) + re.NoError(err) + re.Equal(id, keyspaceGroupBelongTo) + } + } + + // Continually sending TSO requests to the TSO server 1 to make sure the primaries will move back + // to it at the end of test + ctx, cancel := context.WithCancel(context.Background()) + var wg sync.WaitGroup + checkTSO(ctx, re, &wg, mgr1, ids) + + // Create the Second TSO server. + err = suite.registerTSOServer(re, uniqueStr, svcAddr2, cfg2) + re.NoError(err) + mgr2 := suite.newKeyspaceGroupManager(1, uniqueStr, cfg2) + re.NotNil(mgr2) + err = mgr2.Initialize() + re.NoError(err) + // All primaries should eventually move to the second TSO server because of the higher priority. + waitForPrimariesServing(re, []*KeyspaceGroupManager{mgr2, mgr2, mgr2}, ids) + + // Shutdown the second TSO server. + mgr2.Close() + re.NoError(suite.deregisterTSOServer(uniqueStr, svcAddr2)) + // The primaries should move back to the first TSO server. + waitForPrimariesServing(re, []*KeyspaceGroupManager{mgr1, mgr1, mgr1}, ids) + + // Restart the Second TSO server. + err = suite.registerTSOServer(re, uniqueStr, svcAddr2, cfg2) + re.NoError(err) + defer func() { + re.NoError(suite.deregisterTSOServer(uniqueStr, svcAddr2)) + }() + mgr2 = suite.newKeyspaceGroupManager(1, uniqueStr, cfg2) + re.NotNil(mgr2) + defer mgr2.Close() + err = mgr2.Initialize() + re.NoError(err) + // All primaries should eventually move to the second TSO server because of the higher priority. + waitForPrimariesServing(re, []*KeyspaceGroupManager{mgr2, mgr2, mgr2}, ids) + + mgrs := []*KeyspaceGroupManager{mgr2, mgr2, mgr2} + for i, id := range ids { + // Set the keyspace group replica on the first TSO server to have higher priority. + addKeyspaceGroupAssignment( + suite.ctx, suite.etcdClient, id, rootPath, + []string{svcAddr1, svcAddr2}, []int{defaultPriority - 1, defaultPriority - 2}, []uint32{id}) + // The primary of this keyspace group should move back to the first TSO server. + mgrs[i] = mgr1 + waitForPrimariesServing(re, mgrs, ids) + } + + cancel() + wg.Wait() +} + +// Register TSO server. +func (suite *keyspaceGroupManagerTestSuite) registerTSOServer( + re *require.Assertions, clusterID, svcAddr string, cfg *TestServiceConfig, +) error { + // Register TSO server 1 + serviceID := &discovery.ServiceRegistryEntry{ServiceAddr: cfg.GetAdvertiseListenAddr()} + serializedEntry, err := serviceID.Serialize() + re.NoError(err) + serviceKey := discovery.RegistryPath(clusterID, mcsutils.TSOServiceName, svcAddr) + _, err = suite.etcdClient.Put(suite.ctx, serviceKey, serializedEntry) + return err +} + +// Deregister TSO server. +func (suite *keyspaceGroupManagerTestSuite) deregisterTSOServer(clusterID, svcAddr string) error { + serviceKey := discovery.RegistryPath(clusterID, mcsutils.TSOServiceName, svcAddr) + if _, err := suite.etcdClient.Delete(suite.ctx, serviceKey); err != nil { + return err + } + return nil +} + +func checkTSO( + ctx context.Context, re *require.Assertions, wg *sync.WaitGroup, + mgr *KeyspaceGroupManager, ids []uint32, +) { + wg.Add(len(ids)) + for _, id := range ids { + go func(id uint32) { + defer wg.Done() + var ts, lastTS uint64 + for { + select { + case <-ctx.Done(): + // Make sure the lastTS is not empty + re.NotEmpty(lastTS) + return + default: + } + respTS, respGroupID, err := mgr.HandleTSORequest(id, id, GlobalDCLocation, 1) + // omit the error check since there are many kinds of errors during primaries movement + if err != nil { + continue + } + re.Equal(id, respGroupID) + ts = tsoutil.ComposeTS(respTS.Physical, respTS.Logical) + re.Less(lastTS, ts) + lastTS = ts + } + }(id) + } +} + +func waitForPrimariesServing( + re *require.Assertions, mgrs []*KeyspaceGroupManager, ids []uint32, +) { + testutil.Eventually(re, func() bool { + for i := 0; i < 100; i++ { + for j, id := range ids { + if member, err := mgrs[j].GetElectionMember(id, id); err != nil || !member.IsLeader() { + return false + } + if _, _, err := mgrs[j].HandleTSORequest(id, id, GlobalDCLocation, 1); err != nil { + return false + } + } + } + return true + }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) +} diff --git a/pkg/tso/local_allocator.go b/pkg/tso/local_allocator.go index 9c2867966bc..9995d5cec3f 100644 --- a/pkg/tso/local_allocator.go +++ b/pkg/tso/local_allocator.go @@ -72,7 +72,6 @@ func NewLocalTSOAllocator( leadership: leadership, timestampOracle: ×tampOracle{ client: leadership.GetClient(), - rootPath: am.rootPath, tsPath: tsPath, storage: am.storage, saveInterval: am.saveInterval, diff --git a/pkg/tso/tso.go b/pkg/tso/tso.go index aa1a424d8cd..54f0cb927be 100644 --- a/pkg/tso/tso.go +++ b/pkg/tso/tso.go @@ -60,8 +60,7 @@ type tsoObject struct { // timestampOracle is used to maintain the logic of TSO. type timestampOracle struct { - client *clientv3.Client - rootPath string + client *clientv3.Client // When tsPath is empty, it means that it is a global timestampOracle. tsPath string storage endpoint.TSOStorage diff --git a/pkg/unsaferecovery/unsafe_recovery_controller.go b/pkg/unsaferecovery/unsafe_recovery_controller.go index 5db1168c4ce..b3716d145d1 100644 --- a/pkg/unsaferecovery/unsafe_recovery_controller.go +++ b/pkg/unsaferecovery/unsafe_recovery_controller.go @@ -726,9 +726,6 @@ func (u *Controller) getFailedPeers(region *metapb.Region) []*metapb.Peer { var failedPeers []*metapb.Peer for _, peer := range region.Peers { - if peer.Role == metapb.PeerRole_Learner || peer.Role == metapb.PeerRole_DemotingVoter { - continue - } if u.isFailed(peer) { failedPeers = append(failedPeers, peer) } diff --git a/pkg/unsaferecovery/unsafe_recovery_controller_test.go b/pkg/unsaferecovery/unsafe_recovery_controller_test.go index 5cb33bd8eab..3244420af49 100644 --- a/pkg/unsaferecovery/unsafe_recovery_controller_test.go +++ b/pkg/unsaferecovery/unsafe_recovery_controller_test.go @@ -609,6 +609,48 @@ func TestAutoDetectMode(t *testing.T) { } } +// Failed learner replica store should be considered by auto-detect mode. +func TestAutoDetectWithOneLearner(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + opts := mockconfig.NewTestOptions() + cluster := mockcluster.NewCluster(ctx, opts) + coordinator := schedule.NewCoordinator(ctx, cluster, hbstream.NewTestHeartbeatStreams(ctx, cluster.ID, cluster, true)) + coordinator.Run() + for _, store := range newTestStores(1, "6.0.0") { + cluster.PutStore(store) + } + recoveryController := NewController(cluster) + re.NoError(recoveryController.RemoveFailedStores(nil, 60, true)) + + storeReport := pdpb.StoreReport{ + PeerReports: []*pdpb.PeerReport{ + { + RaftState: &raft_serverpb.RaftLocalState{LastIndex: 10, HardState: &eraftpb.HardState{Term: 1, Commit: 10}}, + RegionState: &raft_serverpb.RegionLocalState{ + Region: &metapb.Region{ + Id: 1001, + RegionEpoch: &metapb.RegionEpoch{ConfVer: 7, Version: 10}, + Peers: []*metapb.Peer{ + {Id: 11, StoreId: 1}, {Id: 12, StoreId: 2}, {Id: 13, StoreId: 3, Role: metapb.PeerRole_Learner}}}}}, + }, + } + req := newStoreHeartbeat(1, &storeReport) + req.StoreReport.Step = 1 + resp := &pdpb.StoreHeartbeatResponse{} + recoveryController.HandleStoreHeartbeat(req, resp) + hasStore3AsFailedStore := false + for _, failedStore := range resp.RecoveryPlan.ForceLeader.FailedStores { + if failedStore == 3 { + hasStore3AsFailedStore = true + break + } + } + re.True(hasStore3AsFailedStore) +} + func TestOneLearner(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) diff --git a/pkg/utils/tsoutil/tso_dispatcher.go b/pkg/utils/tsoutil/tso_dispatcher.go index 152a3996538..f9585ba5cdd 100644 --- a/pkg/utils/tsoutil/tso_dispatcher.go +++ b/pkg/utils/tsoutil/tso_dispatcher.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/timerpool" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/logutil" "go.uber.org/zap" @@ -58,7 +59,7 @@ func NewTSODispatcher(tsoProxyHandleDuration, tsoProxyBatchSize prometheus.Histo return tsoDispatcher } -// DispatchRequest is the entry point for dispatching/forwarding a tso request to the detination host +// DispatchRequest is the entry point for dispatching/forwarding a tso request to the destination host func (s *TSODispatcher) DispatchRequest( ctx context.Context, req Request, @@ -69,9 +70,9 @@ func (s *TSODispatcher) DispatchRequest( val, loaded := s.dispatchChs.LoadOrStore(req.getForwardedHost(), make(chan Request, maxMergeRequests)) reqCh := val.(chan Request) if !loaded { - tsDeadlineCh := make(chan deadline, 1) + tsDeadlineCh := make(chan *TSDeadline, 1) go s.dispatch(ctx, tsoProtoFactory, req.getForwardedHost(), req.getClientConn(), reqCh, tsDeadlineCh, doneCh, errCh, tsoPrimaryWatchers...) - go watchTSDeadline(ctx, tsDeadlineCh) + go WatchTSDeadline(ctx, tsDeadlineCh) } reqCh <- req } @@ -82,7 +83,7 @@ func (s *TSODispatcher) dispatch( forwardedHost string, clientConn *grpc.ClientConn, tsoRequestCh <-chan Request, - tsDeadlineCh chan<- deadline, + tsDeadlineCh chan<- *TSDeadline, doneCh <-chan struct{}, errCh chan<- error, tsoPrimaryWatchers ...*etcdutil.LoopWatcher) { @@ -121,11 +122,7 @@ func (s *TSODispatcher) dispatch( requests[i] = <-tsoRequestCh } done := make(chan struct{}) - dl := deadline{ - timer: time.After(DefaultTSOProxyTimeout), - done: done, - cancel: cancel, - } + dl := NewTSDeadline(DefaultTSOProxyTimeout, done, cancel) select { case tsDeadlineCh <- dl: case <-dispatcherCtx.Done(): @@ -199,13 +196,29 @@ func (s *TSODispatcher) finishRequest(requests []Request, physical, firstLogical return nil } -type deadline struct { - timer <-chan time.Time +// TSDeadline is used to watch the deadline of each tso request. +type TSDeadline struct { + timer *time.Timer done chan struct{} cancel context.CancelFunc } -func watchTSDeadline(ctx context.Context, tsDeadlineCh <-chan deadline) { +// NewTSDeadline creates a new TSDeadline. +func NewTSDeadline( + timeout time.Duration, + done chan struct{}, + cancel context.CancelFunc, +) *TSDeadline { + timer := timerpool.GlobalTimerPool.Get(timeout) + return &TSDeadline{ + timer: timer, + done: done, + cancel: cancel, + } +} + +// WatchTSDeadline watches the deadline of each tso request. +func WatchTSDeadline(ctx context.Context, tsDeadlineCh <-chan *TSDeadline) { defer logutil.LogPanic() ctx, cancel := context.WithCancel(ctx) defer cancel() @@ -213,13 +226,15 @@ func watchTSDeadline(ctx context.Context, tsDeadlineCh <-chan deadline) { select { case d := <-tsDeadlineCh: select { - case <-d.timer: + case <-d.timer.C: log.Error("tso proxy request processing is canceled due to timeout", errs.ZapError(errs.ErrProxyTSOTimeout)) 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(): @@ -230,11 +245,12 @@ func watchTSDeadline(ctx context.Context, tsDeadlineCh <-chan deadline) { func checkStream(streamCtx context.Context, cancel context.CancelFunc, done chan struct{}) { defer logutil.LogPanic() - + timer := time.NewTimer(3 * time.Second) + defer timer.Stop() select { case <-done: return - case <-time.After(3 * time.Second): + case <-timer.C: cancel() case <-streamCtx.Done(): } diff --git a/server/api/router.go b/server/api/router.go index 222e2b6f5cb..2b030237340 100644 --- a/server/api/router.go +++ b/server/api/router.go @@ -298,7 +298,6 @@ func createRouter(prefix string, svr *server.Server) *mux.Router { registerFunc(clusterRouter, "/admin/cache/region/{id}", adminHandler.DeleteRegionCache, setMethods(http.MethodDelete), setAuditBackend(localLog, prometheus)) registerFunc(clusterRouter, "/admin/cache/regions", adminHandler.DeleteAllRegionCache, setMethods(http.MethodDelete), setAuditBackend(localLog, prometheus)) registerFunc(apiRouter, "/admin/persist-file/{file_name}", adminHandler.SavePersistFile, setMethods(http.MethodPost), setAuditBackend(localLog, prometheus)) - registerFunc(apiRouter, "/admin/persist-file/{file_name}", adminHandler.SavePersistFile, setMethods(http.MethodPost), setAuditBackend(localLog, prometheus)) registerFunc(apiRouter, "/admin/cluster/markers/snapshot-recovering", adminHandler.IsSnapshotRecovering, setMethods(http.MethodGet), setAuditBackend(localLog, prometheus)) registerFunc(apiRouter, "/admin/cluster/markers/snapshot-recovering", adminHandler.MarkSnapshotRecovering, setMethods(http.MethodPost), setAuditBackend(localLog, prometheus)) registerFunc(apiRouter, "/admin/cluster/markers/snapshot-recovering", adminHandler.UnmarkSnapshotRecovering, setMethods(http.MethodDelete), setAuditBackend(localLog, prometheus)) diff --git a/server/apiv2/handlers/tso_keyspace_group.go b/server/apiv2/handlers/tso_keyspace_group.go index 2d16f6ac360..5de8fd6a9cc 100644 --- a/server/apiv2/handlers/tso_keyspace_group.go +++ b/server/apiv2/handlers/tso_keyspace_group.go @@ -17,6 +17,7 @@ package handlers import ( "net/http" "strconv" + "strings" "sync" "github.com/gin-gonic/gin" @@ -108,8 +109,30 @@ func GetKeyspaceGroups(c *gin.Context) { c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) return } + var kgs []*endpoint.KeyspaceGroup + state, set := c.GetQuery("state") + if set { + state := strings.ToLower(state) + switch state { + case "merge": + for _, keyspaceGroup := range keyspaceGroups { + if keyspaceGroup.MergeState != nil { + kgs = append(kgs, keyspaceGroup) + } + } + case "split": + for _, keyspaceGroup := range keyspaceGroups { + if keyspaceGroup.SplitState != nil { + kgs = append(kgs, keyspaceGroup) + } + } + default: + } + } else { + kgs = keyspaceGroups + } - c.IndentedJSON(http.StatusOK, keyspaceGroups) + c.IndentedJSON(http.StatusOK, kgs) } // GetKeyspaceGroupByID gets keyspace group by ID. @@ -161,6 +184,9 @@ func DeleteKeyspaceGroupByID(c *gin.Context) { type SplitKeyspaceGroupByIDParams struct { NewID uint32 `json:"new-id"` Keyspaces []uint32 `json:"keyspaces"` + // StartKeyspaceID and EndKeyspaceID are used to indicate the range of keyspaces to be split. + StartKeyspaceID uint32 `json:"start-keyspace-id"` + EndKeyspaceID uint32 `json:"end-keyspace-id"` } var patrolKeyspaceAssignmentState struct { @@ -186,10 +212,15 @@ func SplitKeyspaceGroupByID(c *gin.Context) { c.AbortWithStatusJSON(http.StatusBadRequest, "invalid keyspace group id") return } - if len(splitParams.Keyspaces) == 0 { + if len(splitParams.Keyspaces) == 0 && splitParams.StartKeyspaceID == 0 && splitParams.EndKeyspaceID == 0 { c.AbortWithStatusJSON(http.StatusBadRequest, "invalid empty keyspaces") return } + if splitParams.StartKeyspaceID < utils.DefaultKeyspaceID || + splitParams.StartKeyspaceID > splitParams.EndKeyspaceID { + c.AbortWithStatusJSON(http.StatusBadRequest, "invalid start/end keyspace id") + return + } svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) patrolKeyspaceAssignmentState.Lock() @@ -200,7 +231,7 @@ func SplitKeyspaceGroupByID(c *gin.Context) { c.AbortWithStatusJSON(http.StatusInternalServerError, managerUninitializedErr) return } - err = manager.PatrolKeyspaceAssignment() + err = manager.PatrolKeyspaceAssignment(splitParams.StartKeyspaceID, splitParams.EndKeyspaceID) if err != nil { c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) patrolKeyspaceAssignmentState.Unlock() @@ -215,7 +246,9 @@ func SplitKeyspaceGroupByID(c *gin.Context) { return } // Split keyspace group. - err = groupManager.SplitKeyspaceGroupByID(id, splitParams.NewID, splitParams.Keyspaces) + err = groupManager.SplitKeyspaceGroupByID( + id, splitParams.NewID, + splitParams.Keyspaces, splitParams.StartKeyspaceID, splitParams.EndKeyspaceID) if err != nil { c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) return diff --git a/server/grpc_service.go b/server/grpc_service.go index 3096357654b..f66bd37ed11 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -406,16 +406,17 @@ func (s *GrpcServer) forwardTSO(stream pdpb.PD_TsoServer) error { var ( server = &tsoServer{stream: stream} forwardStream tsopb.TSO_TsoClient - cancel context.CancelFunc + forwardCtx context.Context + cancelForward context.CancelFunc lastForwardedHost string ) defer func() { s.concurrentTSOProxyStreamings.Add(-1) - // cancel the forward stream - if cancel != nil { - cancel() + if cancelForward != nil { + cancelForward() } }() + maxConcurrentTSOProxyStreamings := int32(s.GetMaxConcurrentTSOProxyStreamings()) if maxConcurrentTSOProxyStreamings >= 0 { if newCount := s.concurrentTSOProxyStreamings.Add(1); newCount > maxConcurrentTSOProxyStreamings { @@ -423,6 +424,9 @@ func (s *GrpcServer) forwardTSO(stream pdpb.PD_TsoServer) error { } } + tsDeadlineCh := make(chan *tsoutil.TSDeadline, 1) + go tsoutil.WatchTSDeadline(stream.Context(), tsDeadlineCh) + for { select { case <-s.ctx.Done(): @@ -449,22 +453,24 @@ func (s *GrpcServer) forwardTSO(stream pdpb.PD_TsoServer) error { return errors.WithStack(ErrNotFoundTSOAddr) } if forwardStream == nil || lastForwardedHost != forwardedHost { - if cancel != nil { - cancel() + if cancelForward != nil { + cancelForward() } clientConn, err := s.getDelegateClient(s.ctx, forwardedHost) if err != nil { return errors.WithStack(err) } - forwardStream, cancel, err = s.createTSOForwardStream(clientConn) + forwardStream, forwardCtx, cancelForward, err = + s.createTSOForwardStream(stream.Context(), clientConn) if err != nil { return errors.WithStack(err) } lastForwardedHost = forwardedHost } - tsopbResp, err := s.forwardTSORequestWithDeadLine(stream.Context(), request, forwardStream) + tsopbResp, err := s.forwardTSORequestWithDeadLine( + forwardCtx, cancelForward, forwardStream, request, tsDeadlineCh) if err != nil { return errors.WithStack(err) } @@ -502,37 +508,39 @@ func (s *GrpcServer) forwardTSO(stream pdpb.PD_TsoServer) error { } func (s *GrpcServer) forwardTSORequestWithDeadLine( - ctx context.Context, request *pdpb.TsoRequest, forwardStream tsopb.TSO_TsoClient, + forwardCtx context.Context, + cancelForward context.CancelFunc, + forwardStream tsopb.TSO_TsoClient, + request *pdpb.TsoRequest, + tsDeadlineCh chan<- *tsoutil.TSDeadline, ) (*tsopb.TsoResponse, error) { - defer logutil.LogPanic() - // Create a context with deadline for forwarding TSO request to TSO service. - ctxTimeout, cancel := context.WithTimeout(ctx, tsoutil.DefaultTSOProxyTimeout) - defer cancel() - - tsoProxyBatchSize.Observe(float64(request.GetCount())) + done := make(chan struct{}) + dl := tsoutil.NewTSDeadline(tsoutil.DefaultTSOProxyTimeout, done, cancelForward) + select { + case tsDeadlineCh <- dl: + case <-forwardCtx.Done(): + return nil, forwardCtx.Err() + } - // used to receive the result from doSomething function - tsoRespCh := make(chan *tsopbTSOResponse, 1) start := time.Now() - go s.forwardTSORequestAsync(ctxTimeout, request, forwardStream, tsoRespCh) - select { - case <-ctxTimeout.Done(): - tsoProxyForwardTimeoutCounter.Inc() - return nil, ErrForwardTSOTimeout - case tsoResp := <-tsoRespCh: - if tsoResp.err == nil { - tsoProxyHandleDuration.Observe(time.Since(start).Seconds()) + resp, err := s.forwardTSORequest(forwardCtx, request, forwardStream) + close(done) + if err != nil { + if strings.Contains(err.Error(), errs.NotLeaderErr) { + s.tsoPrimaryWatcher.ForceLoad() } - return tsoResp.response, tsoResp.err + return nil, err } + tsoProxyBatchSize.Observe(float64(request.GetCount())) + tsoProxyHandleDuration.Observe(time.Since(start).Seconds()) + return resp, nil } -func (s *GrpcServer) forwardTSORequestAsync( - ctxTimeout context.Context, +func (s *GrpcServer) forwardTSORequest( + ctx context.Context, request *pdpb.TsoRequest, forwardStream tsopb.TSO_TsoClient, - tsoRespCh chan<- *tsopbTSOResponse, -) { +) (*tsopb.TsoResponse, error) { tsopbReq := &tsopb.TsoRequest{ Header: &tsopb.RequestHeader{ ClusterId: request.GetHeader().GetClusterId(), @@ -545,46 +553,32 @@ func (s *GrpcServer) forwardTSORequestAsync( } failpoint.Inject("tsoProxySendToTSOTimeout", func() { - <-ctxTimeout.Done() - failpoint.Return() + // block until watchDeadline routine cancels the context. + <-ctx.Done() }) - if err := forwardStream.Send(tsopbReq); err != nil { - select { - case <-ctxTimeout.Done(): - return - case tsoRespCh <- &tsopbTSOResponse{err: err}: - } - return - } - select { - case <-ctxTimeout.Done(): - return + case <-ctx.Done(): + return nil, ctx.Err() default: } + if err := forwardStream.Send(tsopbReq); err != nil { + return nil, err + } + failpoint.Inject("tsoProxyRecvFromTSOTimeout", func() { - <-ctxTimeout.Done() - failpoint.Return() + // block until watchDeadline routine cancels the context. + <-ctx.Done() }) - response, err := forwardStream.Recv() - if err != nil { - if strings.Contains(err.Error(), errs.NotLeaderErr) { - s.tsoPrimaryWatcher.ForceLoad() - } - } select { - case <-ctxTimeout.Done(): - return - case tsoRespCh <- &tsopbTSOResponse{response: response, err: err}: + case <-ctx.Done(): + return nil, ctx.Err() + default: } -} -type tsopbTSOResponse struct { - response *tsopb.TsoResponse - err error + return forwardStream.Recv() } // tsoServer wraps PD_TsoServer to ensure when any error @@ -612,13 +606,15 @@ func (s *tsoServer) Send(m *pdpb.TsoResponse) error { }) done <- s.stream.Send(m) }() + timer := time.NewTimer(tsoutil.DefaultTSOProxyTimeout) + defer timer.Stop() select { case err := <-done: if err != nil { atomic.StoreInt32(&s.closed, 1) } return errors.WithStack(err) - case <-time.After(tsoutil.DefaultTSOProxyTimeout): + case <-timer.C: atomic.StoreInt32(&s.closed, 1) return ErrForwardTSOTimeout } @@ -639,6 +635,8 @@ func (s *tsoServer) Recv(timeout time.Duration) (*pdpb.TsoRequest, error) { request, err := s.stream.Recv() requestCh <- &pdpbTSORequest{request: request, err: err} }() + timer := time.NewTimer(timeout) + defer timer.Stop() select { case req := <-requestCh: if req.err != nil { @@ -646,7 +644,7 @@ func (s *tsoServer) Recv(timeout time.Duration) (*pdpb.TsoRequest, error) { return nil, errors.WithStack(req.err) } return req.request, nil - case <-time.After(timeout): + case <-timer.C: atomic.StoreInt32(&s.closed, 1) return nil, ErrTSOProxyRecvFromClientTimeout } @@ -2140,13 +2138,15 @@ func forwardRegionHeartbeatClientToServer(forwardStream pdpb.PD_RegionHeartbeatC } } -func (s *GrpcServer) createTSOForwardStream(client *grpc.ClientConn) (tsopb.TSO_TsoClient, context.CancelFunc, error) { +func (s *GrpcServer) createTSOForwardStream( + ctx context.Context, client *grpc.ClientConn, +) (tsopb.TSO_TsoClient, context.Context, context.CancelFunc, error) { done := make(chan struct{}) - ctx, cancel := context.WithCancel(s.ctx) - go checkStream(ctx, cancel, done) - forwardStream, err := tsopb.NewTSOClient(client).Tso(ctx) + forwardCtx, cancelForward := context.WithCancel(ctx) + go checkStream(forwardCtx, cancelForward, done) + forwardStream, err := tsopb.NewTSOClient(client).Tso(forwardCtx) done <- struct{}{} - return forwardStream, cancel, err + return forwardStream, forwardCtx, cancelForward, err } func (s *GrpcServer) createReportBucketsForwardStream(client *grpc.ClientConn) (pdpb.PD_ReportBucketsClient, context.CancelFunc, error) { @@ -2177,10 +2177,12 @@ func forwardReportBucketClientToServer(forwardStream pdpb.PD_ReportBucketsClient // TODO: If goroutine here timeout when tso stream created successfully, we need to handle it correctly. func checkStream(streamCtx context.Context, cancel context.CancelFunc, done chan struct{}) { defer logutil.LogPanic() + timer := time.NewTimer(3 * time.Second) + defer timer.Stop() select { case <-done: return - case <-time.After(3 * time.Second): + case <-timer.C: cancel() case <-streamCtx.Done(): } diff --git a/server/handler.go b/server/handler.go index 1fc543827b2..635901fb04a 100644 --- a/server/handler.go +++ b/server/handler.go @@ -419,7 +419,7 @@ func (h *Handler) RemoveOperator(regionID uint64) error { return ErrOperatorNotFound } - _ = c.RemoveOperator(op) + _ = c.RemoveOperator(op, operator.AdminStop) return nil } diff --git a/tests/integrations/client/go.mod b/tests/integrations/client/go.mod index d6e94929f6f..00c6bd4daaf 100644 --- a/tests/integrations/client/go.mod +++ b/tests/integrations/client/go.mod @@ -119,7 +119,7 @@ require ( github.com/pingcap/errcode v0.3.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5 // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/tests/integrations/client/go.sum b/tests/integrations/client/go.sum index 848c249f032..0e51f7e6dec 100644 --- a/tests/integrations/client/go.sum +++ b/tests/integrations/client/go.sum @@ -395,8 +395,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5 h1:adV4kUWI7v+v/6joR7lfjFngHhS4eiqwr4g3dHCjHtA= -github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= +github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 h1:/jqjj+ydlqjp144LAlHDfHtr7eWJyaNIIXX5viv0RZo= +github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= diff --git a/tests/integrations/mcs/cluster.go b/tests/integrations/mcs/cluster.go index 3cda8c1888a..65a6bf293c3 100644 --- a/tests/integrations/mcs/cluster.go +++ b/tests/integrations/mcs/cluster.go @@ -202,7 +202,8 @@ func (tc *TestTSOCluster) GetServers() map[string]*tso.Server { func (tc *TestTSOCluster) GetKeyspaceGroupMember() (members []endpoint.KeyspaceGroupMember) { for _, server := range tc.servers { members = append(members, endpoint.KeyspaceGroupMember{ - Address: server.GetAddr(), + Address: server.GetAddr(), + Priority: mcsutils.DefaultKeyspaceGroupReplicaPriority, }) } return diff --git a/tests/integrations/mcs/go.mod b/tests/integrations/mcs/go.mod index 2b225f0ced2..70f9bcc3a9a 100644 --- a/tests/integrations/mcs/go.mod +++ b/tests/integrations/mcs/go.mod @@ -119,7 +119,7 @@ require ( github.com/pingcap/errcode v0.3.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5 // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/tests/integrations/mcs/go.sum b/tests/integrations/mcs/go.sum index 178a6dfb953..0fd5cbf2d6f 100644 --- a/tests/integrations/mcs/go.sum +++ b/tests/integrations/mcs/go.sum @@ -394,8 +394,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5 h1:adV4kUWI7v+v/6joR7lfjFngHhS4eiqwr4g3dHCjHtA= -github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= +github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 h1:/jqjj+ydlqjp144LAlHDfHtr7eWJyaNIIXX5viv0RZo= +github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= diff --git a/tests/integrations/mcs/tso/keyspace_group_manager_test.go b/tests/integrations/mcs/tso/keyspace_group_manager_test.go index ed3bfe35280..98c6b90ca28 100644 --- a/tests/integrations/mcs/tso/keyspace_group_manager_test.go +++ b/tests/integrations/mcs/tso/keyspace_group_manager_test.go @@ -29,6 +29,7 @@ import ( "github.com/stretchr/testify/suite" pd "github.com/tikv/pd/client" "github.com/tikv/pd/pkg/election" + "github.com/tikv/pd/pkg/errs" mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/member" "github.com/tikv/pd/pkg/storage/endpoint" @@ -263,7 +264,7 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplit() { err error ) testutil.Eventually(re, func() bool { - ts, err = suite.requestTSO(re, 1, 222, 1) + ts, err = suite.requestTSO(re, 222, 1) return err == nil && tsoutil.CompareTimestamp(&ts, &pdpb.Timestamp{}) > 0 }) ts.Physical += time.Hour.Milliseconds() @@ -282,22 +283,22 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplit() { // Check the split TSO from keyspace group 2. var splitTS pdpb.Timestamp testutil.Eventually(re, func() bool { - splitTS, err = suite.requestTSO(re, 1, 222, 2) + splitTS, err = suite.requestTSO(re, 222, 2) return err == nil && tsoutil.CompareTimestamp(&splitTS, &pdpb.Timestamp{}) > 0 }) - splitTS, err = suite.requestTSO(re, 1, 222, 2) + splitTS, err = suite.requestTSO(re, 222, 2) + re.NoError(err) re.Greater(tsoutil.CompareTimestamp(&splitTS, &ts), 0) } func (suite *tsoKeyspaceGroupManagerTestSuite) requestTSO( re *require.Assertions, - count, keyspaceID, keyspaceGroupID uint32, + keyspaceID, keyspaceGroupID uint32, ) (pdpb.Timestamp, error) { primary := suite.tsoCluster.WaitForPrimaryServing(re, keyspaceID, keyspaceGroupID) kgm := primary.GetKeyspaceGroupManager() re.NotNil(kgm) - ts, _, err := kgm.HandleTSORequest(keyspaceID, keyspaceGroupID, tsopkg.GlobalDCLocation, count) - re.NoError(err) + ts, _, err := kgm.HandleTSORequest(keyspaceID, keyspaceGroupID, tsopkg.GlobalDCLocation, 1) return ts, err } @@ -357,31 +358,58 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplitElection 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}) +} + +func waitFinishSplit( + re *require.Assertions, + server *tests.TestServer, + splitSourceID, splitTargetID uint32, + splitSourceKeyspaces, splitTargetKeyspaces []uint32, +) { + testutil.Eventually(re, func() bool { + kg := handlersutil.MustLoadKeyspaceGroupByID(re, server, splitTargetID) + re.Equal(splitTargetID, kg.ID) + re.Equal(splitTargetKeyspaces, kg.Keyspaces) + return !kg.IsSplitTarget() + }) + testutil.Eventually(re, func() bool { + kg := handlersutil.MustLoadKeyspaceGroupByID(re, server, splitSourceID) + re.Equal(splitSourceID, kg.ID) + re.Equal(splitSourceKeyspaces, kg.Keyspaces) + return !kg.IsSplitSource() + }) } func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplitClient() { re := suite.Require() - // Create the keyspace group 1 with keyspaces [111, 222, 333]. + // Enable the failpoint to slow down the system time to test whether the TSO is monotonic. + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/systemTimeSlow", `return(true)`)) + // Create the keyspace group 1 with keyspaces [444, 555, 666]. 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}, + Keyspaces: []uint32{444, 555, 666}, }, }, }) kg1 := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 1) re.Equal(uint32(1), kg1.ID) - re.Equal([]uint32{111, 222, 333}, kg1.Keyspaces) + re.Equal([]uint32{444, 555, 666}, kg1.Keyspaces) re.False(kg1.IsSplitting()) - // Prepare the client for keyspace 222. - var tsoClient pd.TSOClient - tsoClient, err := pd.NewClientWithKeyspace(suite.ctx, 222, []string{suite.pdLeaderServer.GetAddr()}, pd.SecurityOption{}) + // Make sure the leader of the keyspace group 1 is elected. + member, err := suite.tsoCluster.WaitForPrimaryServing(re, 555, 1).GetMember(555, 1) + 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{}) re.NoError(err) re.NotNil(tsoClient) - // Request the TSO for keyspace 222 concurrently. + // Request the TSO for keyspace 555 concurrently. var ( wg sync.WaitGroup ctx, cancel = context.WithCancel(suite.ctx) @@ -421,19 +449,14 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplitClient() // Split the keyspace group 1 to 2. handlersutil.MustSplitKeyspaceGroup(re, suite.pdLeaderServer, 1, &handlers.SplitKeyspaceGroupByIDParams{ NewID: 2, - Keyspaces: []uint32{222, 333}, + Keyspaces: []uint32{555, 666}, }) - kg2 := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 2) - re.Equal(uint32(2), kg2.ID) - re.Equal([]uint32{222, 333}, kg2.Keyspaces) - re.True(kg2.IsSplitTarget()) - // Finish the split. - handlersutil.MustFinishSplitKeyspaceGroup(re, suite.pdLeaderServer, 2) - // Wait for a while to make sure the client has received the new TSO. - time.Sleep(time.Second) + // 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")) } func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupMembers() { @@ -474,7 +497,7 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) // Init api server config but not start. - tc, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, serverName string) { + tc, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = []string{ "keyspace_a", "keyspace_b", } @@ -513,7 +536,7 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { // Wait pd clients are ready. testutil.Eventually(re, func() bool { count := 0 - clients.Range(func(key, value interface{}) bool { + clients.Range(func(_, _ interface{}) bool { count++ return true }) @@ -535,10 +558,9 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { // Trigger checkTSOSplit to ensure the split is finished. testutil.Eventually(re, func() bool { _, _, err = clientB.(pd.Client).GetTS(ctx) - re.NoError(err) - kg := handlersutil.MustLoadKeyspaceGroupByID(re, leaderServer, 0) - return !kg.IsSplitting() + 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. @@ -550,10 +572,9 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { // Trigger checkTSOSplit to ensure the split is finished. testutil.Eventually(re, func() bool { _, _, err = clientA.(pd.Client).GetTS(ctx) - re.NoError(err) - kg := handlersutil.MustLoadKeyspaceGroupByID(re, leaderServer, 0) - return !kg.IsSplitting() + 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. @@ -569,3 +590,139 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) } + +func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupMerge() { + re := suite.Require() + // Create the keyspace group 1 and 2 with keyspaces [111, 222] and [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}, + }, + { + ID: 2, + UserKind: endpoint.Standard.String(), + Members: suite.tsoCluster.GetKeyspaceGroupMember(), + Keyspaces: []uint32{333}, + }, + }, + }) + // Get a TSO from the keyspace group 1. + var ( + ts pdpb.Timestamp + err error + ) + testutil.Eventually(re, func() bool { + ts, err = suite.requestTSO(re, 222, 1) + return err == nil && tsoutil.CompareTimestamp(&ts, &pdpb.Timestamp{}) > 0 + }) + ts.Physical += time.Hour.Milliseconds() + // Set the TSO of the keyspace group 1 to a large value. + err = suite.tsoCluster.GetPrimaryServer(222, 1).GetHandler().ResetTS(tsoutil.GenerateTS(&ts), false, true, 1) + re.NoError(err) + // Merge the keyspace group 1 and 2 to the default keyspace group. + handlersutil.MustMergeKeyspaceGroup(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID, &handlers.MergeKeyspaceGroupsParams{ + MergeList: []uint32{1, 2}, + }) + // Check the keyspace group 1 and 2 are merged to the default keyspace group. + kg := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID) + re.Equal(mcsutils.DefaultKeyspaceGroupID, kg.ID) + for _, keyspaceID := range []uint32{111, 222, 333} { + re.Contains(kg.Keyspaces, keyspaceID) + } + re.True(kg.IsMergeTarget()) + // Check the merged TSO from the default keyspace group is greater than the TSO from the keyspace group 1. + var mergedTS pdpb.Timestamp + testutil.Eventually(re, func() bool { + mergedTS, err = suite.requestTSO(re, 333, mcsutils.DefaultKeyspaceGroupID) + if err != nil { + re.ErrorIs(err, errs.ErrKeyspaceGroupIsMerging) + } + return err == nil && tsoutil.CompareTimestamp(&mergedTS, &pdpb.Timestamp{}) > 0 + }, testutil.WithTickInterval(5*time.Second), testutil.WithWaitFor(time.Minute)) + re.Greater(tsoutil.CompareTimestamp(&mergedTS, &ts), 0) +} + +func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupMergeClient() { + re := suite.Require() + // 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}, + }, + }, + }) + kg1 := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 1) + 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 + } + }() + // 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. + testutil.Eventually(re, func() bool { + kg := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, mcsutils.DefaultKeyspaceGroupID) + re.Equal(mcsutils.DefaultKeyspaceGroupID, kg.ID) + for _, keyspaceID := range []uint32{111, 222, 333} { + re.Contains(kg.Keyspaces, keyspaceID) + } + return !kg.IsMergeTarget() + }) + // Stop the client. + cancel() + wg.Wait() +} diff --git a/tests/integrations/tso/go.mod b/tests/integrations/tso/go.mod index 06b3c93887e..9b82706d7a4 100644 --- a/tests/integrations/tso/go.mod +++ b/tests/integrations/tso/go.mod @@ -117,7 +117,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5 // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/tests/integrations/tso/go.sum b/tests/integrations/tso/go.sum index 5af6e71ae4d..cdffbdfe8df 100644 --- a/tests/integrations/tso/go.sum +++ b/tests/integrations/tso/go.sum @@ -392,8 +392,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5 h1:adV4kUWI7v+v/6joR7lfjFngHhS4eiqwr4g3dHCjHtA= -github.com/pingcap/tidb-dashboard v0.0.0-20230508075335-d6e0218addd5/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= +github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 h1:/jqjj+ydlqjp144LAlHDfHtr7eWJyaNIIXX5viv0RZo= +github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= diff --git a/tests/pdctl/keyspace/keyspace_group_test.go b/tests/pdctl/keyspace/keyspace_group_test.go index 0c44700f48e..c982dc140af 100644 --- a/tests/pdctl/keyspace/keyspace_group_test.go +++ b/tests/pdctl/keyspace/keyspace_group_test.go @@ -128,6 +128,18 @@ func TestSplitKeyspaceGroup(t *testing.T) { return strings.Contains(string(output), "Success") }) + // get all keyspaces + args := []string{"-u", pdAddr, "keyspace-group"} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Success") + var keyspaceGroups []*endpoint.KeyspaceGroup + err = json.Unmarshal(output, &keyspaceGroups) + re.NoError(err) + re.Len(keyspaceGroups, 2) + re.Equal(keyspaceGroups[0].ID, uint32(0)) + re.Equal(keyspaceGroups[1].ID, uint32(1)) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) re.NoError(failpoint.Disable("github.com/tikv/pd/server/delayStartServerLoop")) } @@ -277,3 +289,164 @@ func TestSetNodeAndPriorityKeyspaceGroup(t *testing.T) { re.NoError(err) re.Contains(string(output), "Failed to parse the priority") } + +func TestMergeKeyspaceGroup(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/server/delayStartServerLoop", `return(true)`)) + keyspaces := make([]string, 0) + // we test the case which exceed the default max txn ops limit in etcd, which is 128. + for i := 0; i < 129; i++ { + keyspaces = append(keyspaces, fmt.Sprintf("keyspace_%d", i)) + } + tc, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, serverName string) { + conf.Keyspace.PreAlloc = keyspaces + }) + re.NoError(err) + err = tc.RunInitialServers() + 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() + re.NoError(err) + cmd := pdctlCmd.GetRootCmd() + + tc.WaitLeader() + leaderServer := tc.GetServer(tc.GetLeader()) + re.NoError(leaderServer.BootstrapCluster()) + + // 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") + }) + + args := []string{"-u", pdAddr, "keyspace-group", "finish-split", "1"} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Success") + + // merge keyspace group. + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group", "merge", "0", "1"} + 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) + var keyspaceGroup endpoint.KeyspaceGroup + err = json.Unmarshal(output, &keyspaceGroup) + re.NoError(err) + re.Len(keyspaceGroup.Keyspaces, 130) + re.Nil(keyspaceGroup.MergeState) + + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) + re.NoError(failpoint.Disable("github.com/tikv/pd/server/delayStartServerLoop")) +} + +func TestKeyspaceGroupState(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/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, 1, func(conf *config.Config, serverName string) { + conf.Keyspace.PreAlloc = keyspaces + }) + re.NoError(err) + err = tc.RunInitialServers() + 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() + re.NoError(err) + cmd := pdctlCmd.GetRootCmd() + + tc.WaitLeader() + leaderServer := tc.GetServer(tc.GetLeader()) + re.NoError(leaderServer.BootstrapCluster()) + + // 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") + }) + args := []string{"-u", pdAddr, "keyspace-group", "finish-split", "1"} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Success") + args = []string{"-u", pdAddr, "keyspace-group", "--state", "split"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Success") + var keyspaceGroups []*endpoint.KeyspaceGroup + err = json.Unmarshal(output, &keyspaceGroups) + re.NoError(err) + re.Len(keyspaceGroups, 0) + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group", "split", "0", "2", "3"} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + return strings.Contains(string(output), "Success") + }) + args = []string{"-u", pdAddr, "keyspace-group", "--state", "split"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Success") + err = json.Unmarshal(output, &keyspaceGroups) + re.NoError(err) + re.Len(keyspaceGroups, 2) + re.Equal(keyspaceGroups[0].ID, uint32(0)) + re.Equal(keyspaceGroups[1].ID, uint32(2)) + + args = []string{"-u", pdAddr, "keyspace-group", "finish-split", "2"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Success") + // merge keyspace group. + testutil.Eventually(re, func() bool { + args := []string{"-u", pdAddr, "keyspace-group", "merge", "0", "1"} + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + return strings.Contains(string(output), "Success") + }) + + args = []string{"-u", pdAddr, "keyspace-group", "--state", "merge"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + strings.Contains(string(output), "Success") + err = json.Unmarshal(output, &keyspaceGroups) + re.NoError(err) + err = json.Unmarshal(output, &keyspaceGroups) + re.NoError(err) + re.Len(keyspaceGroups, 1) + re.Equal(keyspaceGroups[0].ID, uint32(0)) + + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) + 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 b638f1bbba4..900cd84b829 100644 --- a/tests/server/apiv2/handlers/testutil.go +++ b/tests/server/apiv2/handlers/testutil.go @@ -205,7 +205,7 @@ func MustDeleteKeyspaceGroup(re *require.Assertions, server *tests.TestServer, i re.Equal(http.StatusOK, resp.StatusCode, string(data)) } -// MustSplitKeyspaceGroup updates a keyspace group with HTTP API. +// MustSplitKeyspaceGroup splits a keyspace group with HTTP API. func MustSplitKeyspaceGroup(re *require.Assertions, server *tests.TestServer, id uint32, request *handlers.SplitKeyspaceGroupByIDParams) { data, err := json.Marshal(request) re.NoError(err) @@ -232,3 +232,18 @@ func MustFinishSplitKeyspaceGroup(re *require.Assertions, server *tests.TestServ re.NoError(err) re.Equal(http.StatusOK, resp.StatusCode, string(data)) } + +// MustMergeKeyspaceGroup merges keyspace groups with HTTP API. +func MustMergeKeyspaceGroup(re *require.Assertions, server *tests.TestServer, id uint32, request *handlers.MergeKeyspaceGroupsParams) { + data, err := json.Marshal(request) + re.NoError(err) + httpReq, err := http.NewRequest(http.MethodPost, server.GetAddr()+keyspaceGroupsPrefix+fmt.Sprintf("/%d/merge", id), bytes.NewBuffer(data)) + re.NoError(err) + // Send request. + resp, err := dialClient.Do(httpReq) + re.NoError(err) + defer resp.Body.Close() + data, err = io.ReadAll(resp.Body) + re.NoError(err) + re.Equal(http.StatusOK, resp.StatusCode, string(data)) +} diff --git a/tools/pd-ctl/pdctl/command/global.go b/tools/pd-ctl/pdctl/command/global.go index 623ab3edfba..8d888b60b1f 100644 --- a/tools/pd-ctl/pdctl/command/global.go +++ b/tools/pd-ctl/pdctl/command/global.go @@ -192,7 +192,7 @@ func postJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { return nil }) if err != nil { - cmd.Printf("Failed! %s", err) + cmd.Printf("Failed! %s\n", err) return } cmd.Println("Success!") diff --git a/tools/pd-ctl/pdctl/command/keyspace_group_command.go b/tools/pd-ctl/pdctl/command/keyspace_group_command.go index 662a4aa157e..a4be612a301 100644 --- a/tools/pd-ctl/pdctl/command/keyspace_group_command.go +++ b/tools/pd-ctl/pdctl/command/keyspace_group_command.go @@ -15,12 +15,15 @@ package command import ( + "encoding/json" "fmt" "net/http" "net/url" "strconv" + "strings" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/storage/endpoint" ) const keyspaceGroupsPrefix = "pd/api/v2/tso/keyspace-groups" @@ -28,13 +31,18 @@ const keyspaceGroupsPrefix = "pd/api/v2/tso/keyspace-groups" // NewKeyspaceGroupCommand return a keyspace group subcommand of rootCmd func NewKeyspaceGroupCommand() *cobra.Command { cmd := &cobra.Command{ - Use: "keyspace-group ", - Short: "show keyspace group information with the given ID", - Run: showKeyspaceGroupCommandFunc, + Use: "keyspace-group [command] [flags]", + Short: "show keyspace group information", + Run: showKeyspaceGroupsCommandFunc, } cmd.AddCommand(newSplitKeyspaceGroupCommand()) + cmd.AddCommand(newSplitRangeKeyspaceGroupCommand()) + cmd.AddCommand(newFinishSplitKeyspaceGroupCommand()) + cmd.AddCommand(newMergeKeyspaceGroupCommand()) + cmd.AddCommand(newFinishMergeKeyspaceGroupCommand()) cmd.AddCommand(newSetNodesKeyspaceGroupCommand()) cmd.AddCommand(newSetPriorityKeyspaceGroupCommand()) + cmd.Flags().String("state", "", "state filter") return cmd } @@ -47,6 +55,44 @@ func newSplitKeyspaceGroupCommand() *cobra.Command { return r } +func newSplitRangeKeyspaceGroupCommand() *cobra.Command { + r := &cobra.Command{ + Use: "split-range ", + Short: "split the keyspace group with the given ID and transfer the keyspaces in the given range (both ends inclusive) into the newly split one", + Run: splitRangeKeyspaceGroupCommandFunc, + } + return r +} + +func newFinishSplitKeyspaceGroupCommand() *cobra.Command { + r := &cobra.Command{ + Use: "finish-split ", + Short: "finish split the keyspace group with the given ID", + Run: finishSplitKeyspaceGroupCommandFunc, + Hidden: true, + } + return r +} + +func newMergeKeyspaceGroupCommand() *cobra.Command { + r := &cobra.Command{ + Use: "merge []", + Short: "merge the keyspace group with the given IDs into the target one", + Run: mergeKeyspaceGroupCommandFunc, + } + return r +} + +func newFinishMergeKeyspaceGroupCommand() *cobra.Command { + r := &cobra.Command{ + Use: "finish-merge ", + Short: "finish merge the keyspace group with the given ID", + Run: finishMergeKeyspaceGroupCommandFunc, + Hidden: true, + } + return r +} + func newSetNodesKeyspaceGroupCommand() *cobra.Command { r := &cobra.Command{ Use: "set-node [...]", @@ -65,16 +111,48 @@ func newSetPriorityKeyspaceGroupCommand() *cobra.Command { return r } -func showKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { - if len(args) < 1 { +func showKeyspaceGroupsCommandFunc(cmd *cobra.Command, args []string) { + prefix := keyspaceGroupsPrefix + if len(args) > 1 { cmd.Usage() return } - r, err := doRequest(cmd, fmt.Sprintf("%s/%s", keyspaceGroupsPrefix, args[0]), http.MethodGet, http.Header{}) + cFunc := convertToKeyspaceGroups + if len(args) == 1 { + if _, err := strconv.Atoi(args[0]); err != nil { + cmd.Println("keyspace_group_id should be a number") + return + } + prefix = fmt.Sprintf("%s/%s", keyspaceGroupsPrefix, args[0]) + cFunc = convertToKeyspaceGroup + } else { + flags := cmd.Flags() + state, err := flags.GetString("state") + if err != nil { + cmd.Printf("Failed to get state: %s\n", err) + } + stateValue := "" + if len(state) != 0 { + state = strings.ToLower(state) + switch state { + case "merge", "split": + stateValue = fmt.Sprintf("state=%v", state) + default: + cmd.Println("Unknown state: " + state) + return + } + } + + if len(stateValue) != 0 { + prefix = fmt.Sprintf("%v?%v", keyspaceGroupsPrefix, stateValue) + } + } + r, err := doRequest(cmd, prefix, http.MethodGet, http.Header{}) if err != nil { cmd.Printf("Failed to get the keyspace groups information: %s\n", err) return } + r = cFunc(r) cmd.Println(r) } @@ -108,6 +186,98 @@ func splitKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { }) } +func splitRangeKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { + if len(args) < 4 { + cmd.Usage() + return + } + _, err := strconv.ParseUint(args[0], 10, 32) + if err != nil { + cmd.Printf("Failed to parse the old keyspace group ID: %s\n", err) + return + } + newID, err := strconv.ParseUint(args[1], 10, 32) + if err != nil { + cmd.Printf("Failed to parse the new keyspace group ID: %s\n", err) + return + } + startKeyspaceID, err := strconv.ParseUint(args[2], 10, 32) + if err != nil { + cmd.Printf("Failed to parse the start keyspace ID: %s\n", err) + return + } + endKeyspaceID, err := strconv.ParseUint(args[3], 10, 32) + if err != nil { + cmd.Printf("Failed to parse the end keyspace ID: %s\n", err) + return + } + postJSON(cmd, fmt.Sprintf("%s/%s/split", keyspaceGroupsPrefix, args[0]), map[string]interface{}{ + "new-id": uint32(newID), + "start-keyspace-id": uint32(startKeyspaceID), + "end-keyspace-id": uint32(endKeyspaceID), + }) +} + +func finishSplitKeyspaceGroupCommandFunc(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 + } + _, err = doRequest(cmd, fmt.Sprintf("%s/%s/split", keyspaceGroupsPrefix, args[0]), http.MethodDelete, http.Header{}) + if err != nil { + cmd.Println(err) + return + } + cmd.Println("Success!") +} + +func mergeKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { + if len(args) < 2 { + cmd.Usage() + return + } + _, 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 := 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)) + } + postJSON(cmd, fmt.Sprintf("%s/%s/merge", keyspaceGroupsPrefix, args[0]), map[string]interface{}{ + "merge-list": groups, + }) +} + +func finishMergeKeyspaceGroupCommandFunc(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 + } + _, err = doRequest(cmd, fmt.Sprintf("%s/%s/merge", keyspaceGroupsPrefix, args[0]), http.MethodDelete, http.Header{}) + if err != nil { + cmd.Println(err) + return + } + cmd.Println("Success!") +} + func setNodesKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { if len(args) < 2 { cmd.Usage() @@ -161,3 +331,29 @@ func setPriorityKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { "Priority": priority, }) } + +func convertToKeyspaceGroup(content string) string { + kg := endpoint.KeyspaceGroup{} + err := json.Unmarshal([]byte(content), &kg) + if err != nil { + return content + } + byteArr, err := json.MarshalIndent(kg, "", " ") + if err != nil { + return content + } + return string(byteArr) +} + +func convertToKeyspaceGroups(content string) string { + kgs := []*endpoint.KeyspaceGroup{} + err := json.Unmarshal([]byte(content), &kgs) + if err != nil { + return content + } + byteArr, err := json.MarshalIndent(kgs, "", " ") + if err != nil { + return content + } + return string(byteArr) +} diff --git a/tools/pd-ctl/pdctl/command/store_command.go b/tools/pd-ctl/pdctl/command/store_command.go index 79ed9673e83..1dee1c13a72 100644 --- a/tools/pd-ctl/pdctl/command/store_command.go +++ b/tools/pd-ctl/pdctl/command/store_command.go @@ -565,7 +565,6 @@ func labelStoreCommandFunc(cmd *cobra.Command, args []string) { } else if rewrite, _ := cmd.Flags().GetBool("rewrite"); rewrite { prefix += "?force=true" } - cmd.Println(prefix) postJSON(cmd, prefix, labels) } diff --git a/tools/pd-tso-bench/main.go b/tools/pd-tso-bench/main.go index a0041b9ef3e..e82a9dcabba 100644 --- a/tools/pd-tso-bench/main.go +++ b/tools/pd-tso-bench/main.go @@ -33,6 +33,13 @@ import ( "github.com/prometheus/client_golang/prometheus/promhttp" pd "github.com/tikv/pd/client" "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" +) + +const ( + keepaliveTime = 10 * time.Second + keepaliveTimeout = 3 * time.Second ) var ( @@ -95,11 +102,18 @@ func bench(mainCtx context.Context) { 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)