From 6d0aa719fa179f9aff58bcf62841a15d94b6b846 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B7=B7=E6=B2=8CDM?= Date: Fri, 25 Sep 2020 16:07:42 +0800 Subject: [PATCH 01/27] client: rename GetMemberInfo to GetAllMembers (#3029) Signed-off-by: jyz0309 <45495947@qq.com> --- client/client.go | 10 +++++----- client/metrics.go | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/client/client.go b/client/client.go index b7356060282..5d74914ec0a 100644 --- a/client/client.go +++ b/client/client.go @@ -41,8 +41,8 @@ type Region struct { type Client interface { // GetClusterID gets the cluster ID from PD. GetClusterID(ctx context.Context) uint64 - // GetMemberInfo gets the members Info from PD - GetMemberInfo(ctx context.Context) ([]*pdpb.Member, error) + // GetAllMembers gets the members Info from PD + GetAllMembers(ctx context.Context) ([]*pdpb.Member, error) // GetLeaderAddr returns current leader's address. It returns "" before // syncing leader from server. GetLeaderAddr() string @@ -221,9 +221,9 @@ func (c *client) checkStreamTimeout(loopCtx context.Context, cancel context.Canc } } -func (c *client) GetMemberInfo(ctx context.Context) ([]*pdpb.Member, error) { +func (c *client) GetAllMembers(ctx context.Context) ([]*pdpb.Member, error) { start := time.Now() - defer func() { cmdDurationGetMemberInfo.Observe(time.Since(start).Seconds()) }() + defer func() { cmdDurationGetAllMembers.Observe(time.Since(start).Seconds()) }() ctx, cancel := context.WithTimeout(ctx, c.timeout) resp, err := c.leaderClient().GetMembers(ctx, &pdpb.GetMembersRequest{ @@ -231,7 +231,7 @@ func (c *client) GetMemberInfo(ctx context.Context) ([]*pdpb.Member, error) { }) cancel() if err != nil { - cmdFailDurationGetMemberInfo.Observe(time.Since(start).Seconds()) + cmdFailDurationGetAllMembers.Observe(time.Since(start).Seconds()) c.ScheduleCheckLeader() return nil, errors.WithStack(err) } diff --git a/client/metrics.go b/client/metrics.go index 5f7f6060060..c1e9513a02d 100644 --- a/client/metrics.go +++ b/client/metrics.go @@ -59,7 +59,7 @@ var ( cmdDurationTSO = cmdDuration.WithLabelValues("tso") cmdDurationTSOAsyncWait = cmdDuration.WithLabelValues("tso_async_wait") cmdDurationGetRegion = cmdDuration.WithLabelValues("get_region") - cmdDurationGetMemberInfo = cmdDuration.WithLabelValues("get_member_info") + cmdDurationGetAllMembers = cmdDuration.WithLabelValues("get_member_info") cmdDurationGetPrevRegion = cmdDuration.WithLabelValues("get_prev_region") cmdDurationGetRegionByID = cmdDuration.WithLabelValues("get_region_byid") cmdDurationScanRegions = cmdDuration.WithLabelValues("scan_regions") @@ -72,7 +72,7 @@ var ( cmdFailDurationGetRegion = cmdFailedDuration.WithLabelValues("get_region") cmdFailDurationTSO = cmdFailedDuration.WithLabelValues("tso") - cmdFailDurationGetMemberInfo = cmdFailedDuration.WithLabelValues("get_member_info") + cmdFailDurationGetAllMembers = cmdFailedDuration.WithLabelValues("get_member_info") cmdFailDurationGetPrevRegion = cmdFailedDuration.WithLabelValues("get_prev_region") cmdFailedDurationGetRegionByID = cmdFailedDuration.WithLabelValues("get_region_byid") cmdFailedDurationScanRegions = cmdFailedDuration.WithLabelValues("scan_regions") From 0e9a678d533dcbe63fc0a4503ffa19328f68b3d1 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Fri, 25 Sep 2020 18:36:06 +0800 Subject: [PATCH 02/27] tso: use sync.RWMutex to control concurrent reading and writing (#3016) Signed-off-by: JmPotato --- client/client.go | 4 +- server/tso/tso.go | 139 +++++++++++++++++++++++++++------------------- 2 files changed, 84 insertions(+), 59 deletions(-) diff --git a/client/client.go b/client/client.go index 5d74914ec0a..ae85a8f83ec 100644 --- a/client/client.go +++ b/client/client.go @@ -371,8 +371,8 @@ func (c *client) processTSORequests(stream pdpb.PD_TsoClient, requests []*tsoReq // Server returns the highest ts. logical -= int64(resp.GetCount() - 1) if tsLessEqual(physical, logical, c.lastPhysical, c.lastLogical) { - panic(errors.Errorf("timestamp fallback, newly acquired ts (%d,%d) is less or equal to last one (%d, %d)", - physical, logical, c.lastLogical, c.lastLogical)) + panic(errors.Errorf("timestamp fallback, newly acquired ts (%d, %d) is less or equal to last one (%d, %d)", + physical, logical, c.lastPhysical, c.lastLogical)) } c.lastPhysical = physical c.lastLogical = logical + int64(len(requests)) - 1 diff --git a/server/tso/tso.go b/server/tso/tso.go index 5dc00304b3b..3efa6166921 100644 --- a/server/tso/tso.go +++ b/server/tso/tso.go @@ -15,9 +15,9 @@ package tso import ( "path" + "sync" "sync/atomic" "time" - "unsafe" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" @@ -43,8 +43,8 @@ const ( maxLogical = int64(1 << 18) ) -// atomicObject is used to store the current TSO in memory. -type atomicObject struct { +// tsoObject is used to store the current TSO in memory. +type tsoObject struct { physical time.Time logical int64 } @@ -56,8 +56,44 @@ type timestampOracle struct { // TODO: remove saveInterval saveInterval time.Duration maxResetTSGap func() time.Duration - tso unsafe.Pointer - lastSavedTime atomic.Value + // tso info stored in the memory + tsoMux struct { + sync.RWMutex + tso *tsoObject + } + // last timestamp window stored in etcd + lastSavedTime atomic.Value // stored as time.Time +} + +func (t *timestampOracle) setTSOPhysical(next time.Time) { + t.tsoMux.Lock() + defer t.tsoMux.Unlock() + // make sure the ts won't fall back + if t.tsoMux.tso == nil || typeutil.SubTimeByWallClock(next, t.tsoMux.tso.physical) >= updateTimestampGuard { + t.tsoMux.tso = &tsoObject{physical: next} + } +} + +func (t *timestampOracle) getTSO() (time.Time, int64) { + t.tsoMux.RLock() + defer t.tsoMux.RUnlock() + if t.tsoMux.tso == nil { + return typeutil.ZeroTime, 0 + } + return t.tsoMux.tso.physical, t.tsoMux.tso.logical +} + +// generateTSO will add the TSO's logical part with the given count and returns the new TSO result. +func (t *timestampOracle) generateTSO(count int64) (physical int64, logical int64) { + t.tsoMux.Lock() + defer t.tsoMux.Unlock() + if t.tsoMux.tso == nil { + return 0, 0 + } + physical = t.tsoMux.tso.physical.UnixNano() / int64(time.Millisecond) + t.tsoMux.tso.logical += count + logical = t.tsoMux.tso.logical + return physical, logical } func (t *timestampOracle) getTimestampPath() string { @@ -89,9 +125,7 @@ func (t *timestampOracle) saveTimestamp(leadership *election.Leadership, ts time if !resp.Succeeded { return errs.ErrEtcdTxn.FastGenByArgs() } - t.lastSavedTime.Store(ts) - return nil } @@ -128,12 +162,8 @@ func (t *timestampOracle) SyncTimestamp(leadership *election.Leadership) error { tsoCounter.WithLabelValues("sync_ok").Inc() log.Info("sync and save timestamp", zap.Time("last", last), zap.Time("save", save), zap.Time("next", next)) - - current := &atomicObject{ - physical: next, - } - atomic.StorePointer(&t.tso, unsafe.Pointer(current)) - + // save into memory + t.setTSOPhysical(next) return nil } @@ -142,11 +172,9 @@ func (t *timestampOracle) SyncTimestamp(leadership *election.Leadership) error { // 1. When the SyncTimestamp has not been called yet. // 2. When the ResetUserTimestamp has been called already. func (t *timestampOracle) isInitialized() bool { - tsoNow := (*atomicObject)(atomic.LoadPointer(&t.tso)) - if tsoNow == nil || tsoNow.physical == typeutil.ZeroTime { - return false - } - return true + t.tsoMux.RLock() + defer t.tsoMux.RUnlock() + return t.tsoMux.tso != nil } // ResetUserTimestamp update the physical part with specified TSO. @@ -157,28 +185,27 @@ func (t *timestampOracle) ResetUserTimestamp(leadership *election.Leadership, ts } physical, _ := tsoutil.ParseTS(tso) next := physical.Add(time.Millisecond) - prev := (*atomicObject)(atomic.LoadPointer(&t.tso)) - - // do not update - if typeutil.SubTimeByWallClock(next, prev.physical) <= 3*updateTimestampGuard { + prev, _ := t.getTSO() + // do not update if next is less/before than prev + if typeutil.SubTimeByWallClock(next, prev) < updateTimestampGuard { tsoCounter.WithLabelValues("err_reset_small_ts").Inc() - return errs.ErrResetUserTimestamp.FastGenByArgs("the specified ts too small than now") + return errs.ErrResetUserTimestamp.FastGenByArgs("the specified ts is too small than now") } - - if typeutil.SubTimeByWallClock(next, prev.physical) >= t.maxResetTSGap() { + // do not update if next is too greater than prev + if typeutil.SubTimeByWallClock(next, prev) >= t.maxResetTSGap() { tsoCounter.WithLabelValues("err_reset_large_ts").Inc() - return errs.ErrResetUserTimestamp.FastGenByArgs("the specified ts too large than now") - } - - save := next.Add(t.saveInterval) - if err := t.saveTimestamp(leadership, save); err != nil { - tsoCounter.WithLabelValues("err_save_reset_ts").Inc() - return err + return errs.ErrResetUserTimestamp.FastGenByArgs("the specified ts is too large than now") } - update := &atomicObject{ - physical: next, + // save into etcd only if the time difference is big enough + if typeutil.SubTimeByWallClock(next, prev) > 3*updateTimestampGuard { + save := next.Add(t.saveInterval) + if err := t.saveTimestamp(leadership, save); err != nil { + tsoCounter.WithLabelValues("err_save_reset_ts").Inc() + return err + } } - atomic.CompareAndSwapPointer(&t.tso, unsafe.Pointer(prev), unsafe.Pointer(update)) + // save into memory + t.setTSOPhysical(next) tsoCounter.WithLabelValues("reset_tso_ok").Inc() return nil } @@ -195,7 +222,7 @@ func (t *timestampOracle) ResetUserTimestamp(leadership *election.Leadership, ts // 2. The physical time is monotonically increasing. // 3. The physical time is always less than the saved timestamp. func (t *timestampOracle) UpdateTimestamp(leadership *election.Leadership) error { - prev := (*atomicObject)(atomic.LoadPointer(&t.tso)) + prevPhysical, prevLogical := t.getTSO() now := time.Now() failpoint.Inject("fallBackUpdate", func() { @@ -204,9 +231,9 @@ func (t *timestampOracle) UpdateTimestamp(leadership *election.Leadership) error tsoCounter.WithLabelValues("save").Inc() - jetLag := typeutil.SubTimeByWallClock(now, prev.physical) + jetLag := typeutil.SubTimeByWallClock(now, prevPhysical) if jetLag > 3*UpdateTimestampStep { - log.Warn("clock offset", zap.Duration("jet-lag", jetLag), zap.Time("prev-physical", prev.physical), zap.Time("now", now)) + log.Warn("clock offset", zap.Duration("jet-lag", jetLag), zap.Time("prev-physical", prevPhysical), zap.Time("now", now)) tsoCounter.WithLabelValues("slow_save").Inc() } @@ -215,7 +242,6 @@ func (t *timestampOracle) UpdateTimestamp(leadership *election.Leadership) error } var next time.Time - prevLogical := atomic.LoadInt64(&prev.logical) // If the system time is greater, it will be synchronized with the system time. if jetLag > updateTimestampGuard { next = now @@ -223,7 +249,7 @@ func (t *timestampOracle) UpdateTimestamp(leadership *election.Leadership) error // The reason choosing maxLogical/2 here is that it's big enough for common cases. // Because there is enough timestamp can be allocated before next update. log.Warn("the logical time may be not enough", zap.Int64("prev-logical", prevLogical)) - next = prev.physical.Add(time.Millisecond) + next = prevPhysical.Add(time.Millisecond) } else { // It will still use the previous physical time to alloc the timestamp. tsoCounter.WithLabelValues("skip_save").Inc() @@ -239,13 +265,8 @@ func (t *timestampOracle) UpdateTimestamp(leadership *election.Leadership) error return err } } - - current := &atomicObject{ - physical: next, - logical: 0, - } - - atomic.StorePointer(&t.tso, unsafe.Pointer(current)) + // save into memory + t.setTSOPhysical(next) tsoGauge.WithLabelValues("tso").Set(float64(next.Unix())) return nil @@ -265,20 +286,25 @@ func (t *timestampOracle) getTS(leadership *election.Leadership, count uint32) ( }) for i := 0; i < maxRetryCount; i++ { - current := (*atomicObject)(atomic.LoadPointer(&t.tso)) - if current == nil || current.physical == typeutil.ZeroTime { + currentPhysical, currentLogical := t.getTSO() + if currentPhysical == typeutil.ZeroTime { // If it's leader, maybe SyncTimestamp hasn't completed yet if leadership.Check() { log.Info("sync hasn't completed yet, wait for a while") time.Sleep(200 * time.Millisecond) continue } - log.Error("invalid timestamp", zap.Any("timestamp", current), errs.ZapError(errs.ErrInvalidTimestamp)) + log.Error("invalid timestamp", + zap.Any("timestamp-physical", currentPhysical), + zap.Any("timestamp-logical", currentLogical), + errs.ZapError(errs.ErrInvalidTimestamp)) return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory isn't initialized") } - - resp.Physical = current.physical.UnixNano() / int64(time.Millisecond) - resp.Logical = atomic.AddInt64(¤t.logical, int64(count)) + // Get a new TSO result with the given count + resp.Physical, resp.Logical = t.generateTSO(int64(count)) + if resp.Physical == 0 { + return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory has been reset") + } if resp.Logical >= maxLogical { log.Error("logical part outside of max logical interval, please check ntp time", zap.Reflect("response", resp), @@ -298,9 +324,8 @@ func (t *timestampOracle) getTS(leadership *election.Leadership, count uint32) ( // ResetTimestamp is used to reset the timestamp in memory. func (t *timestampOracle) ResetTimestamp() { + t.tsoMux.Lock() + defer t.tsoMux.Unlock() log.Info("reset the timestamp in memory") - zero := &atomicObject{ - physical: typeutil.ZeroTime, - } - atomic.StorePointer(&t.tso, unsafe.Pointer(zero)) + t.tsoMux.tso = nil } From 427dd4c0a79b4fbdba26395f0acb5f460613aeab Mon Sep 17 00:00:00 2001 From: JmPotato Date: Sun, 27 Sep 2020 11:36:46 +0800 Subject: [PATCH 03/27] tso: add SyncMaxTS RPC method (#2988) Signed-off-by: JmPotato --- go.mod | 2 +- go.sum | 4 +- pkg/errs/errno.go | 1 + pkg/tsoutil/tso.go | 18 ++++++-- server/grpc_service.go | 74 ++++++++++++++++++++++++++++++++- server/tso/allocator_manager.go | 20 +++++++++ server/tso/local_allocator.go | 30 +++++++++++++ 7 files changed, 142 insertions(+), 7 deletions(-) diff --git a/go.mod b/go.mod index 35eec7a4543..c6c9344b9e1 100644 --- a/go.mod +++ b/go.mod @@ -32,7 +32,7 @@ require ( github.com/pingcap/errcode v0.0.0-20180921232412-a1a7271709d9 github.com/pingcap/errors v0.11.5-0.20200917111840-a15ef68f753d github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d - github.com/pingcap/kvproto v0.0.0-20200916031750-f9473f2c5379 + github.com/pingcap/kvproto v0.0.0-20200927025644-73dc27044686 github.com/pingcap/log v0.0.0-20200511115504-543df19646ad github.com/pingcap/sysutil v0.0.0-20200715082929-4c47bcac246a github.com/prometheus/client_golang v1.0.0 diff --git a/go.sum b/go.sum index 7472641b335..4db5a2a5dc9 100644 --- a/go.sum +++ b/go.sum @@ -300,8 +300,8 @@ github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d h1:F8vp38kTAckN+ github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200916031750-f9473f2c5379 h1:KAGE4PYxYLL/dnui3sRCcQHNpcpP5aMl0R/NKzATGgI= -github.com/pingcap/kvproto v0.0.0-20200916031750-f9473f2c5379/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20200927025644-73dc27044686 h1:cf7TL5LMMPQew7vPOtvcfam6AyYxwu5uzcOrHMN8z7k= +github.com/pingcap/kvproto v0.0.0-20200927025644-73dc27044686/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9 h1:AJD9pZYm72vMgPcQDww9rkZ1DnWfl0pXV3BOWlkYIjA= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfEgRJ4T9NGgGTxdHpJerent7rM= diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index af438f8bd84..cc4443e20c3 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -30,6 +30,7 @@ var ( // tso errors var ( ErrGetAllocator = errors.Normalize("get allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetAllocator")) + ErrGetLocalAllocator = errors.Normalize("get local allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetLocalAllocator")) ErrResetUserTimestamp = errors.Normalize("reset user timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrResetUserTimestamp")) ErrGenerateTimestamp = errors.Normalize("generate timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrGenerateTimestamp")) ErrInvalidTimestamp = errors.Normalize("invalid timestamp", errors.RFCCodeText("PD:tso:ErrInvalidTimestamp")) diff --git a/pkg/tsoutil/tso.go b/pkg/tsoutil/tso.go index da5a28c5e2f..e0f09707860 100644 --- a/pkg/tsoutil/tso.go +++ b/pkg/tsoutil/tso.go @@ -34,8 +34,20 @@ func ParseTS(ts uint64) (time.Time, uint64) { // ParseTimestamp parses `pdpb.Timestamp` to `time.Time` func ParseTimestamp(ts pdpb.Timestamp) (time.Time, uint64) { - logical := uint64(ts.Logical) - physical := ts.Physical - physicalTime := time.Unix(int64(physical/1000), int64(physical)%1000*time.Millisecond.Nanoseconds()) + logical := uint64(ts.GetLogical()) + physicalTime := time.Unix(ts.GetPhysical()/1000, ts.GetPhysical()%1000*time.Millisecond.Nanoseconds()) return physicalTime, logical } + +// GenerateTS generate an `uint64` TS by passing a `pdpb.Timestamp`. +func GenerateTS(ts *pdpb.Timestamp) uint64 { + return uint64(ts.GetPhysical())<<18 | uint64(ts.GetLogical())&0x3FFFF +} + +// GenerateTimestamp generate a `pdpb.Timestamp` by passing `time.Time` and `uint64` +func GenerateTimestamp(physical time.Time, logical uint64) *pdpb.Timestamp { + return &pdpb.Timestamp{ + Physical: physical.UnixNano() / int64(time.Millisecond), + Logical: int64(logical), + } +} diff --git a/server/grpc_service.go b/server/grpc_service.go index 9efbb5ff319..52b0c81c1a8 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -93,7 +93,7 @@ func (s *Server) Tso(stream pdpb.PD_TsoServer) error { return status.Errorf(codes.FailedPrecondition, "mismatch cluster id, need %d but got %d", s.clusterID, request.GetHeader().GetClusterId()) } count := request.GetCount() - ts, err := s.tsoAllocatorManager.HandleTSORequest(config.GlobalDCLocation, count) + ts, err := s.tsoAllocatorManager.HandleTSORequest(request.GetDcLocation(), count) if err != nil { return status.Errorf(codes.Unknown, err.Error()) } @@ -908,3 +908,75 @@ func (s *Server) incompatibleVersion(tag string) *pdpb.ResponseHeader { Message: msg, }) } + +// SyncMaxTS is a RPC method used to synchronize the timestamp of TSO between the +// Global TSO Allocator and multi Local TSO Allocator leaders. It contains two +// phases: +// 1. Collect timestamps among all Local TSO Allocator leaders, and choose the +// greatest one as MaxTS. +// 2. Send the MaxTS to all Local TSO Allocator leaders. They will compare MaxTS +// with its current TSO in memory to make sure their local TSOs are not less +// than MaxTS by writing MaxTS into memory to finish the global TSO synchronization. +func (s *Server) SyncMaxTS(ctx context.Context, request *pdpb.SyncMaxTSRequest) (*pdpb.SyncMaxTSResponse, error) { + if err := s.validateInternalRequest(request.GetHeader()); err != nil { + return nil, err + } + tsoAllocatorManager := s.GetTSOAllocatorManager() + // Get all Local TSO Allocator leaders + allocatorLeaders, err := tsoAllocatorManager.GetLocalAllocatorLeaders() + if err != nil { + return nil, err + } + var processedDCs []string + if request.GetMaxTs() == nil || request.GetMaxTs().Physical == 0 { + // The first phase of synchronization: collect the max local ts + var maxLocalTS pdpb.Timestamp + for _, allocator := range allocatorLeaders { + // No longer leader, just skip here because + // the global allocator will check if all DCs are handled. + if !allocator.IsStillAllocatorLeader() { + continue + } + currentLocalTSO, err := allocator.GetCurrentTSO() + if err != nil { + return nil, err + } + if currentLocalTSO.Physical > maxLocalTS.Physical { + maxLocalTS.Physical = currentLocalTSO.Physical + } + processedDCs = append(processedDCs, allocator.GetDCLocation()) + } + return &pdpb.SyncMaxTSResponse{ + Header: s.header(), + MaxLocalTs: &maxLocalTS, + Dcs: processedDCs, + }, nil + } + // The second phase of synchronization: do the writing + for _, allocator := range allocatorLeaders { + if !allocator.IsStillAllocatorLeader() { + continue + } + if err := allocator.WriteTSO(request.GetMaxTs()); err != nil { + return nil, err + } + processedDCs = append(processedDCs, allocator.GetDCLocation()) + } + return &pdpb.SyncMaxTSResponse{ + Header: s.header(), + Dcs: processedDCs, + }, nil +} + +// validateInternalRequest checks if server is closed, which is used to validate +// the gRPC communication between PD servers internally. +func (s *Server) validateInternalRequest(header *pdpb.RequestHeader) error { + if s.IsClosed() { + return errors.WithStack(ErrNotStarted) + } + leaderID := s.GetLeader().GetMemberId() + if leaderID != header.GetSenderId() { + return status.Errorf(codes.FailedPrecondition, "mismatch leader id, need %d but got %d", leaderID, header.GetSenderId()) + } + return nil +} diff --git a/server/tso/allocator_manager.go b/server/tso/allocator_manager.go index f1e71b113f8..a0d50af6dba 100644 --- a/server/tso/allocator_manager.go +++ b/server/tso/allocator_manager.go @@ -392,6 +392,9 @@ func (am *AllocatorManager) deleteAllocatorGroup(dcLocation string) { func (am *AllocatorManager) HandleTSORequest(dcLocation string, count uint32) (pdpb.Timestamp, error) { am.RLock() defer am.RUnlock() + if len(dcLocation) == 0 { + dcLocation = config.GlobalDCLocation + } allocatorGroup, exist := am.allocatorGroups[dcLocation] if !exist { err := errs.ErrGetAllocator.FastGenByArgs(fmt.Sprintf("%s allocator not found, generate timestamp failed", dcLocation)) @@ -443,3 +446,20 @@ func (am *AllocatorManager) GetAllocators(filters ...AllocatorGroupFilter) []All } return allocators } + +// GetLocalAllocatorLeaders returns all Local TSO Allocator leaders this server holds. +func (am *AllocatorManager) GetLocalAllocatorLeaders() ([]*LocalTSOAllocator, error) { + localAllocators := am.GetAllocators( + FilterDCLocation(config.GlobalDCLocation), + FilterUnavailableLeadership(), + FilterUninitialized()) + localAllocatorLeaders := make([]*LocalTSOAllocator, len(localAllocators)) + for _, localAllocator := range localAllocators { + localAllocatorLeader, ok := localAllocator.(*LocalTSOAllocator) + if !ok { + return nil, errs.ErrGetLocalAllocator.FastGenByArgs("invalid local tso allocator found") + } + localAllocatorLeaders = append(localAllocatorLeaders, localAllocatorLeader) + } + return localAllocatorLeaders, nil +} diff --git a/server/tso/local_allocator.go b/server/tso/local_allocator.go index 4817befdbdb..63e420fa958 100644 --- a/server/tso/local_allocator.go +++ b/server/tso/local_allocator.go @@ -21,6 +21,8 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/tsoutil" + "github.com/tikv/pd/pkg/typeutil" "github.com/tikv/pd/server/election" "github.com/tikv/pd/server/member" "go.uber.org/zap" @@ -60,6 +62,11 @@ func NewLocalTSOAllocator(member *member.Member, leadership *election.Leadership } } +// GetDCLocation returns the local allocator's dc-location. +func (lta *LocalTSOAllocator) GetDCLocation() string { + return lta.dcLocation +} + // Initialize will initialize the created local TSO allocator. func (lta *LocalTSOAllocator) Initialize() error { return lta.timestampOracle.SyncTimestamp(lta.leadership) @@ -117,6 +124,29 @@ func (lta *LocalTSOAllocator) GetMember() *pdpb.Member { return lta.member.Member() } +// GetCurrentTSO returns current TSO in memory. +func (lta *LocalTSOAllocator) GetCurrentTSO() (pdpb.Timestamp, error) { + currentPhysical, currentLogical := lta.timestampOracle.getTSO() + if currentPhysical == typeutil.ZeroTime { + return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory isn't initialized") + } + return *tsoutil.GenerateTimestamp(currentPhysical, uint64(currentLogical)), nil +} + +// WriteTSO is used to set the maxTS as current TSO in memory. +func (lta *LocalTSOAllocator) WriteTSO(maxTS *pdpb.Timestamp) error { + currentTSO, err := lta.GetCurrentTSO() + if err != nil { + return err + } + // If current local TSO has already been greater than + // maxTS, then do not update it. + if currentTSO.Physical >= maxTS.Physical { + return nil + } + return lta.SetTSO(tsoutil.GenerateTS(maxTS)) +} + // EnableAllocatorLeader sets the Local TSO Allocator itself to a leader. func (lta *LocalTSOAllocator) EnableAllocatorLeader() { lta.setAllocatorLeader(lta.member.Member()) From adc0dd83c20d40150b92b12d3909b87edb61b640 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Sun, 27 Sep 2020 17:28:59 +0800 Subject: [PATCH 04/27] tools: enhance the pd-tso-bench (#3032) Signed-off-by: JmPotato --- tools/pd-tso-bench/README.md | 49 +++++---- tools/pd-tso-bench/main.go | 199 +++++++++++++++++++++++++---------- 2 files changed, 175 insertions(+), 73 deletions(-) diff --git a/tools/pd-tso-bench/README.md b/tools/pd-tso-bench/README.md index 7b0322c9ce7..de20bcf3e80 100644 --- a/tools/pd-tso-bench/README.md +++ b/tools/pd-tso-bench/README.md @@ -15,32 +15,43 @@ This section describes how to benchmark the GetTS performance. ### Flags description ``` --pd string - Specify a PD address (default: "http://127.0.0.1:2379") --C int - Specify the concurrency (default: "1000") --interval duration - Specify the interval to output the statistics (default: "1s") +-c int + concurrency (default 1000) -cacert string - Specify the path to the trusted CA certificate file in PEM format + path of file that contains list of trusted SSL CAs -cert string - Specify the path to the SSL certificate file in PEM format + path of file that contains X509 certificate in PEM format +-client int + the number of pd clients involved in each benchmark (default 1) +-count int + the count number that the test will run (default 1) +-duration duration + how many seconds the test will last (default 1m0s) +-interval duration + interval to output the statistics (default 1s) -key string - Specify the path to the SSL certificate key file in PEM format, which is the private key of the certificate specified by `--cert` + path of file that contains X509 key in PEM format +-pd string + pd address (default "127.0.0.1:2379") +-v output statistics info every interval and output metrics info at the end ``` Benchmark the GetTS performance: - ./pd-tso-bench + ./pd-tso-bench -v -duration 5s It will print some benchmark results like: -```bash -count:606148, max:9, min:0, >1ms:487565, >2ms:108403, >5ms:902, >10ms:0, >30ms:0 -count:714375, max:5, min:0, >1ms:690071, >2ms:13864, >5ms:1, >10ms:0, >30ms:0 -count:634645, max:6, min:0, >1ms:528354, >2ms:98148, >5ms:46, >10ms:0, >30ms:0 -count:565745, max:10, min:0, >1ms:420304, >2ms:135403, >5ms:3792, >10ms:1, >30ms:0 -count:583051, max:11, min:0, >1ms:439761, >2ms:135822, >5ms:1657, >10ms:1, >30ms:0 -count:630377, max:6, min:0, >1ms:526209, >2ms:95165, >5ms:396, >10ms:0, >30ms:0 -count:688006, max:4, min:0, >1ms:626094, >2ms:49262, >5ms:0, >10ms:0, >30ms:0 -... + +```shell +Start benchmark #0, duration: 5s +Create 3 client(s) for benchmark +count:907656, max:3, min:0, >1ms:545903, >2ms:7191, >5ms:0, >10ms:0, >30ms:0 >50ms:0 >100ms:0 >200ms:0 >400ms:0 >800ms:0 >1s:0 +count:892034, max:4, min:0, >1ms:585632, >2ms:11359, >5ms:0, >10ms:0, >30ms:0 >50ms:0 >100ms:0 >200ms:0 >400ms:0 >800ms:0 >1s:0 +count:909465, max:5, min:0, >1ms:564465, >2ms:9572, >5ms:14, >10ms:0, >30ms:0 >50ms:0 >100ms:0 >200ms:0 >400ms:0 >800ms:0 >1s:0 +count:867047, max:6, min:0, >1ms:546294, >2ms:22527, >5ms:1728, >10ms:0, >30ms:0 >50ms:0 >100ms:0 >200ms:0 >400ms:0 >800ms:0 >1s:0 +count:482854, max:9, min:0, >1ms:277221, >2ms:162617, >5ms:15097, >10ms:0, >30ms:0 >50ms:0 >100ms:0 >200ms:0 >400ms:0 >800ms:0 >1s:0 + +Total: +count:4059056, max:9, min:0, >1ms:2519515, >2ms:213266, >5ms:16839, >10ms:0, >30ms:0 >50ms:0 >100ms:0 >200ms:0 >400ms:0 >800ms:0 >1s:0 +count:4059056, >1ms:62.07%, >2ms:5.25%, >5ms:0.41%, >10ms:0.00%, >30ms:0.00% >50ms:0.00% >100ms:0.00% >200ms:0.00% >400ms:0.00% >800ms:0.00% >1s:0.00% ``` \ No newline at end of file diff --git a/tools/pd-tso-bench/main.go b/tools/pd-tso-bench/main.go index be996fb44d9..0c3054f119e 100644 --- a/tools/pd-tso-bench/main.go +++ b/tools/pd-tso-bench/main.go @@ -34,13 +34,17 @@ import ( ) var ( - pdAddrs = flag.String("pd", "127.0.0.1:2379", "pd address") - concurrency = flag.Int("C", 1000, "concurrency") - interval = flag.Duration("interval", time.Second, "interval to output the statistics") - caPath = flag.String("cacert", "", "path of file that contains list of trusted SSL CAs") - certPath = flag.String("cert", "", "path of file that contains X509 certificate in PEM format") - keyPath = flag.String("key", "", "path of file that contains X509 key in PEM format") - wg sync.WaitGroup + pdAddrs = flag.String("pd", "127.0.0.1:2379", "pd address") + clientNumber = flag.Int("client", 1, "the number of pd clients involved in each benchmark") + concurrency = flag.Int("c", 1000, "concurrency") + count = flag.Int("count", 1, "the count number that the test will run") + duration = flag.Duration("duration", 60*time.Second, "how many seconds the test will last") + verbose = flag.Bool("v", false, "output statistics info every interval and output metrics info at the end") + interval = flag.Duration("interval", time.Second, "interval to output the statistics") + caPath = flag.String("cacert", "", "path of file that contains list of trusted SSL CAs") + certPath = flag.String("cert", "", "path of file that contains X509 certificate in PEM format") + keyPath = flag.String("key", "", "path of file that contains X509 key in PEM format") + wg sync.WaitGroup ) var promServer *httptest.Server @@ -54,52 +58,73 @@ func collectMetrics(server *httptest.Server) string { } func main() { - promServer = httptest.NewServer(promhttp.Handler()) flag.Parse() + ctx, cancel := context.WithCancel(context.Background()) + + sc := make(chan os.Signal, 1) + signal.Notify(sc, + syscall.SIGHUP, + syscall.SIGINT, + syscall.SIGTERM, + syscall.SIGQUIT) + go func() { + <-sc + cancel() + }() + + for i := 0; i < *count; i++ { + fmt.Printf("\nStart benchmark #%d, duration: %+vs\n", i, (*duration).Seconds()) + bench(ctx) + } +} + +func bench(mainCtx context.Context) { + promServer = httptest.NewServer(promhttp.Handler()) - pdCli, err := pd.NewClient([]string{*pdAddrs}, pd.SecurityOption{ - CAPath: *caPath, - CertPath: *certPath, - KeyPath: *keyPath, - }) - if err != nil { - log.Fatal(fmt.Sprintf("%v", err)) + // Initialize all clients + fmt.Printf("Create %d client(s) for benchmark\n", *count) + pdClients := make([]pd.Client, *clientNumber) + for idx := range pdClients { + pdCli, err := pd.NewClient([]string{*pdAddrs}, pd.SecurityOption{ + CAPath: *caPath, + CertPath: *certPath, + KeyPath: *keyPath, + }) + if err != nil { + log.Fatal(fmt.Sprintf("create pd client #%d failed: %v", idx, err)) + } + pdClients[idx] = pdCli } - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancel(mainCtx) // To avoid the first time high latency. - for i := 0; i < *concurrency; i++ { - _, _, err = pdCli.GetTS(ctx) + for idx, pdCli := range pdClients { + _, _, err := pdCli.GetTS(ctx) if err != nil { - log.Fatal("get tso failed", zap.Error(err)) + log.Fatal("get first time tso failed", zap.Int("client-number", idx), zap.Error(err)) } } - durCh := make(chan time.Duration, *concurrency*2) + durCh := make(chan time.Duration, 2*(*concurrency)*(*clientNumber)) - wg.Add(*concurrency) - for i := 0; i < *concurrency; i++ { - go reqWorker(ctx, pdCli, durCh) + wg.Add((*concurrency) * (*clientNumber)) + for _, pdCli := range pdClients { + for i := 0; i < *concurrency; i++ { + go reqWorker(ctx, pdCli, durCh) + } } wg.Add(1) go showStats(ctx, durCh) - sc := make(chan os.Signal, 1) - signal.Notify(sc, - syscall.SIGHUP, - syscall.SIGINT, - syscall.SIGTERM, - syscall.SIGQUIT) - - go func() { - <-sc - cancel() - }() + time.Sleep(*duration) + cancel() wg.Wait() - pdCli.Close() + for _, pdCli := range pdClients { + pdCli.Close() + } } func showStats(ctx context.Context, durCh chan time.Duration) { @@ -117,36 +142,53 @@ func showStats(ctx context.Context, durCh chan time.Duration) { select { case <-ticker.C: //runtime.GC() - println(s.String()) + if *verbose { + fmt.Println(s.Counter()) + } total.merge(s) s = newStats() case d := <-durCh: s.update(d) case <-statCtx.Done(): - println("\nTotal:") - println(total.String()) - println(collectMetrics(promServer)) + fmt.Println("\nTotal:") + fmt.Println(total.Counter()) + fmt.Println(total.Percentage()) + if *verbose { + fmt.Println(collectMetrics(promServer)) + } return } } } const ( - twoDur = time.Millisecond * 2 - fiveDur = time.Millisecond * 5 - tenDur = time.Millisecond * 10 - thirtyDur = time.Millisecond * 30 + twoDur = time.Millisecond * 2 + fiveDur = time.Millisecond * 5 + tenDur = time.Millisecond * 10 + thirtyDur = time.Millisecond * 30 + fiftyDur = time.Millisecond * 50 + oneHundredDur = time.Millisecond * 100 + twoHundredDur = time.Millisecond * 200 + fourHundredDur = time.Millisecond * 400 + eightHundredDur = time.Millisecond * 800 + oneThousandDur = time.Millisecond * 1000 ) type stats struct { - maxDur time.Duration - minDur time.Duration - count int - milliCnt int - twoMilliCnt int - fiveMilliCnt int - tenMSCnt int - thirtyCnt int + maxDur time.Duration + minDur time.Duration + count int + milliCnt int + twoMilliCnt int + fiveMilliCnt int + tenMSCnt int + thirtyCnt int + fiftyCnt int + oneHundredCnt int + twoHundredCnt int + fourHundredCnt int + eightHundredCnt int + oneThousandCnt int } func newStats() *stats { @@ -166,6 +208,36 @@ func (s *stats) update(dur time.Duration) { s.minDur = dur } + if dur > oneThousandDur { + s.oneThousandCnt++ + return + } + + if dur > eightHundredDur { + s.eightHundredCnt++ + return + } + + if dur > fourHundredDur { + s.fourHundredCnt++ + return + } + + if dur > twoHundredDur { + s.twoHundredCnt++ + return + } + + if dur > oneHundredDur { + s.oneHundredCnt++ + return + } + + if dur > fiftyDur { + s.fiftyCnt++ + return + } + if dur > thirtyDur { s.thirtyCnt++ return @@ -206,12 +278,31 @@ func (s *stats) merge(other *stats) { s.fiveMilliCnt += other.fiveMilliCnt s.tenMSCnt += other.tenMSCnt s.thirtyCnt += other.thirtyCnt + s.fiftyCnt += other.fiftyCnt + s.oneHundredCnt += other.oneHundredCnt + s.twoHundredCnt += other.twoHundredCnt + s.fourHundredCnt += other.fourHundredCnt + s.eightHundredCnt += other.eightHundredCnt + s.oneThousandCnt += other.oneThousandCnt } -func (s *stats) String() string { - return fmt.Sprintf("count:%d, max:%d, min:%d, >1ms:%d, >2ms:%d, >5ms:%d, >10ms:%d, >30ms:%d", +func (s *stats) Counter() string { + return fmt.Sprintf( + "count:%d, max:%d, min:%d, >1ms:%d, >2ms:%d, >5ms:%d, >10ms:%d, >30ms:%d >50ms:%d >100ms:%d >200ms:%d >400ms:%d >800ms:%d >1s:%d", s.count, s.maxDur.Nanoseconds()/int64(time.Millisecond), s.minDur.Nanoseconds()/int64(time.Millisecond), - s.milliCnt, s.twoMilliCnt, s.fiveMilliCnt, s.tenMSCnt, s.thirtyCnt) + s.milliCnt, s.twoMilliCnt, s.fiveMilliCnt, s.tenMSCnt, s.thirtyCnt, s.fiftyCnt, s.oneHundredCnt, s.twoHundredCnt, s.fourHundredCnt, + s.eightHundredCnt, s.oneThousandCnt) +} + +func (s *stats) Percentage() string { + return fmt.Sprintf( + "count:%d, >1ms:%2.2f%%, >2ms:%2.2f%%, >5ms:%2.2f%%, >10ms:%2.2f%%, >30ms:%2.2f%% >50ms:%2.2f%% >100ms:%2.2f%% >200ms:%2.2f%% >400ms:%2.2f%% >800ms:%2.2f%% >1s:%2.2f%%", s.count, + s.calculate(s.milliCnt), s.calculate(s.twoMilliCnt), s.calculate(s.fiveMilliCnt), s.calculate(s.tenMSCnt), s.calculate(s.thirtyCnt), s.calculate(s.fiftyCnt), + s.calculate(s.oneHundredCnt), s.calculate(s.twoHundredCnt), s.calculate(s.fourHundredCnt), s.calculate(s.eightHundredCnt), s.calculate(s.oneThousandCnt)) +} + +func (s *stats) calculate(count int) float64 { + return float64(count) * 100 / float64(s.count) } func reqWorker(ctx context.Context, pdCli pd.Client, durCh chan time.Duration) { From 458b7aed2637b1c49f82a38292c86217d0a8ff0f Mon Sep 17 00:00:00 2001 From: Zheming Li Date: Mon, 28 Sep 2020 12:23:42 +0800 Subject: [PATCH 05/27] pdctl: add state filter support in store command (#3018) * pdctl: add state filter support in store command Signed-off-by: Zheming Li --- server/api/store.go | 3 ++- tests/pdctl/store/store_test.go | 9 +++++++ tools/pd-ctl/pdctl/command/store_command.go | 26 +++++++++++++++++++++ 3 files changed, 37 insertions(+), 1 deletion(-) diff --git a/server/api/store.go b/server/api/store.go index b839f38d59b..cad231b5fa7 100644 --- a/server/api/store.go +++ b/server/api/store.go @@ -143,6 +143,7 @@ func newStoreHandler(handler *server.Handler, rd *render.Render) *storeHandler { // @Produce json // @Success 200 {object} StoreInfo // @Failure 400 {string} string "The input is invalid." +// @Failure 404 {string} string "The store does not exist." // @Failure 500 {string} string "PD server failed to proceed the request." // @Router /store/{id} [get] func (h *storeHandler) Get(w http.ResponseWriter, r *http.Request) { @@ -156,7 +157,7 @@ func (h *storeHandler) Get(w http.ResponseWriter, r *http.Request) { store := rc.GetStore(storeID) if store == nil { - h.rd.JSON(w, http.StatusInternalServerError, server.ErrStoreNotFound(storeID).Error()) + h.rd.JSON(w, http.StatusNotFound, server.ErrStoreNotFound(storeID).Error()) return } diff --git a/tests/pdctl/store/store_test.go b/tests/pdctl/store/store_test.go index 86baacd72b2..c8788e48bdc 100644 --- a/tests/pdctl/store/store_test.go +++ b/tests/pdctl/store/store_test.go @@ -88,6 +88,14 @@ func (s *storeTestSuite) TestStore(c *C) { c.Assert(json.Unmarshal(output, &storesInfo), IsNil) pdctl.CheckStoresInfo(c, storesInfo.Stores, stores[:2]) + // store --state= command + args = []string{"-u", pdAddr, "store", "--state", "Up,Tombstone"} + _, output, err = pdctl.ExecuteCommandC(cmd, args...) + c.Assert(err, IsNil) + storesInfo = new(api.StoresInfo) + c.Assert(json.Unmarshal(output, &storesInfo), IsNil) + pdctl.CheckStoresInfo(c, storesInfo.Stores, stores) + // store command args = []string{"-u", pdAddr, "store", "1"} _, output, err = pdctl.ExecuteCommandC(cmd, args...) @@ -298,4 +306,5 @@ func (s *storeTestSuite) TestStore(c *C) { err = json.Unmarshal(output, scene) c.Assert(err, IsNil) c.Assert(scene.Idle, Equals, 100) + } diff --git a/tools/pd-ctl/pdctl/command/store_command.go b/tools/pd-ctl/pdctl/command/store_command.go index 146f0e584c7..91f4671bac1 100644 --- a/tools/pd-ctl/pdctl/command/store_command.go +++ b/tools/pd-ctl/pdctl/command/store_command.go @@ -19,7 +19,9 @@ import ( "net/http" "path" "strconv" + "strings" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/spf13/cobra" ) @@ -43,6 +45,7 @@ func NewStoreCommand() *cobra.Command { s.AddCommand(NewRemoveTombStoneCommand()) s.AddCommand(NewStoreLimitSceneCommand()) s.Flags().String("jq", "", "jq query") + s.Flags().StringSlice("state", nil, "state filter") return s } @@ -109,6 +112,7 @@ func NewStoresCommand() *cobra.Command { s.AddCommand(NewSetStoresCommand()) s.AddCommand(NewShowStoresCommand()) s.Flags().String("jq", "", "jq query") + s.Flags().StringSlice("state", nil, "state filter") return s } @@ -228,12 +232,34 @@ func storeLimitSceneCommandFunc(cmd *cobra.Command, args []string) { func showStoreCommandFunc(cmd *cobra.Command, args []string) { prefix := storesPrefix + if len(args) > 1 { + cmd.Usage() + return + } if len(args) == 1 { if _, err := strconv.Atoi(args[0]); err != nil { cmd.Println("store_id should be a number") return } prefix = fmt.Sprintf(storePrefix, args[0]) + } else { + flags := cmd.Flags() + states, err := flags.GetStringSlice("state") + if err != nil { + cmd.Printf("Failed to get state: %s\n", err) + } + stateValues := make([]string, 0, len(states)) + for _, state := range states { + stateValue, ok := metapb.StoreState_value[state] + if !ok { + cmd.Println("Unknown state: " + state) + return + } + stateValues = append(stateValues, fmt.Sprintf("state=%v", stateValue)) + } + if len(stateValues) != 0 { + prefix = fmt.Sprintf("%v?%v", storesPrefix, strings.Join(stateValues, "&")) + } } r, err := doRequest(cmd, prefix, http.MethodGet) if err != nil { From 177f8f8436999be42872e886825df9f2f67ad249 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Mon, 28 Sep 2020 13:55:12 +0800 Subject: [PATCH 06/27] api: add check learner-peer API (#3036) Signed-off-by: Ryan Leung --- server/api/region.go | 17 ++ server/api/router.go | 4 +- server/handler.go | 9 + tests/pdctl/region/region_test.go | 228 ++++++------------- tools/pd-ctl/pdctl/command/region_command.go | 2 +- 5 files changed, 101 insertions(+), 159 deletions(-) diff --git a/server/api/region.go b/server/api/region.go index 81b5c643c02..26f6d7a4cc4 100644 --- a/server/api/region.go +++ b/server/api/region.go @@ -327,6 +327,23 @@ func (h *regionsHandler) GetDownPeerRegions(w http.ResponseWriter, r *http.Reque h.rd.JSON(w, http.StatusOK, regionsInfo) } +// @Tags region +// @Summary List all regions that has learner peer. +// @Produce json +// @Success 200 {object} RegionsInfo +// @Failure 500 {string} string "PD server failed to proceed the request." +// @Router /regions/check/learner-peer [get] +func (h *regionsHandler) GetLearnerPeerRegions(w http.ResponseWriter, r *http.Request) { + handler := h.svr.GetHandler() + regions, err := handler.GetLearnerPeerRegions() + if err != nil { + h.rd.JSON(w, http.StatusInternalServerError, err.Error()) + return + } + regionsInfo := convertToAPIRegions(regions) + h.rd.JSON(w, http.StatusOK, regionsInfo) +} + // @Tags region // @Summary List all regions that has offline peer. // @Produce json diff --git a/server/api/router.go b/server/api/router.go index c01a80a08ad..b45dcedd6cc 100644 --- a/server/api/router.go +++ b/server/api/router.go @@ -160,8 +160,10 @@ func createRouter(ctx context.Context, prefix string, svr *server.Server) *mux.R clusterRouter.HandleFunc("/regions/check/extra-peer", regionsHandler.GetExtraPeerRegions).Methods("GET") clusterRouter.HandleFunc("/regions/check/pending-peer", regionsHandler.GetPendingPeerRegions).Methods("GET") clusterRouter.HandleFunc("/regions/check/down-peer", regionsHandler.GetDownPeerRegions).Methods("GET") - clusterRouter.HandleFunc("/regions/check/offline-peer", regionsHandler.GetOfflinePeer).Methods("GET") + clusterRouter.HandleFunc("/regions/check/learner-peer", regionsHandler.GetLearnerPeerRegions).Methods("GET") clusterRouter.HandleFunc("/regions/check/empty-region", regionsHandler.GetEmptyRegion).Methods("GET") + clusterRouter.HandleFunc("/regions/check/offline-peer", regionsHandler.GetOfflinePeer).Methods("GET") + clusterRouter.HandleFunc("/regions/check/hist-size", regionsHandler.GetSizeHistogram).Methods("GET") clusterRouter.HandleFunc("/regions/check/hist-keys", regionsHandler.GetKeysHistogram).Methods("GET") clusterRouter.HandleFunc("/regions/sibling/{id}", regionsHandler.GetRegionSiblings).Methods("GET") diff --git a/server/handler.go b/server/handler.go index 713e105df90..c3b3afced3d 100644 --- a/server/handler.go +++ b/server/handler.go @@ -781,6 +781,15 @@ func (h *Handler) GetDownPeerRegions() ([]*core.RegionInfo, error) { return c.GetRegionStatsByType(statistics.DownPeer), nil } +// GetLearnerPeerRegions gets the region with learner peer. +func (h *Handler) GetLearnerPeerRegions() ([]*core.RegionInfo, error) { + c := h.s.GetRaftCluster() + if c == nil { + return nil, errs.ErrNotBootstrapped.FastGenByArgs() + } + return c.GetRegionStatsByType(statistics.LearnerPeer), nil +} + // GetExtraPeerRegions gets the region exceeds the specified number of peers. func (h *Handler) GetExtraPeerRegions() ([]*core.RegionInfo, error) { c := h.s.GetRaftCluster() diff --git a/tests/pdctl/region/region_test.go b/tests/pdctl/region/region_test.go index 2dc76c1096d..e90c588a137 100644 --- a/tests/pdctl/region/region_test.go +++ b/tests/pdctl/region/region_test.go @@ -95,168 +95,82 @@ func (s *regionTestSuite) TestRegion(c *C) { r3 := pdctl.MustPutRegion(c, cluster, 3, 1, []byte("c"), []byte("d"), core.SetWrittenBytes(500), core.SetReadBytes(800), core.SetRegionConfVer(3), core.SetRegionVersion(2), core.SetApproximateSize(30), core.WithDownPeers([]*pdpb.PeerStats{{Peer: downPeer, DownSeconds: 3600}}), - core.WithPendingPeers([]*metapb.Peer{downPeer})) + core.WithPendingPeers([]*metapb.Peer{downPeer}), core.WithLearners([]*metapb.Peer{{Id: 3, StoreId: 1}})) r4 := pdctl.MustPutRegion(c, cluster, 4, 1, []byte("d"), []byte("e"), core.SetWrittenBytes(100), core.SetReadBytes(100), core.SetRegionConfVer(1), core.SetRegionVersion(1), core.SetApproximateSize(10)) defer cluster.Destroy() - // region command - args := []string{"-u", pdAddr, "region"} - _, output, err := pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo := api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - regions := leaderServer.GetRegions() - pdctl.CheckRegionsInfo(c, regionsInfo, regions) - - // region command - args = []string{"-u", pdAddr, "region", "1"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionInfo := api.RegionInfo{} - c.Assert(json.Unmarshal(output, ®ionInfo), IsNil) - region := leaderServer.GetRegionInfoByID(1) - c.Assert(api.NewRegionInfo(region), DeepEquals, ®ionInfo) - - // region sibling command - args = []string{"-u", pdAddr, "region", "sibling", "2"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - region = leaderServer.GetRegionInfoByID(2) - regions = leaderServer.GetAdjacentRegions(region) - pdctl.CheckRegionsInfo(c, regionsInfo, regions) - - // region store command - args = []string{"-u", pdAddr, "region", "store", "1"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - regions = leaderServer.GetStoreRegions(1) - pdctl.CheckRegionsInfo(c, regionsInfo, regions) - - // region topread [limit] command - args = []string{"-u", pdAddr, "region", "topread", "2"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - regions = api.TopNRegions(leaderServer.GetRegions(), func(a, b *core.RegionInfo) bool { return a.GetBytesRead() < b.GetBytesRead() }, 2) - pdctl.CheckRegionsInfo(c, regionsInfo, regions) - - // region topwrite [limit] command - args = []string{"-u", pdAddr, "region", "topwrite", "2"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - regions = api.TopNRegions(leaderServer.GetRegions(), func(a, b *core.RegionInfo) bool { return a.GetBytesWritten() < b.GetBytesWritten() }, 2) - pdctl.CheckRegionsInfo(c, regionsInfo, regions) - - // region topconfver [limit] command - args = []string{"-u", pdAddr, "region", "topconfver", "2"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - regions = api.TopNRegions(leaderServer.GetRegions(), func(a, b *core.RegionInfo) bool { - return a.GetMeta().GetRegionEpoch().GetConfVer() < b.GetMeta().GetRegionEpoch().GetConfVer() - }, 2) - pdctl.CheckRegionsInfo(c, regionsInfo, regions) - - // region topversion [limit] command - args = []string{"-u", pdAddr, "region", "topversion", "2"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - regions = api.TopNRegions(leaderServer.GetRegions(), func(a, b *core.RegionInfo) bool { - return a.GetMeta().GetRegionEpoch().GetVersion() < b.GetMeta().GetRegionEpoch().GetVersion() - }, 2) - pdctl.CheckRegionsInfo(c, regionsInfo, regions) - - // region topsize [limit] command - args = []string{"-u", pdAddr, "region", "topsize", "2"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - regions = api.TopNRegions(leaderServer.GetRegions(), func(a, b *core.RegionInfo) bool { - return a.GetApproximateSize() < b.GetApproximateSize() - }, 2) - pdctl.CheckRegionsInfo(c, regionsInfo, regions) - - // region check extra-peer command - args = []string{"-u", pdAddr, "region", "check", "extra-peer"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - pdctl.CheckRegionsInfo(c, regionsInfo, []*core.RegionInfo{r1}) - - // region check miss-peer command - args = []string{"-u", pdAddr, "region", "check", "miss-peer"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - pdctl.CheckRegionsInfo(c, regionsInfo, []*core.RegionInfo{r2, r3, r4}) - - // region check pending-peer command - args = []string{"-u", pdAddr, "region", "check", "pending-peer"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - pdctl.CheckRegionsInfo(c, regionsInfo, []*core.RegionInfo{r3}) - - // region check down-peer command - args = []string{"-u", pdAddr, "region", "check", "down-peer"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - pdctl.CheckRegionsInfo(c, regionsInfo, []*core.RegionInfo{r3}) - - // region key --format=raw command - args = []string{"-u", pdAddr, "region", "key", "--format=raw", "b"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionInfo = api.RegionInfo{} - c.Assert(json.Unmarshal(output, ®ionInfo), IsNil) - c.Assert(®ionInfo, DeepEquals, api.NewRegionInfo(r2)) - - // region key --format=hex command - args = []string{"-u", pdAddr, "region", "key", "--format=hex", "62"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionInfo = api.RegionInfo{} - c.Assert(json.Unmarshal(output, ®ionInfo), IsNil) - c.Assert(®ionInfo, DeepEquals, api.NewRegionInfo(r2)) + var testRegionsCases = []struct { + args []string + expect []*core.RegionInfo + }{ + // region command + {[]string{"region"}, leaderServer.GetRegions()}, + // region sibling command + {[]string{"region", "sibling", "2"}, leaderServer.GetAdjacentRegions(leaderServer.GetRegionInfoByID(2))}, + // region store command + {[]string{"region", "store", "1"}, leaderServer.GetStoreRegions(1)}, + // region topread [limit] command + {[]string{"region", "topread", "2"}, api.TopNRegions(leaderServer.GetRegions(), func(a, b *core.RegionInfo) bool { return a.GetBytesRead() < b.GetBytesRead() }, 2)}, + // region topwrite [limit] command + {[]string{"region", "topwrite", "2"}, api.TopNRegions(leaderServer.GetRegions(), func(a, b *core.RegionInfo) bool { return a.GetBytesWritten() < b.GetBytesWritten() }, 2)}, + // region topconfver [limit] command + {[]string{"region", "topconfver", "2"}, api.TopNRegions(leaderServer.GetRegions(), func(a, b *core.RegionInfo) bool { + return a.GetMeta().GetRegionEpoch().GetConfVer() < b.GetMeta().GetRegionEpoch().GetConfVer() + }, 2)}, + // region topversion [limit] command + {[]string{"region", "topversion", "2"}, api.TopNRegions(leaderServer.GetRegions(), func(a, b *core.RegionInfo) bool { + return a.GetMeta().GetRegionEpoch().GetVersion() < b.GetMeta().GetRegionEpoch().GetVersion() + }, 2)}, + // region topsize [limit] command + {[]string{"region", "topsize", "2"}, api.TopNRegions(leaderServer.GetRegions(), func(a, b *core.RegionInfo) bool { + return a.GetApproximateSize() < b.GetApproximateSize() + }, 2)}, + // region check extra-peer command + {[]string{"region", "check", "extra-peer"}, []*core.RegionInfo{r1}}, + // region check miss-peer command + {[]string{"region", "check", "miss-peer"}, []*core.RegionInfo{r2, r3, r4}}, + // region check pending-peer command + {[]string{"region", "check", "pending-peer"}, []*core.RegionInfo{r3}}, + // region check down-peer command + {[]string{"region", "check", "down-peer"}, []*core.RegionInfo{r3}}, + // region check learner-peer command + {[]string{"region", "check", "learner-peer"}, []*core.RegionInfo{r3}}, + // region startkey --format=raw command + {[]string{"region", "startkey", "--format=raw", "b", "2"}, []*core.RegionInfo{r2, r3}}, + // region startkey --format=hex command + {[]string{"region", "startkey", "--format=hex", "63", "2"}, []*core.RegionInfo{r3, r4}}, + } - // issue #2351 - args = []string{"-u", pdAddr, "region", "key", "--format=hex", "622f62"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionInfo = api.RegionInfo{} - c.Assert(json.Unmarshal(output, ®ionInfo), IsNil) - c.Assert(®ionInfo, DeepEquals, api.NewRegionInfo(r2)) + for _, testCase := range testRegionsCases { + args := append([]string{"-u", pdAddr}, testCase.args...) + _, output, e := pdctl.ExecuteCommandC(cmd, args...) + c.Assert(e, IsNil) + regionsInfo := api.RegionsInfo{} + c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) + pdctl.CheckRegionsInfo(c, regionsInfo, testCase.expect) + } - // region startkey --format=raw command - args = []string{"-u", pdAddr, "region", "startkey", "--format=raw", "b", "2"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - pdctl.CheckRegionsInfo(c, regionsInfo, []*core.RegionInfo{r2, r3}) + var testRegionCases = []struct { + args []string + expect *api.RegionInfo + }{ + // region command + {[]string{"region", "1"}, api.NewRegionInfo(leaderServer.GetRegionInfoByID(1))}, + // region key --format=raw command + {[]string{"region", "key", "--format=raw", "b"}, api.NewRegionInfo(r2)}, + // region key --format=hex command + {[]string{"region", "key", "--format=hex", "62"}, api.NewRegionInfo(r2)}, + // issue #2351 + {[]string{"region", "key", "--format=hex", "622f62"}, api.NewRegionInfo(r2)}, + } - // region startkey --format=hex command - args = []string{"-u", pdAddr, "region", "startkey", "--format=hex", "63", "2"} - _, output, err = pdctl.ExecuteCommandC(cmd, args...) - c.Assert(err, IsNil) - regionsInfo = api.RegionsInfo{} - c.Assert(json.Unmarshal(output, ®ionsInfo), IsNil) - pdctl.CheckRegionsInfo(c, regionsInfo, []*core.RegionInfo{r3, r4}) + for _, testCase := range testRegionCases { + args := append([]string{"-u", pdAddr}, testCase.args...) + _, output, e := pdctl.ExecuteCommandC(cmd, args...) + c.Assert(e, IsNil) + regionInfo := api.RegionInfo{} + c.Assert(json.Unmarshal(output, ®ionInfo), IsNil) + c.Assert(®ionInfo, DeepEquals, testCase.expect) + } } diff --git a/tools/pd-ctl/pdctl/command/region_command.go b/tools/pd-ctl/pdctl/command/region_command.go index a2f9062dac7..e09568b219b 100644 --- a/tools/pd-ctl/pdctl/command/region_command.go +++ b/tools/pd-ctl/pdctl/command/region_command.go @@ -414,7 +414,7 @@ func showRegionsFromStartKeyCommandFunc(cmd *cobra.Command, args []string) { // NewRegionWithCheckCommand returns a region with check subcommand of regionCmd func NewRegionWithCheckCommand() *cobra.Command { r := &cobra.Command{ - Use: "check [miss-peer|extra-peer|down-peer|pending-peer|offline-peer|empty-region|hist-size|hist-keys]", + Use: "check [miss-peer|extra-peer|down-peer|learner-peer|pending-peer|offline-peer|empty-region|hist-size|hist-keys]", Short: "show the region with check specific status", Run: showRegionWithCheckCommandFunc, } From 84dde9857e40b50d7e9208e6851db07d58d02c0d Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Mon, 28 Sep 2020 15:26:13 +0800 Subject: [PATCH 07/27] api: reduce duplicated functions (#3040) Signed-off-by: Ryan Leung --- server/api/region.go | 15 +++++------ server/handler.go | 60 +++----------------------------------------- 2 files changed, 11 insertions(+), 64 deletions(-) diff --git a/server/api/region.go b/server/api/region.go index 26f6d7a4cc4..0cf7adca043 100644 --- a/server/api/region.go +++ b/server/api/region.go @@ -29,6 +29,7 @@ import ( "github.com/tikv/pd/pkg/apiutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/core" + "github.com/tikv/pd/server/statistics" "github.com/unrolled/render" ) @@ -267,7 +268,7 @@ func (h *regionsHandler) GetStoreRegions(w http.ResponseWriter, r *http.Request) // @Router /regions/check/miss-peer [get] func (h *regionsHandler) GetMissPeerRegions(w http.ResponseWriter, r *http.Request) { handler := h.svr.GetHandler() - regions, err := handler.GetMissPeerRegions() + regions, err := handler.GetRegionsByType(statistics.MissPeer) if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) return @@ -284,7 +285,7 @@ func (h *regionsHandler) GetMissPeerRegions(w http.ResponseWriter, r *http.Reque // @Router /regions/check/extra-peer [get] func (h *regionsHandler) GetExtraPeerRegions(w http.ResponseWriter, r *http.Request) { handler := h.svr.GetHandler() - regions, err := handler.GetExtraPeerRegions() + regions, err := handler.GetRegionsByType(statistics.ExtraPeer) if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) return @@ -301,7 +302,7 @@ func (h *regionsHandler) GetExtraPeerRegions(w http.ResponseWriter, r *http.Requ // @Router /regions/check/pending-peer [get] func (h *regionsHandler) GetPendingPeerRegions(w http.ResponseWriter, r *http.Request) { handler := h.svr.GetHandler() - regions, err := handler.GetPendingPeerRegions() + regions, err := handler.GetRegionsByType(statistics.PendingPeer) if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) return @@ -318,7 +319,7 @@ func (h *regionsHandler) GetPendingPeerRegions(w http.ResponseWriter, r *http.Re // @Router /regions/check/down-peer [get] func (h *regionsHandler) GetDownPeerRegions(w http.ResponseWriter, r *http.Request) { handler := h.svr.GetHandler() - regions, err := handler.GetDownPeerRegions() + regions, err := handler.GetRegionsByType(statistics.DownPeer) if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) return @@ -335,7 +336,7 @@ func (h *regionsHandler) GetDownPeerRegions(w http.ResponseWriter, r *http.Reque // @Router /regions/check/learner-peer [get] func (h *regionsHandler) GetLearnerPeerRegions(w http.ResponseWriter, r *http.Request) { handler := h.svr.GetHandler() - regions, err := handler.GetLearnerPeerRegions() + regions, err := handler.GetRegionsByType(statistics.LearnerPeer) if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) return @@ -352,7 +353,7 @@ func (h *regionsHandler) GetLearnerPeerRegions(w http.ResponseWriter, r *http.Re // @Router /regions/check/offline-peer [get] func (h *regionsHandler) GetOfflinePeer(w http.ResponseWriter, r *http.Request) { handler := h.svr.GetHandler() - regions, err := handler.GetOfflinePeer() + regions, err := handler.GetRegionsByType(statistics.OfflinePeer) if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) return @@ -369,7 +370,7 @@ func (h *regionsHandler) GetOfflinePeer(w http.ResponseWriter, r *http.Request) // @Router /regions/check/empty-region [get] func (h *regionsHandler) GetEmptyRegion(w http.ResponseWriter, r *http.Request) { handler := h.svr.GetHandler() - regions, err := handler.GetEmptyRegion() + regions, err := handler.GetRegionsByType(statistics.EmptyRegion) if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) return diff --git a/server/handler.go b/server/handler.go index c3b3afced3d..0bf17a82e43 100644 --- a/server/handler.go +++ b/server/handler.go @@ -772,49 +772,13 @@ func (h *Handler) AddScatterRegionOperator(regionID uint64, group string) error return nil } -// GetDownPeerRegions gets the region with down peer. -func (h *Handler) GetDownPeerRegions() ([]*core.RegionInfo, error) { +// GetRegionsByType gets the region with specified type. +func (h *Handler) GetRegionsByType(typ statistics.RegionStatisticType) ([]*core.RegionInfo, error) { c := h.s.GetRaftCluster() if c == nil { return nil, errs.ErrNotBootstrapped.FastGenByArgs() } - return c.GetRegionStatsByType(statistics.DownPeer), nil -} - -// GetLearnerPeerRegions gets the region with learner peer. -func (h *Handler) GetLearnerPeerRegions() ([]*core.RegionInfo, error) { - c := h.s.GetRaftCluster() - if c == nil { - return nil, errs.ErrNotBootstrapped.FastGenByArgs() - } - return c.GetRegionStatsByType(statistics.LearnerPeer), nil -} - -// GetExtraPeerRegions gets the region exceeds the specified number of peers. -func (h *Handler) GetExtraPeerRegions() ([]*core.RegionInfo, error) { - c := h.s.GetRaftCluster() - if c == nil { - return nil, errs.ErrNotBootstrapped.FastGenByArgs() - } - return c.GetRegionStatsByType(statistics.ExtraPeer), nil -} - -// GetMissPeerRegions gets the region less than the specified number of peers. -func (h *Handler) GetMissPeerRegions() ([]*core.RegionInfo, error) { - c := h.s.GetRaftCluster() - if c == nil { - return nil, errs.ErrNotBootstrapped.FastGenByArgs() - } - return c.GetRegionStatsByType(statistics.MissPeer), nil -} - -// GetPendingPeerRegions gets the region with pending peer. -func (h *Handler) GetPendingPeerRegions() ([]*core.RegionInfo, error) { - c := h.s.GetRaftCluster() - if c == nil { - return nil, errs.ErrNotBootstrapped.FastGenByArgs() - } - return c.GetRegionStatsByType(statistics.PendingPeer), nil + return c.GetRegionStatsByType(typ), nil } // GetSchedulerConfigHandler gets the handler of schedulers. @@ -832,24 +796,6 @@ func (h *Handler) GetSchedulerConfigHandler() http.Handler { return mux } -// GetOfflinePeer gets the region with offline peer. -func (h *Handler) GetOfflinePeer() ([]*core.RegionInfo, error) { - c := h.s.GetRaftCluster() - if c == nil { - return nil, errs.ErrNotBootstrapped.FastGenByArgs() - } - return c.GetRegionStatsByType(statistics.OfflinePeer), nil -} - -// GetEmptyRegion gets the region with empty size. -func (h *Handler) GetEmptyRegion() ([]*core.RegionInfo, error) { - c := h.s.GetRaftCluster() - if c == nil { - return nil, errs.ErrNotBootstrapped.FastGenByArgs() - } - return c.GetRegionStatsByType(statistics.EmptyRegion), nil -} - // ResetTS resets the ts with specified tso. func (h *Handler) ResetTS(ts uint64) error { tsoAllocator, err := h.s.tsoAllocatorManager.GetAllocator(config.GlobalDCLocation) From 7661eae3e5fcd8f1056da030043835aeb2ce9fe9 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Mon, 28 Sep 2020 20:34:42 +0800 Subject: [PATCH 08/27] config: unify the redact log configuration with tikv (#3037) Signed-off-by: Song Gao --- client/base_client.go | 2 +- pkg/grpcutil/grpcutil.go | 8 ++++---- server/config/config.go | 18 +++++++++--------- server/config/config_test.go | 6 +++++- server/region_syncer/client.go | 2 +- server/region_syncer/server.go | 16 ++++++++-------- server/server.go | 6 +++--- tests/client/client_tls_test.go | 2 +- 8 files changed, 32 insertions(+), 28 deletions(-) diff --git a/client/base_client.go b/client/base_client.go index 64510799ad7..8069e792041 100644 --- a/client/base_client.go +++ b/client/base_client.go @@ -267,7 +267,7 @@ func (c *baseClient) getOrCreateGRPCConn(addr string) (*grpc.ClientConn, error) if ok { return conn, nil } - tlsCfg, err := grpcutil.SecurityConfig{ + tlsCfg, err := grpcutil.TLSConfig{ CAPath: c.security.CAPath, CertPath: c.security.CertPath, KeyPath: c.security.KeyPath, diff --git a/pkg/grpcutil/grpcutil.go b/pkg/grpcutil/grpcutil.go index 1f7455db0e3..fb2a1cf85c7 100644 --- a/pkg/grpcutil/grpcutil.go +++ b/pkg/grpcutil/grpcutil.go @@ -24,8 +24,8 @@ import ( "google.golang.org/grpc/credentials" ) -// SecurityConfig is the configuration for supporting tls. -type SecurityConfig struct { +// TLSConfig is the configuration for supporting tls. +type TLSConfig struct { // CAPath is the path of file that contains list of trusted SSL CAs. if set, following four settings shouldn't be empty CAPath string `toml:"cacert-path" json:"cacert-path"` // CertPath is the path of file that contains X509 certificate in PEM format. @@ -37,7 +37,7 @@ type SecurityConfig struct { } // ToTLSConfig generates tls config. -func (s SecurityConfig) ToTLSConfig() (*tls.Config, error) { +func (s TLSConfig) ToTLSConfig() (*tls.Config, error) { if len(s.CertPath) == 0 && len(s.KeyPath) == 0 { return nil, nil } @@ -61,7 +61,7 @@ func (s SecurityConfig) ToTLSConfig() (*tls.Config, error) { } // GetOneAllowedCN only gets the first one CN. -func (s SecurityConfig) GetOneAllowedCN() (string, error) { +func (s TLSConfig) GetOneAllowedCN() (string, error) { switch len(s.CertAllowedCN) { case 1: return s.CertAllowedCN[0], nil diff --git a/server/config/config.go b/server/config/config.go index 9b577b0e2d3..eb59311df8b 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -122,7 +122,7 @@ type Config struct { // an election, thus minimizing disruptions. PreVote bool `toml:"enable-prevote"` - Security grpcutil.SecurityConfig `toml:"security" json:"security"` + Security SecurityConfig `toml:"security" json:"security"` LabelProperty LabelPropertyConfig `toml:"label-property" json:"label-property"` @@ -145,8 +145,6 @@ type Config struct { Dashboard DashboardConfig `toml:"dashboard" json:"dashboard"` ReplicationMode ReplicationModeConfig `toml:"replication-mode" json:"replication-mode"` - // EnableRedactLog indicates that whether redact log, 0 is disable. 1 is enable. - EnableRedactLog bool `toml:"enable-redact-log" json:"enable-redact-log"` } // NewConfig creates a new config. @@ -224,7 +222,6 @@ const ( defaultDRWaitStoreTimeout = time.Minute defaultDRWaitSyncTimeout = time.Minute defaultDRWaitAsyncTimeout = 2 * time.Minute - defaultEnableRedactLog = false ) var ( @@ -547,10 +544,6 @@ func (c *Config) Adjust(meta *toml.MetaData) error { c.ReplicationMode.adjust(configMetaData.Child("replication-mode")) - if !configMetaData.IsDefined("enable-redact-log") { - c.EnableRedactLog = defaultEnableRedactLog - } - return nil } @@ -1151,7 +1144,7 @@ func (c *Config) SetupLogger() error { } c.logger = lg c.logProps = p - logutil.SetRedactLog(c.EnableRedactLog) + logutil.SetRedactLog(c.Security.RedactInfoLog) return nil } @@ -1373,3 +1366,10 @@ func (c *LocalTSOConfig) Validate() error { } return nil } + +// SecurityConfig indicates the security configuration for pd server +type SecurityConfig struct { + grpcutil.TLSConfig + // RedactInfoLog indicates that whether enabling redact log + RedactInfoLog bool `toml:"redact-info-log" json:"redact-info-log"` +} diff --git a/server/config/config_test.go b/server/config/config_test.go index 079e114e88d..ef387eb10cd 100644 --- a/server/config/config_test.go +++ b/server/config/config_test.go @@ -44,6 +44,11 @@ func (s *testConfigSuite) SetUpSuite(c *C) { RegisterScheduler("adjacent-region") } +func (s *testConfigSuite) TestSecurity(c *C) { + cfg := NewConfig() + c.Assert(cfg.Security.RedactInfoLog, Equals, false) +} + func (s *testConfigSuite) TestTLS(c *C) { cfg := NewConfig() tls, err := cfg.Security.ToTLSConfig() @@ -165,7 +170,6 @@ leader-schedule-limit = 0 c.Assert(cfg.PreVote, IsTrue) c.Assert(cfg.Schedule.MaxMergeRegionKeys, Equals, uint64(defaultMaxMergeRegionKeys)) c.Assert(cfg.PDServerCfg.MetricStorage, Equals, "http://127.0.0.1:9090") - c.Assert(cfg.EnableRedactLog, Equals, defaultEnableRedactLog) // Check undefined config fields cfgData = ` diff --git a/server/region_syncer/client.go b/server/region_syncer/client.go index 299ee79f15c..7e0cfc007f2 100644 --- a/server/region_syncer/client.go +++ b/server/region_syncer/client.go @@ -61,7 +61,7 @@ func (s *RegionSyncer) reset() { func (s *RegionSyncer) establish(addr string) (*grpc.ClientConn, error) { s.reset() ctx, cancel := context.WithCancel(s.server.LoopContext()) - tlsCfg, err := s.securityConfig.ToTLSConfig() + tlsCfg, err := s.tlsConfig.ToTLSConfig() if err != nil { cancel() return nil, err diff --git a/server/region_syncer/server.go b/server/region_syncer/server.go index e3c23531d23..018dbbaee7c 100644 --- a/server/region_syncer/server.go +++ b/server/region_syncer/server.go @@ -61,7 +61,7 @@ type Server interface { GetStorage() *core.Storage Name() string GetRegions() []*core.RegionInfo - GetSecurityConfig() *grpcutil.SecurityConfig + GetTLSConfig() *grpcutil.TLSConfig GetBasicCluster() *core.BasicCluster } @@ -76,7 +76,7 @@ type RegionSyncer struct { wg sync.WaitGroup history *historyBuffer limit *ratelimit.Bucket - securityConfig *grpcutil.SecurityConfig + tlsConfig *grpcutil.TLSConfig } // NewRegionSyncer returns a region syncer. @@ -86,12 +86,12 @@ type RegionSyncer struct { // no longer etcd but go-leveldb. func NewRegionSyncer(s Server) *RegionSyncer { return &RegionSyncer{ - streams: make(map[string]ServerStream), - server: s, - closed: make(chan struct{}), - history: newHistoryBuffer(defaultHistoryBufferSize, s.GetStorage().GetRegionStorage()), - limit: ratelimit.NewBucketWithRate(defaultBucketRate, defaultBucketCapacity), - securityConfig: s.GetSecurityConfig(), + streams: make(map[string]ServerStream), + server: s, + closed: make(chan struct{}), + history: newHistoryBuffer(defaultHistoryBufferSize, s.GetStorage().GetRegionStorage()), + limit: ratelimit.NewBucketWithRate(defaultBucketRate, defaultBucketCapacity), + tlsConfig: s.GetTLSConfig(), } } diff --git a/server/server.go b/server/server.go index 0b5cb5fa603..79b7c827245 100644 --- a/server/server.go +++ b/server/server.go @@ -955,9 +955,9 @@ func (s *Server) GetClusterVersion() semver.Version { return *s.persistOptions.GetClusterVersion() } -// GetSecurityConfig get the security config. -func (s *Server) GetSecurityConfig() *grpcutil.SecurityConfig { - return &s.cfg.Security +// GetTLSConfig get the security config. +func (s *Server) GetTLSConfig() *grpcutil.TLSConfig { + return &s.cfg.Security.TLSConfig } // GetServerRootPath returns the server root path. diff --git a/tests/client/client_tls_test.go b/tests/client/client_tls_test.go index 999c9492bfe..89648266e12 100644 --- a/tests/client/client_tls_test.go +++ b/tests/client/client_tls_test.go @@ -128,7 +128,7 @@ func (s *clientTLSTestSuite) testTLSReload( tlsInfo := cloneFunc() // 1. start cluster with valid certs clus, err := tests.NewTestCluster(s.ctx, 1, func(conf *config.Config, serverName string) { - conf.Security = grpcutil.SecurityConfig{ + conf.Security.TLSConfig = grpcutil.TLSConfig{ KeyPath: tlsInfo.KeyFile, CertPath: tlsInfo.CertFile, CAPath: tlsInfo.TrustedCAFile, From 4199e7261f8bec1c3f8a386c219217d010e0ba9f Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Tue, 29 Sep 2020 11:59:23 +0800 Subject: [PATCH 09/27] cluster: check if region is valid before handling it (#3041) Signed-off-by: Ryan Leung --- server/cluster/cluster_worker.go | 7 ------- server/grpc_service.go | 11 +++++++++++ 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/server/cluster/cluster_worker.go b/server/cluster/cluster_worker.go index b17dc58ebe7..1a7158bd628 100644 --- a/server/cluster/cluster_worker.go +++ b/server/cluster/cluster_worker.go @@ -35,13 +35,6 @@ func (c *RaftCluster) HandleRegionHeartbeat(region *core.RegionInfo) error { return err } - // If the region peer count is 0, then we should not handle this. - if len(region.GetPeers()) == 0 { - log.Warn("invalid region, zero region peer count", - logutil.ZapRedactStringer("region-meta", core.RegionToHexMeta(region.GetMeta()))) - return errors.Errorf("invalid region, zero region peer count: %v", logutil.RedactStringer(core.RegionToHexMeta(region.GetMeta()))) - } - c.RLock() co := c.coordinator c.RUnlock() diff --git a/server/grpc_service.go b/server/grpc_service.go index 52b0c81c1a8..21d7a330d24 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/logutil" "github.com/tikv/pd/pkg/tsoutil" "github.com/tikv/pd/server/cluster" "github.com/tikv/pd/server/config" @@ -419,6 +420,16 @@ func (s *Server) RegionHeartbeat(stream pdpb.PD_RegionHeartbeatServer) error { continue } + // If the region peer count is 0, then we should not handle this. + if len(region.GetPeers()) == 0 { + log.Warn("invalid region, zero region peer count", + logutil.ZapRedactStringer("region-meta", core.RegionToHexMeta(region.GetMeta()))) + regionHeartbeatCounter.WithLabelValues(storeAddress, storeLabel, "report", "err").Inc() + msg := fmt.Sprintf("invalid region, zero region peer count: %v", logutil.RedactStringer(core.RegionToHexMeta(region.GetMeta()))) + s.hbStreams.SendErr(pdpb.ErrorType_UNKNOWN, msg, request.GetLeader()) + continue + } + start := time.Now() err = rc.HandleRegionHeartbeat(region) From a5d1b1e89b1fcf061a592e650c22ad77b08fc3d0 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Tue, 29 Sep 2020 12:48:23 +0800 Subject: [PATCH 10/27] scheduler: support regions relocating by range (#3028) Signed-off-by: Song Gao --- server/api/operator.go | 26 +++++++++- server/handler.go | 55 ++++++++++++++++++++++ server/schedule/region_scatterer.go | 48 +++++++++++++++++++ server/schedule/region_scatterer_test.go | 60 ++++++++++++++++++++++++ 4 files changed, 188 insertions(+), 1 deletion(-) diff --git a/server/api/operator.go b/server/api/operator.go index 8d841fbc2e8..1318fbbcbdd 100644 --- a/server/api/operator.go +++ b/server/api/operator.go @@ -280,11 +280,35 @@ func (h *operatorHandler) Post(w http.ResponseWriter, r *http.Request) { h.r.JSON(w, http.StatusInternalServerError, err.Error()) return } + case "scatter-regions": + // support both receiving key ranges or regionIDs + startKey, _ := input["start_key"].(string) + endKey, _ := input["end_key"].(string) + regionIDs, _ := input["region_ids"].([]uint64) + group, _ := input["group"].(string) + retryLimit, ok := input["retry_limit"].(int) + if !ok { + // retry 5 times if retryLimit not defined + retryLimit = 5 + } + processedPercentage, err := h.AddScatterRegionsOperators(regionIDs, startKey, endKey, group, retryLimit) + errorMessage := "" + if err != nil { + errorMessage = err.Error() + } + s := struct { + ProcessedPercentage int `json:"processed-percentage"` + Error string `json:"error"` + }{ + ProcessedPercentage: processedPercentage, + Error: errorMessage, + } + h.r.JSON(w, http.StatusOK, &s) + return default: h.r.JSON(w, http.StatusBadRequest, "unknown operator") return } - h.r.JSON(w, http.StatusOK, "The operator is created.") } diff --git a/server/handler.go b/server/handler.go index 0bf17a82e43..c61115d52f0 100644 --- a/server/handler.go +++ b/server/handler.go @@ -16,6 +16,7 @@ package server import ( "bytes" "encoding/hex" + "fmt" "net/http" "path" "strconv" @@ -772,6 +773,60 @@ func (h *Handler) AddScatterRegionOperator(regionID uint64, group string) error return nil } +// AddScatterRegionsOperators add operators to scatter regions and return the processed percentage and error +func (h *Handler) AddScatterRegionsOperators(regionIDs []uint64, startRawKey, endRawKey, group string, retryLimit int) (int, error) { + c, err := h.GetRaftCluster() + if err != nil { + return 0, err + } + var failureRegionID []string + var regions []*core.RegionInfo + // If startKey and endKey are both defined, use them first. + if len(startRawKey) > 0 && len(endRawKey) > 0 { + startKey, err := hex.DecodeString(startRawKey) + if err != nil { + return 0, err + } + endKey, err := hex.DecodeString(endRawKey) + if err != nil { + return 0, err + } + regions = c.ScanRegions(startKey, endKey, -1) + } else { + for _, id := range regionIDs { + region := c.GetRegion(id) + if region == nil { + failureRegionID = append(failureRegionID, fmt.Sprintf("%v", id)) + continue + } + regions = append(regions, region) + } + } + // check region hot status + regionMap := make(map[uint64]*core.RegionInfo, len(regions)) + for _, region := range regions { + // If region is Hot, add it into unProcessedRegions + if c.IsRegionHot(region) { + failureRegionID = append(failureRegionID, fmt.Sprintf("%v", region.GetID())) + continue + } + regionMap[region.GetID()] = region + } + failures := make(map[uint64]error, len(regionMap)) + // If there existed any region failed to relocated after retry, add it into unProcessedRegions + ops := c.GetRegionScatter().ScatterRegions(regionMap, failures, group, retryLimit) + for regionID := range failures { + failureRegionID = append(failureRegionID, fmt.Sprintf("%v", regionID)) + } + // If there existed any operator failed to be added into Operator Controller, add its regions into unProcessedRegions + for _, op := range ops { + if ok := c.GetOperatorController().AddOperator(op); !ok { + failureRegionID = append(failureRegionID, fmt.Sprintf("%v", op.RegionID())) + } + } + return 100 - (len(failureRegionID) * 100 / len(regions)), errors.New("unprocessed regions:[" + strings.Join(failureRegionID, ",") + "]") +} + // GetRegionsByType gets the region with specified type. func (h *Handler) GetRegionsByType(typ statistics.RegionStatisticType) ([]*core.RegionInfo, error) { c := h.s.GetRaftCluster() diff --git a/server/schedule/region_scatterer.go b/server/schedule/region_scatterer.go index 432407402c1..8163544822c 100644 --- a/server/schedule/region_scatterer.go +++ b/server/schedule/region_scatterer.go @@ -17,11 +17,14 @@ import ( "math" "math/rand" "sync" + "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/typeutil" "github.com/tikv/pd/server/core" "github.com/tikv/pd/server/schedule/filter" "github.com/tikv/pd/server/schedule/operator" @@ -145,6 +148,47 @@ func newEngineContext(filters ...filter.Filter) engineContext { } } +const maxSleepDuration = 1 * time.Minute +const initialSleepDuration = 100 * time.Millisecond +const maxRetryLimit = 30 + +// ScatterRegions relocates the regions. If the group is defined, the regions' leader with the same group would be scattered +// in a group level instead of cluster level. +// RetryTimes indicates the retry times if any of the regions failed to relocate during scattering. There will be +// time.Sleep between each retry. +// Failures indicates the regions which are failed to be relocated, the key of the failures indicates the regionID +// and the value of the failures indicates the failure error. +func (r *RegionScatterer) ScatterRegions(regions map[uint64]*core.RegionInfo, failures map[uint64]error, group string, retryLimit int) []*operator.Operator { + if retryLimit > maxRetryLimit { + retryLimit = maxRetryLimit + } + ops := make([]*operator.Operator, 0, len(regions)) + for currentRetry := 0; currentRetry < retryLimit; currentRetry++ { + for _, region := range regions { + op, err := r.Scatter(region, group) + failpoint.Inject("scatterFail", func() { + if region.GetID() == 1 { + err = errors.New("mock error") + } + }) + if err != nil { + failures[region.GetID()] = err + continue + } + ops = append(ops, op) + delete(regions, region.GetID()) + delete(failures, region.GetID()) + } + // all regions have been relocated, break the loop. + if len(regions) < 1 { + break + } + // Wait for a while if there are some regions failed to be relocated + time.Sleep(typeutil.MinDuration(maxSleepDuration, time.Duration(math.Pow(2, float64(currentRetry)))*initialSleepDuration)) + } + return ops +} + // Scatter relocates the region. If the group is defined, the regions' leader with the same group would be scattered // in a group level instead of cluster level. func (r *RegionScatterer) Scatter(region *core.RegionInfo, group string) (*operator.Operator, error) { @@ -157,6 +201,10 @@ func (r *RegionScatterer) Scatter(region *core.RegionInfo, group string) (*opera return nil, errors.Errorf("region %d has no leader", region.GetID()) } + if r.cluster.IsRegionHot(region) { + return nil, errors.Errorf("region %d is hot", region.GetID()) + } + return r.scatterRegion(region, group), nil } diff --git a/server/schedule/region_scatterer_test.go b/server/schedule/region_scatterer_test.go index 08feffb5dd8..15c186c2c79 100644 --- a/server/schedule/region_scatterer_test.go +++ b/server/schedule/region_scatterer_test.go @@ -6,6 +6,7 @@ import ( "math" . "github.com/pingcap/check" + "github.com/pingcap/failpoint" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/server/config" "github.com/tikv/pd/server/core" @@ -341,3 +342,62 @@ func (s *testScatterRegionSuite) TestScatterGroup(c *C) { } } } + +func (s *testScatterRegionSuite) TestScattersGroup(c *C) { + opt := config.NewTestOptions() + tc := mockcluster.NewCluster(opt) + // Add 5 stores. + for i := uint64(1); i <= 5; i++ { + tc.AddRegionStore(i, 0) + } + testcases := []struct { + name string + failure bool + }{ + { + name: "have failure", + failure: true, + }, + { + name: "no failure", + failure: false, + }, + } + group := "group" + for _, testcase := range testcases { + scatterer := NewRegionScatterer(tc) + regions := map[uint64]*core.RegionInfo{} + for i := 1; i <= 100; i++ { + regions[uint64(i)] = tc.AddLeaderRegion(uint64(i), 1, 2, 3) + } + c.Log(testcase.name) + failures := map[uint64]error{} + if testcase.failure { + c.Assert(failpoint.Enable("github.com/tikv/pd/server/schedule/scatterFail", `return(true)`), IsNil) + } + + scatterer.ScatterRegions(regions, failures, group, 3) + max := uint64(0) + min := uint64(math.MaxUint64) + for _, count := range scatterer.ordinaryEngine.selectedLeader.groupDistribution[group] { + if count > max { + max = count + } + if count < min { + min = count + } + } + // 100 regions divided 5 stores, each store expected to have about 20 regions. + c.Assert(min, LessEqual, uint64(20)) + c.Assert(max, GreaterEqual, uint64(20)) + c.Assert(max-min, LessEqual, uint64(3)) + if testcase.failure { + c.Assert(len(failures), Equals, 1) + _, ok := failures[1] + c.Assert(ok, Equals, true) + c.Assert(failpoint.Disable("github.com/tikv/pd/server/schedule/scatterFail"), IsNil) + } else { + c.Assert(len(failures), Equals, 0) + } + } +} From a4ab63cebc24d9138aa654cc1b0bdba7ee9691bb Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 30 Sep 2020 12:04:55 +0800 Subject: [PATCH 11/27] server: move heartbeat test to heartbeat package (#3010) Signed-off-by: disksing --- server/heartbeat_stream_test.go | 159 ------------------ .../hbstream/heartbeat_streams_test.go | 79 +++++++++ 2 files changed, 79 insertions(+), 159 deletions(-) delete mode 100644 server/heartbeat_stream_test.go create mode 100644 server/schedule/hbstream/heartbeat_streams_test.go diff --git a/server/heartbeat_stream_test.go b/server/heartbeat_stream_test.go deleted file mode 100644 index a1649a8f600..00000000000 --- a/server/heartbeat_stream_test.go +++ /dev/null @@ -1,159 +0,0 @@ -// Copyright 2018 TiKV Project Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package server - -import ( - "context" - "time" - - . "github.com/pingcap/check" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/pingcap/log" - "github.com/tikv/pd/pkg/testutil" - "github.com/tikv/pd/pkg/typeutil" - "github.com/tikv/pd/server/core" - "go.uber.org/zap" -) - -var _ = Suite(&testHeartbeatStreamSuite{}) - -type testHeartbeatStreamSuite struct { - svr *Server - grpcPDClient pdpb.PDClient - region *metapb.Region -} - -// TODO: refactor and move to server/schedule/hbstream -func (s *testHeartbeatStreamSuite) TestActivity(c *C) { - var err error - var cleanup func() - s.svr, cleanup, err = NewTestServer(c) - c.Assert(err, IsNil) - defer cleanup() - s.svr.cfg.HeartbeatStreamBindInterval = typeutil.NewDuration(time.Second) - mustWaitLeader(c, []*Server{s.svr}) - s.grpcPDClient = testutil.MustNewGrpcClient(c, s.svr.GetAddr()) - - bootstrapReq := &pdpb.BootstrapRequest{ - Header: testutil.NewRequestHeader(s.svr.clusterID), - Store: &metapb.Store{Id: 1, Address: "127.0.0.1:0"}, - Region: &metapb.Region{Id: 2, Peers: []*metapb.Peer{{Id: 3, StoreId: 1, Role: metapb.PeerRole_Voter}}}, - } - _, err = s.svr.bootstrapCluster(bootstrapReq) - c.Assert(err, IsNil) - s.region = bootstrapReq.Region - - // Add a new store and an addPeer operator. - req := &pdpb.PutStoreRequest{ - Header: testutil.NewRequestHeader(s.svr.clusterID), - Store: &metapb.Store{Id: 2, Address: "127.0.0.1:1"}, - } - _, err = s.grpcPDClient.PutStore(context.Background(), req) - c.Assert(err, IsNil) - err = s.svr.GetRaftCluster().HandleRegionHeartbeat(core.NewRegionInfo(s.region, s.region.GetPeers()[0])) - c.Assert(err, IsNil) - err = newHandler(s.svr).AddAddPeerOperator(s.region.GetId(), 2) - c.Assert(err, IsNil) - - stream1, stream2 := newRegionHeartbeatClient(c, s.grpcPDClient), newRegionHeartbeatClient(c, s.grpcPDClient) - defer stream1.close() - defer stream2.close() - checkActiveStream := func() int { - // 1 means stream1 got a valid response - // 2 means stream2 got a valid response - // 3 means got an invalid response - select { - case resp := <-stream1.respCh: - if resp.GetHeader().GetError() != nil { - return 3 - } - return 1 - case resp := <-stream2.respCh: - if resp.GetHeader().GetError() != nil { - return 3 - } - return 2 - case <-time.After(time.Second): - return 0 - } - } - hbReq := &pdpb.RegionHeartbeatRequest{ - Header: testutil.NewRequestHeader(s.svr.clusterID), - Leader: s.region.Peers[0], - Region: s.region, - } - invalidRegion := &metapb.Region{Id: 0} - invalidReq := &pdpb.RegionHeartbeatRequest{ - Header: testutil.NewRequestHeader(s.svr.clusterID), - Leader: s.region.Peers[0], - Region: invalidRegion, - } - // Active stream is stream1. - c.Assert(stream1.stream.Send(hbReq), IsNil) - c.Assert(checkActiveStream(), Equals, 1) - // Rebind to stream2. - c.Assert(stream2.stream.Send(hbReq), IsNil) - c.Assert(checkActiveStream(), Equals, 2) - // SendErr to stream2. - c.Assert(stream2.stream.Send(invalidReq), IsNil) - c.Assert(checkActiveStream(), Equals, 3) - // Rebind to stream1 if no more heartbeats sent through stream2. - testutil.WaitUntil(c, func(c *C) bool { - c.Assert(stream1.stream.Send(hbReq), IsNil) - return checkActiveStream() == 1 - }) -} - -type regionHeartbeatClient struct { - stream pdpb.PD_RegionHeartbeatClient - respCh chan *pdpb.RegionHeartbeatResponse -} - -func newRegionHeartbeatClient(c *C, grpcClient pdpb.PDClient) *regionHeartbeatClient { - stream, err := grpcClient.RegionHeartbeat(context.Background()) - c.Assert(err, IsNil) - ch := make(chan *pdpb.RegionHeartbeatResponse) - go func() { - for { - res, err := stream.Recv() - if err != nil { - return - } - ch <- res - } - }() - return ®ionHeartbeatClient{ - stream: stream, - respCh: ch, - } -} - -func (c *regionHeartbeatClient) close() { - if err := c.stream.CloseSend(); err != nil { - log.Error("failed to terminate client stream", zap.Error(err)) - } -} - -func (c *regionHeartbeatClient) SendRecv(msg *pdpb.RegionHeartbeatRequest, timeout time.Duration) *pdpb.RegionHeartbeatResponse { - if err := c.stream.Send(msg); err != nil { - log.Error("send heartbeat message fail", zap.Error(err)) - } - select { - case <-time.After(timeout): - return nil - case res := <-c.respCh: - return res - } -} diff --git a/server/schedule/hbstream/heartbeat_streams_test.go b/server/schedule/hbstream/heartbeat_streams_test.go new file mode 100644 index 00000000000..c824e18c6f7 --- /dev/null +++ b/server/schedule/hbstream/heartbeat_streams_test.go @@ -0,0 +1,79 @@ +// Copyright 2017 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package hbstream + +import ( + "context" + "testing" + + "github.com/gogo/protobuf/proto" + . "github.com/pingcap/check" + "github.com/pingcap/kvproto/pkg/eraftpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/mock/mockcluster" + "github.com/tikv/pd/pkg/mock/mockhbstream" + "github.com/tikv/pd/pkg/testutil" + "github.com/tikv/pd/server/config" +) + +func TestHeaertbeatStreams(t *testing.T) { + TestingT(t) +} + +var _ = Suite(&testHeartbeatStreamSuite{}) + +type testHeartbeatStreamSuite struct { +} + +func (s *testHeartbeatStreamSuite) TestActivity(c *C) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + cluster := mockcluster.NewCluster(config.NewTestOptions()) + cluster.AddRegionStore(1, 1) + cluster.AddRegionStore(2, 0) + cluster.AddLeaderRegion(1, 1) + region := cluster.GetRegion(1) + msg := &pdpb.RegionHeartbeatResponse{ + ChangePeer: &pdpb.ChangePeer{Peer: &metapb.Peer{Id: 2, StoreId: 2}, ChangeType: eraftpb.ConfChangeType_AddLearnerNode}, + } + + hbs := NewTestHeartbeatStreams(ctx, cluster.ID, cluster, true) + stream1, stream2 := mockhbstream.NewHeartbeatStream(), mockhbstream.NewHeartbeatStream() + + // Active stream is stream1. + hbs.BindStream(1, stream1) + testutil.WaitUntil(c, func(c *C) bool { + hbs.SendMsg(region, proto.Clone(msg).(*pdpb.RegionHeartbeatResponse)) + return stream1.Recv() != nil && stream2.Recv() == nil + }) + // Rebind to stream2. + hbs.BindStream(1, stream2) + testutil.WaitUntil(c, func(c *C) bool { + hbs.SendMsg(region, proto.Clone(msg).(*pdpb.RegionHeartbeatResponse)) + return stream1.Recv() == nil && stream2.Recv() != nil + }) + // SendErr to stream2. + hbs.SendErr(pdpb.ErrorType_UNKNOWN, "test error", &metapb.Peer{Id: 1, StoreId: 1}) + res := stream2.Recv() + c.Assert(res, NotNil) + c.Assert(res.GetHeader().GetError(), NotNil) + // Switch back to 1 again. + hbs.BindStream(1, stream1) + testutil.WaitUntil(c, func(c *C) bool { + hbs.SendMsg(region, proto.Clone(msg).(*pdpb.RegionHeartbeatResponse)) + return stream1.Recv() != nil && stream2.Recv() == nil + }) +} From 43baea981b406df26cd49e8b99cc42354f0a6696 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Wed, 30 Sep 2020 16:47:18 +0800 Subject: [PATCH 12/27] tso: add election priority for Local TSO Allocator (#2953) Signed-off-by: JmPotato --- server/tso/allocator_manager.go | 147 +++++++++++++++++++++++++++++ server/tso/filter.go | 5 + tests/cluster.go | 54 +++++++++++ tests/server/tso/allocator_test.go | 11 +-- tests/server/tso/manager_test.go | 62 +++++++++++- 5 files changed, 269 insertions(+), 10 deletions(-) diff --git a/server/tso/allocator_manager.go b/server/tso/allocator_manager.go index a0d50af6dba..c6a5312e938 100644 --- a/server/tso/allocator_manager.go +++ b/server/tso/allocator_manager.go @@ -36,6 +36,7 @@ import ( ) const ( + checkPriorityStep = 1 * time.Minute checkAllocatorStep = 1 * time.Second dcLocationConfigEtcdPrefix = "dc-location" defaultAllocatorLeaderLease = 3 @@ -228,6 +229,23 @@ func (am *AllocatorManager) allocatorLeaderLoop(ctx context.Context, allocator * log.Info("local tso allocator leader has changed, try to re-campaign a local tso allocator leader", zap.String("dc-location", allocator.dcLocation)) } + // Check the next-leader key + nextLeader, err := am.getNextLeaderID(allocator.dcLocation) + if err != nil { + log.Error("get next leader from etcd failed", + zap.String("dc-location", allocator.dcLocation), + errs.ZapError(err)) + time.Sleep(200 * time.Millisecond) + continue + } + if nextLeader != 0 && nextLeader != am.member.ID() { + log.Info("skip campaigning of the local tso allocator leader and check later", + zap.String("server-name", am.member.Member().Name), + zap.Uint64("server-id", am.member.ID()), + zap.Uint64("next-leader-id", nextLeader)) + time.Sleep(200 * time.Millisecond) + continue + } am.campaignAllocatorLeader(ctx, allocator) } } @@ -259,6 +277,8 @@ func (am *AllocatorManager) campaignAllocatorLeader(loopCtx context.Context, all return } allocator.EnableAllocatorLeader() + // The next leader is me, delete it to finish campaigning + am.deleteNextLeaderID(allocator.dcLocation) log.Info("local tso allocator leader is ready to serve", zap.String("dc-location", allocator.dcLocation), zap.String("name", am.member.Member().Name)) @@ -292,6 +312,8 @@ func (am *AllocatorManager) AllocatorDaemon(serverCtx context.Context) { defer tsTicker.Stop() checkerTicker := time.NewTicker(checkAllocatorStep) defer checkerTicker.Stop() + priorityTicker := time.NewTicker(checkPriorityStep) + defer priorityTicker.Stop() for { select { @@ -299,6 +321,8 @@ func (am *AllocatorManager) AllocatorDaemon(serverCtx context.Context) { am.allocatorUpdater() case <-checkerTicker.C: am.allocatorPatroller(serverCtx) + case <-priorityTicker.C: + am.priorityChecker() case <-serverCtx.Done(): return } @@ -378,6 +402,129 @@ func (am *AllocatorManager) allocatorPatroller(serverCtx context.Context) { } } +// priorityChecker is used to check the election priority of a Local TSO Allocator. +// In the normal case, if we want to elect a Local TSO Allocator for a certain DC, +// such as dc-1, we need to make sure the follow priority rules: +// 1. The PD server with dc-location="dc-1" needs to be elected as the allocator +// leader with the highest priority. +// 2. If all PD servers with dc-location="dc-1" are down, then the other PD servers +// of DC could be elected. +func (am *AllocatorManager) priorityChecker() { + serverID := am.member.ID() + myServerDCLocation, err := am.getServerDCLocation(serverID) + if err != nil { + log.Error("skip checking allocator priority, failed to get server's dc-location", + zap.Uint64("server-id", serverID), + errs.ZapError(err)) + return + } + // Check all Local TSO Allocator followers to see if their priorities is higher than the leaders + // Filter out allocators with leadership and initialized + allocatorGroups := am.getAllocatorGroups(FilterDCLocation(config.GlobalDCLocation), FilterAvailableLeadership()) + for _, allocatorGroup := range allocatorGroups { + localTSOAllocator, _ := allocatorGroup.allocator.(*LocalTSOAllocator) + leaderServerID := localTSOAllocator.GetAllocatorLeader().GetMemberId() + // No leader, maybe the leader is not been watched yet + if leaderServerID == 0 { + continue + } + leaderServerDCLocation, err := am.getServerDCLocation(leaderServerID) + if err != nil { + log.Error("failed to get local tso allocator leader's dc-location", + zap.Uint64("server-id", serverID), + errs.ZapError(err)) + continue + } + // For example, an allocator leader for dc-1 is elected by a server of dc-2, then the server of dc-1 will + // find this allocator's dc-location isn't the same with server of dc-2 but is same with itself. + if allocatorGroup.dcLocation != leaderServerDCLocation && allocatorGroup.dcLocation == myServerDCLocation { + log.Info("try to move the local tso allocator", + zap.Uint64("old-leader-id", leaderServerID), + zap.String("old-dc-location", leaderServerDCLocation), + zap.Uint64("next-leader-id", serverID), + zap.String("next-dc-location", myServerDCLocation)) + nextLeaderKey := path.Join(am.rootPath, allocatorGroup.dcLocation, "next-leader") + // Grant a etcd lease with checkPriorityStep * 1.5 + nextLeaderLease := clientv3.NewLease(am.member.Client()) + ctx, cancel := context.WithTimeout(am.member.Client().Ctx(), etcdutil.DefaultRequestTimeout) + leaseResp, err := nextLeaderLease.Grant(ctx, int64(checkPriorityStep.Seconds()*1.5)) + cancel() + if err != nil { + err = errs.ErrEtcdGrantLease.Wrap(err).GenWithStackByCause() + log.Error("failed to grant the lease of the next leader id key", errs.ZapError(err)) + continue + } + resp, err := kv.NewSlowLogTxn(am.member.Client()). + If(clientv3.Compare(clientv3.CreateRevision(nextLeaderKey), "=", 0)). + Then(clientv3.OpPut(nextLeaderKey, fmt.Sprint(serverID), clientv3.WithLease(leaseResp.ID))). + Commit() + if err != nil { + err = errs.ErrEtcdTxn.Wrap(err).GenWithStackByCause() + log.Error("failed to write next leader id into etcd", errs.ZapError(err)) + continue + } + if !resp.Succeeded { + log.Warn("write next leader id into etcd unsuccessfully") + } + } + } + // Check next leader and resign + // Filter out allocators with leadership + allocatorGroups = am.getAllocatorGroups(FilterDCLocation(config.GlobalDCLocation), FilterUnavailableLeadership()) + for _, allocatorGroup := range allocatorGroups { + nextLeader, err := am.getNextLeaderID(allocatorGroup.dcLocation) + if err != nil { + log.Error("get next leader from etcd failed", + zap.String("dc-location", allocatorGroup.dcLocation), + errs.ZapError(err)) + continue + } + // nextLeader is not empty and isn't same with the server ID, resign the leader + if nextLeader != 0 && nextLeader != serverID { + am.resetAllocatorGroup(allocatorGroup.dcLocation) + } + } +} + +func (am *AllocatorManager) getServerDCLocation(serverID uint64) (string, error) { + dcLocationMap, err := am.GetClusterDCLocations() + if err != nil { + return "", err + } + for dcLocation, serverIDs := range dcLocationMap { + if slice.AnyOf(serverIDs, func(i int) bool { return serverIDs[i] == serverID }) { + return dcLocation, nil + } + } + return "", nil +} + +func (am *AllocatorManager) getNextLeaderID(dcLocation string) (uint64, error) { + nextLeaderKey := path.Join(am.rootPath, dcLocation, "next-leader") + nextLeaderValue, err := etcdutil.GetValue(am.member.Client(), nextLeaderKey) + if err != nil { + return 0, err + } + if len(nextLeaderValue) == 0 { + return 0, nil + } + return strconv.ParseUint(string(nextLeaderValue), 10, 64) +} + +func (am *AllocatorManager) deleteNextLeaderID(dcLocation string) error { + nextLeaderKey := path.Join(am.rootPath, dcLocation, "next-leader") + resp, err := kv.NewSlowLogTxn(am.member.Client()). + Then(clientv3.OpDelete(nextLeaderKey)). + Commit() + if err != nil { + return errs.ErrEtcdKVDelete.Wrap(err).GenWithStackByCause() + } + if !resp.Succeeded { + return errs.ErrEtcdTxn.FastGenByArgs() + } + return nil +} + func (am *AllocatorManager) deleteAllocatorGroup(dcLocation string) { am.Lock() defer am.Unlock() diff --git a/server/tso/filter.go b/server/tso/filter.go index 14ec8907b64..d141e363c2f 100644 --- a/server/tso/filter.go +++ b/server/tso/filter.go @@ -23,6 +23,11 @@ func FilterUninitialized() func(ag *allocatorGroup) bool { return func(ag *allocatorGroup) bool { return !ag.allocator.IsInitialize() } } +// FilterAvailableLeadership will filter out the allocatorGroup whose leadership is available. +func FilterAvailableLeadership() func(ag *allocatorGroup) bool { + return func(ag *allocatorGroup) bool { return ag.leadership.Check() } +} + // FilterUnavailableLeadership will filter out the allocatorGroup whose leadership is unavailable. func FilterUnavailableLeadership() func(ag *allocatorGroup) bool { return func(ag *allocatorGroup) bool { return !ag.leadership.Check() } diff --git a/tests/cluster.go b/tests/cluster.go index 31e98efdbd3..b331926b0e4 100644 --- a/tests/cluster.go +++ b/tests/cluster.go @@ -192,6 +192,21 @@ func (s *TestServer) GetLeader() *pdpb.Member { return s.server.GetLeader() } +// GetAllocatorLeader returns current allocator leader +// of PD cluster for given dc-location. +func (s *TestServer) GetAllocatorLeader(dcLocation string) *pdpb.Member { + // For the leader of Global TSO Allocator, it's the PD leader + if dcLocation == config.GlobalDCLocation { + return s.GetLeader() + } + tsoAllocatorManager := s.GetTSOAllocatorManager() + allocator, err := tsoAllocatorManager.GetAllocator(dcLocation) + if err != nil { + return nil + } + return allocator.(*tso.LocalTSOAllocator).GetAllocatorLeader() +} + // GetCluster returns PD cluster. func (s *TestServer) GetCluster() *metapb.Cluster { s.RLock() @@ -220,6 +235,19 @@ func (s *TestServer) IsLeader() bool { return !s.server.IsClosed() && s.server.GetMember().IsLeader() } +// IsAllocatorLeader returns whether the server is a TSO Allocator leader or not. +func (s *TestServer) IsAllocatorLeader(dcLocation string) bool { + if dcLocation == config.GlobalDCLocation { + return s.IsLeader() + } + tsoAllocatorManager := s.GetTSOAllocatorManager() + allocator, err := tsoAllocatorManager.GetAllocator(dcLocation) + if err != nil { + return false + } + return !s.server.IsClosed() && allocator.(*tso.LocalTSOAllocator).IsStillAllocatorLeader() +} + // GetEtcdLeader returns the builtin etcd leader. func (s *TestServer) GetEtcdLeader() (string, error) { s.RLock() @@ -486,6 +514,32 @@ func (c *TestCluster) ResignLeader() error { return errors.New("no leader") } +// WaitAllocatorLeader is used to get the Local TSO Allocator leader. +// If it exceeds the maximum number of loops, it will return an empty string. +func (c *TestCluster) WaitAllocatorLeader(dcLocation string) string { + for i := 0; i < 100; i++ { + counter := make(map[string]int) + running := 0 + for _, s := range c.servers { + if s.state == Running { + running++ + } + serverName := s.GetAllocatorLeader(dcLocation).GetName() + if serverName != "" { + counter[serverName]++ + } + } + for serverName, num := range counter { + if num == running && c.GetServer(serverName).IsAllocatorLeader(dcLocation) { + time.Sleep(WaitLeaderReturnDelay) + return serverName + } + } + time.Sleep(WaitLeaderCheckInterval) + } + return "" +} + // GetCluster returns PD cluster. func (c *TestCluster) GetCluster() *metapb.Cluster { leader := c.GetLeader() diff --git a/tests/server/tso/allocator_test.go b/tests/server/tso/allocator_test.go index 791dcaf6aee..3507d480037 100644 --- a/tests/server/tso/allocator_test.go +++ b/tests/server/tso/allocator_test.go @@ -15,7 +15,6 @@ package tso_test import ( "context" - "time" . "github.com/pingcap/check" "github.com/tikv/pd/pkg/slice" @@ -25,8 +24,6 @@ import ( "github.com/tikv/pd/tests" ) -const waitAllocatorCheckInterval = 2 * time.Second - var _ = Suite(&testAllocatorSuite{}) type testAllocatorSuite struct { @@ -61,9 +58,11 @@ func (s *testAllocatorSuite) TestAllocatorLeader(c *C) { err = cluster.RunInitialServers() c.Assert(err, IsNil) - - // Wait for a while to check - time.Sleep(waitAllocatorCheckInterval) + // Wait for each DC's Local TSO Allocator leader + for _, dcLocation := range dcLocationConfig { + leaderName := cluster.WaitAllocatorLeader(dcLocation) + c.Assert(len(leaderName), Greater, 0) + } // To check whether we have enough Local TSO Allocator leaders allAllocatorLeaders := make([]tso.Allocator, 0, dcLocationNum) for _, server := range cluster.GetServers() { diff --git a/tests/server/tso/manager_test.go b/tests/server/tso/manager_test.go index 147709f0f5d..cee087a9a81 100644 --- a/tests/server/tso/manager_test.go +++ b/tests/server/tso/manager_test.go @@ -15,6 +15,7 @@ package tso_test import ( "context" + "time" . "github.com/pingcap/check" "github.com/tikv/pd/server" @@ -42,11 +43,9 @@ func (s *testManagerSuite) TearDownSuite(c *C) { // and test whether we can get the whole dc-location config from each server. func (s *testManagerSuite) TestClusterDCLocations(c *C) { testCase := struct { - serverNumber int dcLocationNumber int dcLocationConfig map[string]string }{ - serverNumber: 6, dcLocationNumber: 3, dcLocationConfig: map[string]string{ "pd1": "dc-1", @@ -57,7 +56,8 @@ func (s *testManagerSuite) TestClusterDCLocations(c *C) { "pd6": "dc-3", }, } - cluster, err := tests.NewTestCluster(s.ctx, testCase.serverNumber, func(conf *config.Config, serverName string) { + serverNumber := len(testCase.dcLocationConfig) + cluster, err := tests.NewTestCluster(s.ctx, serverNumber, func(conf *config.Config, serverName string) { conf.LocalTSO.EnableLocalTSO = true conf.LocalTSO.DCLocation = testCase.dcLocationConfig[serverName] }) @@ -85,6 +85,60 @@ func (s *testManagerSuite) TestClusterDCLocations(c *C) { c.Assert(obtainedDCLocation, Equals, expectedDCLocation) } } - c.Assert(obtainedServerNumber, Equals, testCase.serverNumber) + c.Assert(obtainedServerNumber, Equals, serverNumber) + } +} + +const waitAllocatorPriorityCheckInterval = 3 * time.Minute + +var _ = Suite(&testPrioritySuite{}) + +type testPrioritySuite struct { + ctx context.Context + cancel context.CancelFunc +} + +func (s *testPrioritySuite) SetUpSuite(c *C) { + s.ctx, s.cancel = context.WithCancel(context.Background()) + server.EnableZap = true +} + +func (s *testPrioritySuite) TearDownSuite(c *C) { + s.cancel() +} + +func (s *testPrioritySuite) TestAllocatorPriority(c *C) { + dcLocationConfig := map[string]string{ + "pd1": "dc-1", + "pd2": "dc-2", + "pd3": "dc-3", + } + serverNumber := len(dcLocationConfig) + cluster, err := tests.NewTestCluster(s.ctx, serverNumber, func(conf *config.Config, serverName string) { + conf.LocalTSO.EnableLocalTSO = true + conf.LocalTSO.DCLocation = dcLocationConfig[serverName] + }) + defer cluster.Destroy() + c.Assert(err, IsNil) + + err = cluster.RunInitialServers() + c.Assert(err, IsNil) + + // Before the priority is checked, we may have allocators typology like this: + // pd1: dc-1, dc-2 and dc-3 allocator leader + // pd2: None + // pd3: None + // After the priority is checked, we should have allocators typology like this: + // pd1: dc-1 allocator leader + // pd2: dc-2 allocator leader + // pd3: dc-3 allocator leader + + // Because the default priority checking period is 1 minute, + // so we sleep longer here. + time.Sleep(waitAllocatorPriorityCheckInterval) + + for serverName, dcLocation := range dcLocationConfig { + currentLeaderName := cluster.WaitAllocatorLeader(dcLocation) + c.Assert(currentLeaderName, Equals, serverName) } } From 9ed91110360740b371b7ae7ad44a999a6880c0c1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Wed, 7 Oct 2020 10:55:09 +0800 Subject: [PATCH 13/27] scheduler: Add auto-gc for Region Scatter (#3038) Signed-off-by: longfangsong --- pkg/cache/ttl.go | 5 ++ server/cluster/coordinator.go | 2 +- server/schedule/region_scatterer.go | 86 ++++++++++++++++-------- server/schedule/region_scatterer_test.go | 47 ++++++++++--- 4 files changed, 102 insertions(+), 38 deletions(-) diff --git a/pkg/cache/ttl.go b/pkg/cache/ttl.go index 390965af5ad..7ef340daa07 100644 --- a/pkg/cache/ttl.go +++ b/pkg/cache/ttl.go @@ -246,3 +246,8 @@ func (c *TTLString) Pop() (string, interface{}, bool) { } return key, v, true } + +// Get return the value by key id +func (c *TTLString) Get(id string) (interface{}, bool) { + return c.ttlCache.get(id) +} diff --git a/server/cluster/coordinator.go b/server/cluster/coordinator.go index 00603c78c40..d99feec614a 100644 --- a/server/cluster/coordinator.go +++ b/server/cluster/coordinator.go @@ -76,7 +76,7 @@ func newCoordinator(ctx context.Context, cluster *RaftCluster, hbStreams *hbstre cancel: cancel, cluster: cluster, checkers: schedule.NewCheckerController(ctx, cluster, cluster.ruleManager, opController), - regionScatterer: schedule.NewRegionScatterer(cluster), + regionScatterer: schedule.NewRegionScatterer(ctx, cluster), schedulers: make(map[string]*scheduleController), opController: opController, hbStreams: hbStreams, diff --git a/server/schedule/region_scatterer.go b/server/schedule/region_scatterer.go index 8163544822c..8f84605a2f1 100644 --- a/server/schedule/region_scatterer.go +++ b/server/schedule/region_scatterer.go @@ -14,6 +14,7 @@ package schedule import ( + "context" "math" "math/rand" "sync" @@ -23,6 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/typeutil" "github.com/tikv/pd/server/core" @@ -34,45 +36,69 @@ import ( const regionScatterName = "region-scatter" +var gcInterval = time.Minute +var gcTTL = time.Minute * 3 + type selectedStores struct { mu sync.Mutex // If checkExist is true, after each putting operation, an entry with the key constructed by group and storeID would be put // into "stores" map. And the entry with the same key (storeID, group) couldn't be put before "stores" being reset checkExist bool - // TODO: support auto-gc for the stores - stores map[string]map[uint64]struct{} // group -> StoreID -> struct{} - // TODO: support auto-gc for the groupDistribution - groupDistribution map[string]map[uint64]uint64 // group -> StoreID -> count + + stores *cache.TTLString // value type: map[uint64]struct{}, group -> StoreID -> struct{} + groupDistribution *cache.TTLString // value type: map[uint64]uint64, group -> StoreID -> count } -func newSelectedStores(checkExist bool) *selectedStores { +func newSelectedStores(ctx context.Context, checkExist bool) *selectedStores { return &selectedStores{ checkExist: checkExist, - stores: make(map[string]map[uint64]struct{}), - groupDistribution: make(map[string]map[uint64]uint64), + stores: cache.NewStringTTL(ctx, gcInterval, gcTTL), + groupDistribution: cache.NewStringTTL(ctx, gcInterval, gcTTL), + } +} + +func (s *selectedStores) getStore(group string) (map[uint64]struct{}, bool) { + if result, ok := s.stores.Get(group); ok { + return result.(map[uint64]struct{}), true + } + return nil, false +} + +func (s *selectedStores) getGroupDistribution(group string) (map[uint64]uint64, bool) { + if result, ok := s.groupDistribution.Get(group); ok { + return result.(map[uint64]uint64), true + } + return nil, false +} + +func (s *selectedStores) getStoreOrDefault(group string) map[uint64]struct{} { + if result, ok := s.getStore(group); ok { + return result + } + return make(map[uint64]struct{}) +} + +func (s *selectedStores) getGroupDistributionOrDefault(group string) map[uint64]uint64 { + if result, ok := s.getGroupDistribution(group); ok { + return result } + return make(map[uint64]uint64) } func (s *selectedStores) put(id uint64, group string) bool { s.mu.Lock() defer s.mu.Unlock() if s.checkExist { - placed, ok := s.stores[group] - if !ok { - placed = map[uint64]struct{}{} - } + placed := s.getStoreOrDefault(group) if _, ok := placed[id]; ok { return false } placed[id] = struct{}{} - s.stores[group] = placed - } - distribution, ok := s.groupDistribution[group] - if !ok { - distribution = make(map[uint64]uint64) + s.stores.Put(group, placed) } + distribution := s.getGroupDistributionOrDefault(group) distribution[id] = distribution[id] + 1 - s.groupDistribution[group] = distribution + s.groupDistribution.Put(group, distribution) return true } @@ -82,13 +108,13 @@ func (s *selectedStores) reset() { if !s.checkExist { return } - s.stores = make(map[string]map[uint64]struct{}) + s.stores.Clear() } func (s *selectedStores) get(id uint64, group string) uint64 { s.mu.Lock() defer s.mu.Unlock() - distribution, ok := s.groupDistribution[group] + distribution, ok := s.getGroupDistribution(group) if !ok { return 0 } @@ -106,7 +132,7 @@ func (s *selectedStores) newFilters(scope, group string) []filter.Filter { return nil } cloned := make(map[uint64]struct{}) - if groupPlaced, ok := s.stores[group]; ok { + if groupPlaced, ok := s.getStore(group); ok { for id := range groupPlaced { cloned[id] = struct{}{} } @@ -116,6 +142,7 @@ func (s *selectedStores) newFilters(scope, group string) []filter.Filter { // RegionScatterer scatters regions. type RegionScatterer struct { + ctx context.Context name string cluster opt.Cluster ordinaryEngine engineContext @@ -124,11 +151,12 @@ type RegionScatterer struct { // NewRegionScatterer creates a region scatterer. // RegionScatter is used for the `Lightning`, it will scatter the specified regions before import data. -func NewRegionScatterer(cluster opt.Cluster) *RegionScatterer { +func NewRegionScatterer(ctx context.Context, cluster opt.Cluster) *RegionScatterer { return &RegionScatterer{ + ctx: ctx, name: regionScatterName, cluster: cluster, - ordinaryEngine: newEngineContext(filter.NewOrdinaryEngineFilter(regionScatterName)), + ordinaryEngine: newEngineContext(ctx, filter.NewOrdinaryEngineFilter(regionScatterName)), specialEngines: make(map[string]engineContext), } } @@ -139,12 +167,12 @@ type engineContext struct { selectedLeader *selectedStores } -func newEngineContext(filters ...filter.Filter) engineContext { +func newEngineContext(ctx context.Context, filters ...filter.Filter) engineContext { filters = append(filters, filter.StoreStateFilter{ActionScope: regionScatterName}) return engineContext{ filters: filters, - selectedPeer: newSelectedStores(true), - selectedLeader: newSelectedStores(false), + selectedPeer: newSelectedStores(ctx, true), + selectedLeader: newSelectedStores(ctx, false), } } @@ -256,12 +284,12 @@ func (r *RegionScatterer) scatterRegion(region *core.RegionInfo, group string) * targetLeader := r.selectAvailableLeaderStores(group, targetPeers, r.ordinaryEngine) for engine, peers := range specialPeers { - context, ok := r.specialEngines[engine] + ctx, ok := r.specialEngines[engine] if !ok { - context = newEngineContext(filter.NewEngineFilter(r.name, engine)) - r.specialEngines[engine] = context + ctx = newEngineContext(r.ctx, filter.NewEngineFilter(r.name, engine)) + r.specialEngines[engine] = ctx } - scatterWithSameEngine(peers, context) + scatterWithSameEngine(peers, ctx) } op, err := operator.CreateScatterRegionOperator("scatter-region", r.cluster, region, targetPeers, targetLeader) diff --git a/server/schedule/region_scatterer_test.go b/server/schedule/region_scatterer_test.go index 15c186c2c79..b5a3342f019 100644 --- a/server/schedule/region_scatterer_test.go +++ b/server/schedule/region_scatterer_test.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "math" + "time" . "github.com/pingcap/check" "github.com/pingcap/failpoint" @@ -95,7 +96,9 @@ func (s *testScatterRegionSuite) scatter(c *C, numStores, numRegions uint64, use tc.AddLeaderRegion(i, 1, 2, 3) } - scatterer := NewRegionScatterer(tc) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + scatterer := NewRegionScatterer(ctx, tc) for i := uint64(1); i <= numRegions; i++ { region := tc.GetRegion(i) @@ -161,7 +164,9 @@ func (s *testScatterRegionSuite) scatterSpecial(c *C, numOrdinaryStores, numSpec ) } - scatterer := NewRegionScatterer(tc) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + scatterer := NewRegionScatterer(ctx, tc) for i := uint64(1); i <= numRegions; i++ { region := tc.GetRegion(i) @@ -226,7 +231,7 @@ func (s *testScatterRegionSuite) TestStoreLimit(c *C) { tc.AddLeaderRegion(i, seq.next(), seq.next(), seq.next()) } - scatterer := NewRegionScatterer(tc) + scatterer := NewRegionScatterer(ctx, tc) for i := uint64(1); i <= 5; i++ { region := tc.GetRegion(i) @@ -266,7 +271,8 @@ func (s *testScatterRegionSuite) TestScatterCheck(c *C) { } for _, testcase := range testcases { c.Logf(testcase.name) - scatterer := NewRegionScatterer(tc) + ctx, cancel := context.WithCancel(context.Background()) + scatterer := NewRegionScatterer(ctx, tc) _, err := scatterer.Scatter(testcase.checkRegion, "") if testcase.needFix { c.Assert(err, NotNil) @@ -276,6 +282,7 @@ func (s *testScatterRegionSuite) TestScatterCheck(c *C) { c.Assert(tc.CheckRegionUnderSuspect(1), Equals, false) } tc.ResetSuspectRegions() + cancel() } } @@ -307,7 +314,8 @@ func (s *testScatterRegionSuite) TestScatterGroup(c *C) { for _, testcase := range testcases { c.Logf(testcase.name) - scatterer := NewRegionScatterer(tc) + ctx, cancel := context.WithCancel(context.Background()) + scatterer := NewRegionScatterer(ctx, tc) regionID := 1 for i := 0; i < 100; i++ { for j := 0; j < testcase.groupCount; j++ { @@ -327,7 +335,8 @@ func (s *testScatterRegionSuite) TestScatterGroup(c *C) { group := fmt.Sprintf("group-%v", i) max := uint64(0) min := uint64(math.MaxUint64) - for _, count := range scatterer.ordinaryEngine.selectedLeader.groupDistribution[group] { + groupDistribution, _ := scatterer.ordinaryEngine.selectedLeader.groupDistribution.Get(group) + for _, count := range groupDistribution.(map[uint64]uint64) { if count > max { max = count } @@ -340,6 +349,7 @@ func (s *testScatterRegionSuite) TestScatterGroup(c *C) { c.Assert(max, GreaterEqual, uint64(20)) c.Assert(max-min, LessEqual, uint64(3)) } + cancel() } } @@ -365,7 +375,8 @@ func (s *testScatterRegionSuite) TestScattersGroup(c *C) { } group := "group" for _, testcase := range testcases { - scatterer := NewRegionScatterer(tc) + ctx, cancel := context.WithCancel(context.Background()) + scatterer := NewRegionScatterer(ctx, tc) regions := map[uint64]*core.RegionInfo{} for i := 1; i <= 100; i++ { regions[uint64(i)] = tc.AddLeaderRegion(uint64(i), 1, 2, 3) @@ -379,7 +390,7 @@ func (s *testScatterRegionSuite) TestScattersGroup(c *C) { scatterer.ScatterRegions(regions, failures, group, 3) max := uint64(0) min := uint64(math.MaxUint64) - for _, count := range scatterer.ordinaryEngine.selectedLeader.groupDistribution[group] { + for _, count := range scatterer.ordinaryEngine.selectedLeader.getGroupDistributionOrDefault(group) { if count > max { max = count } @@ -399,5 +410,25 @@ func (s *testScatterRegionSuite) TestScattersGroup(c *C) { } else { c.Assert(len(failures), Equals, 0) } + cancel() } } + +func (s *testScatterRegionSuite) TestSelectedStoreGC(c *C) { + // use a shorter gcTTL and gcInterval during the test + gcInterval = time.Second + gcTTL = time.Second * 3 + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + stores := newSelectedStores(ctx, true) + stores.put(1, "testgroup") + _, ok := stores.getStore("testgroup") + c.Assert(ok, Equals, true) + _, ok = stores.getGroupDistribution("testgroup") + c.Assert(ok, Equals, true) + time.Sleep(gcTTL) + _, ok = stores.getStore("testgroup") + c.Assert(ok, Equals, false) + _, ok = stores.getGroupDistribution("testgroup") + c.Assert(ok, Equals, false) +} From f0b0d97bc7ee77b34352993c718f43ce852d6dea Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 9 Oct 2020 16:27:10 +0800 Subject: [PATCH 14/27] metrics: add heartbeat interval and distribution of region bytes, keys metrics (#3031) Signed-off-by: lhy1024 --- metrics/grafana/pd.json | 383 +++++++++++++++++++++++++++- server/cluster/cluster.go | 3 + server/statistics/hot_peer_cache.go | 17 ++ server/statistics/metrics.go | 54 ++++ server/statistics/store.go | 5 + 5 files changed, 461 insertions(+), 1 deletion(-) diff --git a/metrics/grafana/pd.json b/metrics/grafana/pd.json index 90c8fc71ff5..12a1c661748 100644 --- a/metrics/grafana/pd.json +++ b/metrics/grafana/pd.json @@ -10,11 +10,17 @@ } ], "__requires": [ + { + "type": "panel", + "id": "bargauge", + "name": "Bar gauge", + "version": "" + }, { "type": "grafana", "id": "grafana", "name": "Grafana", - "version": "6.1.6" + "version": "7.1.5" }, { "type": "panel", @@ -7718,6 +7724,381 @@ "title": "TiDB", "type": "row" }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 23 + }, + "id": 1420, + "panels": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "custom": {}, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 24 + }, + "id": 1407, + "options": { + "displayMode": "lcd", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "mean" + ], + "fields": "", + "values": false + }, + "showUnfilled": true + }, + "pluginVersion": "7.1.5", + "targets": [ + { + "expr": "sum(delta(pd_scheduler_read_byte_hist_bucket{instance=~\"$instance\"}[1m])) by (le)", + "format": "heatmap", + "hide": false, + "interval": "", + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Read Region Byte", + "transparent": true, + "type": "bargauge" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "custom": {}, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 24 + }, + "id": 1411, + "options": { + "displayMode": "lcd", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "mean" + ], + "fields": "", + "values": false + }, + "showUnfilled": true + }, + "pluginVersion": "7.1.5", + "targets": [ + { + "expr": "sum(delta(pd_scheduler_write_byte_hist_bucket{instance=~\"$instance\"}[1m])) by (le)", + "format": "heatmap", + "hide": false, + "interval": "", + "legendFormat": "{{le}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Write Region Byte", + "transparent": true, + "type": "bargauge" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "custom": { + "align": null + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 32 + }, + "id": 1406, + "options": { + "displayMode": "lcd", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "mean" + ], + "fields": "", + "values": false + }, + "showUnfilled": true + }, + "pluginVersion": "7.1.5", + "targets": [ + { + "expr": "sum(delta(pd_scheduler_read_key_hist_bucket{instance=~\"$instance\"}[1m])) by (le)", + "format": "heatmap", + "hide": false, + "interval": "", + "legendFormat": "{{le}}", + "refId": "C" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Read Region Key", + "transparent": true, + "type": "bargauge" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "custom": { + "align": null + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 32 + }, + "id": 1412, + "options": { + "displayMode": "lcd", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "mean" + ], + "fields": "", + "values": false + }, + "showUnfilled": true + }, + "pluginVersion": "7.1.5", + "targets": [ + { + "expr": "sum(delta(pd_scheduler_write_key_hist_bucket{instance=~\"$instance\"}[1m])) by (le)", + "format": "heatmap", + "hide": false, + "interval": "", + "legendFormat": "{{le}}", + "refId": "C" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Write Region Key", + "transparent": true, + "type": "bargauge" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "custom": { + "align": null + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 40 + }, + "id": 1408, + "interval": "", + "options": { + "displayMode": "lcd", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "mean" + ], + "fields": "", + "values": false + }, + "showUnfilled": true + }, + "pluginVersion": "7.1.5", + "repeatDirection": "h", + "targets": [ + { + "expr": "sum(delta(pd_scheduler_store_heartbeat_interval_hist_bucket{instance=~\"$instance\"}[1m])) by (le)", + "format": "heatmap", + "hide": false, + "interval": "", + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Store Heartbeat Interval", + "transparent": true, + "type": "bargauge" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "custom": { + "align": null + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 40 + }, + "id": 1409, + "interval": "", + "options": { + "displayMode": "lcd", + "orientation": "horizontal", + "reduceOptions": { + "calcs": [ + "mean" + ], + "fields": "", + "values": false + }, + "showUnfilled": true + }, + "pluginVersion": "7.1.5", + "repeatDirection": "h", + "targets": [ + { + "expr": "sum(delta(pd_scheduler_region_heartbeat_interval_hist_bucket{instance=~\"$instance\"}[1m])) by (le)", + "format": "heatmap", + "hide": false, + "interval": "", + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Region Heartbeat Interval", + "transparent": true, + "type": "bargauge" + } + ], + "title": "Heartbeat distribution ", + "type": "row" + }, { "collapsed": true, "gridPos": { diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 8a3a3288634..5432b0d9a94 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -512,6 +512,9 @@ func (c *RaftCluster) HandleStoreHeartbeat(stats *pdpb.StoreStats) error { newStore = newStore.Clone(core.SetLastPersistTime(time.Now())) } } + if store := c.core.GetStore(newStore.GetID()); store != nil { + c.storesStats.UpdateStoreHeartbeatMetrics(store) + } c.core.PutStore(newStore) c.storesStats.Observe(newStore.GetID(), newStore.GetStoreStats()) c.storesStats.UpdateTotalBytesRate(c.core.GetStores) diff --git a/server/statistics/hot_peer_cache.go b/server/statistics/hot_peer_cache.go index ee277635f38..4468d0f4793 100644 --- a/server/statistics/hot_peer_cache.go +++ b/server/statistics/hot_peer_cache.go @@ -103,6 +103,21 @@ func (f *hotPeerCache) Update(item *HotPeerStat) { } } +func (f *hotPeerCache) collectRegionMetrics(byteRate, keyRate float64, interval uint64) { + regionHeartbeatIntervalHist.Observe(float64(interval)) + if interval == 0 { + return + } + if f.kind == ReadFlow { + readByteHist.Observe(byteRate) + readKeyHist.Observe(keyRate) + } + if f.kind == WriteFlow { + writeByteHist.Observe(byteRate) + writeKeyHist.Observe(keyRate) + } +} + // CheckRegionFlow checks the flow information of region. func (f *hotPeerCache) CheckRegionFlow(region *core.RegionInfo, storesStats *StoresStats) (ret []*HotPeerStat) { totalBytes := float64(f.getTotalBytes(region)) @@ -114,6 +129,8 @@ func (f *hotPeerCache) CheckRegionFlow(region *core.RegionInfo, storesStats *Sto byteRate := totalBytes / float64(interval) keyRate := totalKeys / float64(interval) + f.collectRegionMetrics(byteRate, keyRate, interval) + // old region is in the front and new region is in the back // which ensures it will hit the cache if moving peer or transfer leader occurs with the same replica number diff --git a/server/statistics/metrics.go b/server/statistics/metrics.go index f1cc01005ae..c996d17ff56 100644 --- a/server/statistics/metrics.go +++ b/server/statistics/metrics.go @@ -71,6 +71,54 @@ var ( Name: "label_level", Help: "Number of regions in the different label level.", }, []string{"type"}) + readByteHist = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "pd", + Subsystem: "scheduler", + Name: "read_byte_hist", + Help: "The distribution of region read bytes", + Buckets: prometheus.ExponentialBuckets(1, 8, 12), + }) + writeByteHist = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "pd", + Subsystem: "scheduler", + Name: "write_byte_hist", + Help: "The distribution of region write bytes", + Buckets: prometheus.ExponentialBuckets(1, 8, 12), + }) + readKeyHist = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "pd", + Subsystem: "scheduler", + Name: "read_key_hist", + Help: "The distribution of region read keys", + Buckets: prometheus.ExponentialBuckets(1, 2, 18), + }) + writeKeyHist = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "pd", + Subsystem: "scheduler", + Name: "write_key_hist", + Help: "The distribution of region write keys", + Buckets: prometheus.ExponentialBuckets(1, 2, 18), + }) + regionHeartbeatIntervalHist = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "pd", + Subsystem: "scheduler", + Name: "region_heartbeat_interval_hist", + Help: "Bucketed histogram of the batch size of handled requests.", + Buckets: prometheus.LinearBuckets(0, 30, 20), + }) + storeHeartbeatIntervalHist = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "pd", + Subsystem: "scheduler", + Name: "store_heartbeat_interval_hist", + Help: "Bucketed histogram of the batch size of handled requests.", + Buckets: prometheus.LinearBuckets(0, 5, 12), + }) ) func init() { @@ -81,4 +129,10 @@ func init() { prometheus.MustRegister(placementStatusGauge) prometheus.MustRegister(configStatusGauge) prometheus.MustRegister(regionLabelLevelGauge) + prometheus.MustRegister(readByteHist) + prometheus.MustRegister(readKeyHist) + prometheus.MustRegister(writeKeyHist) + prometheus.MustRegister(writeByteHist) + prometheus.MustRegister(regionHeartbeatIntervalHist) + prometheus.MustRegister(storeHeartbeatIntervalHist) } diff --git a/server/statistics/store.go b/server/statistics/store.go index d86d1f9bf88..ac1859a9fc9 100644 --- a/server/statistics/store.go +++ b/server/statistics/store.go @@ -286,6 +286,11 @@ func (s *StoresStats) FilterUnhealthyStore(cluster core.StoreSetInformer) { } } +// UpdateStoreHeartbeatMetrics is used to update store heartbeat interval metrics +func (s *StoresStats) UpdateStoreHeartbeatMetrics(store *core.StoreInfo) { + storeHeartbeatIntervalHist.Observe(time.Since(store.GetLastHeartbeatTS()).Seconds()) +} + // RollingStoreStats are multiple sets of recent historical records with specified windows size. type RollingStoreStats struct { sync.RWMutex From 90a934a477a117af10252265abf25b0bf348161c Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Fri, 9 Oct 2020 16:43:11 +0800 Subject: [PATCH 15/27] tso: Support configuring tso update physical interval (#3027) Signed-off-by: MyonKeminta --- server/config/config.go | 19 +++++++++++++++++ server/config/config_test.go | 25 ++++++++++++++++++++++ server/server.go | 2 +- server/tso/allocator_manager.go | 38 ++++++++++++++++++++++----------- server/tso/global_allocator.go | 17 ++++++++++----- server/tso/local_allocator.go | 18 +++++++++++----- server/tso/tso.go | 11 +++++----- 7 files changed, 101 insertions(+), 29 deletions(-) diff --git a/server/config/config.go b/server/config/config.go index eb59311df8b..1962592b056 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -86,6 +86,12 @@ type Config struct { // TSOSaveInterval is the interval to save timestamp. TSOSaveInterval typeutil.Duration `toml:"tso-save-interval" json:"tso-save-interval"` + // The interval to update physical part of timestamp. Usually, this config should not be set. + // It's only useful for test purposes. + // This config is only valid in 50ms to 10s. If it's configured too long or too short, it will + // be automatically clamped to the range. + TSOUpdatePhysicalInterval typeutil.Duration `toml:"tso-update-physical-interval" json:"tso-update-physical-interval"` + // Local TSO service related configuration. LocalTSO LocalTSOConfig `toml:"local-tso" json:"local-tso"` @@ -222,6 +228,11 @@ const ( defaultDRWaitStoreTimeout = time.Minute defaultDRWaitSyncTimeout = time.Minute defaultDRWaitAsyncTimeout = 2 * time.Minute + + // DefaultTSOUpdatePhysicalInterval is the default value of the config `TSOUpdatePhysicalInterval`. + DefaultTSOUpdatePhysicalInterval = 50 * time.Millisecond + maxTSOUpdatePhysicalInterval = 10 * time.Second + minTSOUpdatePhysicalInterval = 50 * time.Millisecond ) var ( @@ -499,6 +510,14 @@ func (c *Config) Adjust(meta *toml.MetaData) error { adjustDuration(&c.TSOSaveInterval, time.Duration(defaultLeaderLease)*time.Second) + adjustDuration(&c.TSOUpdatePhysicalInterval, DefaultTSOUpdatePhysicalInterval) + + if c.TSOUpdatePhysicalInterval.Duration > maxTSOUpdatePhysicalInterval { + c.TSOUpdatePhysicalInterval.Duration = maxTSOUpdatePhysicalInterval + } else if c.TSOUpdatePhysicalInterval.Duration < minTSOUpdatePhysicalInterval { + c.TSOUpdatePhysicalInterval.Duration = minTSOUpdatePhysicalInterval + } + if err := c.LocalTSO.Validate(); err != nil { return err } diff --git a/server/config/config_test.go b/server/config/config_test.go index ef387eb10cd..0ff8d80fc5e 100644 --- a/server/config/config_test.go +++ b/server/config/config_test.go @@ -171,6 +171,8 @@ leader-schedule-limit = 0 c.Assert(cfg.Schedule.MaxMergeRegionKeys, Equals, uint64(defaultMaxMergeRegionKeys)) c.Assert(cfg.PDServerCfg.MetricStorage, Equals, "http://127.0.0.1:9090") + c.Assert(cfg.TSOUpdatePhysicalInterval.Duration, Equals, DefaultTSOUpdatePhysicalInterval) + // Check undefined config fields cfgData = ` type = "pd" @@ -228,6 +230,29 @@ address = "localhost:9090" c.Assert(cfg.Metric.PushInterval.Duration, Equals, 35*time.Second) c.Assert(cfg.Metric.PushAddress, Equals, "localhost:9090") + + // Test clamping TSOUpdatePhysicalInterval value + cfgData = ` +tso-update-physical-interval = "10ms" +` + cfg = NewConfig() + meta, err = toml.Decode(cfgData, &cfg) + c.Assert(err, IsNil) + err = cfg.Adjust(&meta) + c.Assert(err, IsNil) + + c.Assert(cfg.TSOUpdatePhysicalInterval.Duration, Equals, minTSOUpdatePhysicalInterval) + + cfgData = ` +tso-update-physical-interval = "15s" +` + cfg = NewConfig() + meta, err = toml.Decode(cfgData, &cfg) + c.Assert(err, IsNil) + err = cfg.Adjust(&meta) + c.Assert(err, IsNil) + + c.Assert(cfg.TSOUpdatePhysicalInterval.Duration, Equals, maxTSOUpdatePhysicalInterval) } func (s *testConfigSuite) TestMigrateFlags(c *C) { diff --git a/server/server.go b/server/server.go index 79b7c827245..73b660c5ca8 100644 --- a/server/server.go +++ b/server/server.go @@ -351,7 +351,7 @@ func (s *Server) startServer(ctx context.Context) error { s.member.SetMemberGitHash(s.member.ID(), versioninfo.PDGitHash) s.idAllocator = id.NewAllocatorImpl(s.client, s.rootPath, s.member.MemberValue()) s.tsoAllocatorManager = tso.NewAllocatorManager( - s.member, s.rootPath, s.cfg.TSOSaveInterval.Duration, + s.member, s.rootPath, s.cfg.TSOSaveInterval.Duration, s.cfg.TSOUpdatePhysicalInterval.Duration, func() time.Duration { return s.persistOptions.GetMaxResetTSGap() }, ) if err = s.tsoAllocatorManager.SetLocalTSOConfig(s.cfg.LocalTSO); err != nil { diff --git a/server/tso/allocator_manager.go b/server/tso/allocator_manager.go index c6a5312e938..87ae16ad1bb 100644 --- a/server/tso/allocator_manager.go +++ b/server/tso/allocator_manager.go @@ -75,19 +75,27 @@ type AllocatorManager struct { // for election use member *member.Member // TSO config - rootPath string - saveInterval time.Duration - maxResetTSGap func() time.Duration + rootPath string + saveInterval time.Duration + updatePhysicalInterval time.Duration + maxResetTSGap func() time.Duration } // NewAllocatorManager creates a new TSO Allocator Manager. -func NewAllocatorManager(m *member.Member, rootPath string, saveInterval time.Duration, maxResetTSGap func() time.Duration) *AllocatorManager { +func NewAllocatorManager( + m *member.Member, + rootPath string, + saveInterval time.Duration, + updatePhysicalInterval time.Duration, + maxResetTSGap func() time.Duration, +) *AllocatorManager { allocatorManager := &AllocatorManager{ - allocatorGroups: make(map[string]*allocatorGroup), - member: m, - rootPath: rootPath, - saveInterval: saveInterval, - maxResetTSGap: maxResetTSGap, + allocatorGroups: make(map[string]*allocatorGroup), + member: m, + rootPath: rootPath, + saveInterval: saveInterval, + updatePhysicalInterval: updatePhysicalInterval, + maxResetTSGap: maxResetTSGap, } return allocatorManager } @@ -165,11 +173,17 @@ func (am *AllocatorManager) getLocalTSOConfigPath() string { func (am *AllocatorManager) SetUpAllocator(parentCtx context.Context, dcLocation string, leadership *election.Leadership) error { am.Lock() defer am.Unlock() + + if am.updatePhysicalInterval != config.DefaultTSOUpdatePhysicalInterval { + log.Warn("tso update physical interval is non-default", + zap.Duration("update-physical-interval", am.updatePhysicalInterval)) + } + var allocator Allocator if dcLocation == config.GlobalDCLocation { - allocator = NewGlobalTSOAllocator(leadership, am.getAllocatorPath(dcLocation), am.saveInterval, am.maxResetTSGap) + allocator = NewGlobalTSOAllocator(leadership, am.getAllocatorPath(dcLocation), am.saveInterval, am.updatePhysicalInterval, am.maxResetTSGap) } else { - allocator = NewLocalTSOAllocator(am.member, leadership, dcLocation, am.saveInterval, am.maxResetTSGap) + allocator = NewLocalTSOAllocator(am.member, leadership, dcLocation, am.saveInterval, am.updatePhysicalInterval, am.maxResetTSGap) } // Update or create a new allocatorGroup am.allocatorGroups[dcLocation] = &allocatorGroup{ @@ -308,7 +322,7 @@ func (am *AllocatorManager) campaignAllocatorLeader(loopCtx context.Context, all // AllocatorDaemon is used to update every allocator's TSO and check whether we have // any new local allocator that needs to be set up. func (am *AllocatorManager) AllocatorDaemon(serverCtx context.Context) { - tsTicker := time.NewTicker(UpdateTimestampStep) + tsTicker := time.NewTicker(am.updatePhysicalInterval) defer tsTicker.Stop() checkerTicker := time.NewTicker(checkAllocatorStep) defer checkerTicker.Stop() diff --git a/server/tso/global_allocator.go b/server/tso/global_allocator.go index d434736849a..2af0d28681b 100644 --- a/server/tso/global_allocator.go +++ b/server/tso/global_allocator.go @@ -49,14 +49,21 @@ type GlobalTSOAllocator struct { } // NewGlobalTSOAllocator creates a new global TSO allocator. -func NewGlobalTSOAllocator(leadership *election.Leadership, rootPath string, saveInterval time.Duration, maxResetTSGap func() time.Duration) Allocator { +func NewGlobalTSOAllocator( + leadership *election.Leadership, + rootPath string, + saveInterval time.Duration, + updatePhysicalInterval time.Duration, + maxResetTSGap func() time.Duration, +) Allocator { return &GlobalTSOAllocator{ leadership: leadership, timestampOracle: ×tampOracle{ - client: leadership.GetClient(), - rootPath: rootPath, - saveInterval: saveInterval, - maxResetTSGap: maxResetTSGap, + client: leadership.GetClient(), + rootPath: rootPath, + saveInterval: saveInterval, + updatePhysicalInterval: updatePhysicalInterval, + maxResetTSGap: maxResetTSGap, }, } } diff --git a/server/tso/local_allocator.go b/server/tso/local_allocator.go index 63e420fa958..cdb930aec95 100644 --- a/server/tso/local_allocator.go +++ b/server/tso/local_allocator.go @@ -47,14 +47,22 @@ type LocalTSOAllocator struct { } // NewLocalTSOAllocator creates a new local TSO allocator. -func NewLocalTSOAllocator(member *member.Member, leadership *election.Leadership, dcLocation string, saveInterval time.Duration, maxResetTSGap func() time.Duration) Allocator { +func NewLocalTSOAllocator( + member *member.Member, + leadership *election.Leadership, + dcLocation string, + saveInterval time.Duration, + updatePhysicalInterval time.Duration, + maxResetTSGap func() time.Duration, +) Allocator { return &LocalTSOAllocator{ leadership: leadership, timestampOracle: ×tampOracle{ - client: leadership.GetClient(), - rootPath: leadership.GetLeaderKey(), - saveInterval: saveInterval, - maxResetTSGap: maxResetTSGap, + client: leadership.GetClient(), + rootPath: leadership.GetLeaderKey(), + saveInterval: saveInterval, + updatePhysicalInterval: updatePhysicalInterval, + maxResetTSGap: maxResetTSGap, }, member: member, rootPath: leadership.GetLeaderKey(), diff --git a/server/tso/tso.go b/server/tso/tso.go index 3efa6166921..e7cb208535a 100644 --- a/server/tso/tso.go +++ b/server/tso/tso.go @@ -33,8 +33,6 @@ import ( const ( timestampKey = "timestamp" - // UpdateTimestampStep is used to update timestamp. - UpdateTimestampStep = 50 * time.Millisecond // updateTimestampGuard is the min timestamp interval. updateTimestampGuard = time.Millisecond // maxLogical is the max upper limit for logical time. @@ -54,8 +52,9 @@ type timestampOracle struct { client *clientv3.Client rootPath string // TODO: remove saveInterval - saveInterval time.Duration - maxResetTSGap func() time.Duration + saveInterval time.Duration + updatePhysicalInterval time.Duration + maxResetTSGap func() time.Duration // tso info stored in the memory tsoMux struct { sync.RWMutex @@ -232,7 +231,7 @@ func (t *timestampOracle) UpdateTimestamp(leadership *election.Leadership) error tsoCounter.WithLabelValues("save").Inc() jetLag := typeutil.SubTimeByWallClock(now, prevPhysical) - if jetLag > 3*UpdateTimestampStep { + if jetLag > 3*t.updatePhysicalInterval { log.Warn("clock offset", zap.Duration("jet-lag", jetLag), zap.Time("prev-physical", prevPhysical), zap.Time("now", now)) tsoCounter.WithLabelValues("slow_save").Inc() } @@ -310,7 +309,7 @@ func (t *timestampOracle) getTS(leadership *election.Leadership, count uint32) ( zap.Reflect("response", resp), zap.Int("retry-count", i), errs.ZapError(errs.ErrLogicOverflow)) tsoCounter.WithLabelValues("logical_overflow").Inc() - time.Sleep(UpdateTimestampStep) + time.Sleep(t.updatePhysicalInterval) continue } // In case lease expired after the first check. From 858ef8c322a947fd53bc885920f6f271f9f05116 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Sat, 10 Oct 2020 15:43:05 +0800 Subject: [PATCH 16/27] api: add status query parameter for the scheduler API (#3055) Signed-off-by: Ryan Leung --- server/api/scheduler.go | 24 ++++++++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/server/api/scheduler.go b/server/api/scheduler.go index 72cee53c043..ce42727c21f 100644 --- a/server/api/scheduler.go +++ b/server/api/scheduler.go @@ -45,7 +45,7 @@ func newSchedulerHandler(svr *server.Server, r *render.Render) *schedulerHandler } // @Tags scheduler -// @Summary List running schedulers. +// @Summary List all schedulers by status. // @Produce json // @Success 200 {array} string // @Failure 500 {string} string "PD server failed to proceed the request." @@ -56,7 +56,27 @@ func (h *schedulerHandler) List(w http.ResponseWriter, r *http.Request) { h.r.JSON(w, http.StatusInternalServerError, err.Error()) return } - h.r.JSON(w, http.StatusOK, schedulers) + + status := r.URL.Query().Get("status") + switch status { + case "paused": + var pausedSchedulers []string + for _, scheduler := range schedulers { + paused, err := h.IsSchedulerPaused(scheduler) + if err != nil { + h.r.JSON(w, http.StatusInternalServerError, err.Error()) + return + } + + if paused { + pausedSchedulers = append(pausedSchedulers, scheduler) + } + } + h.r.JSON(w, http.StatusOK, pausedSchedulers) + return + default: + h.r.JSON(w, http.StatusOK, schedulers) + } } // FIXME: details of input json body params From 4331304c0439be9ab9c8f1b03bd782c6265a8a5e Mon Sep 17 00:00:00 2001 From: Song Gao Date: Sat, 10 Oct 2020 17:59:11 +0800 Subject: [PATCH 17/27] api: Add scatter regions http api (#3051) Signed-off-by: Song Gao --- server/api/region.go | 98 ++++++++++++++++++++++++++++++++------- server/api/region_test.go | 23 +++++++++ server/api/router.go | 1 + server/api/util.go | 18 +++++++ server/handler.go | 5 -- 5 files changed, 122 insertions(+), 23 deletions(-) diff --git a/server/api/region.go b/server/api/region.go index 0cf7adca043..fd4c0f57dcb 100644 --- a/server/api/region.go +++ b/server/api/region.go @@ -15,12 +15,12 @@ package api import ( "container/heap" - "encoding/hex" "fmt" "net/http" "net/url" "sort" "strconv" + "strings" "github.com/gorilla/mux" "github.com/pingcap/kvproto/pkg/metapb" @@ -593,22 +593,6 @@ func (h *regionsHandler) AccelerateRegionsScheduleInRange(w http.ResponseWriter, if err := apiutil.ReadJSONRespondError(h.rd, w, r.Body, &input); err != nil { return } - parseKey := func(name string, input map[string]interface{}) (string, string, error) { - k, ok := input[name] - if !ok { - return "", "", fmt.Errorf("missing %s", name) - } - rawKey, ok := k.(string) - if !ok { - return "", "", fmt.Errorf("bad format %s", name) - } - returned, err := hex.DecodeString(rawKey) - if err != nil { - return "", "", fmt.Errorf("split key %s is not in hex format", name) - } - return string(returned), rawKey, nil - } - startKey, rawStartKey, err := parseKey("start_key", input) if err != nil { h.rd.JSON(w, http.StatusBadRequest, err.Error()) @@ -634,7 +618,7 @@ func (h *regionsHandler) AccelerateRegionsScheduleInRange(w http.ResponseWriter, limit = maxRegionLimit } - regions := rc.ScanRegions([]byte(startKey), []byte(endKey), limit) + regions := rc.ScanRegions(startKey, endKey, limit) if len(regions) > 0 { regionsIDList := make([]uint64, 0, len(regions)) for _, region := range regions { @@ -664,6 +648,84 @@ func (h *regionsHandler) GetTopNRegions(w http.ResponseWriter, r *http.Request, h.rd.JSON(w, http.StatusOK, regionsInfo) } +// @Tags region +// @Summary Scatter regions by given key ranges or regions id distributed by given group with given retry limit +// @Accept json +// @Param body body object true "json params" +// @Produce json +// @Success 200 {string} string "Scatter regions by given key ranges or regions id distributed by given group with given retry limit" +// @Failure 400 {string} string "The input is invalid." +// @Router /regions/scatter [post] +func (h *regionsHandler) ScatterRegions(w http.ResponseWriter, r *http.Request) { + rc := getCluster(r.Context()) + var input map[string]interface{} + if err := apiutil.ReadJSONRespondError(h.rd, w, r.Body, &input); err != nil { + return + } + var regionMap map[uint64]*core.RegionInfo + _, ok1 := input["start_key"].(string) + _, ok2 := input["end_key"].(string) + regionsCount := 0 + if ok1 && ok2 { + startKey, _, err := parseKey("start_key", input) + if err != nil { + h.rd.JSON(w, http.StatusBadRequest, err.Error()) + return + } + + endKey, _, err := parseKey("end_key", input) + if err != nil { + h.rd.JSON(w, http.StatusBadRequest, err.Error()) + return + } + regions := rc.ScanRegions(startKey, endKey, -1) + regionMap = make(map[uint64]*core.RegionInfo, len(regions)) + for _, region := range regions { + regionMap[region.GetID()] = region + } + regionsCount = len(regionMap) + } else { + regionsID := input["regions_id"].([]uint64) + regionMap = make(map[uint64]*core.RegionInfo, len(regionsID)) + for _, id := range regionsID { + regionMap[id] = rc.GetRegion(id) + } + regionsCount = len(regionsID) + } + if regionsCount < 1 { + h.rd.JSON(w, http.StatusBadRequest, "empty regions") + return + } + group, ok := input["group"].(string) + if !ok { + group = "" + } + retryLimit, ok := input["retry_limit"].(int) + if !ok { + retryLimit = 5 + } + failures := make(map[uint64]error, len(regionMap)) + var failureRegionID []string + ops := rc.GetRegionScatter().ScatterRegions(regionMap, failures, group, retryLimit) + for regionID := range failures { + failureRegionID = append(failureRegionID, fmt.Sprintf("%v", regionID)) + } + // If there existed any operator failed to be added into Operator Controller, add its regions into unProcessedRegions + for _, op := range ops { + if ok := rc.GetOperatorController().AddOperator(op); !ok { + failureRegionID = append(failureRegionID, fmt.Sprintf("%v", op.RegionID())) + } + } + s := struct { + ProcessedPercentage int `json:"processed-percentage"` + Error string `json:"error"` + }{ + ProcessedPercentage: 100 - (len(failureRegionID) * 100 / regionsCount), + Error: "unprocessed regions:[" + strings.Join(failureRegionID, ",") + "]", + } + h.rd.JSON(w, http.StatusOK, &s) +} + // RegionHeap implements heap.Interface, used for selecting top n regions. type RegionHeap struct { regions []*core.RegionInfo diff --git a/server/api/region_test.go b/server/api/region_test.go index 1b414c946f1..6f917b1a8fb 100644 --- a/server/api/region_test.go +++ b/server/api/region_test.go @@ -274,6 +274,29 @@ func (s *testRegionSuite) TestAccelerateRegionsScheduleInRange(c *C) { c.Assert(len(idList), Equals, 2) } +func (s *testRegionSuite) TestScatterRegions(c *C) { + r1 := newTestRegionInfo(601, 13, []byte("b1"), []byte("b2")) + r1.GetMeta().Peers = append(r1.GetMeta().Peers, &metapb.Peer{Id: 5, StoreId: 13}, &metapb.Peer{Id: 6, StoreId: 13}) + r2 := newTestRegionInfo(602, 13, []byte("b2"), []byte("b3")) + r2.GetMeta().Peers = append(r2.GetMeta().Peers, &metapb.Peer{Id: 7, StoreId: 13}, &metapb.Peer{Id: 8, StoreId: 13}) + r3 := newTestRegionInfo(603, 13, []byte("b4"), []byte("b4")) + r3.GetMeta().Peers = append(r3.GetMeta().Peers, &metapb.Peer{Id: 9, StoreId: 13}, &metapb.Peer{Id: 10, StoreId: 13}) + mustRegionHeartbeat(c, s.svr, r1) + mustRegionHeartbeat(c, s.svr, r2) + mustRegionHeartbeat(c, s.svr, r3) + mustPutStore(c, s.svr, 13, metapb.StoreState_Up, []*metapb.StoreLabel{}) + mustPutStore(c, s.svr, 14, metapb.StoreState_Up, []*metapb.StoreLabel{}) + mustPutStore(c, s.svr, 15, metapb.StoreState_Up, []*metapb.StoreLabel{}) + body := fmt.Sprintf(`{"start_key":"%s", "end_key": "%s"}`, hex.EncodeToString([]byte("b1")), hex.EncodeToString([]byte("b3"))) + + err := postJSON(testDialClient, fmt.Sprintf("%s/regions/scatter", s.urlPrefix), []byte(body)) + c.Assert(err, IsNil) + op1 := s.svr.GetRaftCluster().GetOperatorController().GetOperator(601) + c.Assert(op1 != nil, Equals, true) + op2 := s.svr.GetRaftCluster().GetOperatorController().GetOperator(602) + c.Assert(op2 != nil, Equals, true) +} + func (s *testRegionSuite) checkTopRegions(c *C, url string, regionIDs []uint64) { regions := &RegionsInfo{} err := readJSON(testDialClient, url, regions) diff --git a/server/api/router.go b/server/api/router.go index b45dcedd6cc..cffecff5599 100644 --- a/server/api/router.go +++ b/server/api/router.go @@ -168,6 +168,7 @@ func createRouter(ctx context.Context, prefix string, svr *server.Server) *mux.R clusterRouter.HandleFunc("/regions/check/hist-keys", regionsHandler.GetKeysHistogram).Methods("GET") clusterRouter.HandleFunc("/regions/sibling/{id}", regionsHandler.GetRegionSiblings).Methods("GET") clusterRouter.HandleFunc("/regions/accelerate-schedule", regionsHandler.AccelerateRegionsScheduleInRange).Methods("POST") + clusterRouter.HandleFunc("/regions/scatter", regionsHandler.ScatterRegions).Methods("POST") apiRouter.Handle("/version", newVersionHandler(rd)).Methods("GET") apiRouter.Handle("/status", newStatusHandler(svr, rd)).Methods("GET") diff --git a/server/api/util.go b/server/api/util.go index f7e26524d43..e5f5b36ef6c 100644 --- a/server/api/util.go +++ b/server/api/util.go @@ -15,7 +15,9 @@ package api import ( "bytes" + "encoding/hex" "encoding/json" + "fmt" "io/ioutil" "net/http" "net/url" @@ -108,3 +110,19 @@ func doDelete(client *http.Client, url string) (*http.Response, error) { res.Body.Close() return res, nil } + +func parseKey(name string, input map[string]interface{}) ([]byte, string, error) { + k, ok := input[name] + if !ok { + return nil, "", fmt.Errorf("missing %s", name) + } + rawKey, ok := k.(string) + if !ok { + return nil, "", fmt.Errorf("bad format %s", name) + } + returned, err := hex.DecodeString(rawKey) + if err != nil { + return nil, "", fmt.Errorf("split key %s is not in hex format", name) + } + return returned, rawKey, nil +} diff --git a/server/handler.go b/server/handler.go index c61115d52f0..a00deee4f38 100644 --- a/server/handler.go +++ b/server/handler.go @@ -805,11 +805,6 @@ func (h *Handler) AddScatterRegionsOperators(regionIDs []uint64, startRawKey, en // check region hot status regionMap := make(map[uint64]*core.RegionInfo, len(regions)) for _, region := range regions { - // If region is Hot, add it into unProcessedRegions - if c.IsRegionHot(region) { - failureRegionID = append(failureRegionID, fmt.Sprintf("%v", region.GetID())) - continue - } regionMap[region.GetID()] = region } failures := make(map[uint64]error, len(regionMap)) From 2d1adee662f3c95e14ae42ac2e955f1a70d1e341 Mon Sep 17 00:00:00 2001 From: ZenoTan Date: Sat, 10 Oct 2020 18:20:22 +0800 Subject: [PATCH 18/27] Bug fix Signed-off-by: ZenoTan --- server/schedule/checker/learner_checker.go | 7 +++---- server/schedule/operator/create_operator.go | 3 +++ 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/server/schedule/checker/learner_checker.go b/server/schedule/checker/learner_checker.go index 2034b41d2a8..d2a5b59cecf 100644 --- a/server/schedule/checker/learner_checker.go +++ b/server/schedule/checker/learner_checker.go @@ -36,15 +36,14 @@ func NewLearnerChecker(cluster opt.Cluster) *LearnerChecker { // Check verifies a region's role, creating an Operator if need. func (l *LearnerChecker) Check(region *core.RegionInfo) *operator.Operator { for _, p := range region.GetLearners() { - if region.GetPendingLearner(p.GetId()) != nil { - continue - } op, err := operator.CreatePromoteLearnerOperator("promote-learner", l.cluster, region, p) if err != nil { log.Debug("fail to create promote learner operator", errs.ZapError(err)) return nil } - return op + if op != nil { + return op + } } return nil } diff --git a/server/schedule/operator/create_operator.go b/server/schedule/operator/create_operator.go index 0b8cc126a5f..7537005b4ae 100644 --- a/server/schedule/operator/create_operator.go +++ b/server/schedule/operator/create_operator.go @@ -34,6 +34,9 @@ func CreateAddPeerOperator(desc string, cluster opt.Cluster, region *core.Region // CreatePromoteLearnerOperator creates an operator that promotes a learner. func CreatePromoteLearnerOperator(desc string, cluster opt.Cluster, region *core.RegionInfo, peer *metapb.Peer) (*Operator, error) { + if region.GetPendingPeer(peer.GetId()) != nil { + return nil, nil + } return NewBuilder(desc, cluster, region). PromoteLearner(peer.GetStoreId()). Build(0) From 6a418cee3dc8358e10872ee19fc5470d522d102f Mon Sep 17 00:00:00 2001 From: yiwu-arbug Date: Sat, 10 Oct 2020 22:35:10 -0700 Subject: [PATCH 19/27] security: Encrypt region boundary keys, Part 2 - server changes (#2976) Signed-off-by: Yi Wu --- conf/config.toml | 47 ++++++++ pkg/encryption/config.go | 154 ++++++++++++++++++++++++++ pkg/encryption/config_test.go | 55 +++++++++ pkg/encryption/crypter.go | 30 ++--- pkg/encryption/crypter_test.go | 6 +- pkg/encryption/kms.go | 19 ++++ pkg/encryption/region_crypter.go | 36 +++--- pkg/encryption/region_crypter_test.go | 26 +++-- pkg/errs/errno.go | 1 + server/config/config.go | 6 +- server/core/region_storage.go | 55 ++++++--- server/core/storage.go | 117 +++++++++++++++---- server/encryptionkm/key_manager.go | 54 +++++++++ server/server.go | 20 +++- 14 files changed, 540 insertions(+), 86 deletions(-) create mode 100644 pkg/encryption/config.go create mode 100644 pkg/encryption/config_test.go create mode 100644 pkg/encryption/kms.go create mode 100644 server/encryptionkm/key_manager.go diff --git a/conf/config.toml b/conf/config.toml index 11c45d2ddea..88d32cb1f80 100644 --- a/conf/config.toml +++ b/conf/config.toml @@ -32,6 +32,53 @@ key-path = "" cert-allowed-cn = ["example.com"] +[security.encryption] +## Encryption method to use for PD data. One of "plaintext", "aes128-ctr", "aes192-ctr" and "aes256-ctr". +## Defaults to "plaintext" if not set. +# data-encryption-method = "plaintext" +## Specifies how often PD rotates data encryption key. Default is 7 days. +# data-key-rotation-period = "168h" + +## Specifies master key if encryption is enabled. There are three types of master key: +## +## * "plaintext": +## +## Plaintext as master key means no master key is given and only applicable when +## encryption is not enabled, i.e. data-encryption-method = "plaintext". This type doesn't +## have sub-config items. Example: +## +## [security.encryption.master-key] +## type = "plaintext" +## +## * "kms": +## +## Use a KMS service to supply master key. Currently only AWS KMS is supported. This type of +## master key is recommended for production use. Example: +## +## [security.encryption.master-key] +## type = "kms" +## ## KMS CMK key id. Must be a valid KMS CMK where the TiKV process has access to. +## ## In production is recommended to grant access of the CMK to TiKV using IAM. +## key-id = "1234abcd-12ab-34cd-56ef-1234567890ab" +## ## AWS region of the KMS CMK. +## region = "us-west-2" +## ## (Optional) AWS KMS service endpoint. Only required when non-default KMS endpoint is +## ## desired. +## endpoint = "https://kms.us-west-2.amazonaws.com" +## +## * "file": +## +## Supply a custom encryption key stored in a file. It is recommended NOT to use in production, +## as it breaks the purpose of encryption at rest, unless the file is stored in tempfs. +## The file must contain a 256-bits (32 bytes, regardless of key length implied by +## data-encryption-method) key encoded as hex string and end with newline ("\n"). Example: +## +## [security.encryption.master-key] +## type = "file" +## path = "/path/to/master/key/file" +# [security.encryption.master-key] +# type = "plaintext" + [log] level = "info" diff --git a/pkg/encryption/config.go b/pkg/encryption/config.go new file mode 100644 index 00000000000..c58a04bce36 --- /dev/null +++ b/pkg/encryption/config.go @@ -0,0 +1,154 @@ +// Copyright 2020 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package encryption + +import ( + "time" + + "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/typeutil" +) + +const ( + methodPlaintext = "plaintext" + methodAes128Ctr = "aes128-ctr" + methodAes192Ctr = "aes192-ctr" + methodAes256Ctr = "aes256-ctr" + + masterKeyTypePlaintext = "plaintext" + masterKeyTypeKMS = "kms" + masterKeyTypeFile = "file" + + defaultDataEncryptionMethod = methodPlaintext + defaultDataKeyRotationPeriod = "168h" // 7 days +) + +// Config define the encryption config structure. +type Config struct { + // Encryption method to use for PD data. + DataEncryptionMethod string `toml:"data-encryption-method" json:"data-encryption-method"` + // Specifies how often PD rotates data encryption key. + DataKeyRotationPeriod typeutil.Duration `toml:"data-key-rotation-period" json:"data-key-rotation-period"` + // Specifies master key if encryption is enabled. + MasterKey MasterKeyConfig `toml:"master-key" json:"master-key"` +} + +// Adjust validates the config and sets default values. +func (c *Config) Adjust() error { + if len(c.DataEncryptionMethod) == 0 { + c.DataEncryptionMethod = defaultDataEncryptionMethod + } else { + if _, err := c.GetMethod(); err != nil { + return err + } + } + if c.DataKeyRotationPeriod.Duration == 0 { + duration, err := time.ParseDuration(defaultDataKeyRotationPeriod) + if err != nil { + return errs.ErrEncryptionInvalidConfig.Wrap(err).GenWithStack( + "fail to parse default value of data-key-rotation-period %s", + defaultDataKeyRotationPeriod) + } + c.DataKeyRotationPeriod.Duration = duration + } else if c.DataKeyRotationPeriod.Duration < 0 { + return errs.ErrEncryptionInvalidConfig.GenWithStack( + "negative data-key-rotation-period %d", + c.DataKeyRotationPeriod.Duration) + } + if len(c.MasterKey.Type) == 0 { + c.MasterKey.Type = masterKeyTypePlaintext + } else { + if _, err := c.GetMasterKey(); err != nil { + return err + } + } + return nil +} + +// GetMethod gets the encryption method. +func (c *Config) GetMethod() (encryptionpb.EncryptionMethod, error) { + switch c.DataEncryptionMethod { + case methodPlaintext: + return encryptionpb.EncryptionMethod_PLAINTEXT, nil + case methodAes128Ctr: + return encryptionpb.EncryptionMethod_AES128_CTR, nil + case methodAes192Ctr: + return encryptionpb.EncryptionMethod_AES192_CTR, nil + case methodAes256Ctr: + return encryptionpb.EncryptionMethod_AES256_CTR, nil + default: + return encryptionpb.EncryptionMethod_UNKNOWN, + errs.ErrEncryptionInvalidMethod.GenWithStack("unknown method") + } +} + +// GetMasterKey gets the master key config. +func (c *Config) GetMasterKey() (*encryptionpb.MasterKey, error) { + switch c.MasterKey.Type { + case masterKeyTypePlaintext: + return &encryptionpb.MasterKey{ + Backend: &encryptionpb.MasterKey_Plaintext{ + Plaintext: &encryptionpb.MasterKeyPlaintext{}, + }, + }, nil + case masterKeyTypeKMS: + return &encryptionpb.MasterKey{ + Backend: &encryptionpb.MasterKey_Kms{ + Kms: &encryptionpb.MasterKeyKms{ + Vendor: kmsVendorAWS, + KeyId: c.MasterKey.KmsKeyID, + Region: c.MasterKey.KmsRegion, + Endpoint: c.MasterKey.KmsEndpoint, + }, + }, + }, nil + case masterKeyTypeFile: + return &encryptionpb.MasterKey{ + Backend: &encryptionpb.MasterKey_File{ + File: &encryptionpb.MasterKeyFile{ + Path: c.MasterKey.FilePath, + }, + }, + }, nil + default: + return nil, errs.ErrEncryptionInvalidConfig.GenWithStack( + "unrecognized encryption master key type: %s", c.MasterKey.Type) + } +} + +// MasterKeyConfig defines master key config structure. +type MasterKeyConfig struct { + // Master key type, one of "plaintext", "kms" or "file". + Type string `toml:"type" json:"type"` + + MasterKeyKMSConfig + MasterKeyFileConfig +} + +// MasterKeyKMSConfig defines a KMS master key config structure. +type MasterKeyKMSConfig struct { + // KMS CMK key id. + KmsKeyID string `toml:"key-id" json:"key-id"` + // KMS region of the CMK. + KmsRegion string `toml:"region" json:"region"` + // Custom endpoint to access KMS. + KmsEndpoint string `toml:"endpoint" json:"endpoint"` +} + +// MasterKeyFileConfig defines a file-based master key config structure. +type MasterKeyFileConfig struct { + // Master key file path. + FilePath string `toml:"path" json:"path"` +} diff --git a/pkg/encryption/config_test.go b/pkg/encryption/config_test.go new file mode 100644 index 00000000000..79253c23144 --- /dev/null +++ b/pkg/encryption/config_test.go @@ -0,0 +1,55 @@ +// Copyright 2020 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package encryption + +import ( + "testing" + "time" + + . "github.com/pingcap/check" + "github.com/tikv/pd/pkg/typeutil" +) + +func TestConfig(t *testing.T) { + TestingT(t) +} + +type testConfigSuite struct{} + +var _ = Suite(&testConfigSuite{}) + +func (s *testConfigSuite) TestAdjustDefaultValue(c *C) { + config := &Config{} + err := config.Adjust() + c.Assert(err, IsNil) + c.Assert(config.DataEncryptionMethod, Equals, methodPlaintext) + defaultRotationPeriod, _ := time.ParseDuration(defaultDataKeyRotationPeriod) + c.Assert(config.DataKeyRotationPeriod.Duration, Equals, defaultRotationPeriod) + c.Assert(config.MasterKey.Type, Equals, masterKeyTypePlaintext) +} + +func (s *testConfigSuite) TestAdjustInvalidDataEncryptionMethod(c *C) { + config := &Config{DataEncryptionMethod: "unknown"} + c.Assert(config.Adjust(), NotNil) +} + +func (s *testConfigSuite) TestAdjustNegativeRotationDuration(c *C) { + config := &Config{DataKeyRotationPeriod: typeutil.NewDuration(time.Duration(int64(-1)))} + c.Assert(config.Adjust(), NotNil) +} + +func (s *testConfigSuite) TestAdjustInvalidMasterKeyType(c *C) { + config := &Config{MasterKey: MasterKeyConfig{Type: "unknown"}} + c.Assert(config.Adjust(), NotNil) +} diff --git a/pkg/encryption/crypter.go b/pkg/encryption/crypter.go index eb8ebe87a48..d35b75dbff9 100644 --- a/pkg/encryption/crypter.go +++ b/pkg/encryption/crypter.go @@ -20,6 +20,7 @@ import ( "encoding/binary" "io" "time" + "unsafe" "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/tikv/pd/pkg/errs" @@ -67,11 +68,11 @@ func KeyLength(method encryptionpb.EncryptionMethod) (int, error) { } } -// IvCtr represent IV bytes for CTR mode. -type IvCtr []byte +// IvCTR represent IV bytes for CTR mode. +type IvCTR []byte -// IvGcm represent IV bytes for GCM mode. -type IvGcm []byte +// IvGCM represent IV bytes for GCM mode. +type IvGCM []byte func newIV(ivLength int) ([]byte, error) { iv := make([]byte, ivLength) @@ -86,13 +87,13 @@ func newIV(ivLength int) ([]byte, error) { return iv, nil } -// NewIvCtr randomly generate an IV for CTR mode. -func NewIvCtr() (IvCtr, error) { +// NewIvCTR randomly generate an IV for CTR mode. +func NewIvCTR() (IvCTR, error) { return newIV(ivLengthCTR) } -// NewIvGcm randomly generate an IV for GCM mode. -func NewIvGcm() (IvGcm, error) { +// NewIvGCM randomly generate an IV for GCM mode. +func NewIvGCM() (IvGCM, error) { return newIV(ivLengthGCM) } @@ -104,14 +105,15 @@ func NewDataKey( if err != nil { return } - keyIDBuf := make([]byte, 8) + keyIDBufSize := unsafe.Sizeof(uint64(0)) + keyIDBuf := make([]byte, keyIDBufSize) n, err := io.ReadFull(rand.Reader, keyIDBuf) if err != nil { err = errs.ErrEncryptionNewDataKey.Wrap(err).GenWithStack( "fail to generate data key id") return } - if n != 8 { + if n != int(keyIDBufSize) { err = errs.ErrEncryptionNewDataKey.GenWithStack( "no enough random bytes to generate data key id, bytes %d", n) return @@ -145,7 +147,7 @@ func NewDataKey( func aesGcmEncryptImpl( key []byte, plaintext []byte, - iv IvGcm, + iv IvGCM, ) (ciphertext []byte, err error) { block, err := aes.NewCipher(key) if err != nil { @@ -166,8 +168,8 @@ func aesGcmEncryptImpl( func AesGcmEncrypt( key []byte, plaintext []byte, -) (ciphertext []byte, iv IvGcm, err error) { - iv, err = NewIvGcm() +) (ciphertext []byte, iv IvGCM, err error) { + iv, err = NewIvGCM() if err != nil { return } @@ -180,7 +182,7 @@ func AesGcmEncrypt( func AesGcmDecrypt( key []byte, ciphertext []byte, - iv IvGcm, + iv IvGCM, ) (plaintext []byte, err error) { if len(iv) != ivLengthGCM { err = errs.ErrEncryptionGCMDecrypt.GenWithStack("unexpected gcm iv length %d", len(iv)) diff --git a/pkg/encryption/crypter_test.go b/pkg/encryption/crypter_test.go index 65b87e10ec8..d140117a435 100644 --- a/pkg/encryption/crypter_test.go +++ b/pkg/encryption/crypter_test.go @@ -55,10 +55,10 @@ func (s *testCrypterSuite) TestKeyLength(c *C) { } func (s *testCrypterSuite) TestNewIv(c *C) { - ivCtr, err := NewIvCtr() + ivCtr, err := NewIvCTR() c.Assert(err, IsNil) c.Assert(len([]byte(ivCtr)), Equals, ivLengthCTR) - ivGcm, err := NewIvGcm() + ivGcm, err := NewIvGCM() c.Assert(err, IsNil) c.Assert(len([]byte(ivGcm)), Equals, ivLengthGCM) } @@ -88,7 +88,7 @@ func (s *testCrypterSuite) TestAesGcmCrypter(c *C) { // encrypt ivBytes, err := hex.DecodeString("ba432b70336c40c39ba14c1b") c.Assert(err, IsNil) - iv := IvGcm(ivBytes) + iv := IvGCM(ivBytes) ciphertext, err := aesGcmEncryptImpl(key, plaintext, iv) c.Assert(err, IsNil) c.Assert(len([]byte(iv)), Equals, ivLengthGCM) diff --git a/pkg/encryption/kms.go b/pkg/encryption/kms.go new file mode 100644 index 00000000000..eb613cceb14 --- /dev/null +++ b/pkg/encryption/kms.go @@ -0,0 +1,19 @@ +// Copyright 2020 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package encryption + +const ( + // We only support AWS KMS right now. + kmsVendorAWS = "AWS" +) diff --git a/pkg/encryption/region_crypter.go b/pkg/encryption/region_crypter.go index 31115f9d657..6762086d525 100644 --- a/pkg/encryption/region_crypter.go +++ b/pkg/encryption/region_crypter.go @@ -17,6 +17,7 @@ import ( "crypto/aes" "crypto/cipher" + "github.com/gogo/protobuf/proto" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/kvproto/pkg/metapb" @@ -36,46 +37,48 @@ func processRegionKeys(region *metapb.Region, key *encryptionpb.DataKey, iv []by return nil } -// EncryptRegion encrypt the region start key and end key in-place, -// using the current key return from the key manager. Encryption meta is updated accordingly. -// Note: Call may need to make deep copy of the object if changing the object is undesired. -func EncryptRegion(region *metapb.Region, keyManager KeyManager) error { +// EncryptRegion encrypt the region start key and end key, using the current key return from the +// key manager. The return is an encypted copy of the region, with Encryption meta updated. +func EncryptRegion(region *metapb.Region, keyManager KeyManager) (*metapb.Region, error) { if region == nil { - return errs.ErrEncryptionEncryptRegion.GenWithStack("trying to encrypt nil region") + return nil, errs.ErrEncryptionEncryptRegion.GenWithStack("trying to encrypt nil region") } if region.EncryptionMeta != nil { - return errs.ErrEncryptionEncryptRegion.GenWithStack( + return nil, errs.ErrEncryptionEncryptRegion.GenWithStack( "region already encrypted, region id = %d", region.Id) } if keyManager == nil { // encryption is not enabled. - return nil + return region, nil } keyID, key, err := keyManager.GetCurrentKey() if err != nil { - return err + return nil, err } if key == nil { // encryption is not enabled. - return nil + return region, nil } err = CheckEncryptionMethodSupported(key.Method) if err != nil { - return err + return nil, err } - iv, err := NewIvCtr() + iv, err := NewIvCTR() if err != nil { - return err + return nil, err } - err = processRegionKeys(region, key, iv) + // Deep copy region before altering it. + outRegion := proto.Clone(region).(*metapb.Region) + // Encrypt and update in-place. + err = processRegionKeys(outRegion, key, iv) if err != nil { - return err + return nil, err } - region.EncryptionMeta = &encryptionpb.EncryptionMeta{ + outRegion.EncryptionMeta = &encryptionpb.EncryptionMeta{ KeyId: keyID, Iv: iv, } - return nil + return outRegion, nil } // DecryptRegion decrypt the region start key and end key, if the region object was encrypted. @@ -100,6 +103,7 @@ func DecryptRegion(region *metapb.Region, keyManager KeyManager) error { if err != nil { return err } + // Decrypt and update in-place. err = processRegionKeys(region, key, region.EncryptionMeta.Iv) if err != nil { return err diff --git a/pkg/encryption/region_crypter_test.go b/pkg/encryption/region_crypter_test.go index 45f94fde228..9778ae02981 100644 --- a/pkg/encryption/region_crypter_test.go +++ b/pkg/encryption/region_crypter_test.go @@ -78,8 +78,11 @@ func (m *testKeyManager) GetKey(keyID uint64) (*encryptionpb.DataKey, error) { func (s *testRegionCrypterSuite) TestNilRegion(c *C) { m := newTestKeyManager() - c.Assert(EncryptRegion(nil, m), Not(IsNil)) - c.Assert(DecryptRegion(nil, m), Not(IsNil)) + region, err := EncryptRegion(nil, m) + c.Assert(err, NotNil) + c.Assert(region, IsNil) + err = DecryptRegion(nil, m) + c.Assert(err, NotNil) } func (s *testRegionCrypterSuite) TestEncryptRegionWithoutKeyManager(c *C) { @@ -89,7 +92,7 @@ func (s *testRegionCrypterSuite) TestEncryptRegionWithoutKeyManager(c *C) { EndKey: []byte("xyz"), EncryptionMeta: nil, } - err := EncryptRegion(region, nil) + region, err := EncryptRegion(region, nil) c.Assert(err, IsNil) // check the region isn't changed c.Assert(string(region.StartKey), Equals, "abc") @@ -106,7 +109,7 @@ func (s *testRegionCrypterSuite) TestEncryptRegionWhileEncryptionDisabled(c *C) } m := newTestKeyManager() m.EncryptionEnabled = false - err := EncryptRegion(region, m) + region, err := EncryptRegion(region, m) c.Assert(err, IsNil) // check the region isn't changed c.Assert(string(region.StartKey), Equals, "abc") @@ -126,24 +129,25 @@ func (s *testRegionCrypterSuite) TestEncryptRegion(c *C) { copy(region.StartKey, startKey) copy(region.EndKey, endKey) m := newTestKeyManager() - err := EncryptRegion(region, m) + outRegion, err := EncryptRegion(region, m) c.Assert(err, IsNil) + c.Assert(outRegion, Not(Equals), region) // check region is encrypted - c.Assert(region.EncryptionMeta, Not(IsNil)) - c.Assert(region.EncryptionMeta.KeyId, Equals, uint64(2)) - c.Assert(len(region.EncryptionMeta.Iv), Equals, ivLengthCTR) + c.Assert(outRegion.EncryptionMeta, Not(IsNil)) + c.Assert(outRegion.EncryptionMeta.KeyId, Equals, uint64(2)) + c.Assert(outRegion.EncryptionMeta.Iv, HasLen, ivLengthCTR) // Check encrypted content _, currentKey, err := m.GetCurrentKey() c.Assert(err, IsNil) block, err := aes.NewCipher(currentKey.Key) c.Assert(err, IsNil) - stream := cipher.NewCTR(block, region.EncryptionMeta.Iv) + stream := cipher.NewCTR(block, outRegion.EncryptionMeta.Iv) ciphertextStartKey := make([]byte, len(startKey)) stream.XORKeyStream(ciphertextStartKey, startKey) - c.Assert(string(region.StartKey), Equals, string(ciphertextStartKey)) + c.Assert(string(outRegion.StartKey), Equals, string(ciphertextStartKey)) ciphertextEndKey := make([]byte, len(endKey)) stream.XORKeyStream(ciphertextEndKey, endKey) - c.Assert(string(region.EndKey), Equals, string(ciphertextEndKey)) + c.Assert(string(outRegion.EndKey), Equals, string(ciphertextEndKey)) } func (s *testRegionCrypterSuite) TestDecryptRegionNotEncrypted(c *C) { diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index cc4443e20c3..84e82a0e53d 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -268,6 +268,7 @@ var ( // encryption var ( ErrEncryptionInvalidMethod = errors.Normalize("invalid encryption method", errors.RFCCodeText("PD:encryption:ErrEncryptionInvalidMethod")) + ErrEncryptionInvalidConfig = errors.Normalize("invalid config", errors.RFCCodeText("PD:encryption:ErrEncryptionInvalidConfig")) ErrEncryptionGenerateIV = errors.Normalize("fail to generate iv", errors.RFCCodeText("PD:encryption:ErrEncryptionGenerateIV")) ErrEncryptionNewDataKey = errors.Normalize("fail to generate data key", errors.RFCCodeText("PD:encryption:ErrEncryptionNewDataKey")) ErrEncryptionGCMEncrypt = errors.Normalize("GCM encryption fail", errors.RFCCodeText("PD:encryption:ErrEncryptionGCMEncrypt")) diff --git a/server/config/config.go b/server/config/config.go index 1962592b056..fab3b2df165 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -26,6 +26,7 @@ import ( "sync" "time" + "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/grpcutil" "github.com/tikv/pd/pkg/logutil" @@ -563,6 +564,8 @@ func (c *Config) Adjust(meta *toml.MetaData) error { c.ReplicationMode.adjust(configMetaData.Child("replication-mode")) + c.Security.Encryption.Adjust() + return nil } @@ -1390,5 +1393,6 @@ func (c *LocalTSOConfig) Validate() error { type SecurityConfig struct { grpcutil.TLSConfig // RedactInfoLog indicates that whether enabling redact log - RedactInfoLog bool `toml:"redact-info-log" json:"redact-info-log"` + RedactInfoLog bool `toml:"redact-info-log" json:"redact-info-log"` + Encryption encryption.Config `toml:"encryption" json:"encryption"` } diff --git a/server/core/region_storage.go b/server/core/region_storage.go index be6de0d037f..d5ee5d546bb 100644 --- a/server/core/region_storage.go +++ b/server/core/region_storage.go @@ -21,7 +21,9 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/server/encryptionkm" "github.com/tikv/pd/server/kv" ) @@ -30,14 +32,15 @@ var dirtyFlushTick = time.Second // RegionStorage is used to save regions. type RegionStorage struct { *kv.LeveldbKV - mu sync.RWMutex - batchRegions map[string]*metapb.Region - batchSize int - cacheSize int - flushRate time.Duration - flushTime time.Time - regionStorageCtx context.Context - regionStorageCancel context.CancelFunc + encryptionKeyManager *encryptionkm.KeyManager + mu sync.RWMutex + batchRegions map[string]*metapb.Region + batchSize int + cacheSize int + flushRate time.Duration + flushTime time.Time + regionStorageCtx context.Context + regionStorageCancel context.CancelFunc } const ( @@ -48,20 +51,25 @@ const ( ) // NewRegionStorage returns a region storage that is used to save regions. -func NewRegionStorage(ctx context.Context, path string) (*RegionStorage, error) { +func NewRegionStorage( + ctx context.Context, + path string, + encryptionKeyManager *encryptionkm.KeyManager, +) (*RegionStorage, error) { levelDB, err := kv.NewLeveldbKV(path) if err != nil { return nil, err } regionStorageCtx, regionStorageCancel := context.WithCancel(ctx) s := &RegionStorage{ - LeveldbKV: levelDB, - batchSize: defaultBatchSize, - flushRate: defaultFlushRegionRate, - batchRegions: make(map[string]*metapb.Region, defaultBatchSize), - flushTime: time.Now().Add(defaultFlushRegionRate), - regionStorageCtx: regionStorageCtx, - regionStorageCancel: regionStorageCancel, + LeveldbKV: levelDB, + encryptionKeyManager: encryptionKeyManager, + batchSize: defaultBatchSize, + flushRate: defaultFlushRegionRate, + batchRegions: make(map[string]*metapb.Region, defaultBatchSize), + flushTime: time.Now().Add(defaultFlushRegionRate), + regionStorageCtx: regionStorageCtx, + regionStorageCancel: regionStorageCancel, } s.backgroundFlush() return s, nil @@ -96,6 +104,10 @@ func (s *RegionStorage) backgroundFlush() { // SaveRegion saves one region to storage. func (s *RegionStorage) SaveRegion(region *metapb.Region) error { + region, err := encryption.EncryptRegion(region, s.encryptionKeyManager) + if err != nil { + return err + } s.mu.Lock() defer s.mu.Unlock() if s.cacheSize < s.batchSize-1 { @@ -106,7 +118,7 @@ func (s *RegionStorage) SaveRegion(region *metapb.Region) error { return nil } s.batchRegions[regionPath(region.GetId())] = region - err := s.flush() + err = s.flush() if err != nil { return err @@ -118,7 +130,11 @@ func deleteRegion(kv kv.Base, region *metapb.Region) error { return kv.Remove(regionPath(region.GetId())) } -func loadRegions(kv kv.Base, f func(region *RegionInfo) []*RegionInfo) error { +func loadRegions( + kv kv.Base, + encryptionKeyManager *encryptionkm.KeyManager, + f func(region *RegionInfo) []*RegionInfo, +) error { nextID := uint64(0) endKey := regionPath(math.MaxUint64) @@ -141,6 +157,9 @@ func loadRegions(kv kv.Base, f func(region *RegionInfo) []*RegionInfo) error { if err := region.Unmarshal([]byte(s)); err != nil { return errs.ErrProtoUnmarshal.Wrap(err).GenWithStackByArgs() } + if err = encryption.DecryptRegion(region, encryptionKeyManager); err != nil { + return err + } nextID = region.GetId() + 1 overlaps := f(NewRegionInfo(region, nil)) diff --git a/server/core/storage.go b/server/core/storage.go index 70702297de4..00efe2fa404 100644 --- a/server/core/storage.go +++ b/server/core/storage.go @@ -27,7 +27,9 @@ import ( "github.com/gogo/protobuf/proto" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/server/encryptionkm" "github.com/tikv/pd/server/kv" "go.etcd.io/etcd/clientv3" ) @@ -42,6 +44,7 @@ const ( replicationPath = "replication_mode" componentPath = "component" customScheduleConfigPath = "scheduler_config" + encryptionKeysPath = "encryption_keys" ) const ( @@ -52,23 +55,47 @@ const ( // Storage wraps all kv operations, keep it stateless. type Storage struct { kv.Base - regionStorage *RegionStorage - useRegionStorage int32 - regionLoaded int32 - mu sync.Mutex + regionStorage *RegionStorage + encryptionKeyManager *encryptionkm.KeyManager + useRegionStorage int32 + regionLoaded int32 + mu sync.Mutex } -// NewStorage creates Storage instance with Base. -func NewStorage(base kv.Base) *Storage { - return &Storage{ - Base: base, +// StorageOpt represents available options to create Storage. +type StorageOpt struct { + regionStorage *RegionStorage + encryptionKeyManager *encryptionkm.KeyManager +} + +// StorageOption configures StorageOpt +type StorageOption func(*StorageOpt) + +// WithRegionStorage sets RegionStorage to the Storage +func WithRegionStorage(regionStorage *RegionStorage) StorageOption { + return func(opt *StorageOpt) { + opt.regionStorage = regionStorage } } -// SetRegionStorage sets the region storage. -func (s *Storage) SetRegionStorage(regionStorage *RegionStorage) *Storage { - s.regionStorage = regionStorage - return s +// WithEncryptionKeyManager sets EncryptionManager to the Storage +func WithEncryptionKeyManager(encryptionKeyManager *encryptionkm.KeyManager) StorageOption { + return func(opt *StorageOpt) { + opt.encryptionKeyManager = encryptionKeyManager + } +} + +// NewStorage creates Storage instance with Base. +func NewStorage(base kv.Base, opts ...StorageOption) *Storage { + options := &StorageOpt{} + for _, opt := range opts { + opt(options) + } + return &Storage{ + Base: base, + regionStorage: options.regionStorage, + encryptionKeyManager: options.encryptionKeyManager, + } } // GetRegionStorage gets the region storage. @@ -107,6 +134,11 @@ func (s *Storage) storeRegionWeightPath(storeID uint64) string { return path.Join(schedulePath, "store_weight", fmt.Sprintf("%020d", storeID), "region") } +// EncryptionKeysPath returns the path to save encryption keys. +func (s *Storage) EncryptionKeysPath() string { + return path.Join(encryptionKeysPath, "keys") +} + // SaveScheduleConfig saves the config of scheduler. func (s *Storage) SaveScheduleConfig(scheduleName string, data []byte) error { configPath := path.Join(customScheduleConfigPath, scheduleName) @@ -151,30 +183,30 @@ func (s *Storage) DeleteStore(store *metapb.Store) error { } // LoadRegion loads one region from storage. -func (s *Storage) LoadRegion(regionID uint64, region *metapb.Region) (bool, error) { +func (s *Storage) LoadRegion(regionID uint64, region *metapb.Region) (ok bool, err error) { if atomic.LoadInt32(&s.useRegionStorage) > 0 { - return loadProto(s.regionStorage, regionPath(regionID), region) + return loadRegion(s.regionStorage, s.encryptionKeyManager, regionID, region) } - return loadProto(s.Base, regionPath(regionID), region) + return loadRegion(s.Base, s.encryptionKeyManager, regionID, region) } // LoadRegions loads all regions from storage to RegionsInfo. func (s *Storage) LoadRegions(f func(region *RegionInfo) []*RegionInfo) error { if atomic.LoadInt32(&s.useRegionStorage) > 0 { - return loadRegions(s.regionStorage, f) + return loadRegions(s.regionStorage, s.encryptionKeyManager, f) } - return loadRegions(s.Base, f) + return loadRegions(s.Base, s.encryptionKeyManager, f) } // LoadRegionsOnce loads all regions from storage to RegionsInfo.Only load one time from regionStorage. func (s *Storage) LoadRegionsOnce(f func(region *RegionInfo) []*RegionInfo) error { if atomic.LoadInt32(&s.useRegionStorage) == 0 { - return loadRegions(s.Base, f) + return loadRegions(s.Base, s.encryptionKeyManager, f) } s.mu.Lock() defer s.mu.Unlock() if s.regionLoaded == 0 { - if err := loadRegions(s.regionStorage, f); err != nil { + if err := loadRegions(s.regionStorage, s.encryptionKeyManager, f); err != nil { return err } s.regionLoaded = 1 @@ -187,7 +219,7 @@ func (s *Storage) SaveRegion(region *metapb.Region) error { if atomic.LoadInt32(&s.useRegionStorage) > 0 { return s.regionStorage.SaveRegion(region) } - return saveProto(s.Base, regionPath(region.GetId()), region) + return saveRegion(s.Base, s.encryptionKeyManager, region) } // DeleteRegion deletes one region from storage. @@ -401,7 +433,13 @@ func (s *Storage) Flush() error { // Close closes the s. func (s *Storage) Close() error { if s.regionStorage != nil { - return s.regionStorage.Close() + err := s.regionStorage.Close() + if err != nil { + return err + } + } + if s.encryptionKeyManager != nil { + s.encryptionKeyManager.Close() } return nil } @@ -540,3 +578,40 @@ func saveProto(s kv.Base, key string, msg proto.Message) error { } return s.Save(key, string(value)) } + +func loadRegion( + kv kv.Base, + encryptionKeyManager *encryptionkm.KeyManager, + regionID uint64, + region *metapb.Region, +) (ok bool, err error) { + value, err := kv.Load(regionPath(regionID)) + if err != nil { + return false, err + } + if value == "" { + return false, nil + } + err = proto.Unmarshal([]byte(value), region) + if err != nil { + return true, errs.ErrProtoUnmarshal.Wrap(err).GenWithStackByArgs() + } + err = encryption.DecryptRegion(region, encryptionKeyManager) + return true, err +} + +func saveRegion( + kv kv.Base, + encryptionKeyManager *encryptionkm.KeyManager, + region *metapb.Region, +) error { + region, err := encryption.EncryptRegion(region, encryptionKeyManager) + if err != nil { + return err + } + value, err := proto.Marshal(region) + if err != nil { + return errs.ErrProtoMarshal.Wrap(err).GenWithStackByArgs() + } + return kv.Save(regionPath(region.GetId()), string(value)) +} diff --git a/server/encryptionkm/key_manager.go b/server/encryptionkm/key_manager.go new file mode 100644 index 00000000000..945f5348118 --- /dev/null +++ b/server/encryptionkm/key_manager.go @@ -0,0 +1,54 @@ +// Copyright 2020 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package encryptionkm + +import ( + "github.com/pingcap/kvproto/pkg/encryptionpb" + lib "github.com/tikv/pd/pkg/encryption" + "github.com/tikv/pd/server/election" + "github.com/tikv/pd/server/kv" +) + +// KeyManager maintains the list to encryption keys. It handles encryption key generation and +// rotation, persisting and loading encryption keys. +type KeyManager struct{} + +// NewKeyManager creates a new key manager. +func NewKeyManager(kv kv.Base, config *lib.Config) (*KeyManager, error) { + // TODO: Implement + return &KeyManager{}, nil +} + +// GetCurrentKey get the current encryption key. The key is nil if encryption is not enabled. +func (m *KeyManager) GetCurrentKey() (keyID uint64, key *encryptionpb.DataKey, err error) { + // TODO: Implement + return 0, nil, nil +} + +// GetKey get the encryption key with the specific key id. +func (m *KeyManager) GetKey(keyID uint64) (key *encryptionpb.DataKey, err error) { + // TODO: Implement + return nil, nil +} + +// SetLeadership sets the PD leadership of the current node. PD leader is responsible to update +// encryption keys, e.g. key rotation. +func (m *KeyManager) SetLeadership(leadership *election.Leadership) { + // TODO: Implement +} + +// Close close the key manager on PD server shutdown +func (m *KeyManager) Close() { + // TODO: Implement +} diff --git a/server/server.go b/server/server.go index 73b660c5ca8..f26acddabdb 100644 --- a/server/server.go +++ b/server/server.go @@ -48,6 +48,7 @@ import ( "github.com/tikv/pd/server/cluster" "github.com/tikv/pd/server/config" "github.com/tikv/pd/server/core" + "github.com/tikv/pd/server/encryptionkm" "github.com/tikv/pd/server/id" "github.com/tikv/pd/server/kv" "github.com/tikv/pd/server/member" @@ -116,6 +117,8 @@ type Server struct { // store, region and peer, because we just need // a unique ID. idAllocator *id.AllocatorImpl + // for encryption + encryptionKeyManager *encryptionkm.KeyManager // for storage operation. storage *core.Storage // for basicCluster operation. @@ -358,12 +361,22 @@ func (s *Server) startServer(ctx context.Context) error { return err } kvBase := kv.NewEtcdKVBase(s.client, s.rootPath) + encryptionKeyManager, err := encryptionkm.NewKeyManager(kvBase, &s.cfg.Security.Encryption) + if err != nil { + return err + } + s.encryptionKeyManager = encryptionKeyManager path := filepath.Join(s.cfg.DataDir, "region-meta") - regionStorage, err := core.NewRegionStorage(ctx, path) + regionStorage, err := core.NewRegionStorage(ctx, path, encryptionKeyManager) if err != nil { return err } - s.storage = core.NewStorage(kvBase).SetRegionStorage(regionStorage) + + s.storage = core.NewStorage( + kvBase, + core.WithRegionStorage(regionStorage), + core.WithEncryptionKeyManager(encryptionKeyManager), + ) s.basicCluster = core.NewBasicCluster() s.cluster = cluster.NewRaftCluster(ctx, s.GetClusterRootPath(), s.clusterID, syncer.NewRegionSyncer(s), s.client, s.httpClient) s.hbStreams = hbstream.NewHeartbeatStreams(ctx, s.clusterID, s.cluster) @@ -1156,6 +1169,9 @@ func (s *Server) campaignLeader() { log.Error("failed to reload configuration", errs.ZapError(err)) return } + + s.encryptionKeyManager.SetLeadership(s.member.GetLeadership()) + // Try to create raft cluster. if err := s.createRaftCluster(); err != nil { log.Error("failed to create raft cluster", errs.ZapError(err)) From 0c935f960d7dd042d03ab72f3869cc1c056ef967 Mon Sep 17 00:00:00 2001 From: ZenoTan Date: Mon, 12 Oct 2020 16:45:31 +0800 Subject: [PATCH 20/27] Address comment Signed-off-by: ZenoTan --- server/schedule/checker/learner_checker.go | 7 ++++--- server/schedule/operator/builder.go | 9 +++++++++ server/schedule/operator/builder_test.go | 10 ++++++++++ server/schedule/operator/create_operator.go | 3 --- 4 files changed, 23 insertions(+), 6 deletions(-) diff --git a/server/schedule/checker/learner_checker.go b/server/schedule/checker/learner_checker.go index d2a5b59cecf..0795d3eba53 100644 --- a/server/schedule/checker/learner_checker.go +++ b/server/schedule/checker/learner_checker.go @@ -36,14 +36,15 @@ func NewLearnerChecker(cluster opt.Cluster) *LearnerChecker { // Check verifies a region's role, creating an Operator if need. func (l *LearnerChecker) Check(region *core.RegionInfo) *operator.Operator { for _, p := range region.GetLearners() { + //if region.GetPendingLearner(p.GetId()) != nil { + // continue + //} op, err := operator.CreatePromoteLearnerOperator("promote-learner", l.cluster, region, p) if err != nil { log.Debug("fail to create promote learner operator", errs.ZapError(err)) return nil } - if op != nil { - return op - } + return op } return nil } diff --git a/server/schedule/operator/builder.go b/server/schedule/operator/builder.go index 58779402ee5..e1949e5d09c 100644 --- a/server/schedule/operator/builder.go +++ b/server/schedule/operator/builder.go @@ -45,6 +45,7 @@ type Builder struct { // operation record originPeers peersMap + pendingPeers peersMap originLeaderStoreID uint64 targetPeers peersMap targetLeaderStoreID uint64 @@ -95,6 +96,7 @@ func NewBuilder(desc string, cluster opt.Cluster, region *core.RegionInfo, opts // origin peers err := b.err originPeers := newPeersMap() + pendingPeers := newPeersMap() for _, p := range region.GetPeers() { if p == nil || p.GetStoreId() == 0 { @@ -104,6 +106,10 @@ func NewBuilder(desc string, cluster opt.Cluster, region *core.RegionInfo, opts originPeers.Set(p) } + for _, p := range region.GetPendingPeers() { + pendingPeers.Set(p) + } + // origin leader originLeaderStoreID := region.GetLeader().GetStoreId() if _, ok := originPeers[originLeaderStoreID]; err == nil && !ok { @@ -132,6 +138,7 @@ func NewBuilder(desc string, cluster opt.Cluster, region *core.RegionInfo, opts b.rules = rules b.originPeers = originPeers + b.pendingPeers = pendingPeers b.originLeaderStoreID = originLeaderStoreID b.targetPeers = originPeers.Copy() b.allowDemote = supportJointConsensus @@ -182,6 +189,8 @@ func (b *Builder) PromoteLearner(storeID uint64) *Builder { b.err = errors.Errorf("cannot promote peer %d: not found", storeID) } else if !core.IsLearner(peer) { b.err = errors.Errorf("cannot promote peer %d: is not learner", storeID) + } else if _, ok := b.pendingPeers[storeID]; ok { + b.err = errors.Errorf("cannot promote peer %d: pending", storeID) } else { b.targetPeers.Set(&metapb.Peer{ Id: peer.GetId(), diff --git a/server/schedule/operator/builder_test.go b/server/schedule/operator/builder_test.go index 12caf607f04..c94e162c094 100644 --- a/server/schedule/operator/builder_test.go +++ b/server/schedule/operator/builder_test.go @@ -442,3 +442,13 @@ func (s *testBuilderSuite) TestBuild(c *C) { } } } + +// Test for issue 3039 +func (s *testBuilderSuite) TestPromotePending(c *C) { + p := &metapb.Peer{Id: 2, StoreId: 2, Role: metapb.PeerRole_Learner} + region := core.NewRegionInfo(&metapb.Region{Id: 1, Peers: []*metapb.Peer{{Id: 1, StoreId: 1}, + p}}, &metapb.Peer{Id: 1, StoreId: 1}, core.WithPendingPeers([]*metapb.Peer{p})) + builder := NewBuilder("test", s.cluster, region) + builder.PromoteLearner(2) + c.Assert(builder.err, NotNil) +} diff --git a/server/schedule/operator/create_operator.go b/server/schedule/operator/create_operator.go index 7537005b4ae..0b8cc126a5f 100644 --- a/server/schedule/operator/create_operator.go +++ b/server/schedule/operator/create_operator.go @@ -34,9 +34,6 @@ func CreateAddPeerOperator(desc string, cluster opt.Cluster, region *core.Region // CreatePromoteLearnerOperator creates an operator that promotes a learner. func CreatePromoteLearnerOperator(desc string, cluster opt.Cluster, region *core.RegionInfo, peer *metapb.Peer) (*Operator, error) { - if region.GetPendingPeer(peer.GetId()) != nil { - return nil, nil - } return NewBuilder(desc, cluster, region). PromoteLearner(peer.GetStoreId()). Build(0) From 37dbd4c1f9fb71b60979915f30ba681eb099e0ca Mon Sep 17 00:00:00 2001 From: ZenoTan Date: Mon, 12 Oct 2020 16:49:41 +0800 Subject: [PATCH 21/27] Fix Signed-off-by: ZenoTan --- server/schedule/checker/learner_checker.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/server/schedule/checker/learner_checker.go b/server/schedule/checker/learner_checker.go index 0795d3eba53..64f44d0ddb2 100644 --- a/server/schedule/checker/learner_checker.go +++ b/server/schedule/checker/learner_checker.go @@ -36,9 +36,6 @@ func NewLearnerChecker(cluster opt.Cluster) *LearnerChecker { // Check verifies a region's role, creating an Operator if need. func (l *LearnerChecker) Check(region *core.RegionInfo) *operator.Operator { for _, p := range region.GetLearners() { - //if region.GetPendingLearner(p.GetId()) != nil { - // continue - //} op, err := operator.CreatePromoteLearnerOperator("promote-learner", l.cluster, region, p) if err != nil { log.Debug("fail to create promote learner operator", errs.ZapError(err)) From 0adcfe0292173897469ee84fa3bdab8bfbba7388 Mon Sep 17 00:00:00 2001 From: ZenoTan Date: Mon, 12 Oct 2020 17:12:53 +0800 Subject: [PATCH 22/27] Fix ci Signed-off-by: ZenoTan --- server/schedule/checker/learner_checker.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/schedule/checker/learner_checker.go b/server/schedule/checker/learner_checker.go index 64f44d0ddb2..8a9efe3e044 100644 --- a/server/schedule/checker/learner_checker.go +++ b/server/schedule/checker/learner_checker.go @@ -39,7 +39,7 @@ func (l *LearnerChecker) Check(region *core.RegionInfo) *operator.Operator { op, err := operator.CreatePromoteLearnerOperator("promote-learner", l.cluster, region, p) if err != nil { log.Debug("fail to create promote learner operator", errs.ZapError(err)) - return nil + continue } return op } From 8558f34613cbfc2ebac07105e1d479ce16d91d72 Mon Sep 17 00:00:00 2001 From: Zheming Li Date: Tue, 13 Oct 2020 17:10:24 +0800 Subject: [PATCH 23/27] api: auto fill group id in rules when unset (#3061) Signed-off-by: Zheming Li --- server/api/rule.go | 9 +++++++ server/api/rule_test.go | 52 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 61 insertions(+) diff --git a/server/api/rule.go b/server/api/rule.go index 01e81c0d41d..7a211f89995 100644 --- a/server/api/rule.go +++ b/server/api/rule.go @@ -454,6 +454,9 @@ func (h *ruleHandler) SetAllGroupBundles(w http.ResponseWriter, r *http.Request) h.rd.JSON(w, http.StatusBadRequest, err.Error()) return } + if len(rule.GroupID) == 0 { + rule.GroupID = g.ID + } if rule.GroupID != g.ID { h.rd.JSON(w, http.StatusBadRequest, fmt.Sprintf("rule group %s does not match group ID %s", rule.GroupID, g.ID)) return @@ -533,6 +536,9 @@ func (h *ruleHandler) SetGroupBundle(w http.ResponseWriter, r *http.Request) { if err := apiutil.ReadJSONRespondError(h.rd, w, r.Body, &group); err != nil { return } + if len(group.ID) == 0 { + group.ID = groupID + } if group.ID != groupID { h.rd.JSON(w, http.StatusBadRequest, fmt.Sprintf("group id %s does not match request URI %s", group.ID, groupID)) return @@ -542,6 +548,9 @@ func (h *ruleHandler) SetGroupBundle(w http.ResponseWriter, r *http.Request) { h.rd.JSON(w, http.StatusBadRequest, err.Error()) return } + if len(rule.GroupID) == 0 { + rule.GroupID = groupID + } if rule.GroupID != groupID { h.rd.JSON(w, http.StatusBadRequest, fmt.Sprintf("rule group %s does not match group ID %s", rule.GroupID, groupID)) return diff --git a/server/api/rule_test.go b/server/api/rule_test.go index f091704483a..22c13666eeb 100644 --- a/server/api/rule_test.go +++ b/server/api/rule_test.go @@ -707,6 +707,58 @@ func (s *testRuleSuite) TestBundle(c *C) { c.Assert(err, IsNil) c.Assert(bundles, HasLen, 1) compareBundle(c, bundles[0], b1) + + // Set + id := "rule-without-group-id" + b4 := placement.GroupBundle{ + Index: 4, + Rules: []*placement.Rule{ + {ID: "bar", Index: 1, Override: true, Role: "voter", Count: 1}, + }, + } + data, err = json.Marshal(b4) + c.Assert(err, IsNil) + err = postJSON(testDialClient, s.urlPrefix+"/placement-rule/"+id, data) + c.Assert(err, IsNil) + + b4.ID = id + b4.Rules[0].GroupID = b4.ID + + // Get + err = readJSON(testDialClient, s.urlPrefix+"/placement-rule/"+id, &bundle) + c.Assert(err, IsNil) + compareBundle(c, bundle, b4) + + // GetAll again + err = readJSON(testDialClient, s.urlPrefix+"/placement-rule", &bundles) + c.Assert(err, IsNil) + c.Assert(bundles, HasLen, 2) + compareBundle(c, bundles[0], b1) + compareBundle(c, bundles[1], b4) + + // SetAll + b5 := placement.GroupBundle{ + ID: "rule-without-group-id-2", + Index: 5, + Rules: []*placement.Rule{ + {ID: "bar", Index: 1, Override: true, Role: "voter", Count: 1}, + }, + } + data, err = json.Marshal([]placement.GroupBundle{b1, b4, b5}) + c.Assert(err, IsNil) + err = postJSON(testDialClient, s.urlPrefix+"/placement-rule", data) + c.Assert(err, IsNil) + + b5.Rules[0].GroupID = b5.ID + + // GetAll again + err = readJSON(testDialClient, s.urlPrefix+"/placement-rule", &bundles) + c.Assert(err, IsNil) + c.Assert(bundles, HasLen, 3) + compareBundle(c, bundles[0], b1) + compareBundle(c, bundles[1], b4) + compareBundle(c, bundles[2], b5) + } func (s *testRuleSuite) TestBundleBadRequest(c *C) { From e0747f9a50d52992bf7b9e5a7b1bfbb7f8d90b0d Mon Sep 17 00:00:00 2001 From: JmPotato Date: Wed, 14 Oct 2020 11:47:55 +0800 Subject: [PATCH 24/27] tso: implement the global TSO generation algorithm (#3033) Signed-off-by: JmPotato --- pkg/errs/errno.go | 1 + pkg/tsoutil/tso.go | 14 ++ server/grpc_service.go | 6 +- server/server.go | 2 +- server/tso/allocator_manager.go | 13 +- server/tso/global_allocator.go | 201 +++++++++++++++++- server/tso/local_allocator.go | 5 +- server/tso/tso.go | 54 +++-- .../tso/{tso_test.go => global_tso_test.go} | 199 +++++++++++++---- tests/server/tso/local_tso_test.go | 121 +++++++++++ 10 files changed, 542 insertions(+), 74 deletions(-) rename tests/server/tso/{tso_test.go => global_tso_test.go} (63%) create mode 100644 tests/server/tso/local_tso_test.go diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index 84e82a0e53d..7ddafd92705 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -31,6 +31,7 @@ var ( var ( ErrGetAllocator = errors.Normalize("get allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetAllocator")) ErrGetLocalAllocator = errors.Normalize("get local allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetLocalAllocator")) + ErrSyncMaxTS = errors.Normalize("sync max ts failed, %s", errors.RFCCodeText("PD:tso:ErrSyncMaxTS")) ErrResetUserTimestamp = errors.Normalize("reset user timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrResetUserTimestamp")) ErrGenerateTimestamp = errors.Normalize("generate timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrGenerateTimestamp")) ErrInvalidTimestamp = errors.Normalize("invalid timestamp", errors.RFCCodeText("PD:tso:ErrInvalidTimestamp")) diff --git a/pkg/tsoutil/tso.go b/pkg/tsoutil/tso.go index e0f09707860..246fcfd6b1e 100644 --- a/pkg/tsoutil/tso.go +++ b/pkg/tsoutil/tso.go @@ -51,3 +51,17 @@ func GenerateTimestamp(physical time.Time, logical uint64) *pdpb.Timestamp { Logical: int64(logical), } } + +// CompareTimestamp is used to compare two timestamps. +// If tsoOne > tsoTwo, returns 1. +// If tsoOne = tsoTwo, returns 0. +// If tsoOne < tsoTwo, returns -1. +func CompareTimestamp(tsoOne, tsoTwo *pdpb.Timestamp) int { + if tsoOne.GetPhysical() > tsoTwo.GetPhysical() || (tsoOne.GetPhysical() == tsoTwo.GetPhysical() && tsoOne.GetLogical() > tsoTwo.GetLogical()) { + return 1 + } + if tsoOne.GetPhysical() == tsoTwo.GetPhysical() && tsoOne.GetLogical() == tsoTwo.GetLogical() { + return 0 + } + return -1 +} diff --git a/server/grpc_service.go b/server/grpc_service.go index 21d7a330d24..7eb2ac912a2 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -939,7 +939,7 @@ func (s *Server) SyncMaxTS(ctx context.Context, request *pdpb.SyncMaxTSRequest) return nil, err } var processedDCs []string - if request.GetMaxTs() == nil || request.GetMaxTs().Physical == 0 { + if request.GetMaxTs() == nil || request.GetMaxTs().GetPhysical() == 0 { // The first phase of synchronization: collect the max local ts var maxLocalTS pdpb.Timestamp for _, allocator := range allocatorLeaders { @@ -952,8 +952,8 @@ func (s *Server) SyncMaxTS(ctx context.Context, request *pdpb.SyncMaxTSRequest) if err != nil { return nil, err } - if currentLocalTSO.Physical > maxLocalTS.Physical { - maxLocalTS.Physical = currentLocalTSO.Physical + if tsoutil.CompareTimestamp(¤tLocalTSO, &maxLocalTS) > 0 { + maxLocalTS = currentLocalTSO } processedDCs = append(processedDCs, allocator.GetDCLocation()) } diff --git a/server/server.go b/server/server.go index f26acddabdb..c43378ef15a 100644 --- a/server/server.go +++ b/server/server.go @@ -356,7 +356,7 @@ func (s *Server) startServer(ctx context.Context) error { s.tsoAllocatorManager = tso.NewAllocatorManager( s.member, s.rootPath, s.cfg.TSOSaveInterval.Duration, s.cfg.TSOUpdatePhysicalInterval.Duration, func() time.Duration { return s.persistOptions.GetMaxResetTSGap() }, - ) + s.GetTLSConfig()) if err = s.tsoAllocatorManager.SetLocalTSOConfig(s.cfg.LocalTSO); err != nil { return err } diff --git a/server/tso/allocator_manager.go b/server/tso/allocator_manager.go index 87ae16ad1bb..4bf246e6bf3 100644 --- a/server/tso/allocator_manager.go +++ b/server/tso/allocator_manager.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/log" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/etcdutil" + "github.com/tikv/pd/pkg/grpcutil" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/server/config" "github.com/tikv/pd/server/election" @@ -79,6 +80,7 @@ type AllocatorManager struct { saveInterval time.Duration updatePhysicalInterval time.Duration maxResetTSGap func() time.Duration + securityConfig *grpcutil.TLSConfig } // NewAllocatorManager creates a new TSO Allocator Manager. @@ -88,6 +90,7 @@ func NewAllocatorManager( saveInterval time.Duration, updatePhysicalInterval time.Duration, maxResetTSGap func() time.Duration, + sc *grpcutil.TLSConfig, ) *AllocatorManager { allocatorManager := &AllocatorManager{ allocatorGroups: make(map[string]*allocatorGroup), @@ -96,6 +99,7 @@ func NewAllocatorManager( saveInterval: saveInterval, updatePhysicalInterval: updatePhysicalInterval, maxResetTSGap: maxResetTSGap, + securityConfig: sc, } return allocatorManager } @@ -181,7 +185,7 @@ func (am *AllocatorManager) SetUpAllocator(parentCtx context.Context, dcLocation var allocator Allocator if dcLocation == config.GlobalDCLocation { - allocator = NewGlobalTSOAllocator(leadership, am.getAllocatorPath(dcLocation), am.saveInterval, am.updatePhysicalInterval, am.maxResetTSGap) + allocator = NewGlobalTSOAllocator(am, leadership, am.getAllocatorPath(dcLocation), am.saveInterval, am.updatePhysicalInterval, am.maxResetTSGap) } else { allocator = NewLocalTSOAllocator(am.member, leadership, dcLocation, am.saveInterval, am.updatePhysicalInterval, am.maxResetTSGap) } @@ -552,11 +556,11 @@ func (am *AllocatorManager) deleteAllocatorGroup(dcLocation string) { // HandleTSORequest forwards TSO allocation requests to correct TSO Allocators. func (am *AllocatorManager) HandleTSORequest(dcLocation string, count uint32) (pdpb.Timestamp, error) { am.RLock() - defer am.RUnlock() if len(dcLocation) == 0 { dcLocation = config.GlobalDCLocation } allocatorGroup, exist := am.allocatorGroups[dcLocation] + am.RUnlock() if !exist { err := errs.ErrGetAllocator.FastGenByArgs(fmt.Sprintf("%s allocator not found, generate timestamp failed", dcLocation)) return pdpb.Timestamp{}, err @@ -612,9 +616,8 @@ func (am *AllocatorManager) GetAllocators(filters ...AllocatorGroupFilter) []All func (am *AllocatorManager) GetLocalAllocatorLeaders() ([]*LocalTSOAllocator, error) { localAllocators := am.GetAllocators( FilterDCLocation(config.GlobalDCLocation), - FilterUnavailableLeadership(), - FilterUninitialized()) - localAllocatorLeaders := make([]*LocalTSOAllocator, len(localAllocators)) + FilterUnavailableLeadership()) + localAllocatorLeaders := make([]*LocalTSOAllocator, 0, len(localAllocators)) for _, localAllocator := range localAllocators { localAllocatorLeader, ok := localAllocator.(*LocalTSOAllocator) if !ok { diff --git a/server/tso/global_allocator.go b/server/tso/global_allocator.go index 2af0d28681b..1ab96d92def 100644 --- a/server/tso/global_allocator.go +++ b/server/tso/global_allocator.go @@ -14,10 +14,21 @@ package tso import ( + "context" + "fmt" + "sync" "time" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/grpcutil" + "github.com/tikv/pd/pkg/slice" + "github.com/tikv/pd/pkg/tsoutil" + "github.com/tikv/pd/pkg/typeutil" "github.com/tikv/pd/server/election" + "go.uber.org/zap" + "google.golang.org/grpc" ) // Allocator is a Timestamp Oracle allocator. @@ -46,17 +57,25 @@ type GlobalTSOAllocator struct { // to determine whether a TSO request could be processed. leadership *election.Leadership timestampOracle *timestampOracle + // for global TSO synchronization + allocatorManager *AllocatorManager + // for gRPC use + localAllocatorConn struct { + sync.RWMutex + clientConns map[string]*grpc.ClientConn + } } // NewGlobalTSOAllocator creates a new global TSO allocator. func NewGlobalTSOAllocator( + am *AllocatorManager, leadership *election.Leadership, rootPath string, saveInterval time.Duration, updatePhysicalInterval time.Duration, maxResetTSGap func() time.Duration, ) Allocator { - return &GlobalTSOAllocator{ + gta := &GlobalTSOAllocator{ leadership: leadership, timestampOracle: ×tampOracle{ client: leadership.GetClient(), @@ -65,7 +84,10 @@ func NewGlobalTSOAllocator( updatePhysicalInterval: updatePhysicalInterval, maxResetTSGap: maxResetTSGap, }, + allocatorManager: am, } + gta.localAllocatorConn.clientConns = make(map[string]*grpc.ClientConn) + return gta } // Initialize will initialize the created global TSO allocator. @@ -91,8 +113,181 @@ func (gta *GlobalTSOAllocator) SetTSO(tso uint64) error { // GenerateTSO is used to generate a given number of TSOs. // Make sure you have initialized the TSO allocator before calling. func (gta *GlobalTSOAllocator) GenerateTSO(count uint32) (pdpb.Timestamp, error) { - // Todo: implement the synchronization algorithm for global TSO generation - return gta.timestampOracle.getTS(gta.leadership, count) + // To check if we have any dc-location configured in the cluster + dcLocationMap, err := gta.allocatorManager.GetClusterDCLocations() + if err != nil { + log.Error("get cluster dc-locations info failed", errs.ZapError(err)) + return pdpb.Timestamp{}, err + } + // No dc-locations configured in the cluster + if len(dcLocationMap) == 0 { + return gta.timestampOracle.getTS(gta.leadership, count) + } + // Send maxTS to all Local TSO Allocator leaders to prewrite + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + maxTSO := &pdpb.Timestamp{} + // Collect the MaxTS with all Local TSO Allocator leaders first + if err = gta.syncMaxTS(ctx, dcLocationMap, maxTSO); err != nil { + return pdpb.Timestamp{}, err + } + maxTSO.Logical += int64(count) + // Sync the MaxTS with all Local TSO Allocator leaders then + if err := gta.syncMaxTS(ctx, dcLocationMap, maxTSO); err != nil { + return pdpb.Timestamp{}, err + } + var currentGlobalTSO pdpb.Timestamp + if currentGlobalTSO, err = gta.getCurrentTSO(); err != nil { + return pdpb.Timestamp{}, err + } + if tsoutil.CompareTimestamp(¤tGlobalTSO, maxTSO) < 0 { + // Update the global TSO in memory + if err := gta.SetTSO(tsoutil.GenerateTS(maxTSO)); err != nil { + return pdpb.Timestamp{}, err + } + } + return *maxTSO, nil +} + +const ( + dialTimeout = 3 * time.Second + rpcTimeout = 3 * time.Second +) + +func (gta *GlobalTSOAllocator) syncMaxTS(ctx context.Context, dcLocationMap map[string][]uint64, maxTSO *pdpb.Timestamp) error { + maxRetryCount := 1 + for i := 0; i < maxRetryCount; i++ { + // Collect all allocator leaders' client URLs + allocatorLeaders, err := gta.allocatorManager.GetLocalAllocatorLeaders() + if err != nil { + return err + } + leaderURLs := make([]string, 0, len(allocatorLeaders)) + for _, allocator := range allocatorLeaders { + // Check if its client URLs are empty + if len(allocator.GetMember().GetClientUrls()) < 1 { + continue + } + leaderURL := allocator.GetMember().GetClientUrls()[0] + if slice.NoneOf(leaderURLs, func(i int) bool { return leaderURLs[i] == leaderURL }) { + leaderURLs = append(leaderURLs, leaderURL) + } + } + // Prepare to make RPC requests concurrently + respCh := make(chan *pdpb.SyncMaxTSResponse, len(leaderURLs)) + errCh := make(chan error, len(leaderURLs)) + var errList []error + wg := sync.WaitGroup{} + for _, leaderURL := range leaderURLs { + leaderConn, err := gta.getOrCreateGRPCConn(ctx, leaderURL) + if err != nil { + return err + } + wg.Add(1) + go func(ctx context.Context, conn *grpc.ClientConn, respCh chan<- *pdpb.SyncMaxTSResponse, errCh chan<- error) { + request := &pdpb.SyncMaxTSRequest{ + Header: &pdpb.RequestHeader{ + SenderId: gta.allocatorManager.member.ID(), + }, + } + if maxTSO.GetPhysical() != 0 { + request.MaxTs = maxTSO + } + syncCtx, cancel := context.WithTimeout(ctx, rpcTimeout) + resp, err := pdpb.NewPDClient(conn).SyncMaxTS(syncCtx, request) + cancel() + if err != nil { + errCh <- err + log.Error("sync max ts rpc failed, got an error", zap.String("local-allocator-leader-url", leaderConn.Target()), zap.Error(err)) + } + respCh <- resp + if resp == nil { + log.Error("sync max ts rpc failed, got a nil response", zap.String("local-allocator-leader-url", leaderConn.Target())) + } + wg.Done() + }(ctx, leaderConn, respCh, errCh) + } + wg.Wait() + close(respCh) + close(errCh) + // If any error occurs, the synchronization process will fail + if err := <-errCh; err != nil { + errList = append(errList, err) + } + if len(errList) > 0 { + return errs.ErrSyncMaxTS.FastGenWithCause(errList) + } + var syncedDCs []string + for resp := range respCh { + if resp == nil { + return errs.ErrSyncMaxTS.FastGenWithCause("got nil response") + } + syncedDCs = append(syncedDCs, resp.GetDcs()...) + // Compare and get the max one + if resp.GetMaxLocalTs() != nil && resp.GetMaxLocalTs().GetPhysical() != 0 { + if tsoutil.CompareTimestamp(resp.GetMaxLocalTs(), maxTSO) > 0 { + *maxTSO = *(resp.GetMaxLocalTs()) + } + } + } + if !gta.checkSyncedDCs(dcLocationMap, syncedDCs) { + // Only retry one time when synchronization is incomplete + if maxRetryCount == 1 { + log.Warn("unsynced dc-locations found, will retry", zap.Strings("syncedDCs", syncedDCs)) + maxRetryCount++ + continue + } + return errs.ErrSyncMaxTS.FastGenWithCause(fmt.Sprintf("unsynced dc-locations found, synced dc-locations: %+v", syncedDCs)) + } + } + return nil +} + +func (gta *GlobalTSOAllocator) checkSyncedDCs(dcLocationMap map[string][]uint64, syncedDCs []string) bool { + unsyncedDCs := make([]string, 0) + for dcLocation := range dcLocationMap { + if slice.NoneOf(syncedDCs, func(i int) bool { return syncedDCs[i] == dcLocation }) { + unsyncedDCs = append(unsyncedDCs, dcLocation) + } + } + log.Info("check unsynced dc-locations", zap.Strings("unsyncedDCs", unsyncedDCs), zap.Strings("syncedDCs", syncedDCs)) + return len(unsyncedDCs) == 0 +} + +func (gta *GlobalTSOAllocator) getOrCreateGRPCConn(ctx context.Context, addr string) (*grpc.ClientConn, error) { + gta.localAllocatorConn.RLock() + conn, ok := gta.localAllocatorConn.clientConns[addr] + gta.localAllocatorConn.RUnlock() + if ok { + return conn, nil + } + tlsCfg, err := gta.allocatorManager.securityConfig.ToTLSConfig() + if err != nil { + return nil, err + } + ctxWithTimeout, cancel := context.WithTimeout(ctx, dialTimeout) + defer cancel() + cc, err := grpcutil.GetClientConn(ctxWithTimeout, addr, tlsCfg) + if err != nil { + return nil, err + } + gta.localAllocatorConn.Lock() + defer gta.localAllocatorConn.Unlock() + if old, ok := gta.localAllocatorConn.clientConns[addr]; ok { + cc.Close() + log.Debug("use old connection", zap.String("target", cc.Target()), zap.String("state", cc.GetState().String())) + return old, nil + } + gta.localAllocatorConn.clientConns[addr] = cc + return cc, nil +} + +func (gta *GlobalTSOAllocator) getCurrentTSO() (pdpb.Timestamp, error) { + currentPhysical, currentLogical := gta.timestampOracle.getTSO() + if currentPhysical == typeutil.ZeroTime { + return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory isn't initialized") + } + return *tsoutil.GenerateTimestamp(currentPhysical, uint64(currentLogical)), nil } // Reset is used to reset the TSO allocator. diff --git a/server/tso/local_allocator.go b/server/tso/local_allocator.go index cdb930aec95..aeaed38153d 100644 --- a/server/tso/local_allocator.go +++ b/server/tso/local_allocator.go @@ -147,9 +147,8 @@ func (lta *LocalTSOAllocator) WriteTSO(maxTS *pdpb.Timestamp) error { if err != nil { return err } - // If current local TSO has already been greater than - // maxTS, then do not update it. - if currentTSO.Physical >= maxTS.Physical { + // If current local TSO has already been greater or equal to maxTS, then do not update it. + if tsoutil.CompareTimestamp(¤tTSO, maxTS) >= 0 { return nil } return lta.SetTSO(tsoutil.GenerateTS(maxTS)) diff --git a/server/tso/tso.go b/server/tso/tso.go index e7cb208535a..b12555cddd1 100644 --- a/server/tso/tso.go +++ b/server/tso/tso.go @@ -68,11 +68,28 @@ func (t *timestampOracle) setTSOPhysical(next time.Time) { t.tsoMux.Lock() defer t.tsoMux.Unlock() // make sure the ts won't fall back - if t.tsoMux.tso == nil || typeutil.SubTimeByWallClock(next, t.tsoMux.tso.physical) >= updateTimestampGuard { + if t.tsoMux.tso == nil || typeutil.SubTimeByWallClock(next, t.tsoMux.tso.physical) > 0 { t.tsoMux.tso = &tsoObject{physical: next} } } +// setTSO is only used to update the TSO in memory, please make sure you handle +// the time window persisted in etcd well also while using this method. +func (t *timestampOracle) setTSO(nextPhysical time.Time, nextLogical int64) { + t.tsoMux.Lock() + defer t.tsoMux.Unlock() + // make sure the ts won't fall back + if t.tsoMux.tso == nil { + t.tsoMux.tso = &tsoObject{physical: nextPhysical, logical: nextLogical} + } + if typeutil.SubTimeByWallClock(nextPhysical, t.tsoMux.tso.physical) > 0 { + t.tsoMux.tso = &tsoObject{physical: nextPhysical} + } + if typeutil.SubTimeByWallClock(nextPhysical, t.tsoMux.tso.physical) == 0 && nextLogical > t.tsoMux.tso.logical { + t.tsoMux.tso = &tsoObject{physical: nextPhysical, logical: nextLogical} + } +} + func (t *timestampOracle) getTSO() (time.Time, int64) { t.tsoMux.RLock() defer t.tsoMux.RUnlock() @@ -176,35 +193,40 @@ func (t *timestampOracle) isInitialized() bool { return t.tsoMux.tso != nil } -// ResetUserTimestamp update the physical part with specified TSO. +// ResetUserTimestamp update the TSO in memory with specified TSO. func (t *timestampOracle) ResetUserTimestamp(leadership *election.Leadership, tso uint64) error { if !leadership.Check() { tsoCounter.WithLabelValues("err_lease_reset_ts").Inc() return errs.ErrResetUserTimestamp.FastGenByArgs("lease expired") } - physical, _ := tsoutil.ParseTS(tso) - next := physical.Add(time.Millisecond) - prev, _ := t.getTSO() - // do not update if next is less/before than prev - if typeutil.SubTimeByWallClock(next, prev) < updateTimestampGuard { + nextPhysical, nextLogical := tsoutil.ParseTS(tso) + nextPhysical = nextPhysical.Add(updateTimestampGuard) + prevPhysical, prevLogical := t.getTSO() + // do not update if next logical time is less/before than prev + if typeutil.SubTimeByWallClock(nextPhysical, prevPhysical) == 0 && int64(nextLogical) <= prevLogical { + tsoCounter.WithLabelValues("err_reset_small_counter").Inc() + return errs.ErrResetUserTimestamp.FastGenByArgs("the specified counter is smaller than now") + } + // do not update if next physical time is less/before than prev + if typeutil.SubTimeByWallClock(nextPhysical, prevPhysical) < 0 { tsoCounter.WithLabelValues("err_reset_small_ts").Inc() - return errs.ErrResetUserTimestamp.FastGenByArgs("the specified ts is too small than now") + return errs.ErrResetUserTimestamp.FastGenByArgs("the specified ts is smaller than now") } - // do not update if next is too greater than prev - if typeutil.SubTimeByWallClock(next, prev) >= t.maxResetTSGap() { + // do not update if physical time is too greater than prev + if typeutil.SubTimeByWallClock(nextPhysical, prevPhysical) >= t.maxResetTSGap() { tsoCounter.WithLabelValues("err_reset_large_ts").Inc() - return errs.ErrResetUserTimestamp.FastGenByArgs("the specified ts is too large than now") + return errs.ErrResetUserTimestamp.FastGenByArgs("the specified ts is too larger than now") } // save into etcd only if the time difference is big enough - if typeutil.SubTimeByWallClock(next, prev) > 3*updateTimestampGuard { - save := next.Add(t.saveInterval) + if typeutil.SubTimeByWallClock(nextPhysical, prevPhysical) > 3*updateTimestampGuard { + save := nextPhysical.Add(t.saveInterval) if err := t.saveTimestamp(leadership, save); err != nil { tsoCounter.WithLabelValues("err_save_reset_ts").Inc() return err } } // save into memory - t.setTSOPhysical(next) + t.setTSO(nextPhysical, int64(nextLogical)) tsoCounter.WithLabelValues("reset_tso_ok").Inc() return nil } @@ -301,10 +323,10 @@ func (t *timestampOracle) getTS(leadership *election.Leadership, count uint32) ( } // Get a new TSO result with the given count resp.Physical, resp.Logical = t.generateTSO(int64(count)) - if resp.Physical == 0 { + if resp.GetPhysical() == 0 { return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory has been reset") } - if resp.Logical >= maxLogical { + if resp.GetLogical() >= maxLogical { log.Error("logical part outside of max logical interval, please check ntp time", zap.Reflect("response", resp), zap.Int("retry-count", i), errs.ZapError(errs.ErrLogicOverflow)) diff --git a/tests/server/tso/tso_test.go b/tests/server/tso/global_tso_test.go similarity index 63% rename from tests/server/tso/tso_test.go rename to tests/server/tso/global_tso_test.go index 31836f405bb..6f2ad472636 100644 --- a/tests/server/tso/tso_test.go +++ b/tests/server/tso/global_tso_test.go @@ -1,4 +1,4 @@ -// Copyright 2016 TiKV Project Authors. +// Copyright 2020 TiKV Project Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -25,6 +25,7 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/tikv/pd/pkg/testutil" "github.com/tikv/pd/server" + "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" "go.uber.org/goleak" ) @@ -37,24 +38,35 @@ func TestMain(m *testing.M) { goleak.VerifyTestMain(m, testutil.LeakOptions...) } -var _ = Suite(&testTsoSuite{}) +// There are three kinds of ways to generate a TSO: +// 1. Normal Global TSO, the normal way to get a global TSO from the PD leader, +// a.k.a the single Global TSO Allocator. +// 2. Normal Local TSO, the new way to get a local TSO may from any of PD servers, +// a.k.a the Local TSO Allocator leader. +// 3. Synchronized global TSO, the new way to get a global TSO from the PD leader, +// which will coordinate and synchronize a TSO with other Local TSO Allocator +// leaders. -type testTsoSuite struct { +const tsoCount = 10 + +var _ = Suite(&testNormalGlobalTSOSuite{}) + +type testNormalGlobalTSOSuite struct { ctx context.Context cancel context.CancelFunc } -func (s *testTsoSuite) SetUpSuite(c *C) { +func (s *testNormalGlobalTSOSuite) SetUpSuite(c *C) { s.ctx, s.cancel = context.WithCancel(context.Background()) server.EnableZap = true } -func (s *testTsoSuite) TearDownSuite(c *C) { +func (s *testNormalGlobalTSOSuite) TearDownSuite(c *C) { s.cancel() } -func (s *testTsoSuite) testGetTimestamp(c *C, n int) *pdpb.Timestamp { - var err error +// TestNormalGlobalTSO is used to test the normal way of global TSO generation. +func (s *testNormalGlobalTSOSuite) TestNormalGlobalTSO(c *C) { cluster, err := tests.NewTestCluster(s.ctx, 1) defer cluster.Destroy() c.Assert(err, IsNil) @@ -68,28 +80,11 @@ func (s *testTsoSuite) testGetTimestamp(c *C, n int) *pdpb.Timestamp { clusterID := leaderServer.GetClusterID() req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(clusterID), - Count: uint32(n), + Header: testutil.NewRequestHeader(clusterID), + Count: uint32(tsoCount), + DcLocation: config.GlobalDCLocation, } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - tsoClient, err := grpcPDClient.Tso(ctx) - c.Assert(err, IsNil) - defer tsoClient.CloseSend() - err = tsoClient.Send(req) - c.Assert(err, IsNil) - resp, err := tsoClient.Recv() - c.Assert(err, IsNil) - c.Assert(resp.GetCount(), Equals, uint32(n)) - - res := resp.GetTimestamp() - c.Assert(res.GetLogical(), Greater, int64(0)) - - return res -} - -func (s *testTsoSuite) TestTso(c *C) { var wg sync.WaitGroup for i := 0; i < 10; i++ { wg.Add(1) @@ -102,7 +97,7 @@ func (s *testTsoSuite) TestTso(c *C) { } for j := 0; j < 30; j++ { - ts := s.testGetTimestamp(c, 10) + ts := s.testGetNormalGlobalTimestamp(c, grpcPDClient, req) c.Assert(ts.GetPhysical(), Not(Less), last.GetPhysical()) if ts.GetPhysical() == last.GetPhysical() { c.Assert(ts.GetLogical(), Greater, last.GetLogical()) @@ -112,11 +107,26 @@ func (s *testTsoSuite) TestTso(c *C) { } }() } - wg.Wait() } -func (s *testTsoSuite) TestConcurrcyRequest(c *C) { +func (s *testNormalGlobalTSOSuite) testGetNormalGlobalTimestamp(c *C, pdCli pdpb.PDClient, req *pdpb.TsoRequest) *pdpb.Timestamp { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + tsoClient, err := pdCli.Tso(ctx) + c.Assert(err, IsNil) + defer tsoClient.CloseSend() + err = tsoClient.Send(req) + c.Assert(err, IsNil) + resp, err := tsoClient.Recv() + c.Assert(err, IsNil) + c.Assert(resp.GetCount(), Equals, req.GetCount()) + res := resp.GetTimestamp() + c.Assert(res.GetLogical(), Greater, int64(0)) + return res +} + +func (s *testNormalGlobalTSOSuite) TestConcurrencyRequest(c *C) { cluster, err := tests.NewTestCluster(s.ctx, 1) defer cluster.Destroy() c.Assert(err, IsNil) @@ -144,8 +154,7 @@ func (s *testTsoSuite) TestConcurrcyRequest(c *C) { wg.Wait() } -func (s *testTsoSuite) TestTsoCount0(c *C) { - var err error +func (s *testNormalGlobalTSOSuite) TestZeroTSOCount(c *C) { cluster, err := tests.NewTestCluster(s.ctx, 1) defer cluster.Destroy() c.Assert(err, IsNil) @@ -158,7 +167,10 @@ func (s *testTsoSuite) TestTsoCount0(c *C) { grpcPDClient := testutil.MustNewGrpcClient(c, leaderServer.GetAddr()) clusterID := leaderServer.GetClusterID() - req := &pdpb.TsoRequest{Header: testutil.NewRequestHeader(clusterID)} + req := &pdpb.TsoRequest{ + Header: testutil.NewRequestHeader(clusterID), + DcLocation: config.GlobalDCLocation, + } ctx, cancel := context.WithCancel(context.Background()) defer cancel() tsoClient, err := grpcPDClient.Tso(ctx) @@ -170,7 +182,7 @@ func (s *testTsoSuite) TestTsoCount0(c *C) { c.Assert(err, NotNil) } -func (s *testTsoSuite) TestRequestFollower(c *C) { +func (s *testNormalGlobalTSOSuite) TestRequestFollower(c *C) { cluster, err := tests.NewTestCluster(s.ctx, 2) c.Assert(err, IsNil) defer cluster.Destroy() @@ -190,8 +202,9 @@ func (s *testTsoSuite) TestRequestFollower(c *C) { grpcPDClient := testutil.MustNewGrpcClient(c, followerServer.GetAddr()) clusterID := followerServer.GetClusterID() req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(clusterID), - Count: 1, + Header: testutil.NewRequestHeader(clusterID), + Count: 1, + DcLocation: config.GlobalDCLocation, } ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -212,7 +225,7 @@ func (s *testTsoSuite) TestRequestFollower(c *C) { // In some cases, when a TSO request arrives, the SyncTimestamp may not finish yet. // This test is used to simulate this situation and verify that the retry mechanism. -func (s *testTsoSuite) TestDelaySyncTimestamp(c *C) { +func (s *testNormalGlobalTSOSuite) TestDelaySyncTimestamp(c *C) { cluster, err := tests.NewTestCluster(s.ctx, 2) c.Assert(err, IsNil) defer cluster.Destroy() @@ -234,8 +247,9 @@ func (s *testTsoSuite) TestDelaySyncTimestamp(c *C) { grpcPDClient := testutil.MustNewGrpcClient(c, nextLeaderServer.GetAddr()) clusterID := nextLeaderServer.GetClusterID() req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(clusterID), - Count: 1, + Header: testutil.NewRequestHeader(clusterID), + Count: 1, + DcLocation: config.GlobalDCLocation, } ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -298,8 +312,9 @@ func (s *testTimeFallBackSuite) TearDownSuite(c *C) { func (s *testTimeFallBackSuite) testGetTimestamp(c *C, n int) *pdpb.Timestamp { clusterID := s.server.GetClusterID() req := &pdpb.TsoRequest{ - Header: testutil.NewRequestHeader(clusterID), - Count: uint32(n), + Header: testutil.NewRequestHeader(clusterID), + Count: uint32(n), + DcLocation: config.GlobalDCLocation, } ctx, cancel := context.WithCancel(context.Background()) @@ -365,7 +380,6 @@ func (s *testFollowerTsoSuite) TearDownSuite(c *C) { func (s *testFollowerTsoSuite) TestRequest(c *C) { c.Assert(failpoint.Enable("github.com/tikv/pd/server/tso/skipRetryGetTS", `return(true)`), IsNil) - var err error cluster, err := tests.NewTestCluster(s.ctx, 2) defer cluster.Destroy() c.Assert(err, IsNil) @@ -385,7 +399,11 @@ func (s *testFollowerTsoSuite) TestRequest(c *C) { grpcPDClient := testutil.MustNewGrpcClient(c, followerServer.GetAddr()) clusterID := followerServer.GetClusterID() - req := &pdpb.TsoRequest{Header: testutil.NewRequestHeader(clusterID), Count: 1} + req := &pdpb.TsoRequest{ + Header: testutil.NewRequestHeader(clusterID), + Count: 1, + DcLocation: config.GlobalDCLocation, + } ctx, cancel := context.WithCancel(context.Background()) defer cancel() tsoClient, err := grpcPDClient.Tso(ctx) @@ -398,3 +416,98 @@ func (s *testFollowerTsoSuite) TestRequest(c *C) { c.Assert(strings.Contains(err.Error(), "generate timestamp failed"), IsTrue) failpoint.Disable("github.com/tikv/pd/server/tso/skipRetryGetTS") } + +var _ = Suite(&testSynchronizedGlobalTSO{}) + +type testSynchronizedGlobalTSO struct { + ctx context.Context + cancel context.CancelFunc + leaderServer *tests.TestServer + dcClientMap map[string]pdpb.PDClient +} + +func (s *testSynchronizedGlobalTSO) SetUpSuite(c *C) { + s.ctx, s.cancel = context.WithCancel(context.Background()) + s.dcClientMap = make(map[string]pdpb.PDClient) + server.EnableZap = true +} + +func (s *testSynchronizedGlobalTSO) TearDownSuite(c *C) { + s.cancel() +} + +// TestSynchronizedGlobalTSO is used to test the synchronized way of global TSO generation. +func (s *testSynchronizedGlobalTSO) TestSynchronizedGlobalTSO(c *C) { + dcLocationConfig := map[string]string{ + "pd1": "dc-1", + "pd2": "dc-2", + "pd3": "dc-3", + } + dcLocationNum := len(dcLocationConfig) + cluster, err := tests.NewTestCluster(s.ctx, dcLocationNum, func(conf *config.Config, serverName string) { + conf.LocalTSO.EnableLocalTSO = true + conf.LocalTSO.DCLocation = dcLocationConfig[serverName] + }) + defer cluster.Destroy() + c.Assert(err, IsNil) + + err = cluster.RunInitialServers() + c.Assert(err, IsNil) + for _, dcLocation := range dcLocationConfig { + pdName := cluster.WaitAllocatorLeader(dcLocation) + c.Assert(len(pdName), Greater, 0) + s.dcClientMap[dcLocation] = testutil.MustNewGrpcClient(c, cluster.GetServer(pdName).GetAddr()) + } + cluster.WaitLeader() + s.leaderServer = cluster.GetServer(cluster.GetLeader()) + c.Assert(s.leaderServer, NotNil) + s.dcClientMap[config.GlobalDCLocation] = testutil.MustNewGrpcClient(c, s.leaderServer.GetAddr()) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + // Get some local TSOs first + oldLocalTSOs := make([]*pdpb.Timestamp, 0, dcLocationNum) + for _, dcLocation := range dcLocationConfig { + oldLocalTSOs = append(oldLocalTSOs, s.testGetTimestamp(ctx, c, tsoCount, dcLocation)) + } + // Get a global TSO then + globalTSO := s.testGetTimestamp(ctx, c, tsoCount, config.GlobalDCLocation) + for _, oldLocalTSO := range oldLocalTSOs { + c.Assert(globalTSO.GetPhysical(), GreaterEqual, oldLocalTSO.GetPhysical()) + if globalTSO.GetPhysical() == oldLocalTSO.GetPhysical() { + c.Assert(globalTSO.GetLogical(), Greater, oldLocalTSO.GetLogical()) + } + } + // Get some local TSOs again + newLocalTSOs := make([]*pdpb.Timestamp, 0, dcLocationNum) + for _, dcLocation := range dcLocationConfig { + newLocalTSOs = append(newLocalTSOs, s.testGetTimestamp(ctx, c, tsoCount, dcLocation)) + } + for _, newLocalTSO := range newLocalTSOs { + c.Assert(globalTSO.GetPhysical(), LessEqual, newLocalTSO.GetPhysical()) + if globalTSO.GetPhysical() == newLocalTSO.GetPhysical() { + c.Assert(globalTSO.GetLogical(), Less, newLocalTSO.GetLogical()) + } + } +} + +func (s *testSynchronizedGlobalTSO) testGetTimestamp(ctx context.Context, c *C, n int, dcLocation string) *pdpb.Timestamp { + req := &pdpb.TsoRequest{ + Header: testutil.NewRequestHeader(s.leaderServer.GetClusterID()), + Count: tsoCount, + DcLocation: dcLocation, + } + pdClient, ok := s.dcClientMap[dcLocation] + c.Assert(ok, IsTrue) + tsoClient, err := pdClient.Tso(ctx) + c.Assert(err, IsNil) + defer tsoClient.CloseSend() + err = tsoClient.Send(req) + c.Assert(err, IsNil) + resp, err := tsoClient.Recv() + c.Assert(err, IsNil) + c.Assert(resp.GetCount(), Equals, uint32(n)) + res := resp.GetTimestamp() + c.Assert(res.GetLogical(), Greater, int64(0)) + return res +} diff --git a/tests/server/tso/local_tso_test.go b/tests/server/tso/local_tso_test.go new file mode 100644 index 00000000000..bf924452f45 --- /dev/null +++ b/tests/server/tso/local_tso_test.go @@ -0,0 +1,121 @@ +// Copyright 2020 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tso_test + +import ( + "context" + "sync" + "time" + + . "github.com/pingcap/check" + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/testutil" + "github.com/tikv/pd/server" + "github.com/tikv/pd/server/config" + "github.com/tikv/pd/tests" +) + +var _ = Suite(&testLocalTSOSuite{}) + +type testLocalTSOSuite struct { + ctx context.Context + cancel context.CancelFunc +} + +func (s *testLocalTSOSuite) SetUpSuite(c *C) { + s.ctx, s.cancel = context.WithCancel(context.Background()) + server.EnableZap = true +} + +func (s *testLocalTSOSuite) TearDownSuite(c *C) { + s.cancel() +} + +// TestNormalGlobalTSO is used to test the normal way of global TSO generation. +func (s *testLocalTSOSuite) TestLocalTSO(c *C) { + dcLocationConfig := map[string]string{ + "pd1": "dc-1", + "pd2": "dc-2", + "pd3": "dc-3", + } + dcLocationNum := len(dcLocationConfig) + cluster, err := tests.NewTestCluster(s.ctx, dcLocationNum, func(conf *config.Config, serverName string) { + conf.LocalTSO.EnableLocalTSO = true + conf.LocalTSO.DCLocation = dcLocationConfig[serverName] + }) + defer cluster.Destroy() + c.Assert(err, IsNil) + + err = cluster.RunInitialServers() + c.Assert(err, IsNil) + + dcClientMap := make(map[string]pdpb.PDClient) + for _, dcLocation := range dcLocationConfig { + pdName := cluster.WaitAllocatorLeader(dcLocation) + c.Assert(len(pdName), Greater, 0) + dcClientMap[dcLocation] = testutil.MustNewGrpcClient(c, cluster.GetServer(pdName).GetAddr()) + } + + cluster.WaitLeader() + leaderServer := cluster.GetServer(cluster.GetLeader()) + + var wg sync.WaitGroup + for i := 0; i < 10; i++ { + wg.Add(1) + go func() { + defer wg.Done() + lastList := make(map[string]*pdpb.Timestamp) + for _, dcLocation := range dcLocationConfig { + lastList[dcLocation] = &pdpb.Timestamp{ + Physical: 0, + Logical: 0, + } + } + for j := 0; j < 30; j++ { + for _, dcLocation := range dcLocationConfig { + req := &pdpb.TsoRequest{ + Header: testutil.NewRequestHeader(leaderServer.GetClusterID()), + Count: tsoCount, + DcLocation: dcLocation, + } + ts := s.testGetLocalTimestamp(c, dcClientMap[dcLocation], req) + lastTS := lastList[dcLocation] + c.Assert(ts.GetPhysical(), Not(Less), lastTS.GetPhysical()) + if ts.GetPhysical() == lastTS.GetPhysical() { + c.Assert(ts.GetLogical(), Greater, lastTS.GetLogical()) + } + lastList[dcLocation] = ts + } + time.Sleep(10 * time.Millisecond) + } + }() + } + wg.Wait() +} + +func (s *testLocalTSOSuite) testGetLocalTimestamp(c *C, pdCli pdpb.PDClient, req *pdpb.TsoRequest) *pdpb.Timestamp { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + tsoClient, err := pdCli.Tso(ctx) + c.Assert(err, IsNil) + defer tsoClient.CloseSend() + err = tsoClient.Send(req) + c.Assert(err, IsNil) + resp, err := tsoClient.Recv() + c.Assert(err, IsNil) + c.Assert(resp.GetCount(), Equals, req.GetCount()) + res := resp.GetTimestamp() + c.Assert(res.GetLogical(), Greater, int64(0)) + return res +} From d12132da9c593234828feeb57d272a88e4beb7e6 Mon Sep 17 00:00:00 2001 From: ZenoTan Date: Wed, 14 Oct 2020 16:12:00 +0800 Subject: [PATCH 25/27] Use unhealthy Signed-off-by: ZenoTan --- server/schedule/operator/builder.go | 16 ++++++++++------ server/schedule/operator/builder_test.go | 6 ++++++ 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/server/schedule/operator/builder.go b/server/schedule/operator/builder.go index e1949e5d09c..b27d87f096b 100644 --- a/server/schedule/operator/builder.go +++ b/server/schedule/operator/builder.go @@ -45,7 +45,7 @@ type Builder struct { // operation record originPeers peersMap - pendingPeers peersMap + unhealthyPeers peersMap originLeaderStoreID uint64 targetPeers peersMap targetLeaderStoreID uint64 @@ -96,7 +96,7 @@ func NewBuilder(desc string, cluster opt.Cluster, region *core.RegionInfo, opts // origin peers err := b.err originPeers := newPeersMap() - pendingPeers := newPeersMap() + unhealthyPeers := newPeersMap() for _, p := range region.GetPeers() { if p == nil || p.GetStoreId() == 0 { @@ -107,7 +107,11 @@ func NewBuilder(desc string, cluster opt.Cluster, region *core.RegionInfo, opts } for _, p := range region.GetPendingPeers() { - pendingPeers.Set(p) + unhealthyPeers.Set(p) + } + + for _, p := range region.GetDownPeers() { + unhealthyPeers.Set(p.Peer) } // origin leader @@ -138,7 +142,7 @@ func NewBuilder(desc string, cluster opt.Cluster, region *core.RegionInfo, opts b.rules = rules b.originPeers = originPeers - b.pendingPeers = pendingPeers + b.unhealthyPeers = unhealthyPeers b.originLeaderStoreID = originLeaderStoreID b.targetPeers = originPeers.Copy() b.allowDemote = supportJointConsensus @@ -189,8 +193,8 @@ func (b *Builder) PromoteLearner(storeID uint64) *Builder { b.err = errors.Errorf("cannot promote peer %d: not found", storeID) } else if !core.IsLearner(peer) { b.err = errors.Errorf("cannot promote peer %d: is not learner", storeID) - } else if _, ok := b.pendingPeers[storeID]; ok { - b.err = errors.Errorf("cannot promote peer %d: pending", storeID) + } else if _, ok := b.unhealthyPeers[storeID]; ok { + b.err = errors.Errorf("cannot promote peer %d: unhealthy", storeID) } else { b.targetPeers.Set(&metapb.Peer{ Id: peer.GetId(), diff --git a/server/schedule/operator/builder_test.go b/server/schedule/operator/builder_test.go index c94e162c094..a9e2dcbc43f 100644 --- a/server/schedule/operator/builder_test.go +++ b/server/schedule/operator/builder_test.go @@ -16,6 +16,7 @@ package operator import ( . "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/server/config" "github.com/tikv/pd/server/core" @@ -451,4 +452,9 @@ func (s *testBuilderSuite) TestPromotePending(c *C) { builder := NewBuilder("test", s.cluster, region) builder.PromoteLearner(2) c.Assert(builder.err, NotNil) + region = core.NewRegionInfo(&metapb.Region{Id: 1, Peers: []*metapb.Peer{{Id: 1, StoreId: 1}, + p}}, &metapb.Peer{Id: 1, StoreId: 1}, core.WithDownPeers([]*pdpb.PeerStats{{Peer: p}})) + builder = NewBuilder("test", s.cluster, region) + builder.PromoteLearner(2) + c.Assert(builder.err, NotNil) } From 055a5f5dd6bdd09b4f8511ab160ba65d624db191 Mon Sep 17 00:00:00 2001 From: ZenoTan Date: Wed, 14 Oct 2020 16:36:18 +0800 Subject: [PATCH 26/27] fix Signed-off-by: ZenoTan --- server/schedule/operator/builder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/schedule/operator/builder.go b/server/schedule/operator/builder.go index b27d87f096b..7ed14402e4d 100644 --- a/server/schedule/operator/builder.go +++ b/server/schedule/operator/builder.go @@ -45,7 +45,7 @@ type Builder struct { // operation record originPeers peersMap - unhealthyPeers peersMap + unhealthyPeers peersMap originLeaderStoreID uint64 targetPeers peersMap targetLeaderStoreID uint64 From 3ff4b5c8a603cfe87c4eae9de7a8d10fa2716d21 Mon Sep 17 00:00:00 2001 From: ZenoTan Date: Wed, 14 Oct 2020 18:15:12 +0800 Subject: [PATCH 27/27] Change func name Signed-off-by: ZenoTan --- server/schedule/operator/builder_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/schedule/operator/builder_test.go b/server/schedule/operator/builder_test.go index a9e2dcbc43f..3670f515f42 100644 --- a/server/schedule/operator/builder_test.go +++ b/server/schedule/operator/builder_test.go @@ -445,7 +445,7 @@ func (s *testBuilderSuite) TestBuild(c *C) { } // Test for issue 3039 -func (s *testBuilderSuite) TestPromotePending(c *C) { +func (s *testBuilderSuite) TestPromoteUnhealthyPeer(c *C) { p := &metapb.Peer{Id: 2, StoreId: 2, Role: metapb.PeerRole_Learner} region := core.NewRegionInfo(&metapb.Region{Id: 1, Peers: []*metapb.Peer{{Id: 1, StoreId: 1}, p}}, &metapb.Peer{Id: 1, StoreId: 1}, core.WithPendingPeers([]*metapb.Peer{p}))