From cac7e855596e77e980e4ad060766cf0000e928ff Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Wed, 26 Apr 2023 20:07:45 +0800 Subject: [PATCH 01/24] etcdutil: add watch loop Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil.go | 160 +++++++++++++++++++++++++++++++++ 1 file changed, 160 insertions(+) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index b65f8b901a4..440a7ae7f88 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -20,6 +20,7 @@ import ( "math/rand" "net/http" "net/url" + "sync" "time" "github.com/gogo/protobuf/proto" @@ -27,9 +28,11 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/etcdserver" + "go.etcd.io/etcd/mvcc/mvccpb" "go.etcd.io/etcd/pkg/types" "go.uber.org/zap" ) @@ -350,3 +353,160 @@ func InitOrGetClusterID(c *clientv3.Client, key string) (uint64, error) { return typeutil.BytesToUint64(response.Kvs[0].Value) } + +// LoopWatcher loads data from etcd and sets a watcher for it. +type LoopWatcher struct { + ctx context.Context + wg *sync.WaitGroup + client *clientv3.Client + key string + name string + forceLoadCh chan struct{} +} + +// NewLoopWatcher creates a new LoopWatcher. +func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Client, name, key string) *LoopWatcher { + return &LoopWatcher{ + ctx: ctx, + client: client, + name: name, + key: key, + wg: wg, + forceLoadCh: make(chan struct{}, 1), + } +} + +type etcdFunc func(*mvccpb.KeyValue) error + +const ( + maxLoadRetryTimes = 30 + loadRetryInterval = time.Millisecond * 100 + watchEtcdChangeRetryInterval = time.Second +) + +// StartWatchLoop starts a loop to watch the key. +func (lw *LoopWatcher) StartWatchLoop(putFn, deleteFn etcdFunc, opts ...clientv3.OpOption) { + defer logutil.LogPanic() + defer lw.wg.Done() + ctx, cancel := context.WithCancel(lw.ctx) + defer cancel() + + var ( + revision int64 + err error + ) + ticker := time.NewTicker(loadRetryInterval) + defer ticker.Stop() + for i := 0; i < maxLoadRetryTimes; i++ { + revision, err = lw.load(putFn, opts...) + if err == nil { + break + } + select { + case <-ctx.Done(): + return + case <-ticker.C: + } + } + if err != nil || revision == 0 { + log.Warn("watched key doesn't exist in watch loop when loading", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) + } + log.Info("start to watch loop", zap.String("name", lw.name), zap.String("key", lw.key)) + for { + select { + case <-ctx.Done(): + log.Info("server is closed, exit watch loop", zap.String("name", lw.name), zap.String("key", lw.key)) + return + default: + } + nextRevision, err := lw.watch(ctx, revision, putFn, deleteFn, opts...) + if err != nil { + log.Error("watcher canceled unexpectedly and a new watcher will start after a while for watch loop", + zap.String("name", lw.name), + zap.String("key", lw.key), + zap.Int64("next-revision", nextRevision), + zap.Time("retry-at", time.Now().Add(watchEtcdChangeRetryInterval)), + zap.Error(err)) + revision = nextRevision + time.Sleep(watchEtcdChangeRetryInterval) + } + } +} + +func (lw *LoopWatcher) watch(ctx context.Context, revision int64, putFn, deleteFn etcdFunc, opts ...clientv3.OpOption) (nextRevision int64, err error) { + watcher := clientv3.NewWatcher(lw.client) + defer watcher.Close() + + for { + WatchChan: + watchChan := watcher.Watch(ctx, lw.key, append(opts, clientv3.WithRev(revision))...) + select { + case <-ctx.Done(): + return revision, nil + case <-lw.forceLoadCh: + revision, err = lw.load(putFn, opts...) + if err != nil { + log.Warn("force load key failed in watch loop", zap.String("name", lw.name), + zap.String("key", lw.key), zap.Error(err)) + } + goto WatchChan + case wresp := <-watchChan: + if wresp.CompactRevision != 0 { + log.Warn("required revision has been compacted, use the compact revision in watch loop", + zap.Int64("required-revision", revision), + zap.Int64("compact-revision", wresp.CompactRevision)) + revision = wresp.CompactRevision + goto WatchChan + } + if wresp.Err() != nil { + log.Error("watcher is canceled in watch loop", + zap.Int64("revision", revision), + errs.ZapError(errs.ErrEtcdWatcherCancel, wresp.Err())) + return revision, wresp.Err() + } + for _, event := range wresp.Events { + switch event.Type { + case clientv3.EventTypePut: + if err := putFn(event.Kv); err != nil { + log.Error("put failed in watch loop", zap.String("name", lw.name), + zap.String("key", lw.key), zap.Error(err)) + } + case clientv3.EventTypeDelete: + if err := deleteFn(event.Kv); err != nil { + log.Error("delete failed in watch loop", zap.String("name", lw.name), + zap.String("key", lw.key), zap.Error(err)) + } + } + } + revision = wresp.Header.Revision + 1 + } + } +} + +func (lw *LoopWatcher) load(putFn etcdFunc, opts ...clientv3.OpOption) (nextRevision int64, err error) { + resp, err := EtcdKVGet(lw.client, lw.key, opts...) + if err != nil { + log.Error("load failed in watch loop", zap.String("name", lw.name), + zap.String("key", lw.key), zap.Error(err)) + return 0, err + } + for _, item := range resp.Kvs { + err = putFn(item) + if err != nil { + log.Error("put failed in watch loop when loading", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) + continue + } + } + return resp.Header.Revision + 1, err +} + +func (lw *LoopWatcher) ForceLoad() { + select { + case lw.forceLoadCh <- struct{}{}: + default: + } +} + +func (lw *LoopWatcher) Close() { + close(lw.forceLoadCh) +} From 9f3f6838211ce162efcdb81e466e31dbc844f461 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Wed, 26 Apr 2023 22:28:57 +0800 Subject: [PATCH 02/24] update Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil.go | 8 +- pkg/utils/tsoutil/tso_dispatcher.go | 13 +-- server/grpc_service.go | 2 +- server/server.go | 151 +++++----------------------- 4 files changed, 35 insertions(+), 139 deletions(-) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 440a7ae7f88..2286758788e 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -376,8 +376,6 @@ func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Cl } } -type etcdFunc func(*mvccpb.KeyValue) error - const ( maxLoadRetryTimes = 30 loadRetryInterval = time.Millisecond * 100 @@ -385,7 +383,7 @@ const ( ) // StartWatchLoop starts a loop to watch the key. -func (lw *LoopWatcher) StartWatchLoop(putFn, deleteFn etcdFunc, opts ...clientv3.OpOption) { +func (lw *LoopWatcher) StartWatchLoop(putFn, deleteFn func(*mvccpb.KeyValue) error, opts ...clientv3.OpOption) { defer logutil.LogPanic() defer lw.wg.Done() ctx, cancel := context.WithCancel(lw.ctx) @@ -433,7 +431,7 @@ func (lw *LoopWatcher) StartWatchLoop(putFn, deleteFn etcdFunc, opts ...clientv3 } } -func (lw *LoopWatcher) watch(ctx context.Context, revision int64, putFn, deleteFn etcdFunc, opts ...clientv3.OpOption) (nextRevision int64, err error) { +func (lw *LoopWatcher) watch(ctx context.Context, revision int64, putFn, deleteFn func(*mvccpb.KeyValue) error, opts ...clientv3.OpOption) (nextRevision int64, err error) { watcher := clientv3.NewWatcher(lw.client) defer watcher.Close() @@ -483,7 +481,7 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64, putFn, deleteF } } -func (lw *LoopWatcher) load(putFn etcdFunc, opts ...clientv3.OpOption) (nextRevision int64, err error) { +func (lw *LoopWatcher) load(putFn func(*mvccpb.KeyValue) error, opts ...clientv3.OpOption) (nextRevision int64, err error) { resp, err := EtcdKVGet(lw.client, lw.key, opts...) if err != nil { log.Error("load failed in watch loop", zap.String("name", lw.name), diff --git a/pkg/utils/tsoutil/tso_dispatcher.go b/pkg/utils/tsoutil/tso_dispatcher.go index 0d18c5a3e6e..0af74864f3c 100644 --- a/pkg/utils/tsoutil/tso_dispatcher.go +++ b/pkg/utils/tsoutil/tso_dispatcher.go @@ -64,12 +64,12 @@ func (s *TSODispatcher) DispatchRequest( tsoProtoFactory ProtoFactory, doneCh <-chan struct{}, errCh chan<- error, - updateServicePrimaryAddrChs ...chan<- struct{}) { + callbacks ...func()) { val, loaded := s.dispatchChs.LoadOrStore(req.getForwardedHost(), make(chan Request, maxMergeRequests)) reqCh := val.(chan Request) if !loaded { tsDeadlineCh := make(chan deadline, 1) - go s.dispatch(ctx, tsoProtoFactory, req.getForwardedHost(), req.getClientConn(), reqCh, tsDeadlineCh, doneCh, errCh, updateServicePrimaryAddrChs...) + go s.dispatch(ctx, tsoProtoFactory, req.getForwardedHost(), req.getClientConn(), reqCh, tsDeadlineCh, doneCh, errCh, callbacks...) go watchTSDeadline(ctx, tsDeadlineCh) } reqCh <- req @@ -84,7 +84,7 @@ func (s *TSODispatcher) dispatch( tsDeadlineCh chan<- deadline, doneCh <-chan struct{}, errCh chan<- error, - updateServicePrimaryAddrChs ...chan<- struct{}) { + callbacks ...func()) { defer logutil.LogPanic() dispatcherCtx, ctxCancel := context.WithCancel(ctx) defer ctxCancel() @@ -111,7 +111,7 @@ func (s *TSODispatcher) dispatch( defer cancel() requests := make([]Request, maxMergeRequests+1) - needUpdateServicePrimaryAddr := len(updateServicePrimaryAddrChs) > 0 && updateServicePrimaryAddrChs[0] != nil + needUpdateServicePrimaryAddr := len(callbacks) > 0 && callbacks[0] != nil for { select { case first := <-tsoRequestCh: @@ -139,10 +139,7 @@ func (s *TSODispatcher) dispatch( errs.ZapError(errs.ErrGRPCSend, err)) if needUpdateServicePrimaryAddr { if strings.Contains(err.Error(), errs.NotLeaderErr) || strings.Contains(err.Error(), errs.MismatchLeaderErr) { - select { - case updateServicePrimaryAddrChs[0] <- struct{}{}: - default: - } + callbacks[0]() } } select { diff --git a/server/grpc_service.go b/server/grpc_service.go index 48ac9875415..25af29dbf18 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -224,7 +224,7 @@ func (s *GrpcServer) Tso(stream pdpb.PD_TsoServer) error { } tsoRequest := tsoutil.NewPDProtoRequest(forwardedHost, clientConn, request, stream) - s.tsoDispatcher.DispatchRequest(ctx, tsoRequest, tsoProtoFactory, doneCh, errCh, s.updateServicePrimaryAddrCh) + s.tsoDispatcher.DispatchRequest(ctx, tsoRequest, tsoProtoFactory, doneCh, errCh, s.servicePrimaryLoopWatcher.ForceLoad) continue } diff --git a/server/server.go b/server/server.go index e83831364c7..aa482d40654 100644 --- a/server/server.go +++ b/server/server.go @@ -77,6 +77,7 @@ import ( syncer "github.com/tikv/pd/server/region_syncer" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" + "go.etcd.io/etcd/mvcc/mvccpb" "go.etcd.io/etcd/pkg/types" "go.uber.org/zap" "google.golang.org/grpc" @@ -214,12 +215,10 @@ type Server struct { auditBackends []audit.Backend - registry *registry.ServiceRegistry - mode string - servicePrimaryMap sync.Map /* Store as map[string]string */ - // updateServicePrimaryAddrCh is used to notify the server to update the service primary address. - // Note: it is only used in API service mode. - updateServicePrimaryAddrCh chan struct{} + registry *registry.ServiceRegistry + mode string + servicePrimaryMap sync.Map /* Store as map[string]string */ + servicePrimaryLoopWatcher *etcdutil.LoopWatcher } // HandlerBuilder builds a server HTTP handler. @@ -567,8 +566,28 @@ func (s *Server) startServerLoop(ctx context.Context) { go s.serverMetricsLoop() go s.encryptionKeyManagerLoop() if s.IsAPIServiceMode() { // disable tso service in api server + serviceName := mcs.TSOServiceName + tsoServicePrimaryKey := s.servicePrimaryKey(serviceName) s.serverLoopWg.Add(1) - go s.startWatchServicePrimaryAddrLoop(mcs.TSOServiceName) + s.servicePrimaryLoopWatcher = etcdutil.NewLoopWatcher(s.serverLoopCtx, &s.serverLoopWg, s.client, "service-primary", tsoServicePrimaryKey) + putFn := func(kv *mvccpb.KeyValue) error { + primary := &tsopb.Participant{} // TODO: use Generics + if err := proto.Unmarshal(kv.Value, primary); err != nil { + return err + } else { + listenUrls := primary.GetListenUrls() + if len(listenUrls) > 0 { + // listenUrls[0] is the primary service endpoint of the keyspace group + s.servicePrimaryMap.Store(serviceName, listenUrls[0]) + } + } + return nil + } + deleteFn := func(kv *mvccpb.KeyValue) error { + s.servicePrimaryMap.Delete(serviceName) + return nil + } + go s.servicePrimaryLoopWatcher.StartWatchLoop(putFn, deleteFn) } } @@ -1722,124 +1741,6 @@ func (s *Server) GetServicePrimaryAddr(ctx context.Context, serviceName string) return "", false } -// startWatchServicePrimaryAddrLoop starts a loop to watch the primary address of a given service. -func (s *Server) startWatchServicePrimaryAddrLoop(serviceName string) { - defer logutil.LogPanic() - defer s.serverLoopWg.Done() - ctx, cancel := context.WithCancel(s.serverLoopCtx) - defer cancel() - s.updateServicePrimaryAddrCh = make(chan struct{}, 1) - serviceKey := s.servicePrimaryKey(serviceName) - var ( - revision int64 - err error - ) - for i := 0; i < maxRetryTimesGetServicePrimary; i++ { - revision, err = s.updateServicePrimaryAddr(serviceName) - if revision != 0 && err == nil { // update success - break - } - select { - case <-ctx.Done(): - return - case <-time.After(retryIntervalGetServicePrimary): - } - } - if err != nil { - log.Warn("service primary addr doesn't exist", zap.String("service-key", serviceKey), zap.Error(err)) - } - log.Info("start to watch service primary addr", zap.String("service-key", serviceKey)) - for { - select { - case <-ctx.Done(): - log.Info("server is closed, exist watch service primary addr loop", zap.String("service", serviceName)) - return - default: - } - nextRevision, err := s.watchServicePrimaryAddr(ctx, serviceName, revision) - if err != nil { - log.Error("watcher canceled unexpectedly and a new watcher will start after a while", - zap.Int64("next-revision", nextRevision), - zap.Time("retry-at", time.Now().Add(watchEtcdChangeRetryInterval)), - zap.Error(err)) - revision = nextRevision - time.Sleep(watchEtcdChangeRetryInterval) - } - } -} - -// watchServicePrimaryAddr watches the primary address on etcd. -func (s *Server) watchServicePrimaryAddr(ctx context.Context, serviceName string, revision int64) (nextRevision int64, err error) { - serviceKey := s.servicePrimaryKey(serviceName) - watcher := clientv3.NewWatcher(s.client) - defer watcher.Close() - - for { - WatchChan: - watchChan := watcher.Watch(s.serverLoopCtx, serviceKey, clientv3.WithRev(revision)) - select { - case <-ctx.Done(): - return revision, nil - case <-s.updateServicePrimaryAddrCh: - revision, err = s.updateServicePrimaryAddr(serviceName) - if err != nil { - log.Warn("update service primary addr failed", zap.String("service-key", serviceKey), zap.Error(err)) - } - goto WatchChan - case wresp := <-watchChan: - if wresp.CompactRevision != 0 { - log.Warn("required revision has been compacted, use the compact revision", - zap.Int64("required-revision", revision), - zap.Int64("compact-revision", wresp.CompactRevision)) - revision = wresp.CompactRevision - goto WatchChan - } - if wresp.Err() != nil { - log.Error("watcher is canceled with", - zap.Int64("revision", revision), - errs.ZapError(errs.ErrEtcdWatcherCancel, wresp.Err())) - return revision, wresp.Err() - } - for _, event := range wresp.Events { - switch event.Type { - case clientv3.EventTypePut: - primary := &tsopb.Participant{} - if err := proto.Unmarshal(event.Kv.Value, primary); err != nil { - log.Error("watch service primary addr failed", zap.String("service-key", serviceKey), zap.Error(err)) - } else { - listenUrls := primary.GetListenUrls() - if len(listenUrls) > 0 { - // listenUrls[0] is the primary service endpoint of the keyspace group - s.servicePrimaryMap.Store(serviceName, listenUrls[0]) - } else { - log.Warn("service primary addr doesn't exist", zap.String("service-key", serviceKey)) - } - } - case clientv3.EventTypeDelete: - log.Warn("service primary is deleted", zap.String("service-key", serviceKey)) - s.servicePrimaryMap.Delete(serviceName) - } - } - revision = wresp.Header.Revision + 1 - } - } -} - -// updateServicePrimaryAddr updates the primary address from etcd with get operation. -func (s *Server) updateServicePrimaryAddr(serviceName string) (nextRevision int64, err error) { - serviceKey := s.servicePrimaryKey(serviceName) - primary := &tsopb.Participant{} - ok, revision, err := etcdutil.GetProtoMsgWithModRev(s.client, serviceKey, primary) - listenUrls := primary.GetListenUrls() - if !ok || err != nil || len(listenUrls) == 0 { - return 0, err - } - // listenUrls[0] is the primary service endpoint of the keyspace group - s.servicePrimaryMap.Store(serviceName, listenUrls[0]) - log.Info("update service primary addr", zap.String("service-key", serviceKey), zap.String("primary-addr", listenUrls[0])) - return revision, nil -} - func (s *Server) servicePrimaryKey(serviceName string) string { return fmt.Sprintf("/ms/%d/%s/%s/%s", s.clusterID, serviceName, fmt.Sprintf("%05d", 0), "primary") } From eab3fb68e071a9bbf5aaf5be6013b78d5fff2186 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Thu, 27 Apr 2023 12:15:58 +0800 Subject: [PATCH 03/24] update Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil.go | 30 +++++++++------- server/grpc_service.go | 2 +- server/server.go | 64 ++++++++++++++++++++-------------- 3 files changed, 57 insertions(+), 39 deletions(-) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 2286758788e..5794e0d49ec 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -362,10 +362,13 @@ type LoopWatcher struct { key string name string forceLoadCh chan struct{} + putFn func(*mvccpb.KeyValue) error + deleteFn func(*mvccpb.KeyValue) error + opts []clientv3.OpOption } // NewLoopWatcher creates a new LoopWatcher. -func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Client, name, key string) *LoopWatcher { +func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Client, name, key string, putFn, deleteFn func(*mvccpb.KeyValue) error, opts ...clientv3.OpOption) *LoopWatcher { return &LoopWatcher{ ctx: ctx, client: client, @@ -373,6 +376,9 @@ func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Cl key: key, wg: wg, forceLoadCh: make(chan struct{}, 1), + putFn: putFn, + deleteFn: deleteFn, + opts: opts, } } @@ -383,7 +389,7 @@ const ( ) // StartWatchLoop starts a loop to watch the key. -func (lw *LoopWatcher) StartWatchLoop(putFn, deleteFn func(*mvccpb.KeyValue) error, opts ...clientv3.OpOption) { +func (lw *LoopWatcher) StartWatchLoop() { defer logutil.LogPanic() defer lw.wg.Done() ctx, cancel := context.WithCancel(lw.ctx) @@ -396,7 +402,7 @@ func (lw *LoopWatcher) StartWatchLoop(putFn, deleteFn func(*mvccpb.KeyValue) err ticker := time.NewTicker(loadRetryInterval) defer ticker.Stop() for i := 0; i < maxLoadRetryTimes; i++ { - revision, err = lw.load(putFn, opts...) + revision, err = lw.load() if err == nil { break } @@ -417,7 +423,7 @@ func (lw *LoopWatcher) StartWatchLoop(putFn, deleteFn func(*mvccpb.KeyValue) err return default: } - nextRevision, err := lw.watch(ctx, revision, putFn, deleteFn, opts...) + nextRevision, err := lw.watch(ctx, revision) if err != nil { log.Error("watcher canceled unexpectedly and a new watcher will start after a while for watch loop", zap.String("name", lw.name), @@ -431,18 +437,18 @@ func (lw *LoopWatcher) StartWatchLoop(putFn, deleteFn func(*mvccpb.KeyValue) err } } -func (lw *LoopWatcher) watch(ctx context.Context, revision int64, putFn, deleteFn func(*mvccpb.KeyValue) error, opts ...clientv3.OpOption) (nextRevision int64, err error) { +func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision int64, err error) { watcher := clientv3.NewWatcher(lw.client) defer watcher.Close() for { WatchChan: - watchChan := watcher.Watch(ctx, lw.key, append(opts, clientv3.WithRev(revision))...) + watchChan := watcher.Watch(ctx, lw.key, append(lw.opts, clientv3.WithRev(revision))...) select { case <-ctx.Done(): return revision, nil case <-lw.forceLoadCh: - revision, err = lw.load(putFn, opts...) + revision, err = lw.load() if err != nil { log.Warn("force load key failed in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) @@ -465,12 +471,12 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64, putFn, deleteF for _, event := range wresp.Events { switch event.Type { case clientv3.EventTypePut: - if err := putFn(event.Kv); err != nil { + if err := lw.putFn(event.Kv); err != nil { log.Error("put failed in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) } case clientv3.EventTypeDelete: - if err := deleteFn(event.Kv); err != nil { + if err := lw.deleteFn(event.Kv); err != nil { log.Error("delete failed in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) } @@ -481,15 +487,15 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64, putFn, deleteF } } -func (lw *LoopWatcher) load(putFn func(*mvccpb.KeyValue) error, opts ...clientv3.OpOption) (nextRevision int64, err error) { - resp, err := EtcdKVGet(lw.client, lw.key, opts...) +func (lw *LoopWatcher) load() (nextRevision int64, err error) { + resp, err := EtcdKVGet(lw.client, lw.key, lw.opts...) if err != nil { log.Error("load failed in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) return 0, err } for _, item := range resp.Kvs { - err = putFn(item) + err = lw.putFn(item) if err != nil { log.Error("put failed in watch loop when loading", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) continue diff --git a/server/grpc_service.go b/server/grpc_service.go index 25af29dbf18..f05ed8db3fc 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -224,7 +224,7 @@ func (s *GrpcServer) Tso(stream pdpb.PD_TsoServer) error { } tsoRequest := tsoutil.NewPDProtoRequest(forwardedHost, clientConn, request, stream) - s.tsoDispatcher.DispatchRequest(ctx, tsoRequest, tsoProtoFactory, doneCh, errCh, s.servicePrimaryLoopWatcher.ForceLoad) + s.tsoDispatcher.DispatchRequest(ctx, tsoRequest, tsoProtoFactory, doneCh, errCh, s.tsoPrimaryWatcher.ForceLoad) continue } diff --git a/server/server.go b/server/server.go index aa482d40654..f575c64e0ee 100644 --- a/server/server.go +++ b/server/server.go @@ -215,10 +215,10 @@ type Server struct { auditBackends []audit.Backend - registry *registry.ServiceRegistry - mode string - servicePrimaryMap sync.Map /* Store as map[string]string */ - servicePrimaryLoopWatcher *etcdutil.LoopWatcher + registry *registry.ServiceRegistry + mode string + servicePrimaryMap sync.Map /* Store as map[string]string */ + tsoPrimaryWatcher *etcdutil.LoopWatcher } // HandlerBuilder builds a server HTTP handler. @@ -565,29 +565,10 @@ func (s *Server) startServerLoop(ctx context.Context) { go s.etcdLeaderLoop() go s.serverMetricsLoop() go s.encryptionKeyManagerLoop() - if s.IsAPIServiceMode() { // disable tso service in api server - serviceName := mcs.TSOServiceName - tsoServicePrimaryKey := s.servicePrimaryKey(serviceName) + if s.IsAPIServiceMode() { + s.initTSOPrimaryWatcher() s.serverLoopWg.Add(1) - s.servicePrimaryLoopWatcher = etcdutil.NewLoopWatcher(s.serverLoopCtx, &s.serverLoopWg, s.client, "service-primary", tsoServicePrimaryKey) - putFn := func(kv *mvccpb.KeyValue) error { - primary := &tsopb.Participant{} // TODO: use Generics - if err := proto.Unmarshal(kv.Value, primary); err != nil { - return err - } else { - listenUrls := primary.GetListenUrls() - if len(listenUrls) > 0 { - // listenUrls[0] is the primary service endpoint of the keyspace group - s.servicePrimaryMap.Store(serviceName, listenUrls[0]) - } - } - return nil - } - deleteFn := func(kv *mvccpb.KeyValue) error { - s.servicePrimaryMap.Delete(serviceName) - return nil - } - go s.servicePrimaryLoopWatcher.StartWatchLoop(putFn, deleteFn) + go s.tsoPrimaryWatcher.StartWatchLoop() } } @@ -1745,6 +1726,37 @@ func (s *Server) servicePrimaryKey(serviceName string) string { return fmt.Sprintf("/ms/%d/%s/%s/%s", s.clusterID, serviceName, fmt.Sprintf("%05d", 0), "primary") } +func (s *Server) initTSOPrimaryWatcher() { + serviceName := mcs.TSOServiceName + tsoServicePrimaryKey := s.servicePrimaryKey(serviceName) + putFn := func(kv *mvccpb.KeyValue) error { + primary := &tsopb.Participant{} // TODO: use Generics + if err := proto.Unmarshal(kv.Value, primary); err != nil { + return err + } else { + listenUrls := primary.GetListenUrls() + if len(listenUrls) > 0 { + // listenUrls[0] is the primary service endpoint of the keyspace group + s.servicePrimaryMap.Store(serviceName, listenUrls[0]) + } + } + return nil + } + deleteFn := func(kv *mvccpb.KeyValue) error { + s.servicePrimaryMap.Delete(serviceName) + return nil + } + s.tsoPrimaryWatcher = etcdutil.NewLoopWatcher( + s.serverLoopCtx, + &s.serverLoopWg, + s.client, + "service-primary", + tsoServicePrimaryKey, + putFn, + deleteFn, + ) +} + // RecoverAllocID recover alloc id. set current base id to input id func (s *Server) RecoverAllocID(ctx context.Context, id uint64) error { return s.idAllocator.SetBase(id) From e9bd5f06e5ce7a0497fbef2ba2d90503a3b124c2 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Thu, 27 Apr 2023 12:29:38 +0800 Subject: [PATCH 04/24] fix test Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 5794e0d49ec..492f83132ca 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -505,6 +505,10 @@ func (lw *LoopWatcher) load() (nextRevision int64, err error) { } func (lw *LoopWatcher) ForceLoad() { + if lw == nil { + // avoid panic when lw is nil, it may be called in pd mode for follower forward. + return + } select { case lw.forceLoadCh <- struct{}{}: default: From 072c114579887da803d4106aef5e4b0ac45fbdd6 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Thu, 27 Apr 2023 12:45:53 +0800 Subject: [PATCH 05/24] fix Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil.go | 4 ---- pkg/utils/tsoutil/tso_dispatcher.go | 11 ++++++----- server/grpc_service.go | 2 +- server/server.go | 2 -- 4 files changed, 7 insertions(+), 12 deletions(-) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 492f83132ca..5794e0d49ec 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -505,10 +505,6 @@ func (lw *LoopWatcher) load() (nextRevision int64, err error) { } func (lw *LoopWatcher) ForceLoad() { - if lw == nil { - // avoid panic when lw is nil, it may be called in pd mode for follower forward. - return - } select { case lw.forceLoadCh <- struct{}{}: default: diff --git a/pkg/utils/tsoutil/tso_dispatcher.go b/pkg/utils/tsoutil/tso_dispatcher.go index 0af74864f3c..f43a9557171 100644 --- a/pkg/utils/tsoutil/tso_dispatcher.go +++ b/pkg/utils/tsoutil/tso_dispatcher.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/logutil" "go.uber.org/zap" "google.golang.org/grpc" @@ -64,12 +65,12 @@ func (s *TSODispatcher) DispatchRequest( tsoProtoFactory ProtoFactory, doneCh <-chan struct{}, errCh chan<- error, - callbacks ...func()) { + tsoPrimaryWatchers ...*etcdutil.LoopWatcher) { val, loaded := s.dispatchChs.LoadOrStore(req.getForwardedHost(), make(chan Request, maxMergeRequests)) reqCh := val.(chan Request) if !loaded { tsDeadlineCh := make(chan deadline, 1) - go s.dispatch(ctx, tsoProtoFactory, req.getForwardedHost(), req.getClientConn(), reqCh, tsDeadlineCh, doneCh, errCh, callbacks...) + go s.dispatch(ctx, tsoProtoFactory, req.getForwardedHost(), req.getClientConn(), reqCh, tsDeadlineCh, doneCh, errCh, tsoPrimaryWatchers...) go watchTSDeadline(ctx, tsDeadlineCh) } reqCh <- req @@ -84,7 +85,7 @@ func (s *TSODispatcher) dispatch( tsDeadlineCh chan<- deadline, doneCh <-chan struct{}, errCh chan<- error, - callbacks ...func()) { + tsoPrimaryWatchers ...*etcdutil.LoopWatcher) { defer logutil.LogPanic() dispatcherCtx, ctxCancel := context.WithCancel(ctx) defer ctxCancel() @@ -111,7 +112,7 @@ func (s *TSODispatcher) dispatch( defer cancel() requests := make([]Request, maxMergeRequests+1) - needUpdateServicePrimaryAddr := len(callbacks) > 0 && callbacks[0] != nil + needUpdateServicePrimaryAddr := len(tsoPrimaryWatchers) > 0 && tsoPrimaryWatchers[0] != nil for { select { case first := <-tsoRequestCh: @@ -139,7 +140,7 @@ func (s *TSODispatcher) dispatch( errs.ZapError(errs.ErrGRPCSend, err)) if needUpdateServicePrimaryAddr { if strings.Contains(err.Error(), errs.NotLeaderErr) || strings.Contains(err.Error(), errs.MismatchLeaderErr) { - callbacks[0]() + tsoPrimaryWatchers[0].ForceLoad() } } select { diff --git a/server/grpc_service.go b/server/grpc_service.go index f05ed8db3fc..967c3dcda49 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -224,7 +224,7 @@ func (s *GrpcServer) Tso(stream pdpb.PD_TsoServer) error { } tsoRequest := tsoutil.NewPDProtoRequest(forwardedHost, clientConn, request, stream) - s.tsoDispatcher.DispatchRequest(ctx, tsoRequest, tsoProtoFactory, doneCh, errCh, s.tsoPrimaryWatcher.ForceLoad) + s.tsoDispatcher.DispatchRequest(ctx, tsoRequest, tsoProtoFactory, doneCh, errCh, s.tsoPrimaryWatcher) continue } diff --git a/server/server.go b/server/server.go index f575c64e0ee..27deab0e52f 100644 --- a/server/server.go +++ b/server/server.go @@ -105,8 +105,6 @@ const ( maxRetryTimesGetServicePrimary = 25 // retryIntervalGetServicePrimary is the retry interval for getting primary addr. retryIntervalGetServicePrimary = 100 * time.Millisecond - // TODO: move it to etcdutil - watchEtcdChangeRetryInterval = 1 * time.Second ) // EtcdStartTimeout the timeout of the startup etcd. From 9cf42dedb2c25d564182ce19a0592f91bb875434 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Thu, 27 Apr 2023 13:07:28 +0800 Subject: [PATCH 06/24] update Signed-off-by: lhy1024 --- pkg/keyspace/tso_keyspace_group.go | 162 +++++++---------------------- pkg/utils/etcdutil/etcdutil.go | 6 +- server/server.go | 13 ++- 3 files changed, 47 insertions(+), 134 deletions(-) diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index 36b45f885d6..38572f6ff87 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -32,6 +32,7 @@ import ( "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/logutil" "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" ) @@ -40,10 +41,6 @@ const ( allocNodesToKeyspaceGroupsInterval = 1 * time.Second allocNodesTimeout = 1 * time.Second allocNodesInterval = 10 * time.Millisecond - // TODO: move it to etcdutil - watchEtcdChangeRetryInterval = 1 * time.Second - maxRetryTimes = 25 - retryInterval = 100 * time.Millisecond ) const ( @@ -71,18 +68,14 @@ type GroupManager struct { endpoint.KeyspaceStorage } - client *clientv3.Client - - // tsoServiceKey is the path of TSO service in etcd. - tsoServiceKey string - // tsoServiceEndKey is the end key of TSO service in etcd. - tsoServiceEndKey string - + // nodeBalancer is the balancer for tso nodes. // TODO: add user kind with different balancer // when we ensure where the correspondence between tso node and user kind will be found nodesBalancer balancer.Balancer[string] // serviceRegistryMap stores the mapping from the service registry key to the service address. serviceRegistryMap map[string]string + // tsoNodesWatcher is the watcher for the registered tso servers. + tsoNodesWatcher *etcdutil.LoopWatcher } // NewKeyspaceGroupManager creates a Manager of keyspace group related data. @@ -96,7 +89,6 @@ func NewKeyspaceGroupManager( clusterID uint64, ) *GroupManager { ctx, cancel := context.WithCancel(ctx) - key := discovery.TSOPath(clusterID) groups := make(map[endpoint.UserKind]*indexedHeap) for i := 0; i < int(endpoint.UserKindCount); i++ { groups[endpoint.UserKind(i)] = newIndexedHeap(int(utils.MaxKeyspaceGroupCountInUse)) @@ -105,9 +97,6 @@ func NewKeyspaceGroupManager( ctx: ctx, cancel: cancel, store: store, - client: client, - tsoServiceKey: key, - tsoServiceEndKey: clientv3.GetPrefixRangeEnd(key) + "/", groups: groups, nodesBalancer: balancer.GenByPolicy[string](defaultBalancerPolicy), serviceRegistryMap: make(map[string]string), @@ -115,10 +104,11 @@ func NewKeyspaceGroupManager( // If the etcd client is not nil, start the watch loop for the registered tso servers. // The PD(TSO) Client relies on this info to discover tso servers. - if m.client != nil { - log.Info("start the watch loop for tso service discovery") - m.wg.Add(1) - go m.startWatchLoop(ctx) + if client != nil { + m.wg.Add(2) + m.initTSONodesWatcher(client, clusterID) + go m.tsoNodesWatcher.StartWatchLoop() + go m.allocNodesToAllKeyspaceGroups() } return m @@ -139,12 +129,6 @@ func (m *GroupManager) Bootstrap() error { m.Lock() defer m.Unlock() - // If the etcd client is not nil, start the watch loop. - if m.client != nil { - m.wg.Add(1) - go m.allocNodesToAllKeyspaceGroups() - } - // Ignore the error if default keyspace group already exists in the storage (e.g. PD restart/recover). err := m.saveKeyspaceGroups([]*endpoint.KeyspaceGroup{defaultKeyspaceGroup}, false) if err != nil && err != ErrKeyspaceGroupExists { @@ -241,109 +225,41 @@ func (m *GroupManager) allocNodesToAllKeyspaceGroups() { } } -func (m *GroupManager) startWatchLoop(parentCtx context.Context) { - defer logutil.LogPanic() - defer m.wg.Done() - ctx, cancel := context.WithCancel(parentCtx) - defer cancel() - var ( - resp *clientv3.GetResponse - revision int64 - err error - ) - ticker := time.NewTicker(retryInterval) - defer ticker.Stop() - for i := 0; i < maxRetryTimes; i++ { - resp, err = etcdutil.EtcdKVGet(m.client, m.tsoServiceKey, clientv3.WithRange(m.tsoServiceEndKey)) - if err == nil { - revision = resp.Header.Revision + 1 - for _, item := range resp.Kvs { - s := &discovery.ServiceRegistryEntry{} - if err := json.Unmarshal(item.Value, s); err != nil { - log.Warn("failed to unmarshal service registry entry", zap.Error(err)) - continue - } - m.nodesBalancer.Put(s.ServiceAddr) - m.serviceRegistryMap[string(item.Key)] = s.ServiceAddr - } - break - } - log.Warn("failed to get tso service addrs from etcd and will retry", zap.Error(err)) - select { - case <-m.ctx.Done(): - return - case <-ticker.C: +func (m *GroupManager) initTSONodesWatcher(client *clientv3.Client, clusterID uint64) { + tsoServiceKey := discovery.TSOPath(clusterID) + tsoServiceEndKey := clientv3.GetPrefixRangeEnd(tsoServiceKey) + "/" + + putFn := func(kv *mvccpb.KeyValue) error { + s := &discovery.ServiceRegistryEntry{} + if err := json.Unmarshal(kv.Value, s); err != nil { + log.Warn("failed to unmarshal service registry entry", + zap.String("event-kv-key", string(kv.Key)), zap.Error(err)) + return err } + m.nodesBalancer.Put(s.ServiceAddr) + m.serviceRegistryMap[string(kv.Key)] = s.ServiceAddr + return nil } - if err != nil || revision == 0 { - log.Warn("failed to get tso service addrs from etcd finally when loading", zap.Error(err)) - } - for { - select { - case <-ctx.Done(): - return - default: - } - nextRevision, err := m.watchServiceAddrs(ctx, revision) - if err != nil { - log.Error("watcher canceled unexpectedly and a new watcher will start after a while", - zap.Int64("next-revision", nextRevision), - zap.Time("retry-at", time.Now().Add(watchEtcdChangeRetryInterval)), - zap.Error(err)) - revision = nextRevision - time.Sleep(watchEtcdChangeRetryInterval) + deleteFn := func(kv *mvccpb.KeyValue) error { + key := string(kv.Key) + if serviceAddr, ok := m.serviceRegistryMap[key]; ok { + delete(m.serviceRegistryMap, key) + m.nodesBalancer.Delete(serviceAddr) + return nil } + return errors.Errorf("failed to find the service address for key %s", key) } -} -func (m *GroupManager) watchServiceAddrs(ctx context.Context, revision int64) (int64, error) { - watcher := clientv3.NewWatcher(m.client) - defer watcher.Close() - for { - WatchChan: - watchChan := watcher.Watch(ctx, m.tsoServiceKey, clientv3.WithRange(m.tsoServiceEndKey), clientv3.WithRev(revision)) - select { - case <-ctx.Done(): - return revision, nil - case wresp := <-watchChan: - if wresp.CompactRevision != 0 { - log.Warn("required revision has been compacted, the watcher will watch again with the compact revision", - zap.Int64("required-revision", revision), - zap.Int64("compact-revision", wresp.CompactRevision)) - revision = wresp.CompactRevision - goto WatchChan - } - if wresp.Err() != nil { - log.Error("watch is canceled or closed", - zap.Int64("required-revision", revision), - zap.Error(wresp.Err())) - return revision, wresp.Err() - } - for _, event := range wresp.Events { - switch event.Type { - case clientv3.EventTypePut: - s := &discovery.ServiceRegistryEntry{} - if err := json.Unmarshal(event.Kv.Value, s); err != nil { - log.Warn("failed to unmarshal service registry entry", - zap.String("event-kv-key", string(event.Kv.Key)), zap.Error(err)) - break - } - m.nodesBalancer.Put(s.ServiceAddr) - m.serviceRegistryMap[string(event.Kv.Key)] = s.ServiceAddr - case clientv3.EventTypeDelete: - key := string(event.Kv.Key) - if serviceAddr, ok := m.serviceRegistryMap[key]; ok { - delete(m.serviceRegistryMap, key) - m.nodesBalancer.Delete(serviceAddr) - } else { - log.Warn("can't retrieve service addr from service registry map", - zap.String("event-kv-key", key)) - } - } - } - revision = wresp.Header.Revision + 1 - } - } + m.tsoNodesWatcher = etcdutil.NewLoopWatcher( + m.ctx, + &m.wg, + client, + "tso-nodes-watcher", + tsoServiceKey, + putFn, + deleteFn, + clientv3.WithRange(tsoServiceEndKey), + ) } // CreateKeyspaceGroups creates keyspace groups. diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 5794e0d49ec..64484ebf647 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -419,6 +419,7 @@ func (lw *LoopWatcher) StartWatchLoop() { for { select { case <-ctx.Done(): + close(lw.forceLoadCh) log.Info("server is closed, exit watch loop", zap.String("name", lw.name), zap.String("key", lw.key)) return default: @@ -504,13 +505,10 @@ func (lw *LoopWatcher) load() (nextRevision int64, err error) { return resp.Header.Revision + 1, err } +// ForceLoad forces to load the key. func (lw *LoopWatcher) ForceLoad() { select { case lw.forceLoadCh <- struct{}{}: default: } } - -func (lw *LoopWatcher) Close() { - close(lw.forceLoadCh) -} diff --git a/server/server.go b/server/server.go index 27deab0e52f..60836be8a52 100644 --- a/server/server.go +++ b/server/server.go @@ -1731,12 +1731,11 @@ func (s *Server) initTSOPrimaryWatcher() { primary := &tsopb.Participant{} // TODO: use Generics if err := proto.Unmarshal(kv.Value, primary); err != nil { return err - } else { - listenUrls := primary.GetListenUrls() - if len(listenUrls) > 0 { - // listenUrls[0] is the primary service endpoint of the keyspace group - s.servicePrimaryMap.Store(serviceName, listenUrls[0]) - } + } + listenUrls := primary.GetListenUrls() + if len(listenUrls) > 0 { + // listenUrls[0] is the primary service endpoint of the keyspace group + s.servicePrimaryMap.Store(serviceName, listenUrls[0]) } return nil } @@ -1748,7 +1747,7 @@ func (s *Server) initTSOPrimaryWatcher() { s.serverLoopCtx, &s.serverLoopWg, s.client, - "service-primary", + "tso-primary-watcher", tsoServicePrimaryKey, putFn, deleteFn, From 5f4ac40888db696c4b0e4f160984b6a7b8e865be Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Thu, 27 Apr 2023 20:44:19 +0800 Subject: [PATCH 07/24] update Signed-off-by: lhy1024 --- pkg/tso/keyspace_group_manager.go | 303 +++++-------------------- pkg/tso/keyspace_group_manager_test.go | 78 +------ pkg/utils/etcdutil/etcdutil.go | 92 +++++--- 3 files changed, 124 insertions(+), 349 deletions(-) diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 4c9e8dac4ca..1d8b49df193 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -17,7 +17,6 @@ package tso import ( "context" "encoding/json" - "errors" "fmt" "net/http" "path" @@ -27,7 +26,6 @@ import ( "time" perrors "github.com/pingcap/errors" - "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/election" @@ -43,19 +41,14 @@ import ( "github.com/tikv/pd/pkg/utils/memberutil" "github.com/tikv/pd/pkg/utils/tsoutil" "go.etcd.io/etcd/clientv3" + "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" ) const ( // primaryElectionSuffix is the suffix of the key for keyspace group primary election primaryElectionSuffix = "primary" - // defaultLoadKeyspaceGroupsTimeout is the default timeout for loading the initial - // keyspace group assignment - defaultLoadKeyspaceGroupsTimeout = 30 * time.Second - defaultLoadKeyspaceGroupsBatchSize = int64(400) - defaultLoadFromEtcdRetryInterval = 500 * time.Millisecond - defaultLoadFromEtcdMaxRetryTimes = int(defaultLoadKeyspaceGroupsTimeout / defaultLoadFromEtcdRetryInterval) - watchEtcdChangeRetryInterval = 1 * time.Second + defaultRetryInterval = 500 * time.Millisecond ) type state struct { @@ -205,13 +198,11 @@ type KeyspaceGroupManager struct { tsoSvcStorage *endpoint.StorageEndpoint // cfg is the TSO config cfg ServiceConfig - // loadKeyspaceGroupsTimeout is the timeout for loading the initial keyspace group assignment. - loadKeyspaceGroupsTimeout time.Duration - loadKeyspaceGroupsBatchSize int64 - loadFromEtcdMaxRetryTimes int // groupUpdateRetryList is the list of keyspace groups which failed to update and need to retry. groupUpdateRetryList map[uint32]*endpoint.KeyspaceGroup + + groupWatcher *etcdutil.LoopWatcher } // NewKeyspaceGroupManager creates a new Keyspace Group Manager. @@ -233,19 +224,16 @@ func NewKeyspaceGroupManager( ctx, cancel := context.WithCancel(ctx) kgm := &KeyspaceGroupManager{ - ctx: ctx, - cancel: cancel, - tsoServiceID: tsoServiceID, - etcdClient: etcdClient, - httpClient: httpClient, - electionNamePrefix: electionNamePrefix, - legacySvcRootPath: legacySvcRootPath, - tsoSvcRootPath: tsoSvcRootPath, - cfg: cfg, - loadKeyspaceGroupsTimeout: defaultLoadKeyspaceGroupsTimeout, - loadKeyspaceGroupsBatchSize: defaultLoadKeyspaceGroupsBatchSize, - loadFromEtcdMaxRetryTimes: defaultLoadFromEtcdMaxRetryTimes, - groupUpdateRetryList: make(map[uint32]*endpoint.KeyspaceGroup), + ctx: ctx, + cancel: cancel, + tsoServiceID: tsoServiceID, + etcdClient: etcdClient, + httpClient: httpClient, + electionNamePrefix: electionNamePrefix, + legacySvcRootPath: legacySvcRootPath, + tsoSvcRootPath: tsoSvcRootPath, + cfg: cfg, + groupUpdateRetryList: make(map[uint32]*endpoint.KeyspaceGroup), } kgm.legacySvcStorage = endpoint.NewStorageEndpoint( kv.NewEtcdKVBase(kgm.etcdClient, kgm.legacySvcRootPath), nil) @@ -257,20 +245,55 @@ func NewKeyspaceGroupManager( // Initialize this KeyspaceGroupManager func (kgm *KeyspaceGroupManager) Initialize() error { - // Load the initial keyspace group assignment from storage with time limit - done := make(chan struct{}, 1) - ctx, cancel := context.WithCancel(kgm.ctx) - go kgm.checkInitProgress(ctx, cancel, done) - watchStartRevision, defaultKGConfigured, err := kgm.initAssignment(ctx) - done <- struct{}{} - if err != nil { + rootPath := kgm.legacySvcRootPath + startKey := strings.Join([]string{rootPath, endpoint.KeyspaceGroupIDPath(mcsutils.DefaultKeyspaceGroupID)}, "/") + endKey := strings.Join( + []string{rootPath, clientv3.GetPrefixRangeEnd(endpoint.KeyspaceGroupIDPrefix())}, "/") + + defaultKGConfigured := false + putFn := func(kv *mvccpb.KeyValue) error { + group := &endpoint.KeyspaceGroup{} + if err := json.Unmarshal(kv.Value, group); err != nil { + return errs.ErrJSONUnmarshal.Wrap(err).FastGenWithCause() + } + kgm.updateKeyspaceGroup(group) + if group.ID == mcsutils.DefaultKeyspaceGroupID { + defaultKGConfigured = true + } + return nil + } + deleteFn := func(kv *mvccpb.KeyValue) error { + groupID, err := endpoint.ExtractKeyspaceGroupIDFromPath(string(kv.Key)) + if err != nil { + return err + } + kgm.deleteKeyspaceGroup(groupID) + return nil + } + kgm.groupWatcher = etcdutil.NewLoopWatcher( + kgm.ctx, + &kgm.wg, + kgm.etcdClient, + "keyspace-watcher", + startKey, + putFn, + deleteFn, + clientv3.WithRange(endKey), + ) + + kgm.wg.Add(1) + go kgm.groupWatcher.StartWatchLoop() + + if err := kgm.groupWatcher.WaitLoad(); err != nil { log.Error("failed to initialize keyspace group manager", errs.ZapError(err)) // We might have partially loaded/initialized the keyspace groups. Close the manager to clean up. kgm.Close() return err } - // Initialize the default keyspace group if it isn't configured in the storage. + kgm.wg.Add(1) + go kgm.startRetryUpdateLoop() + if !defaultKGConfigured { log.Info("initializing default keyspace group") group := &endpoint.KeyspaceGroup{ @@ -280,12 +303,6 @@ func (kgm *KeyspaceGroupManager) Initialize() error { } kgm.updateKeyspaceGroup(group) } - - // Watch/apply keyspace group membership/distribution meta changes dynamically. - kgm.wg.Add(1) - go kgm.startKeyspaceGroupsMetaWatchLoop(watchStartRevision) - - log.Info("keyspace group manager initialized") return nil } @@ -305,218 +322,20 @@ func (kgm *KeyspaceGroupManager) Close() { log.Info("keyspace group manager closed") } -func (kgm *KeyspaceGroupManager) checkInitProgress(ctx context.Context, cancel context.CancelFunc, done chan struct{}) { - defer logutil.LogPanic() - - select { - case <-done: - return - case <-time.After(kgm.loadKeyspaceGroupsTimeout): - log.Error("failed to initialize keyspace group manager", - zap.Any("timeout-setting", kgm.loadKeyspaceGroupsTimeout), - errs.ZapError(errs.ErrLoadKeyspaceGroupsTimeout)) - cancel() - case <-ctx.Done(): - } - <-done -} - -// initAssignment loads initial keyspace group assignment from storage and initialize the group manager. -// Return watchStartRevision, the start revision for watching keyspace group membership/distribution change. -func (kgm *KeyspaceGroupManager) initAssignment( - ctx context.Context, -) (watchStartRevision int64, defaultKGConfigured bool, err error) { - var ( - groups []*endpoint.KeyspaceGroup - more bool - keyspaceGroupsLoaded uint32 - revision int64 - ) - - // Load all keyspace groups from etcd and apply the ones assigned to this tso service. - for { - revision, groups, more, err = kgm.loadKeyspaceGroups(ctx, keyspaceGroupsLoaded, kgm.loadKeyspaceGroupsBatchSize) - if err != nil { - return - } - - keyspaceGroupsLoaded += uint32(len(groups)) - - if watchStartRevision == 0 || revision < watchStartRevision { - watchStartRevision = revision - } - - // Update the keyspace groups - for _, group := range groups { - select { - case <-ctx.Done(): - err = errs.ErrLoadKeyspaceGroupsTerminated - return - default: - } - - if group.ID == mcsutils.DefaultKeyspaceGroupID { - defaultKGConfigured = true - } - - kgm.updateKeyspaceGroup(group) - } - - if !more { - break - } - } - - log.Info("loaded keyspace groups", zap.Uint32("keyspace-groups-loaded", keyspaceGroupsLoaded)) - return -} - -// loadKeyspaceGroups loads keyspace groups from the start ID with limit. -// If limit is 0, it will load all keyspace groups from the start ID. -func (kgm *KeyspaceGroupManager) loadKeyspaceGroups( - ctx context.Context, startID uint32, limit int64, -) (revision int64, ksgs []*endpoint.KeyspaceGroup, more bool, err error) { - rootPath := kgm.legacySvcRootPath - startKey := strings.Join([]string{rootPath, endpoint.KeyspaceGroupIDPath(startID)}, "/") - endKey := strings.Join( - []string{rootPath, clientv3.GetPrefixRangeEnd(endpoint.KeyspaceGroupIDPrefix())}, "/") - opOption := []clientv3.OpOption{clientv3.WithRange(endKey), clientv3.WithLimit(limit)} - - var ( - i int - resp *clientv3.GetResponse - ) - for ; i < kgm.loadFromEtcdMaxRetryTimes; i++ { - resp, err = etcdutil.EtcdKVGet(kgm.etcdClient, startKey, opOption...) - - failpoint.Inject("delayLoadKeyspaceGroups", func(val failpoint.Value) { - if sleepIntervalSeconds, ok := val.(int); ok && sleepIntervalSeconds > 0 { - time.Sleep(time.Duration(sleepIntervalSeconds) * time.Second) - } - }) - - failpoint.Inject("loadKeyspaceGroupsTemporaryFail", func(val failpoint.Value) { - if maxFailTimes, ok := val.(int); ok && i < maxFailTimes { - err = errors.New("fail to read from etcd") - failpoint.Continue() - } - }) - - if err == nil && resp != nil { - break - } - - select { - case <-ctx.Done(): - return 0, []*endpoint.KeyspaceGroup{}, false, errs.ErrLoadKeyspaceGroupsTerminated - case <-time.After(defaultLoadFromEtcdRetryInterval): - } - } - - if i == kgm.loadFromEtcdMaxRetryTimes { - return 0, []*endpoint.KeyspaceGroup{}, false, errs.ErrLoadKeyspaceGroupsRetryExhausted.FastGenByArgs(err) - } - - kgs := make([]*endpoint.KeyspaceGroup, 0, len(resp.Kvs)) - for _, item := range resp.Kvs { - kg := &endpoint.KeyspaceGroup{} - if err = json.Unmarshal(item.Value, kg); err != nil { - return 0, nil, false, err - } - kgs = append(kgs, kg) - } - - if resp.Header != nil { - revision = resp.Header.Revision + 1 - } - - return revision, kgs, resp.More, nil -} - -// startKeyspaceGroupsMetaWatchLoop repeatedly watches any change in keyspace group membership/distribution -// and apply the change dynamically. -func (kgm *KeyspaceGroupManager) startKeyspaceGroupsMetaWatchLoop(revision int64) { +func (kgm *KeyspaceGroupManager) startRetryUpdateLoop() { defer logutil.LogPanic() defer kgm.wg.Done() - - // Repeatedly watch/apply keyspace group membership/distribution changes until the context is canceled. + ticker := time.NewTicker(defaultRetryInterval) + defer ticker.Stop() for { select { case <-kgm.ctx.Done(): return - default: - } - - nextRevision, err := kgm.watchKeyspaceGroupsMetaChange(revision) - if err != nil { - log.Error("watcher canceled unexpectedly and a new watcher will start after a while", - zap.Int64("next-revision", nextRevision), - zap.Time("retry-at", time.Now().Add(watchEtcdChangeRetryInterval)), - zap.Error(err)) - time.Sleep(watchEtcdChangeRetryInterval) - } - } -} - -// watchKeyspaceGroupsMetaChange watches any change in keyspace group membership/distribution -// and apply the change dynamically. -func (kgm *KeyspaceGroupManager) watchKeyspaceGroupsMetaChange(revision int64) (int64, error) { - watcher := clientv3.NewWatcher(kgm.etcdClient) - defer watcher.Close() - - ksgPrefix := strings.Join([]string{kgm.legacySvcRootPath, endpoint.KeyspaceGroupIDPrefix()}, "/") - log.Info("start to watch keyspace group meta change", zap.Int64("revision", revision), zap.String("prefix", ksgPrefix)) - - for { - watchChan := watcher.Watch(kgm.ctx, ksgPrefix, clientv3.WithPrefix(), clientv3.WithRev(revision)) - for wresp := range watchChan { - if wresp.CompactRevision != 0 { - log.Warn("Required revision has been compacted, the watcher will watch again with the compact revision", - zap.Int64("required-revision", revision), - zap.Int64("compact-revision", wresp.CompactRevision)) - revision = wresp.CompactRevision - break - } - if wresp.Err() != nil { - log.Error("watch is canceled or closed", - zap.Int64("required-revision", revision), - errs.ZapError(errs.ErrEtcdWatcherCancel, wresp.Err())) - return revision, wresp.Err() - } - for _, event := range wresp.Events { - groupID, err := endpoint.ExtractKeyspaceGroupIDFromPath(string(event.Kv.Key)) - if err != nil { - log.Warn("failed to extract keyspace group ID from the key path", - zap.String("key-path", string(event.Kv.Key)), zap.Error(err)) - continue - } - - switch event.Type { - case clientv3.EventTypePut: - group := &endpoint.KeyspaceGroup{} - if err := json.Unmarshal(event.Kv.Value, group); err != nil { - log.Warn("failed to unmarshal keyspace group", - zap.Uint32("keyspace-group-id", groupID), - zap.Error(errs.ErrJSONUnmarshal.Wrap(err).FastGenWithCause())) - break - } - kgm.updateKeyspaceGroup(group) - case clientv3.EventTypeDelete: - kgm.deleteKeyspaceGroup(groupID) - } - } - // Retry the groups that are not initialized successfully before. + case <-ticker.C: for id, group := range kgm.groupUpdateRetryList { delete(kgm.groupUpdateRetryList, id) kgm.updateKeyspaceGroup(group) } - revision = wresp.Header.Revision + 1 - } - - select { - case <-kgm.ctx.Done(): - return revision, nil - default: } } } diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index d6e8cb4b046..3c15dc38f09 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -29,10 +29,8 @@ import ( "time" "github.com/google/uuid" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/storage/endpoint" @@ -106,8 +104,6 @@ func (suite *keyspaceGroupManagerTestSuite) TestNewKeyspaceGroupManager() { re.Equal(legacySvcRootPath, kgm.legacySvcRootPath) re.Equal(tsoSvcRootPath, kgm.tsoSvcRootPath) re.Equal(suite.cfg, kgm.cfg) - re.Equal(defaultLoadKeyspaceGroupsBatchSize, kgm.loadKeyspaceGroupsBatchSize) - re.Equal(defaultLoadKeyspaceGroupsTimeout, kgm.loadKeyspaceGroupsTimeout) am, err := kgm.GetAllocatorManager(mcsutils.DefaultKeyspaceGroupID) re.NoError(err) @@ -166,72 +162,7 @@ func (suite *keyspaceGroupManagerTestSuite) TestLoadWithDifferentBatchSize() { } } -// TestLoadKeyspaceGroupsTimeout tests there is timeout when loading the initial keyspace group assignment -// from etcd. The initialization of the keyspace group manager should fail. -func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsTimeout() { - re := suite.Require() - - mgr := suite.newUniqueKeyspaceGroupManager(1) - re.NotNil(mgr) - defer mgr.Close() - - addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, uint32(0), []uint32{0}) - - // Set the timeout to 1 second and inject the delayLoadKeyspaceGroups to return 3 seconds to let - // the loading sleep 3 seconds. - mgr.loadKeyspaceGroupsTimeout = time.Second - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/delayLoadKeyspaceGroups", "return(3)")) - err := mgr.Initialize() - // If loading keyspace groups timeout, the initialization should fail with ErrLoadKeyspaceGroupsTerminated. - re.Equal(errs.ErrLoadKeyspaceGroupsTerminated, err) - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/delayLoadKeyspaceGroups")) -} - -// TestLoadKeyspaceGroupsSucceedWithTempFailures tests the initialization should succeed when there are temporary -// failures during loading the initial keyspace group assignment from etcd. -func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsSucceedWithTempFailures() { - re := suite.Require() - - mgr := suite.newUniqueKeyspaceGroupManager(1) - re.NotNil(mgr) - defer mgr.Close() - - addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, uint32(0), []uint32{0}) - - // Set the max retry times to 3 and inject the loadKeyspaceGroupsTemporaryFail to return 2 to let - // loading from etcd fail 2 times but the whole initialization still succeeds. - mgr.loadFromEtcdMaxRetryTimes = 3 - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/loadKeyspaceGroupsTemporaryFail", "return(2)")) - err := mgr.Initialize() - re.NoError(err) - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/loadKeyspaceGroupsTemporaryFail")) -} - -// TestLoadKeyspaceGroupsFailed tests the initialization should fail when there are too many failures -// during loading the initial keyspace group assignment from etcd. -func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsFailed() { - re := suite.Require() - - mgr := suite.newUniqueKeyspaceGroupManager(1) - re.NotNil(mgr) - defer mgr.Close() - - addKeyspaceGroupAssignment( - suite.ctx, suite.etcdClient, true, - mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, uint32(0), []uint32{0}) - - // Set the max retry times to 3 and inject the loadKeyspaceGroupsTemporaryFail to return 3 to let - // loading from etcd fail 3 times which should cause the whole initialization to fail. - mgr.loadFromEtcdMaxRetryTimes = 3 - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/loadKeyspaceGroupsTemporaryFail", "return(3)")) - err := mgr.Initialize() - re.Error(err) - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/loadKeyspaceGroupsTemporaryFail")) -} +// TODO: move test // TestWatchAndDynamicallyApplyChanges tests the keyspace group manager watch and dynamically apply // keyspace groups' membership/distribution meta changes. @@ -668,10 +599,9 @@ func (suite *keyspaceGroupManagerTestSuite) newUniqueKeyspaceGroupManager( electionNamePrefix := "kgm-test-" + uniqueStr keyspaceGroupManager := suite.newKeyspaceGroupManager(tsoServiceID, electionNamePrefix, legacySvcRootPath, tsoSvcRootPath) - - if loadKeyspaceGroupsBatchSize != 0 { - keyspaceGroupManager.loadKeyspaceGroupsBatchSize = loadKeyspaceGroupsBatchSize - } + // if loadKeyspaceGroupsBatchSize != 0 { + // keyspaceGroupManager.loadKeyspaceGroupsBatchSize = loadKeyspaceGroupsBatchSize + // } return keyspaceGroupManager } diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 64484ebf647..c013d5ae6f5 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -354,6 +354,13 @@ func InitOrGetClusterID(c *clientv3.Client, key string) (uint64, error) { return typeutil.BytesToUint64(response.Kvs[0].Value) } +const ( + defaultLoadDataTimeout = 30 * time.Second + defaultLoadFromEtcdRetryInterval = 200 * time.Millisecond + defaultLoadFromEtcdMaxRetryTimes = int(defaultLoadDataTimeout / defaultLoadFromEtcdRetryInterval) + watchEtcdChangeRetryInterval = 1 * time.Second +) + // LoopWatcher loads data from etcd and sets a watcher for it. type LoopWatcher struct { ctx context.Context @@ -362,6 +369,7 @@ type LoopWatcher struct { key string name string forceLoadCh chan struct{} + isLoadedCh chan error putFn func(*mvccpb.KeyValue) error deleteFn func(*mvccpb.KeyValue) error opts []clientv3.OpOption @@ -376,55 +384,32 @@ func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Cl key: key, wg: wg, forceLoadCh: make(chan struct{}, 1), + isLoadedCh: make(chan error, 1), putFn: putFn, deleteFn: deleteFn, - opts: opts, + opts: opts, // todo: add default opts, e.g. clientv3.WithPrefix(), clientv3.Limit(512) } } -const ( - maxLoadRetryTimes = 30 - loadRetryInterval = time.Millisecond * 100 - watchEtcdChangeRetryInterval = time.Second -) - // StartWatchLoop starts a loop to watch the key. func (lw *LoopWatcher) StartWatchLoop() { defer logutil.LogPanic() defer lw.wg.Done() - ctx, cancel := context.WithCancel(lw.ctx) + + ctx, cancel := context.WithTimeout(lw.ctx, defaultLoadDataTimeout) defer cancel() + watchStartRevision := lw.initFromEtcd(ctx) - var ( - revision int64 - err error - ) - ticker := time.NewTicker(loadRetryInterval) - defer ticker.Stop() - for i := 0; i < maxLoadRetryTimes; i++ { - revision, err = lw.load() - if err == nil { - break - } - select { - case <-ctx.Done(): - return - case <-ticker.C: - } - } - if err != nil || revision == 0 { - log.Warn("watched key doesn't exist in watch loop when loading", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) - } log.Info("start to watch loop", zap.String("name", lw.name), zap.String("key", lw.key)) for { select { - case <-ctx.Done(): + case <-lw.ctx.Done(): close(lw.forceLoadCh) log.Info("server is closed, exit watch loop", zap.String("name", lw.name), zap.String("key", lw.key)) return default: } - nextRevision, err := lw.watch(ctx, revision) + nextRevision, err := lw.watch(lw.ctx, watchStartRevision) if err != nil { log.Error("watcher canceled unexpectedly and a new watcher will start after a while for watch loop", zap.String("name", lw.name), @@ -432,12 +417,44 @@ func (lw *LoopWatcher) StartWatchLoop() { zap.Int64("next-revision", nextRevision), zap.Time("retry-at", time.Now().Add(watchEtcdChangeRetryInterval)), zap.Error(err)) - revision = nextRevision + watchStartRevision = nextRevision time.Sleep(watchEtcdChangeRetryInterval) } } } +func (lw *LoopWatcher) initFromEtcd(ctx context.Context) int64 { + var ( + watchStartRevision int64 + err error + ) + ticker := time.NewTicker(defaultLoadFromEtcdRetryInterval) + defer ticker.Stop() + for i := 0; i < defaultLoadFromEtcdMaxRetryTimes; i++ { + watchStartRevision, err = lw.load() + failpoint.Inject("loadKeyspaceGroupsTemporaryFail", func(val failpoint.Value) { + if maxFailTimes, ok := val.(int); ok && i < maxFailTimes { + err = errors.New("fail to read from etcd") + failpoint.Continue() + } + }) + if err == nil { + break + } + select { + case <-ctx.Done(): + lw.isLoadedCh <- errors.Errorf("ctx is done before load data from etcd") + return watchStartRevision + case <-ticker.C: + } + } + if err != nil { + log.Warn("watched key doesn't exist in watch loop when loading", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) + } + lw.isLoadedCh <- err + return watchStartRevision +} + func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision int64, err error) { watcher := clientv3.NewWatcher(lw.client) defer watcher.Close() @@ -462,8 +479,7 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision zap.Int64("compact-revision", wresp.CompactRevision)) revision = wresp.CompactRevision goto WatchChan - } - if wresp.Err() != nil { + } else if wresp.Err() != nil { // wresp.Err() contains CompactRevision not equal to 0 log.Error("watcher is canceled in watch loop", zap.Int64("revision", revision), errs.ZapError(errs.ErrEtcdWatcherCancel, wresp.Err())) @@ -490,6 +506,12 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision func (lw *LoopWatcher) load() (nextRevision int64, err error) { resp, err := EtcdKVGet(lw.client, lw.key, lw.opts...) + failpoint.Inject("delayLoadKeyspaceGroups", func(val failpoint.Value) { + if sleepIntervalSeconds, ok := val.(int); ok && sleepIntervalSeconds > 0 { + time.Sleep(time.Duration(sleepIntervalSeconds) * time.Second) + } + }) + if err != nil { log.Error("load failed in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) @@ -512,3 +534,7 @@ func (lw *LoopWatcher) ForceLoad() { default: } } + +func (lw *LoopWatcher) WaitLoad() error { + return <-lw.isLoadedCh +} From efa00e10eb09368da39eb4110dfa10c0f45d52d2 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 5 May 2023 03:20:05 +0800 Subject: [PATCH 08/24] fix test Signed-off-by: lhy1024 --- pkg/tso/keyspace_group_manager.go | 31 ++++++---- pkg/tso/keyspace_group_manager_test.go | 79 ++++++++++++++++++++++++-- pkg/utils/etcdutil/etcdutil.go | 23 ++++++-- 3 files changed, 112 insertions(+), 21 deletions(-) diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 1d8b49df193..48841cea604 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -47,8 +47,9 @@ import ( const ( // primaryElectionSuffix is the suffix of the key for keyspace group primary election - primaryElectionSuffix = "primary" - defaultRetryInterval = 500 * time.Millisecond + primaryElectionSuffix = "primary" + defaultRetryInterval = 500 * time.Millisecond + defaultLoadKeyspaceGroupsBatchSize = int64(400) ) type state struct { @@ -199,6 +200,8 @@ type KeyspaceGroupManager struct { // cfg is the TSO config cfg ServiceConfig + loadKeyspaceGroupsBatchSize int64 + // groupUpdateRetryList is the list of keyspace groups which failed to update and need to retry. groupUpdateRetryList map[uint32]*endpoint.KeyspaceGroup @@ -224,16 +227,17 @@ func NewKeyspaceGroupManager( ctx, cancel := context.WithCancel(ctx) kgm := &KeyspaceGroupManager{ - ctx: ctx, - cancel: cancel, - tsoServiceID: tsoServiceID, - etcdClient: etcdClient, - httpClient: httpClient, - electionNamePrefix: electionNamePrefix, - legacySvcRootPath: legacySvcRootPath, - tsoSvcRootPath: tsoSvcRootPath, - cfg: cfg, - groupUpdateRetryList: make(map[uint32]*endpoint.KeyspaceGroup), + ctx: ctx, + cancel: cancel, + tsoServiceID: tsoServiceID, + etcdClient: etcdClient, + httpClient: httpClient, + electionNamePrefix: electionNamePrefix, + legacySvcRootPath: legacySvcRootPath, + tsoSvcRootPath: tsoSvcRootPath, + cfg: cfg, + loadKeyspaceGroupsBatchSize: defaultLoadKeyspaceGroupsBatchSize, + groupUpdateRetryList: make(map[uint32]*endpoint.KeyspaceGroup), } kgm.legacySvcStorage = endpoint.NewStorageEndpoint( kv.NewEtcdKVBase(kgm.etcdClient, kgm.legacySvcRootPath), nil) @@ -279,6 +283,9 @@ func (kgm *KeyspaceGroupManager) Initialize() error { putFn, deleteFn, clientv3.WithRange(endKey), + clientv3.WithLimit(kgm.loadKeyspaceGroupsBatchSize), + // It loads keyspace groups from the start ID with limit. + // If limit is 0, it will load all keyspace groups from the start ID. ) kgm.wg.Add(1) diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index 3c15dc38f09..e8add57be4b 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -29,8 +29,10 @@ import ( "time" "github.com/google/uuid" + "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/storage/endpoint" @@ -104,6 +106,7 @@ func (suite *keyspaceGroupManagerTestSuite) TestNewKeyspaceGroupManager() { re.Equal(legacySvcRootPath, kgm.legacySvcRootPath) re.Equal(tsoSvcRootPath, kgm.tsoSvcRootPath) re.Equal(suite.cfg, kgm.cfg) + re.Equal(defaultLoadKeyspaceGroupsBatchSize, kgm.loadKeyspaceGroupsBatchSize) am, err := kgm.GetAllocatorManager(mcsutils.DefaultKeyspaceGroupID) re.NoError(err) @@ -162,7 +165,75 @@ func (suite *keyspaceGroupManagerTestSuite) TestLoadWithDifferentBatchSize() { } } -// TODO: move test +// TestLoadKeyspaceGroupsTimeout tests there is timeout when loading the initial keyspace group assignment +// from etcd. The initialization of the keyspace group manager should fail. +func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsTimeout() { + re := suite.Require() + + mgr := suite.newUniqueKeyspaceGroupManager(1) + re.NotNil(mgr) + defer mgr.Close() + + addKeyspaceGroupAssignment( + suite.ctx, suite.etcdClient, true, + mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, uint32(0), []uint32{0}) + + // Set the timeout to 1 second and inject the delayLoad to return 3 seconds to let + // the loading sleep 3 seconds. + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimeout", "return(1)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/delayLoad", "return(3)")) + err := mgr.Initialize() + // If loading keyspace groups timeout, the initialization should fail with ErrLoadKeyspaceGroupsTerminated. + re.Equal(errs.ErrLoadKeyspaceGroupsTerminated, err) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/delayLoad")) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimeout")) +} + +// TestLoadKeyspaceGroupsSucceedWithTempFailures tests the initialization should succeed when there are temporary +// failures during loading the initial keyspace group assignment from etcd. +func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsSucceedWithTempFailures() { + re := suite.Require() + + mgr := suite.newUniqueKeyspaceGroupManager(1) + re.NotNil(mgr) + defer mgr.Close() + + addKeyspaceGroupAssignment( + suite.ctx, suite.etcdClient, true, + mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, uint32(0), []uint32{0}) + + // Set the max retry times to 3 and inject the loadTemporaryFail to return 2 to let + // loading from etcd fail 2 times but the whole initialization still succeeds. + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimes", "return(3)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadTemporaryFail", "return(2)")) + err := mgr.Initialize() + re.NoError(err) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/loadTemporaryFail")) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimes")) +} + +// TestLoadKeyspaceGroupsFailed tests the initialization should fail when there are too many failures +// during loading the initial keyspace group assignment from etcd. +func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsFailed() { + re := suite.Require() + + mgr := suite.newUniqueKeyspaceGroupManager(1) + re.NotNil(mgr) + defer mgr.Close() + + addKeyspaceGroupAssignment( + suite.ctx, suite.etcdClient, true, + mgr.legacySvcRootPath, mgr.tsoServiceID.ServiceAddr, uint32(0), []uint32{0}) + + // Set the max retry times to 3 and inject the loadTemporaryFail to return 3 to let + // loading from etcd fail 3 times which should cause the whole initialization to fail. + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimes", "return(3)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadTemporaryFail", "return(3)")) + err := mgr.Initialize() + re.Error(err) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/loadTemporaryFail")) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimes")) +} // TestWatchAndDynamicallyApplyChanges tests the keyspace group manager watch and dynamically apply // keyspace groups' membership/distribution meta changes. @@ -599,9 +670,9 @@ func (suite *keyspaceGroupManagerTestSuite) newUniqueKeyspaceGroupManager( electionNamePrefix := "kgm-test-" + uniqueStr keyspaceGroupManager := suite.newKeyspaceGroupManager(tsoServiceID, electionNamePrefix, legacySvcRootPath, tsoSvcRootPath) - // if loadKeyspaceGroupsBatchSize != 0 { - // keyspaceGroupManager.loadKeyspaceGroupsBatchSize = loadKeyspaceGroupsBatchSize - // } + if loadKeyspaceGroupsBatchSize != 0 { + keyspaceGroupManager.loadKeyspaceGroupsBatchSize = loadKeyspaceGroupsBatchSize + } return keyspaceGroupManager } diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index c013d5ae6f5..7d23817b8df 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -387,7 +387,7 @@ func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Cl isLoadedCh: make(chan error, 1), putFn: putFn, deleteFn: deleteFn, - opts: opts, // todo: add default opts, e.g. clientv3.WithPrefix(), clientv3.Limit(512) + opts: opts, } } @@ -396,7 +396,13 @@ func (lw *LoopWatcher) StartWatchLoop() { defer logutil.LogPanic() defer lw.wg.Done() - ctx, cancel := context.WithTimeout(lw.ctx, defaultLoadDataTimeout) + timeout := defaultLoadDataTimeout + failpoint.Inject("loadRetryTimeout", func(val failpoint.Value) { + if v, ok := val.(int); ok { + timeout = time.Duration(v) * time.Second + } + }) + ctx, cancel := context.WithTimeout(lw.ctx, timeout) defer cancel() watchStartRevision := lw.initFromEtcd(ctx) @@ -430,9 +436,16 @@ func (lw *LoopWatcher) initFromEtcd(ctx context.Context) int64 { ) ticker := time.NewTicker(defaultLoadFromEtcdRetryInterval) defer ticker.Stop() - for i := 0; i < defaultLoadFromEtcdMaxRetryTimes; i++ { + + maxRetryTimes := defaultLoadFromEtcdMaxRetryTimes + failpoint.Inject("loadRetryTimes", func(val failpoint.Value) { + if v, ok := val.(int); ok { + maxRetryTimes = v + } + }) + for i := 0; i < maxRetryTimes; i++ { watchStartRevision, err = lw.load() - failpoint.Inject("loadKeyspaceGroupsTemporaryFail", func(val failpoint.Value) { + failpoint.Inject("loadTemporaryFail", func(val failpoint.Value) { if maxFailTimes, ok := val.(int); ok && i < maxFailTimes { err = errors.New("fail to read from etcd") failpoint.Continue() @@ -506,7 +519,7 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision func (lw *LoopWatcher) load() (nextRevision int64, err error) { resp, err := EtcdKVGet(lw.client, lw.key, lw.opts...) - failpoint.Inject("delayLoadKeyspaceGroups", func(val failpoint.Value) { + failpoint.Inject("delayLoad", func(val failpoint.Value) { if sleepIntervalSeconds, ok := val.(int); ok && sleepIntervalSeconds > 0 { time.Sleep(time.Duration(sleepIntervalSeconds) * time.Second) } From 06b280eb944c099b600913ca3c5fe89581afaf7a Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 5 May 2023 03:49:55 +0800 Subject: [PATCH 09/24] fix test Signed-off-by: lhy1024 --- pkg/tso/keyspace_group_manager.go | 2 +- pkg/utils/etcdutil/etcdutil.go | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 48841cea604..8901d4a58a1 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -295,7 +295,7 @@ func (kgm *KeyspaceGroupManager) Initialize() error { log.Error("failed to initialize keyspace group manager", errs.ZapError(err)) // We might have partially loaded/initialized the keyspace groups. Close the manager to clean up. kgm.Close() - return err + return errs.ErrLoadKeyspaceGroupsTerminated } kgm.wg.Add(1) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 7d23817b8df..bb621efce57 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -452,7 +452,9 @@ func (lw *LoopWatcher) initFromEtcd(ctx context.Context) int64 { } }) if err == nil { - break + if deadline, ok := ctx.Deadline(); ok && deadline.After(time.Now()) { + break + } } select { case <-ctx.Done(): @@ -474,7 +476,7 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision for { WatchChan: - watchChan := watcher.Watch(ctx, lw.key, append(lw.opts, clientv3.WithRev(revision))...) + watchChan := watcher.Watch(ctx, lw.key, append(lw.opts, clientv3.WithRev(revision), clientv3.WithLimit(0))...) select { case <-ctx.Done(): return revision, nil @@ -548,6 +550,7 @@ func (lw *LoopWatcher) ForceLoad() { } } +// WaitLoad waits for the result to obtain whether data is loaded. func (lw *LoopWatcher) WaitLoad() error { return <-lw.isLoadedCh } From ff495c1579e0455c88ca75397b33f43a94044d2f Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 5 May 2023 10:12:15 +0800 Subject: [PATCH 10/24] remove unnecessary failpoint Signed-off-by: lhy1024 --- pkg/tso/keyspace_group_manager_test.go | 7 ++- pkg/utils/etcdutil/etcdutil.go | 74 +++++++++++++------------- 2 files changed, 41 insertions(+), 40 deletions(-) diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index e8add57be4b..1eb90d2ea9f 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -180,13 +180,12 @@ func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsTimeout() { // Set the timeout to 1 second and inject the delayLoad to return 3 seconds to let // the loading sleep 3 seconds. - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimeout", "return(1)")) + mgr.groupWatcher.SetLoadTimeout(time.Second) re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/delayLoad", "return(3)")) err := mgr.Initialize() // If loading keyspace groups timeout, the initialization should fail with ErrLoadKeyspaceGroupsTerminated. re.Equal(errs.ErrLoadKeyspaceGroupsTerminated, err) re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/delayLoad")) - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimeout")) } // TestLoadKeyspaceGroupsSucceedWithTempFailures tests the initialization should succeed when there are temporary @@ -204,12 +203,11 @@ func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsSucceedWithTem // Set the max retry times to 3 and inject the loadTemporaryFail to return 2 to let // loading from etcd fail 2 times but the whole initialization still succeeds. - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimes", "return(3)")) + mgr.groupWatcher.SetLoadRetryTimes(3) re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadTemporaryFail", "return(2)")) err := mgr.Initialize() re.NoError(err) re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/loadTemporaryFail")) - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimes")) } // TestLoadKeyspaceGroupsFailed tests the initialization should fail when there are too many failures @@ -670,6 +668,7 @@ func (suite *keyspaceGroupManagerTestSuite) newUniqueKeyspaceGroupManager( electionNamePrefix := "kgm-test-" + uniqueStr keyspaceGroupManager := suite.newKeyspaceGroupManager(tsoServiceID, electionNamePrefix, legacySvcRootPath, tsoSvcRootPath) + if loadKeyspaceGroupsBatchSize != 0 { keyspaceGroupManager.loadKeyspaceGroupsBatchSize = loadKeyspaceGroupsBatchSize } diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index bb621efce57..7dec0ee8cc9 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -355,39 +355,43 @@ func InitOrGetClusterID(c *clientv3.Client, key string) (uint64, error) { } const ( - defaultLoadDataTimeout = 30 * time.Second + defaultLoadDataFromEtcdTimeout = 30 * time.Second defaultLoadFromEtcdRetryInterval = 200 * time.Millisecond - defaultLoadFromEtcdMaxRetryTimes = int(defaultLoadDataTimeout / defaultLoadFromEtcdRetryInterval) + defaultLoadFromEtcdRetryTimes = int(defaultLoadDataFromEtcdTimeout / defaultLoadFromEtcdRetryInterval) watchEtcdChangeRetryInterval = 1 * time.Second ) // LoopWatcher loads data from etcd and sets a watcher for it. type LoopWatcher struct { - ctx context.Context - wg *sync.WaitGroup - client *clientv3.Client - key string - name string - forceLoadCh chan struct{} - isLoadedCh chan error - putFn func(*mvccpb.KeyValue) error - deleteFn func(*mvccpb.KeyValue) error - opts []clientv3.OpOption + ctx context.Context + wg *sync.WaitGroup + client *clientv3.Client + key string + name string + forceLoadCh chan struct{} + isLoadedCh chan error + putFn func(*mvccpb.KeyValue) error + deleteFn func(*mvccpb.KeyValue) error + opts []clientv3.OpOption + loadTimeout time.Duration + loadRetryTimes int } // NewLoopWatcher creates a new LoopWatcher. func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Client, name, key string, putFn, deleteFn func(*mvccpb.KeyValue) error, opts ...clientv3.OpOption) *LoopWatcher { return &LoopWatcher{ - ctx: ctx, - client: client, - name: name, - key: key, - wg: wg, - forceLoadCh: make(chan struct{}, 1), - isLoadedCh: make(chan error, 1), - putFn: putFn, - deleteFn: deleteFn, - opts: opts, + ctx: ctx, + client: client, + name: name, + key: key, + wg: wg, + forceLoadCh: make(chan struct{}, 1), + isLoadedCh: make(chan error, 1), + putFn: putFn, + deleteFn: deleteFn, + opts: opts, + loadTimeout: defaultLoadDataFromEtcdTimeout, + loadRetryTimes: defaultLoadFromEtcdRetryTimes, } } @@ -396,13 +400,7 @@ func (lw *LoopWatcher) StartWatchLoop() { defer logutil.LogPanic() defer lw.wg.Done() - timeout := defaultLoadDataTimeout - failpoint.Inject("loadRetryTimeout", func(val failpoint.Value) { - if v, ok := val.(int); ok { - timeout = time.Duration(v) * time.Second - } - }) - ctx, cancel := context.WithTimeout(lw.ctx, timeout) + ctx, cancel := context.WithTimeout(lw.ctx, lw.loadTimeout) defer cancel() watchStartRevision := lw.initFromEtcd(ctx) @@ -437,13 +435,7 @@ func (lw *LoopWatcher) initFromEtcd(ctx context.Context) int64 { ticker := time.NewTicker(defaultLoadFromEtcdRetryInterval) defer ticker.Stop() - maxRetryTimes := defaultLoadFromEtcdMaxRetryTimes - failpoint.Inject("loadRetryTimes", func(val failpoint.Value) { - if v, ok := val.(int); ok { - maxRetryTimes = v - } - }) - for i := 0; i < maxRetryTimes; i++ { + for i := 0; i < lw.loadRetryTimes; i++ { watchStartRevision, err = lw.load() failpoint.Inject("loadTemporaryFail", func(val failpoint.Value) { if maxFailTimes, ok := val.(int); ok && i < maxFailTimes { @@ -554,3 +546,13 @@ func (lw *LoopWatcher) ForceLoad() { func (lw *LoopWatcher) WaitLoad() error { return <-lw.isLoadedCh } + +// SetLoadRetryTimes sets the retry times when loading data from etcd. +func (lw *LoopWatcher) SetLoadRetryTimes(times int) { + lw.loadRetryTimes = times +} + +// SetLoadTimeout sets the timeout when loading data from etcd. +func (lw *LoopWatcher) SetLoadTimeout(timeout time.Duration) { + lw.loadTimeout = timeout +} From 2cfc882f47ad314bcc0c916cbeb34314a04a5ade Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 5 May 2023 10:36:11 +0800 Subject: [PATCH 11/24] fix test Signed-off-by: lhy1024 --- pkg/tso/keyspace_group_manager.go | 10 +++++++++- pkg/tso/keyspace_group_manager_test.go | 7 +++---- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 8fc9a4b6b4f..a5818ea1024 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -200,7 +200,10 @@ type KeyspaceGroupManager struct { // cfg is the TSO config cfg ServiceConfig + // loadKeyspaceGroupsTimeout is the timeout for loading the initial keyspace group assignment. + loadKeyspaceGroupsTimeout time.Duration loadKeyspaceGroupsBatchSize int64 + loadFromEtcdMaxRetryTimes int // groupUpdateRetryList is the list of keyspace groups which failed to update and need to retry. groupUpdateRetryList map[uint32]*endpoint.KeyspaceGroup @@ -287,7 +290,12 @@ func (kgm *KeyspaceGroupManager) Initialize() error { // It loads keyspace groups from the start ID with limit. // If limit is 0, it will load all keyspace groups from the start ID. ) - + if kgm.loadKeyspaceGroupsTimeout > 0 { + kgm.groupWatcher.SetLoadTimeout(kgm.loadKeyspaceGroupsTimeout) + } + if kgm.loadFromEtcdMaxRetryTimes > 0 { + kgm.groupWatcher.SetLoadRetryTimes(kgm.loadFromEtcdMaxRetryTimes) + } kgm.wg.Add(1) go kgm.groupWatcher.StartWatchLoop() diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index 1eb90d2ea9f..ef649209884 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -180,7 +180,7 @@ func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsTimeout() { // Set the timeout to 1 second and inject the delayLoad to return 3 seconds to let // the loading sleep 3 seconds. - mgr.groupWatcher.SetLoadTimeout(time.Second) + mgr.loadKeyspaceGroupsTimeout = time.Second re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/delayLoad", "return(3)")) err := mgr.Initialize() // If loading keyspace groups timeout, the initialization should fail with ErrLoadKeyspaceGroupsTerminated. @@ -203,7 +203,7 @@ func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsSucceedWithTem // Set the max retry times to 3 and inject the loadTemporaryFail to return 2 to let // loading from etcd fail 2 times but the whole initialization still succeeds. - mgr.groupWatcher.SetLoadRetryTimes(3) + mgr.loadFromEtcdMaxRetryTimes = 3 re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadTemporaryFail", "return(2)")) err := mgr.Initialize() re.NoError(err) @@ -225,12 +225,11 @@ func (suite *keyspaceGroupManagerTestSuite) TestLoadKeyspaceGroupsFailed() { // Set the max retry times to 3 and inject the loadTemporaryFail to return 3 to let // loading from etcd fail 3 times which should cause the whole initialization to fail. - re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimes", "return(3)")) + mgr.loadFromEtcdMaxRetryTimes = 3 re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/loadTemporaryFail", "return(3)")) err := mgr.Initialize() re.Error(err) re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/loadTemporaryFail")) - re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/loadRetryTimes")) } // TestWatchAndDynamicallyApplyChanges tests the keyspace group manager watch and dynamically apply From 7c26ae3c76b8806d15d516f6412e16830876d288 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 5 May 2023 11:46:39 +0800 Subject: [PATCH 12/24] add withLimit by default Signed-off-by: lhy1024 --- pkg/tso/keyspace_group_manager.go | 30 +++++++------- pkg/tso/keyspace_group_manager_test.go | 1 - pkg/utils/etcdutil/etcdutil.go | 55 ++++++++++++++++++-------- 3 files changed, 53 insertions(+), 33 deletions(-) diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index a5818ea1024..323404bfd7e 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -47,9 +47,8 @@ import ( const ( // primaryElectionSuffix is the suffix of the key for keyspace group primary election - primaryElectionSuffix = "primary" - defaultRetryInterval = 500 * time.Millisecond - defaultLoadKeyspaceGroupsBatchSize = int64(400) + primaryElectionSuffix = "primary" + defaultRetryInterval = 500 * time.Millisecond ) type state struct { @@ -230,17 +229,16 @@ func NewKeyspaceGroupManager( ctx, cancel := context.WithCancel(ctx) kgm := &KeyspaceGroupManager{ - ctx: ctx, - cancel: cancel, - tsoServiceID: tsoServiceID, - etcdClient: etcdClient, - httpClient: httpClient, - electionNamePrefix: electionNamePrefix, - legacySvcRootPath: legacySvcRootPath, - tsoSvcRootPath: tsoSvcRootPath, - cfg: cfg, - loadKeyspaceGroupsBatchSize: defaultLoadKeyspaceGroupsBatchSize, - groupUpdateRetryList: make(map[uint32]*endpoint.KeyspaceGroup), + ctx: ctx, + cancel: cancel, + tsoServiceID: tsoServiceID, + etcdClient: etcdClient, + httpClient: httpClient, + electionNamePrefix: electionNamePrefix, + legacySvcRootPath: legacySvcRootPath, + tsoSvcRootPath: tsoSvcRootPath, + cfg: cfg, + groupUpdateRetryList: make(map[uint32]*endpoint.KeyspaceGroup), } kgm.legacySvcStorage = endpoint.NewStorageEndpoint( kv.NewEtcdKVBase(kgm.etcdClient, kgm.legacySvcRootPath), nil) @@ -286,7 +284,6 @@ func (kgm *KeyspaceGroupManager) Initialize() error { putFn, deleteFn, clientv3.WithRange(endKey), - clientv3.WithLimit(kgm.loadKeyspaceGroupsBatchSize), // It loads keyspace groups from the start ID with limit. // If limit is 0, it will load all keyspace groups from the start ID. ) @@ -296,6 +293,9 @@ func (kgm *KeyspaceGroupManager) Initialize() error { if kgm.loadFromEtcdMaxRetryTimes > 0 { kgm.groupWatcher.SetLoadRetryTimes(kgm.loadFromEtcdMaxRetryTimes) } + if kgm.loadKeyspaceGroupsBatchSize > 0 { + kgm.groupWatcher.SetLoadBatchSize(kgm.loadKeyspaceGroupsBatchSize) + } kgm.wg.Add(1) go kgm.groupWatcher.StartWatchLoop() diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index ef649209884..6b8beb3b0ae 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -106,7 +106,6 @@ func (suite *keyspaceGroupManagerTestSuite) TestNewKeyspaceGroupManager() { re.Equal(legacySvcRootPath, kgm.legacySvcRootPath) re.Equal(tsoSvcRootPath, kgm.tsoSvcRootPath) re.Equal(suite.cfg, kgm.cfg) - re.Equal(defaultLoadKeyspaceGroupsBatchSize, kgm.loadKeyspaceGroupsBatchSize) am, err := kgm.GetAllocatorManager(mcsutils.DefaultKeyspaceGroupID) re.NoError(err) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 7dec0ee8cc9..3f844deacfe 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -358,6 +358,7 @@ const ( defaultLoadDataFromEtcdTimeout = 30 * time.Second defaultLoadFromEtcdRetryInterval = 200 * time.Millisecond defaultLoadFromEtcdRetryTimes = int(defaultLoadDataFromEtcdTimeout / defaultLoadFromEtcdRetryInterval) + defaultLoadBatchSize = 400 watchEtcdChangeRetryInterval = 1 * time.Second ) @@ -375,6 +376,7 @@ type LoopWatcher struct { opts []clientv3.OpOption loadTimeout time.Duration loadRetryTimes int + loadBatchSize int64 } // NewLoopWatcher creates a new LoopWatcher. @@ -392,6 +394,7 @@ func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Cl opts: opts, loadTimeout: defaultLoadDataFromEtcdTimeout, loadRetryTimes: defaultLoadFromEtcdRetryTimes, + loadBatchSize: defaultLoadBatchSize, } } @@ -468,7 +471,8 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision for { WatchChan: - watchChan := watcher.Watch(ctx, lw.key, append(lw.opts, clientv3.WithRev(revision), clientv3.WithLimit(0))...) + opts := append(lw.opts, clientv3.WithRev(revision)) + watchChan := watcher.Watch(ctx, lw.key, opts...) select { case <-ctx.Done(): return revision, nil @@ -512,26 +516,38 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision } func (lw *LoopWatcher) load() (nextRevision int64, err error) { - resp, err := EtcdKVGet(lw.client, lw.key, lw.opts...) - failpoint.Inject("delayLoad", func(val failpoint.Value) { - if sleepIntervalSeconds, ok := val.(int); ok && sleepIntervalSeconds > 0 { - time.Sleep(time.Duration(sleepIntervalSeconds) * time.Second) - } - }) - - if err != nil { - log.Error("load failed in watch loop", zap.String("name", lw.name), - zap.String("key", lw.key), zap.Error(err)) - return 0, err + startKey := lw.key + limit := lw.loadBatchSize + if limit <= 0 { + limit = defaultLoadBatchSize } - for _, item := range resp.Kvs { - err = lw.putFn(item) + for { + opts := append(lw.opts, clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend), clientv3.WithLimit(limit)) + resp, err := EtcdKVGet(lw.client, startKey, opts...) + failpoint.Inject("delayLoad", func(val failpoint.Value) { + if sleepIntervalSeconds, ok := val.(int); ok && sleepIntervalSeconds > 0 { + time.Sleep(time.Duration(sleepIntervalSeconds) * time.Second) + } + }) + if err != nil { - log.Error("put failed in watch loop when loading", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) - continue + log.Error("load failed in watch loop", zap.String("name", lw.name), + zap.String("key", lw.key), zap.Error(err)) + return 0, err + } + for _, item := range resp.Kvs { + err = lw.putFn(item) + if err != nil { + log.Error("put failed in watch loop when loading", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) + continue + } + } + count := int64(len(resp.Kvs)) + if count < limit { + return resp.Header.Revision + 1, err } + startKey = string(resp.Kvs[count-1].Key) } - return resp.Header.Revision + 1, err } // ForceLoad forces to load the key. @@ -556,3 +572,8 @@ func (lw *LoopWatcher) SetLoadRetryTimes(times int) { func (lw *LoopWatcher) SetLoadTimeout(timeout time.Duration) { lw.loadTimeout = timeout } + +// SetLoadBatchSize sets the batch size when loading data from etcd. +func (lw *LoopWatcher) SetLoadBatchSize(size int64) { + lw.loadBatchSize = size +} From acd2420dc466ddb2da38df40f6374e82f8645f03 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 5 May 2023 12:49:46 +0800 Subject: [PATCH 13/24] use loopwatcher in keyspaces Signed-off-by: lhy1024 --- pkg/tso/keyspace_group_manager.go | 2 - server/keyspace_service.go | 119 ++++++++++++++---------------- 2 files changed, 56 insertions(+), 65 deletions(-) diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 323404bfd7e..208261374db 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -284,8 +284,6 @@ func (kgm *KeyspaceGroupManager) Initialize() error { putFn, deleteFn, clientv3.WithRange(endKey), - // It loads keyspace groups from the start ID with limit. - // If limit is 0, it will load all keyspace groups from the start ID. ) if kgm.loadKeyspaceGroupsTimeout > 0 { kgm.groupWatcher.SetLoadTimeout(kgm.loadKeyspaceGroupsTimeout) diff --git a/server/keyspace_service.go b/server/keyspace_service.go index 0ecfe45c1d7..bb9b6548141 100644 --- a/server/keyspace_service.go +++ b/server/keyspace_service.go @@ -17,17 +17,17 @@ package server import ( "context" "path" + "sync" "time" "github.com/gogo/protobuf/proto" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/pingcap/log" - "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/storage/endpoint" + "github.com/tikv/pd/pkg/utils/etcdutil" "go.etcd.io/etcd/clientv3" - "go.uber.org/zap" + "go.etcd.io/etcd/mvcc/mvccpb" ) // KeyspaceServer wraps GrpcServer to provide keyspace service. @@ -73,79 +73,72 @@ func (s *KeyspaceServer) WatchKeyspaces(request *keyspacepb.WatchKeyspacesReques if err := s.validateRequest(request.GetHeader()); err != nil { return err } - ctx, cancel := context.WithCancel(s.Context()) defer cancel() + startKey := path.Join(s.rootPath, endpoint.KeyspaceMetaPrefix()) - revision, err := s.sendAllKeyspaceMeta(ctx, stream) - if err != nil { - return err + var firstLoading struct { + sync.Mutex + isFinished bool + keyspaces []*keyspacepb.KeyspaceMeta } + firstLoading.keyspaces = make([]*keyspacepb.KeyspaceMeta, 0) - watcher := clientv3.NewWatcher(s.client) - defer watcher.Close() - - for { - rch := watcher.Watch(ctx, path.Join(s.rootPath, endpoint.KeyspaceMetaPrefix()), clientv3.WithPrefix(), clientv3.WithRev(revision)) - for wresp := range rch { - if wresp.CompactRevision != 0 { - log.Warn("required revision has been compacted, use the compact revision", - zap.Int64("required-revision", revision), - zap.Int64("compact-revision", wresp.CompactRevision)) - revision = wresp.CompactRevision - break - } - if wresp.Canceled { - log.Error("watcher is canceled with", - zap.Int64("revision", revision), - errs.ZapError(errs.ErrEtcdWatcherCancel, wresp.Err())) - return wresp.Err() - } - keyspaces := make([]*keyspacepb.KeyspaceMeta, 0, len(wresp.Events)) - for _, event := range wresp.Events { - if event.Type != clientv3.EventTypePut { - continue - } - meta := &keyspacepb.KeyspaceMeta{} - if err = proto.Unmarshal(event.Kv.Value, meta); err != nil { - return err - } - keyspaces = append(keyspaces, meta) - } - if len(keyspaces) > 0 { - if err = stream.Send(&keyspacepb.WatchKeyspacesResponse{Header: s.header(), Keyspaces: keyspaces}); err != nil { - return err - } - } + putFn := func(kv *mvccpb.KeyValue) error { + meta := &keyspacepb.KeyspaceMeta{} + if err := proto.Unmarshal(kv.Value, meta); err != nil { + return err } - select { - case <-ctx.Done(): - // server closed, return + firstLoading.Lock() + if !firstLoading.isFinished { + firstLoading.keyspaces = append(firstLoading.keyspaces, meta) + firstLoading.Unlock() return nil - default: } + firstLoading.Unlock() + resp := &keyspacepb.WatchKeyspacesResponse{ + Header: s.header(), + Keyspaces: []*keyspacepb.KeyspaceMeta{meta}} + if err := stream.Send(resp); err != nil { + return err + } + return nil + } + deleteFn := func(kv *mvccpb.KeyValue) error { + return nil } -} -func (s *KeyspaceServer) sendAllKeyspaceMeta(ctx context.Context, stream keyspacepb.Keyspace_WatchKeyspacesServer) (int64, error) { - getResp, err := s.client.Get(ctx, path.Join(s.rootPath, endpoint.KeyspaceMetaPrefix()), clientv3.WithPrefix()) - if err != nil { - return 0, err + watcher := etcdutil.NewLoopWatcher( + ctx, + &s.serverLoopWg, + s.client, + "keyspace-server-watcher", + startKey, + putFn, + deleteFn, + clientv3.WithPrefix(), + ) + s.serverLoopWg.Add(1) + go watcher.StartWatchLoop() + if err := watcher.WaitLoad(); err != nil { + cancel() // cancel context to stop watcher + return err } - metas := make([]*keyspacepb.KeyspaceMeta, getResp.Count) - for i, kv := range getResp.Kvs { - meta := &keyspacepb.KeyspaceMeta{} - if err = proto.Unmarshal(kv.Value, meta); err != nil { - return 0, err + + // send all existing keyspaces as first package + firstLoading.Lock() + if len(firstLoading.keyspaces) > 0 { + if err := stream.Send(&keyspacepb.WatchKeyspacesResponse{Header: s.header(), Keyspaces: firstLoading.keyspaces}); err != nil { + cancel() // cancel context to stop watcher + firstLoading.Unlock() + return err } - metas[i] = meta - } - var revision int64 - if getResp.Header != nil { - // start from the next revision - revision = getResp.Header.GetRevision() + 1 + firstLoading.isFinished = true } - return revision, stream.Send(&keyspacepb.WatchKeyspacesResponse{Header: s.header(), Keyspaces: metas}) + firstLoading.Unlock() + + <-ctx.Done() // wait for context done + return nil } // UpdateKeyspaceState updates the state of keyspace specified in the request. From 87949f05692ebab190bac1d07cd87419a5b2edd7 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 5 May 2023 13:10:10 +0800 Subject: [PATCH 14/24] use sync.Map as groupUpdateRetryList Signed-off-by: lhy1024 --- pkg/tso/keyspace_group_manager.go | 38 +++++++++++++++++++------------ server/keyspace_service.go | 8 ++----- 2 files changed, 25 insertions(+), 21 deletions(-) diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 208261374db..8defbec6538 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -205,7 +205,7 @@ type KeyspaceGroupManager struct { loadFromEtcdMaxRetryTimes int // groupUpdateRetryList is the list of keyspace groups which failed to update and need to retry. - groupUpdateRetryList map[uint32]*endpoint.KeyspaceGroup + groupUpdateRetryList sync.Map // store as map[uint32]*endpoint.KeyspaceGroup groupWatcher *etcdutil.LoopWatcher } @@ -229,16 +229,15 @@ func NewKeyspaceGroupManager( ctx, cancel := context.WithCancel(ctx) kgm := &KeyspaceGroupManager{ - ctx: ctx, - cancel: cancel, - tsoServiceID: tsoServiceID, - etcdClient: etcdClient, - httpClient: httpClient, - electionNamePrefix: electionNamePrefix, - legacySvcRootPath: legacySvcRootPath, - tsoSvcRootPath: tsoSvcRootPath, - cfg: cfg, - groupUpdateRetryList: make(map[uint32]*endpoint.KeyspaceGroup), + ctx: ctx, + cancel: cancel, + tsoServiceID: tsoServiceID, + etcdClient: etcdClient, + httpClient: httpClient, + electionNamePrefix: electionNamePrefix, + legacySvcRootPath: legacySvcRootPath, + tsoSvcRootPath: tsoSvcRootPath, + cfg: cfg, } kgm.legacySvcStorage = endpoint.NewStorageEndpoint( kv.NewEtcdKVBase(kgm.etcdClient, kgm.legacySvcRootPath), nil) @@ -345,10 +344,19 @@ func (kgm *KeyspaceGroupManager) startRetryUpdateLoop() { case <-kgm.ctx.Done(): return case <-ticker.C: - for id, group := range kgm.groupUpdateRetryList { - delete(kgm.groupUpdateRetryList, id) + kgm.groupUpdateRetryList.Range(func(key, value interface{}) bool { + id, ok := key.(uint64) + if !ok { + return true + } + group, ok := value.(*endpoint.KeyspaceGroup) + if !ok { + return true + } + kgm.groupUpdateRetryList.Delete(id) kgm.updateKeyspaceGroup(group) - } + return true + }) } } } @@ -416,7 +424,7 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroup(group *endpoint.KeyspaceGro splitSourceAM, splitSourceGroup := kgm.getKeyspaceGroupMeta(splitSource) if !validateSplit(splitSourceAM, group, splitSourceGroup) { // Put the group into the retry list to retry later. - kgm.groupUpdateRetryList[group.ID] = group + kgm.groupUpdateRetryList.Store(group.ID, group) return } participant.SetCampaignChecker(func(leadership *election.Leadership) bool { diff --git a/server/keyspace_service.go b/server/keyspace_service.go index bb9b6548141..205aae90af0 100644 --- a/server/keyspace_service.go +++ b/server/keyspace_service.go @@ -96,13 +96,9 @@ func (s *KeyspaceServer) WatchKeyspaces(request *keyspacepb.WatchKeyspacesReques return nil } firstLoading.Unlock() - resp := &keyspacepb.WatchKeyspacesResponse{ + return stream.Send(&keyspacepb.WatchKeyspacesResponse{ Header: s.header(), - Keyspaces: []*keyspacepb.KeyspaceMeta{meta}} - if err := stream.Send(resp); err != nil { - return err - } - return nil + Keyspaces: []*keyspacepb.KeyspaceMeta{meta}}) } deleteFn := func(kv *mvccpb.KeyValue) error { return nil From edb213a68ca6642d4919e91621a3492b59cf9f61 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 5 May 2023 13:40:24 +0800 Subject: [PATCH 15/24] add post function Signed-off-by: lhy1024 --- pkg/keyspace/tso_keyspace_group.go | 1 + pkg/tso/keyspace_group_manager.go | 47 +++++++++++------------------- pkg/utils/etcdutil/etcdutil.go | 15 ++++++++-- server/keyspace_service.go | 39 ++++++------------------- server/server.go | 1 + 5 files changed, 41 insertions(+), 62 deletions(-) diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index b314f1bfccf..afccd7371a1 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -258,6 +258,7 @@ func (m *GroupManager) initTSONodesWatcher(client *clientv3.Client, clusterID ui tsoServiceKey, putFn, deleteFn, + func() error { return nil }, clientv3.WithRange(tsoServiceEndKey), ) } diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 8defbec6538..992dba6ef0e 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -274,6 +274,22 @@ func (kgm *KeyspaceGroupManager) Initialize() error { kgm.deleteKeyspaceGroup(groupID) return nil } + postEventFn := func() error { + kgm.groupUpdateRetryList.Range(func(key, value interface{}) bool { + id, ok := key.(uint64) + if !ok { + return true + } + group, ok := value.(*endpoint.KeyspaceGroup) + if !ok { + return true + } + kgm.groupUpdateRetryList.Delete(id) + kgm.updateKeyspaceGroup(group) + return true + }) + return nil + } kgm.groupWatcher = etcdutil.NewLoopWatcher( kgm.ctx, &kgm.wg, @@ -282,6 +298,7 @@ func (kgm *KeyspaceGroupManager) Initialize() error { startKey, putFn, deleteFn, + postEventFn, clientv3.WithRange(endKey), ) if kgm.loadKeyspaceGroupsTimeout > 0 { @@ -303,9 +320,6 @@ func (kgm *KeyspaceGroupManager) Initialize() error { return errs.ErrLoadKeyspaceGroupsTerminated } - kgm.wg.Add(1) - go kgm.startRetryUpdateLoop() - if !defaultKGConfigured { log.Info("initializing default keyspace group") group := &endpoint.KeyspaceGroup{ @@ -334,33 +348,6 @@ func (kgm *KeyspaceGroupManager) Close() { log.Info("keyspace group manager closed") } -func (kgm *KeyspaceGroupManager) startRetryUpdateLoop() { - defer logutil.LogPanic() - defer kgm.wg.Done() - ticker := time.NewTicker(defaultRetryInterval) - defer ticker.Stop() - for { - select { - case <-kgm.ctx.Done(): - return - case <-ticker.C: - kgm.groupUpdateRetryList.Range(func(key, value interface{}) bool { - id, ok := key.(uint64) - if !ok { - return true - } - group, ok := value.(*endpoint.KeyspaceGroup) - if !ok { - return true - } - kgm.groupUpdateRetryList.Delete(id) - kgm.updateKeyspaceGroup(group) - return true - }) - } - } -} - func (kgm *KeyspaceGroupManager) isAssignedToMe(group *endpoint.KeyspaceGroup) bool { for _, member := range group.Members { if member.Address == kgm.tsoServiceID.ServiceAddr { diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 3f844deacfe..50b361c0418 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -373,6 +373,7 @@ type LoopWatcher struct { isLoadedCh chan error putFn func(*mvccpb.KeyValue) error deleteFn func(*mvccpb.KeyValue) error + postEventFn func() error opts []clientv3.OpOption loadTimeout time.Duration loadRetryTimes int @@ -380,7 +381,8 @@ type LoopWatcher struct { } // NewLoopWatcher creates a new LoopWatcher. -func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Client, name, key string, putFn, deleteFn func(*mvccpb.KeyValue) error, opts ...clientv3.OpOption) *LoopWatcher { +func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Client, name, key string, + putFn, deleteFn func(*mvccpb.KeyValue) error, postEventFn func() error, opts ...clientv3.OpOption) *LoopWatcher { return &LoopWatcher{ ctx: ctx, client: client, @@ -391,6 +393,7 @@ func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Cl isLoadedCh: make(chan error, 1), putFn: putFn, deleteFn: deleteFn, + postEventFn: postEventFn, opts: opts, loadTimeout: defaultLoadDataFromEtcdTimeout, loadRetryTimes: defaultLoadFromEtcdRetryTimes, @@ -510,6 +513,10 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision } } } + if err := lw.postEventFn(); err != nil { + log.Error("run post event failed in watch loop", zap.String("name", lw.name), + zap.String("key", lw.key), zap.Error(err)) + } revision = wresp.Header.Revision + 1 } } @@ -543,7 +550,11 @@ func (lw *LoopWatcher) load() (nextRevision int64, err error) { } } count := int64(len(resp.Kvs)) - if count < limit { + if count < limit { // no more data + if err := lw.postEventFn(); err != nil { + log.Error("run post event failed in watch loop", zap.String("name", lw.name), + zap.String("key", lw.key), zap.Error(err)) + } return resp.Header.Revision + 1, err } startKey = string(resp.Kvs[count-1].Key) diff --git a/server/keyspace_service.go b/server/keyspace_service.go index 205aae90af0..2a27ce4eedc 100644 --- a/server/keyspace_service.go +++ b/server/keyspace_service.go @@ -17,7 +17,6 @@ package server import ( "context" "path" - "sync" "time" "github.com/gogo/protobuf/proto" @@ -77,32 +76,23 @@ func (s *KeyspaceServer) WatchKeyspaces(request *keyspacepb.WatchKeyspacesReques defer cancel() startKey := path.Join(s.rootPath, endpoint.KeyspaceMetaPrefix()) - var firstLoading struct { - sync.Mutex - isFinished bool - keyspaces []*keyspacepb.KeyspaceMeta - } - firstLoading.keyspaces = make([]*keyspacepb.KeyspaceMeta, 0) - + keyspaces := make([]*keyspacepb.KeyspaceMeta, 0) putFn := func(kv *mvccpb.KeyValue) error { meta := &keyspacepb.KeyspaceMeta{} if err := proto.Unmarshal(kv.Value, meta); err != nil { return err } - firstLoading.Lock() - if !firstLoading.isFinished { - firstLoading.keyspaces = append(firstLoading.keyspaces, meta) - firstLoading.Unlock() - return nil - } - firstLoading.Unlock() - return stream.Send(&keyspacepb.WatchKeyspacesResponse{ - Header: s.header(), - Keyspaces: []*keyspacepb.KeyspaceMeta{meta}}) + keyspaces = append(keyspaces, meta) + return nil } deleteFn := func(kv *mvccpb.KeyValue) error { return nil } + postEventFn := func() error { + return stream.Send(&keyspacepb.WatchKeyspacesResponse{ + Header: s.header(), + Keyspaces: keyspaces}) + } watcher := etcdutil.NewLoopWatcher( ctx, @@ -112,6 +102,7 @@ func (s *KeyspaceServer) WatchKeyspaces(request *keyspacepb.WatchKeyspacesReques startKey, putFn, deleteFn, + postEventFn, clientv3.WithPrefix(), ) s.serverLoopWg.Add(1) @@ -121,18 +112,6 @@ func (s *KeyspaceServer) WatchKeyspaces(request *keyspacepb.WatchKeyspacesReques return err } - // send all existing keyspaces as first package - firstLoading.Lock() - if len(firstLoading.keyspaces) > 0 { - if err := stream.Send(&keyspacepb.WatchKeyspacesResponse{Header: s.header(), Keyspaces: firstLoading.keyspaces}); err != nil { - cancel() // cancel context to stop watcher - firstLoading.Unlock() - return err - } - firstLoading.isFinished = true - } - firstLoading.Unlock() - <-ctx.Done() // wait for context done return nil } diff --git a/server/server.go b/server/server.go index 60836be8a52..3e8d1ee3dd2 100644 --- a/server/server.go +++ b/server/server.go @@ -1751,6 +1751,7 @@ func (s *Server) initTSOPrimaryWatcher() { tsoServicePrimaryKey, putFn, deleteFn, + func() error { return nil }, ) } From 54a06c6f0581042f39744f3292f76aa84fff0715 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 5 May 2023 14:12:42 +0800 Subject: [PATCH 16/24] address comments Signed-off-by: lhy1024 --- pkg/tso/keyspace_group_manager.go | 39 +++++++++++++------------------ pkg/utils/etcdutil/etcdutil.go | 8 ++++--- server/keyspace_service.go | 3 +++ 3 files changed, 24 insertions(+), 26 deletions(-) diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 992dba6ef0e..6990a298589 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -205,7 +205,7 @@ type KeyspaceGroupManager struct { loadFromEtcdMaxRetryTimes int // groupUpdateRetryList is the list of keyspace groups which failed to update and need to retry. - groupUpdateRetryList sync.Map // store as map[uint32]*endpoint.KeyspaceGroup + groupUpdateRetryList map[uint32]*endpoint.KeyspaceGroup groupWatcher *etcdutil.LoopWatcher } @@ -229,15 +229,16 @@ func NewKeyspaceGroupManager( ctx, cancel := context.WithCancel(ctx) kgm := &KeyspaceGroupManager{ - ctx: ctx, - cancel: cancel, - tsoServiceID: tsoServiceID, - etcdClient: etcdClient, - httpClient: httpClient, - electionNamePrefix: electionNamePrefix, - legacySvcRootPath: legacySvcRootPath, - tsoSvcRootPath: tsoSvcRootPath, - cfg: cfg, + ctx: ctx, + cancel: cancel, + tsoServiceID: tsoServiceID, + etcdClient: etcdClient, + httpClient: httpClient, + electionNamePrefix: electionNamePrefix, + legacySvcRootPath: legacySvcRootPath, + tsoSvcRootPath: tsoSvcRootPath, + cfg: cfg, + groupUpdateRetryList: make(map[uint32]*endpoint.KeyspaceGroup), } kgm.legacySvcStorage = endpoint.NewStorageEndpoint( kv.NewEtcdKVBase(kgm.etcdClient, kgm.legacySvcRootPath), nil) @@ -275,19 +276,11 @@ func (kgm *KeyspaceGroupManager) Initialize() error { return nil } postEventFn := func() error { - kgm.groupUpdateRetryList.Range(func(key, value interface{}) bool { - id, ok := key.(uint64) - if !ok { - return true - } - group, ok := value.(*endpoint.KeyspaceGroup) - if !ok { - return true - } - kgm.groupUpdateRetryList.Delete(id) + // Retry the groups that are not initialized successfully before. + for id, group := range kgm.groupUpdateRetryList { + delete(kgm.groupUpdateRetryList, id) kgm.updateKeyspaceGroup(group) - return true - }) + } return nil } kgm.groupWatcher = etcdutil.NewLoopWatcher( @@ -411,7 +404,7 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroup(group *endpoint.KeyspaceGro splitSourceAM, splitSourceGroup := kgm.getKeyspaceGroupMeta(splitSource) if !validateSplit(splitSourceAM, group, splitSourceGroup) { // Put the group into the retry list to retry later. - kgm.groupUpdateRetryList.Store(group.ID, group) + kgm.groupUpdateRetryList[group.ID] = group return } participant.SetCampaignChecker(func(leadership *election.Leadership) bool { diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 50b361c0418..b33cbad3a21 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -549,15 +549,17 @@ func (lw *LoopWatcher) load() (nextRevision int64, err error) { continue } } - count := int64(len(resp.Kvs)) - if count < limit { // no more data + if !resp.More { if err := lw.postEventFn(); err != nil { log.Error("run post event failed in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) } return resp.Header.Revision + 1, err } - startKey = string(resp.Kvs[count-1].Key) + index := int64(len(resp.Kvs)) - 1 + if index >= 0 { + startKey = string(resp.Kvs[index].Key) + } } } diff --git a/server/keyspace_service.go b/server/keyspace_service.go index 2a27ce4eedc..9a7b207e925 100644 --- a/server/keyspace_service.go +++ b/server/keyspace_service.go @@ -89,6 +89,9 @@ func (s *KeyspaceServer) WatchKeyspaces(request *keyspacepb.WatchKeyspacesReques return nil } postEventFn := func() error { + defer func() { + keyspaces = make([]*keyspacepb.KeyspaceMeta, 0) + }() return stream.Send(&keyspacepb.WatchKeyspacesResponse{ Header: s.header(), Keyspaces: keyspaces}) From a5ce1a69e2eaee642bb69c841c5b62326dfe88fa Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 5 May 2023 21:45:33 +0800 Subject: [PATCH 17/24] fix next key in limit Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil.go | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index b33cbad3a21..0e8f4fb48cc 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -414,7 +414,6 @@ func (lw *LoopWatcher) StartWatchLoop() { for { select { case <-lw.ctx.Done(): - close(lw.forceLoadCh) log.Info("server is closed, exit watch loop", zap.String("name", lw.name), zap.String("key", lw.key)) return default: @@ -442,7 +441,7 @@ func (lw *LoopWatcher) initFromEtcd(ctx context.Context) int64 { defer ticker.Stop() for i := 0; i < lw.loadRetryTimes; i++ { - watchStartRevision, err = lw.load() + watchStartRevision, err = lw.load(ctx) failpoint.Inject("loadTemporaryFail", func(val failpoint.Value) { if maxFailTimes, ok := val.(int); ok && i < maxFailTimes { err = errors.New("fail to read from etcd") @@ -480,7 +479,7 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision case <-ctx.Done(): return revision, nil case <-lw.forceLoadCh: - revision, err = lw.load() + revision, err = lw.load(ctx) if err != nil { log.Warn("force load key failed in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) @@ -522,15 +521,19 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision } } -func (lw *LoopWatcher) load() (nextRevision int64, err error) { +func (lw *LoopWatcher) load(ctx context.Context) (nextRevision int64, err error) { startKey := lw.key limit := lw.loadBatchSize - if limit <= 0 { - limit = defaultLoadBatchSize + // If limit is 0, it means no limit. + // If limit is 1, we need to load 2 items to look for the next key. + if limit == 1 { + limit = 2 } + ctx, cancel := context.WithTimeout(ctx, DefaultRequestTimeout) + defer cancel() for { opts := append(lw.opts, clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend), clientv3.WithLimit(limit)) - resp, err := EtcdKVGet(lw.client, startKey, opts...) + resp, err := clientv3.NewKV(lw.client).Get(ctx, startKey, opts...) failpoint.Inject("delayLoad", func(val failpoint.Value) { if sleepIntervalSeconds, ok := val.(int); ok && sleepIntervalSeconds > 0 { time.Sleep(time.Duration(sleepIntervalSeconds) * time.Second) @@ -549,17 +552,15 @@ func (lw *LoopWatcher) load() (nextRevision int64, err error) { continue } } - if !resp.More { + if !resp.More || len(resp.Kvs) == 0 { if err := lw.postEventFn(); err != nil { log.Error("run post event failed in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) } + log.Info("load finished in watch loop", zap.String("name", lw.name), zap.String("key", lw.key)) return resp.Header.Revision + 1, err } - index := int64(len(resp.Kvs)) - 1 - if index >= 0 { - startKey = string(resp.Kvs[index].Key) - } + startKey = string(resp.Kvs[int64(len(resp.Kvs))-1].Key) } } From 70686433be44fe9a1f01fe12f9a5b12c5aaf9d1f Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Sat, 6 May 2023 10:24:00 +0800 Subject: [PATCH 18/24] add more comments Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil.go | 40 +++++++++++++++++++++++----------- 1 file changed, 27 insertions(+), 13 deletions(-) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 0e8f4fb48cc..e3364e2981a 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -364,20 +364,34 @@ const ( // LoopWatcher loads data from etcd and sets a watcher for it. type LoopWatcher struct { - ctx context.Context - wg *sync.WaitGroup - client *clientv3.Client - key string - name string - forceLoadCh chan struct{} - isLoadedCh chan error - putFn func(*mvccpb.KeyValue) error - deleteFn func(*mvccpb.KeyValue) error - postEventFn func() error - opts []clientv3.OpOption - loadTimeout time.Duration + ctx context.Context + wg *sync.WaitGroup + name string + client *clientv3.Client + + // key is the etcd key to watch. + key string + // opts is used to set etcd options. + opts []clientv3.OpOption + + // forceLoadCh is used to force loading data from etcd. + forceLoadCh chan struct{} + // isLoadedCh is used to notify that the data has been loaded from etcd first time. + isLoadedCh chan error + + // putFn is used to handle the put event. + putFn func(*mvccpb.KeyValue) error + // deleteFn is used to handle the delete event. + deleteFn func(*mvccpb.KeyValue) error + // postEventFn is used to call after handling all events. + postEventFn func() error + + // loadTimeout is used to set the timeout for loading data from etcd. + loadTimeout time.Duration + // loadRetryTimes is used to set the retry times for loading data from etcd. loadRetryTimes int - loadBatchSize int64 + // loadBatchSize is used to set the batch size for loading data from etcd. + loadBatchSize int64 } // NewLoopWatcher creates a new LoopWatcher. From c8de713014de94eb4fdfaec33addf5975cd93610 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Sat, 6 May 2023 11:13:22 +0800 Subject: [PATCH 19/24] address comments Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil.go | 43 ++++++++++++++++++---------------- 1 file changed, 23 insertions(+), 20 deletions(-) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index e3364e2981a..f3d424bd649 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -455,17 +455,20 @@ func (lw *LoopWatcher) initFromEtcd(ctx context.Context) int64 { defer ticker.Stop() for i := 0; i < lw.loadRetryTimes; i++ { - watchStartRevision, err = lw.load(ctx) failpoint.Inject("loadTemporaryFail", func(val failpoint.Value) { if maxFailTimes, ok := val.(int); ok && i < maxFailTimes { err = errors.New("fail to read from etcd") failpoint.Continue() } }) - if err == nil { - if deadline, ok := ctx.Deadline(); ok && deadline.After(time.Now()) { - break + failpoint.Inject("delayLoad", func(val failpoint.Value) { + if sleepIntervalSeconds, ok := val.(int); ok && sleepIntervalSeconds > 0 { + time.Sleep(time.Duration(sleepIntervalSeconds) * time.Second) } + }) + watchStartRevision, err = lw.load(ctx) + if err == nil { + break } select { case <-ctx.Done(): @@ -536,37 +539,38 @@ func (lw *LoopWatcher) watch(ctx context.Context, revision int64) (nextRevision } func (lw *LoopWatcher) load(ctx context.Context) (nextRevision int64, err error) { + ctx, cancel := context.WithTimeout(ctx, DefaultRequestTimeout) + defer cancel() startKey := lw.key - limit := lw.loadBatchSize // If limit is 0, it means no limit. - // If limit is 1, we need to load 2 items to look for the next key. - if limit == 1 { - limit = 2 + // If limit is not 0, we need to add 1 to limit to get the next key. + limit := lw.loadBatchSize + if limit != 0 { + limit++ } - ctx, cancel := context.WithTimeout(ctx, DefaultRequestTimeout) - defer cancel() for { + // Sort by key to get the next key and we don't need to worry about the performance, + // Because the default sort is just SortByKey and SortAscend opts := append(lw.opts, clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend), clientv3.WithLimit(limit)) resp, err := clientv3.NewKV(lw.client).Get(ctx, startKey, opts...) - failpoint.Inject("delayLoad", func(val failpoint.Value) { - if sleepIntervalSeconds, ok := val.(int); ok && sleepIntervalSeconds > 0 { - time.Sleep(time.Duration(sleepIntervalSeconds) * time.Second) - } - }) - if err != nil { log.Error("load failed in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) return 0, err } - for _, item := range resp.Kvs { + for i, item := range resp.Kvs { + if resp.More && i == len(resp.Kvs)-1 { + // The last key is the start key of the next batch. + // To avoid to get the same key in the next load, we need to skip the last key. + startKey = string(item.Key) + continue + } err = lw.putFn(item) if err != nil { log.Error("put failed in watch loop when loading", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) - continue } } - if !resp.More || len(resp.Kvs) == 0 { + if !resp.More { if err := lw.postEventFn(); err != nil { log.Error("run post event failed in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) @@ -574,7 +578,6 @@ func (lw *LoopWatcher) load(ctx context.Context) (nextRevision int64, err error) log.Info("load finished in watch loop", zap.String("name", lw.name), zap.String("key", lw.key)) return resp.Header.Revision + 1, err } - startKey = string(resp.Kvs[int64(len(resp.Kvs))-1].Key) } } From 4a230e6b0b34fce2cd74519dcc29846c9f93f3ca Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Sat, 6 May 2023 14:27:35 +0800 Subject: [PATCH 20/24] add more tests Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil.go | 37 +++-- pkg/utils/etcdutil/etcdutil_test.go | 243 ++++++++++++++++++++++++++++ server/keyspace_service.go | 2 +- 3 files changed, 264 insertions(+), 18 deletions(-) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index f3d424bd649..d091b3eb83f 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -359,7 +359,7 @@ const ( defaultLoadFromEtcdRetryInterval = 200 * time.Millisecond defaultLoadFromEtcdRetryTimes = int(defaultLoadDataFromEtcdTimeout / defaultLoadFromEtcdRetryInterval) defaultLoadBatchSize = 400 - watchEtcdChangeRetryInterval = 1 * time.Second + defaultWatchChangeRetryInterval = 1 * time.Second ) // LoopWatcher loads data from etcd and sets a watcher for it. @@ -392,26 +392,29 @@ type LoopWatcher struct { loadRetryTimes int // loadBatchSize is used to set the batch size for loading data from etcd. loadBatchSize int64 + // watchChangeRetryInterval is used to set the retry interval for watching etcd change. + watchChangeRetryInterval time.Duration } // NewLoopWatcher creates a new LoopWatcher. func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Client, name, key string, putFn, deleteFn func(*mvccpb.KeyValue) error, postEventFn func() error, opts ...clientv3.OpOption) *LoopWatcher { return &LoopWatcher{ - ctx: ctx, - client: client, - name: name, - key: key, - wg: wg, - forceLoadCh: make(chan struct{}, 1), - isLoadedCh: make(chan error, 1), - putFn: putFn, - deleteFn: deleteFn, - postEventFn: postEventFn, - opts: opts, - loadTimeout: defaultLoadDataFromEtcdTimeout, - loadRetryTimes: defaultLoadFromEtcdRetryTimes, - loadBatchSize: defaultLoadBatchSize, + ctx: ctx, + client: client, + name: name, + key: key, + wg: wg, + forceLoadCh: make(chan struct{}, 1), + isLoadedCh: make(chan error, 1), + putFn: putFn, + deleteFn: deleteFn, + postEventFn: postEventFn, + opts: opts, + loadTimeout: defaultLoadDataFromEtcdTimeout, + loadRetryTimes: defaultLoadFromEtcdRetryTimes, + loadBatchSize: defaultLoadBatchSize, + watchChangeRetryInterval: defaultWatchChangeRetryInterval, } } @@ -438,10 +441,10 @@ func (lw *LoopWatcher) StartWatchLoop() { zap.String("name", lw.name), zap.String("key", lw.key), zap.Int64("next-revision", nextRevision), - zap.Time("retry-at", time.Now().Add(watchEtcdChangeRetryInterval)), + zap.Time("retry-at", time.Now().Add(lw.watchChangeRetryInterval)), zap.Error(err)) watchStartRevision = nextRevision - time.Sleep(watchEtcdChangeRetryInterval) + time.Sleep(lw.watchChangeRetryInterval) } } } diff --git a/pkg/utils/etcdutil/etcdutil_test.go b/pkg/utils/etcdutil/etcdutil_test.go index e8aa901bee0..071975d5cbf 100644 --- a/pkg/utils/etcdutil/etcdutil_test.go +++ b/pkg/utils/etcdutil/etcdutil_test.go @@ -21,19 +21,28 @@ import ( "io" "net" "strings" + "sync" "sync/atomic" "testing" "time" "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" "github.com/tikv/pd/pkg/utils/tempurl" + "github.com/tikv/pd/pkg/utils/testutil" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" "go.etcd.io/etcd/etcdserver/etcdserverpb" + "go.etcd.io/etcd/mvcc/mvccpb" "go.etcd.io/etcd/pkg/types" + "go.uber.org/goleak" ) +func TestMain(m *testing.M) { + goleak.VerifyTestMain(m, testutil.LeakOptions...) +} + func TestMemberHelpers(t *testing.T) { re := require.New(t) cfg1 := NewTestSingleConfig(t) @@ -408,3 +417,237 @@ func ioCopy(dst io.Writer, src io.Reader, enableDiscard *atomic.Bool) (err error } return err } + +type loopWatcherTestSuite struct { + suite.Suite + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup + cleans []func() + etcd *embed.Etcd + client *clientv3.Client + config *embed.Config +} + +func TestLoopWatcherTestSuite(t *testing.T) { + suite.Run(t, new(loopWatcherTestSuite)) +} + +func (suite *loopWatcherTestSuite) SetupSuite() { + t := suite.T() + suite.ctx, suite.cancel = context.WithCancel(context.Background()) + suite.cleans = make([]func(), 0) + // Start a etcd server and create a client with etcd1 as endpoint. + suite.config = NewTestSingleConfig(t) + suite.startEtcd() + ep1 := suite.config.LCUrls[0].String() + urls, err := types.NewURLs([]string{ep1}) + suite.NoError(err) + suite.client, err = createEtcdClient(nil, urls[0]) + suite.NoError(err) + suite.cleans = append(suite.cleans, func() { + suite.client.Close() + }) +} + +func (suite *loopWatcherTestSuite) TearDownSuite() { + suite.cancel() + suite.wg.Wait() + for _, clean := range suite.cleans { + clean() + } +} + +func (suite *loopWatcherTestSuite) TestLoadWithoutKey() { + watcher := NewLoopWatcher( + suite.ctx, + &suite.wg, + suite.client, + "test", + "TestLoadWithoutKey", + func(kv *mvccpb.KeyValue) error { return nil }, + func(kv *mvccpb.KeyValue) error { return nil }, + func() error { return nil }, + ) + + suite.wg.Add(1) + go watcher.StartWatchLoop() + err := watcher.WaitLoad() + suite.NoError(err) // although no key, watcher returns no error +} + +func (suite *loopWatcherTestSuite) TestCallBack() { + cache := make(map[string]struct{}) + result := make([]string, 0) + watcher := NewLoopWatcher( + suite.ctx, + &suite.wg, + suite.client, + "test", + "TestCallBack", + func(kv *mvccpb.KeyValue) error { + result = append(result, string(kv.Key)) + return nil + }, + func(kv *mvccpb.KeyValue) error { + delete(cache, string(kv.Key)) + return nil + }, + func() error { + for _, r := range result { + cache[r] = struct{}{} + } + result = result[:0] + return nil + }, + clientv3.WithPrefix(), + ) + + suite.wg.Add(1) + go watcher.StartWatchLoop() + err := watcher.WaitLoad() + suite.NoError(err) + + // put 10 keys + for i := 0; i < 10; i++ { + suite.put(fmt.Sprintf("TestCallBack%d", i), "") + } + time.Sleep(time.Second) + suite.Len(cache, 10) + + // delete 10 keys + for i := 0; i < 10; i++ { + key := fmt.Sprintf("TestCallBack%d", i) + _, err = suite.client.Delete(suite.ctx, key) + suite.NoError(err) + } + time.Sleep(time.Second) + suite.Empty(cache) +} + +func (suite *loopWatcherTestSuite) TestWatcherLoadLimit() { + for count := 1; count < 10; count++ { + for limit := 0; limit < 10; limit++ { + ctx, cancel := context.WithCancel(suite.ctx) + for i := 0; i < count; i++ { + suite.put(fmt.Sprintf("TestWatcherLoadLimit%d", i), "") + } + cache := []string{} + watcher := NewLoopWatcher( + ctx, + &suite.wg, + suite.client, + "test", + "TestWatcherLoadLimit", + func(kv *mvccpb.KeyValue) error { + cache = append(cache, string(kv.Key)) + return nil + }, + func(kv *mvccpb.KeyValue) error { + return nil + }, + func() error { + return nil + }, + clientv3.WithPrefix(), + ) + suite.wg.Add(1) + go watcher.StartWatchLoop() + err := watcher.WaitLoad() + suite.NoError(err) + suite.Len(cache, count) + cancel() + } + } +} + +func (suite *loopWatcherTestSuite) TestWatcherBreak() { + cache := make(map[string]string) + watcher := NewLoopWatcher( + suite.ctx, + &suite.wg, + suite.client, + "test", + "TestWatcherBreak", + func(kv *mvccpb.KeyValue) error { cache[string(kv.Key)] = string(kv.Value); return nil }, + func(kv *mvccpb.KeyValue) error { delete(cache, string(kv.Key)); return nil }, + func() error { return nil }, + ) + watcher.watchChangeRetryInterval = 100 * time.Millisecond + + suite.wg.Add(1) + go watcher.StartWatchLoop() + err := watcher.WaitLoad() + suite.NoError(err) + + // Case1: restart the etcd server + suite.Empty(cache) + suite.etcd.Close() + suite.startEtcd() + suite.put("TestWatcherBreak", "1") + time.Sleep(watcher.watchChangeRetryInterval) + suite.Len(cache, 1) + suite.Equal("1", cache["TestWatcherBreak"]) + + // Case2: close the etcd client and put a new value after watcher restarts + suite.client.Close() + suite.client, err = createEtcdClient(nil, suite.config.LCUrls[0]) + suite.NoError(err) + watcher.client = suite.client + suite.put("TestWatcherBreak", "2") + testutil.Eventually(suite.Require(), func() bool { + return cache["TestWatcherBreak"] == "2" + }, testutil.WithWaitFor(time.Second)) + + // Case3: close the etcd client and put a new value before watcher restarts + suite.client.Close() + suite.client, err = createEtcdClient(nil, suite.config.LCUrls[0]) + suite.NoError(err) + suite.put("TestWatcherBreak", "3") + watcher.client = suite.client + testutil.Eventually(suite.Require(), func() bool { + return cache["TestWatcherBreak"] == "3" + }, testutil.WithWaitFor(time.Second)) + + // Case4: close the etcd client and put a new value with compact + suite.client.Close() + suite.client, err = createEtcdClient(nil, suite.config.LCUrls[0]) + suite.NoError(err) + suite.put("TestWatcherBreak", "4") + resp, err := EtcdKVGet(suite.client, "TestWatcherBreak") + suite.NoError(err) + revision := resp.Header.Revision + resp2, err := suite.etcd.Server.Compact(suite.ctx, &etcdserverpb.CompactionRequest{Revision: revision}) + suite.NoError(err) + suite.Equal(revision, resp2.Header.Revision) + watcher.client = suite.client + testutil.Eventually(suite.Require(), func() bool { + return cache["TestWatcherBreak"] == "4" + }, testutil.WithWaitFor(time.Second)) + + // Case5: there is an error data in cache + cache["TestWatcherBreak"] = "error" + watcher.ForceLoad() + testutil.Eventually(suite.Require(), func() bool { + return cache["TestWatcherBreak"] == "4" + }, testutil.WithWaitFor(time.Second)) +} + +func (suite *loopWatcherTestSuite) startEtcd() { + etcd1, err := embed.StartEtcd(suite.config) + suite.NoError(err) + suite.etcd = etcd1 + <-etcd1.Server.ReadyNotify() + suite.cleans = append(suite.cleans, func() { + suite.etcd.Close() + }) +} + +func (suite *loopWatcherTestSuite) put(key, value string) { + kv := clientv3.NewKV(suite.client) + _, err := kv.Put(suite.ctx, key, value) + suite.NoError(err) + resp, err := kv.Get(suite.ctx, key) + suite.NoError(err) + suite.Equal(value, string(resp.Kvs[0].Value)) +} diff --git a/server/keyspace_service.go b/server/keyspace_service.go index 9a7b207e925..4069987510d 100644 --- a/server/keyspace_service.go +++ b/server/keyspace_service.go @@ -90,7 +90,7 @@ func (s *KeyspaceServer) WatchKeyspaces(request *keyspacepb.WatchKeyspacesReques } postEventFn := func() error { defer func() { - keyspaces = make([]*keyspacepb.KeyspaceMeta, 0) + keyspaces = keyspaces[:0] }() return stream.Send(&keyspacepb.WatchKeyspacesResponse{ Header: s.header(), From 0c6a5cfdc17c1326eb2a671b6e957f969ec3a4e5 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Sat, 6 May 2023 14:34:43 +0800 Subject: [PATCH 21/24] address comments Signed-off-by: lhy1024 --- pkg/keyspace/tso_keyspace_group.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index cee8339cfff..51b9fcf0c55 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -96,8 +96,8 @@ func NewKeyspaceGroupManager( // If the etcd client is not nil, start the watch loop for the registered tso servers. // The PD(TSO) Client relies on this info to discover tso servers. if client != nil { - m.wg.Add(2) m.initTSONodesWatcher(client, clusterID) + m.wg.Add(2) go m.tsoNodesWatcher.StartWatchLoop() go m.allocNodesToAllKeyspaceGroups() } From c7473117a59a42490411f46f428fd3158a71ecf9 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Sat, 6 May 2023 14:50:05 +0800 Subject: [PATCH 22/24] address comments and fix tests Signed-off-by: lhy1024 --- pkg/keyspace/tso_keyspace_group.go | 4 +- pkg/utils/etcdutil/etcdutil.go | 2 +- pkg/utils/etcdutil/etcdutil_test.go | 104 ++++++++++++++++++++-------- 3 files changed, 78 insertions(+), 32 deletions(-) diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index 51b9fcf0c55..46810be92d5 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -63,10 +63,10 @@ type GroupManager struct { store endpoint.KeyspaceGroupStorage // nodeBalancer is the balancer for tso nodes. - // TODO: add user kind with different balancer - // when we ensure where the correspondence between tso node and user kind will be found + // TODO: add user kind with different balancer when we ensure where the correspondence between tso node and user kind will be found nodesBalancer balancer.Balancer[string] // serviceRegistryMap stores the mapping from the service registry key to the service address. + // Note: it is only used in tsoNodesWatcher. serviceRegistryMap map[string]string // tsoNodesWatcher is the watcher for the registered tso servers. tsoNodesWatcher *etcdutil.LoopWatcher diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index d091b3eb83f..e1f7a539ae6 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -481,7 +481,7 @@ func (lw *LoopWatcher) initFromEtcd(ctx context.Context) int64 { } } if err != nil { - log.Warn("watched key doesn't exist in watch loop when loading", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) + log.Warn("meet error when loading in watch loop", zap.String("name", lw.name), zap.String("key", lw.key), zap.Error(err)) } lw.isLoadedCh <- err return watchStartRevision diff --git a/pkg/utils/etcdutil/etcdutil_test.go b/pkg/utils/etcdutil/etcdutil_test.go index 071975d5cbf..1a2083a9c02 100644 --- a/pkg/utils/etcdutil/etcdutil_test.go +++ b/pkg/utils/etcdutil/etcdutil_test.go @@ -459,13 +459,24 @@ func (suite *loopWatcherTestSuite) TearDownSuite() { } func (suite *loopWatcherTestSuite) TestLoadWithoutKey() { + cache := struct { + sync.RWMutex + data map[string]struct{} + }{ + data: make(map[string]struct{}), + } watcher := NewLoopWatcher( suite.ctx, &suite.wg, suite.client, "test", "TestLoadWithoutKey", - func(kv *mvccpb.KeyValue) error { return nil }, + func(kv *mvccpb.KeyValue) error { + cache.Lock() + defer cache.Unlock() + cache.data[string(kv.Key)] = struct{}{} + return nil + }, func(kv *mvccpb.KeyValue) error { return nil }, func() error { return nil }, ) @@ -474,10 +485,18 @@ func (suite *loopWatcherTestSuite) TestLoadWithoutKey() { go watcher.StartWatchLoop() err := watcher.WaitLoad() suite.NoError(err) // although no key, watcher returns no error + cache.RLock() + defer cache.RUnlock() + suite.Len(cache.data, 0) } func (suite *loopWatcherTestSuite) TestCallBack() { - cache := make(map[string]struct{}) + cache := struct { + sync.RWMutex + data map[string]struct{} + }{ + data: make(map[string]struct{}), + } result := make([]string, 0) watcher := NewLoopWatcher( suite.ctx, @@ -490,12 +509,16 @@ func (suite *loopWatcherTestSuite) TestCallBack() { return nil }, func(kv *mvccpb.KeyValue) error { - delete(cache, string(kv.Key)) + cache.Lock() + defer cache.Unlock() + delete(cache.data, string(kv.Key)) return nil }, func() error { + cache.Lock() + defer cache.Unlock() for _, r := range result { - cache[r] = struct{}{} + cache.data[r] = struct{}{} } result = result[:0] return nil @@ -513,7 +536,9 @@ func (suite *loopWatcherTestSuite) TestCallBack() { suite.put(fmt.Sprintf("TestCallBack%d", i), "") } time.Sleep(time.Second) - suite.Len(cache, 10) + cache.RLock() + suite.Len(cache.data, 10) + cache.RUnlock() // delete 10 keys for i := 0; i < 10; i++ { @@ -522,7 +547,9 @@ func (suite *loopWatcherTestSuite) TestCallBack() { suite.NoError(err) } time.Sleep(time.Second) - suite.Empty(cache) + cache.RLock() + suite.Empty(cache.data) + cache.RUnlock() } func (suite *loopWatcherTestSuite) TestWatcherLoadLimit() { @@ -532,7 +559,12 @@ func (suite *loopWatcherTestSuite) TestWatcherLoadLimit() { for i := 0; i < count; i++ { suite.put(fmt.Sprintf("TestWatcherLoadLimit%d", i), "") } - cache := []string{} + cache := struct { + sync.RWMutex + data []string + }{ + data: make([]string, 0), + } watcher := NewLoopWatcher( ctx, &suite.wg, @@ -540,7 +572,9 @@ func (suite *loopWatcherTestSuite) TestWatcherLoadLimit() { "test", "TestWatcherLoadLimit", func(kv *mvccpb.KeyValue) error { - cache = append(cache, string(kv.Key)) + cache.Lock() + defer cache.Unlock() + cache.data = append(cache.data, string(kv.Key)) return nil }, func(kv *mvccpb.KeyValue) error { @@ -555,22 +589,42 @@ func (suite *loopWatcherTestSuite) TestWatcherLoadLimit() { go watcher.StartWatchLoop() err := watcher.WaitLoad() suite.NoError(err) - suite.Len(cache, count) + cache.RLock() + suite.Len(cache.data, count) + cache.RUnlock() cancel() } } } func (suite *loopWatcherTestSuite) TestWatcherBreak() { - cache := make(map[string]string) + cache := struct { + sync.RWMutex + data string + }{} + checkCache := func(expect string) { + testutil.Eventually(suite.Require(), func() bool { + cache.RLock() + defer cache.RUnlock() + return cache.data == expect + }, testutil.WithWaitFor(time.Second)) + } + watcher := NewLoopWatcher( suite.ctx, &suite.wg, suite.client, "test", "TestWatcherBreak", - func(kv *mvccpb.KeyValue) error { cache[string(kv.Key)] = string(kv.Value); return nil }, - func(kv *mvccpb.KeyValue) error { delete(cache, string(kv.Key)); return nil }, + func(kv *mvccpb.KeyValue) error { + if string(kv.Key) == "TestWatcherBreak" { + cache.Lock() + defer cache.Unlock() + cache.data = string(kv.Value) + } + return nil + }, + func(kv *mvccpb.KeyValue) error { return nil }, func() error { return nil }, ) watcher.watchChangeRetryInterval = 100 * time.Millisecond @@ -579,15 +633,13 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { go watcher.StartWatchLoop() err := watcher.WaitLoad() suite.NoError(err) + checkCache("") // Case1: restart the etcd server - suite.Empty(cache) suite.etcd.Close() suite.startEtcd() suite.put("TestWatcherBreak", "1") - time.Sleep(watcher.watchChangeRetryInterval) - suite.Len(cache, 1) - suite.Equal("1", cache["TestWatcherBreak"]) + checkCache("1") // Case2: close the etcd client and put a new value after watcher restarts suite.client.Close() @@ -595,9 +647,7 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { suite.NoError(err) watcher.client = suite.client suite.put("TestWatcherBreak", "2") - testutil.Eventually(suite.Require(), func() bool { - return cache["TestWatcherBreak"] == "2" - }, testutil.WithWaitFor(time.Second)) + checkCache("2") // Case3: close the etcd client and put a new value before watcher restarts suite.client.Close() @@ -605,9 +655,7 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { suite.NoError(err) suite.put("TestWatcherBreak", "3") watcher.client = suite.client - testutil.Eventually(suite.Require(), func() bool { - return cache["TestWatcherBreak"] == "3" - }, testutil.WithWaitFor(time.Second)) + checkCache("3") // Case4: close the etcd client and put a new value with compact suite.client.Close() @@ -621,16 +669,14 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { suite.NoError(err) suite.Equal(revision, resp2.Header.Revision) watcher.client = suite.client - testutil.Eventually(suite.Require(), func() bool { - return cache["TestWatcherBreak"] == "4" - }, testutil.WithWaitFor(time.Second)) + checkCache("4") // Case5: there is an error data in cache - cache["TestWatcherBreak"] = "error" + cache.Lock() + cache.data = "error" + cache.Unlock() watcher.ForceLoad() - testutil.Eventually(suite.Require(), func() bool { - return cache["TestWatcherBreak"] == "4" - }, testutil.WithWaitFor(time.Second)) + checkCache("4") } func (suite *loopWatcherTestSuite) startEtcd() { From cb5815b754febf06ea0b8485d70e9bfc29a1c544 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Sat, 6 May 2023 15:37:20 +0800 Subject: [PATCH 23/24] fix other test leak Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil_test.go | 36 +++++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/pkg/utils/etcdutil/etcdutil_test.go b/pkg/utils/etcdutil/etcdutil_test.go index 1a2083a9c02..0ab5d40b624 100644 --- a/pkg/utils/etcdutil/etcdutil_test.go +++ b/pkg/utils/etcdutil/etcdutil_test.go @@ -56,6 +56,9 @@ func TestMemberHelpers(t *testing.T) { client1, err := clientv3.New(clientv3.Config{ Endpoints: []string{ep1}, }) + defer func() { + client1.Close() + }() re.NoError(err) <-etcd1.Server.ReadyNotify() @@ -75,6 +78,9 @@ func TestMemberHelpers(t *testing.T) { client2, err := clientv3.New(clientv3.Config{ Endpoints: []string{ep2}, }) + defer func() { + client2.Close() + }() re.NoError(err) checkMembers(re, client2, []*embed.Etcd{etcd1, etcd2}) @@ -107,6 +113,9 @@ func TestEtcdKVGet(t *testing.T) { client, err := clientv3.New(clientv3.Config{ Endpoints: []string{ep}, }) + defer func() { + client.Close() + }() re.NoError(err) <-etcd.Server.ReadyNotify() @@ -157,6 +166,9 @@ func TestEtcdKVPutWithTTL(t *testing.T) { client, err := clientv3.New(clientv3.Config{ Endpoints: []string{ep}, }) + defer func() { + client.Close() + }() re.NoError(err) <-etcd.Server.ReadyNotify() @@ -197,6 +209,9 @@ func TestInitClusterID(t *testing.T) { client, err := clientv3.New(clientv3.Config{ Endpoints: []string{ep}, }) + defer func() { + client.Close() + }() re.NoError(err) <-etcd.Server.ReadyNotify() @@ -223,6 +238,9 @@ func TestEtcdClientSync(t *testing.T) { // Start a etcd server. cfg1 := NewTestSingleConfig(t) etcd1, err := embed.StartEtcd(cfg1) + defer func() { + etcd1.Close() + }() re.NoError(err) // Create a etcd client with etcd1 as endpoint. @@ -230,6 +248,9 @@ func TestEtcdClientSync(t *testing.T) { urls, err := types.NewURLs([]string{ep1}) re.NoError(err) client1, err := createEtcdClientWithMultiEndpoint(nil, urls) + defer func() { + client1.Close() + }() re.NoError(err) <-etcd1.Server.ReadyNotify() @@ -274,6 +295,9 @@ func TestEtcdScaleInAndOutWithoutMultiPoint(t *testing.T) { // Start a etcd server. cfg1 := NewTestSingleConfig(t) etcd1, err := embed.StartEtcd(cfg1) + defer func() { + etcd1.Close() + }() re.NoError(err) ep1 := cfg1.LCUrls[0].String() <-etcd1.Server.ReadyNotify() @@ -282,8 +306,14 @@ func TestEtcdScaleInAndOutWithoutMultiPoint(t *testing.T) { urls, err := types.NewURLs([]string{ep1}) re.NoError(err) client1, err := createEtcdClient(nil, urls[0]) // execute member change operation with this client + defer func() { + client1.Close() + }() re.NoError(err) client2, err := createEtcdClient(nil, urls[0]) // check member change with this client + defer func() { + client2.Close() + }() re.NoError(err) // Add a new member and check members @@ -301,6 +331,9 @@ func checkEtcdWithHangLeader(t *testing.T) error { // Start a etcd server. cfg1 := NewTestSingleConfig(t) etcd1, err := embed.StartEtcd(cfg1) + defer func() { + etcd1.Close() + }() re.NoError(err) ep1 := cfg1.LCUrls[0].String() <-etcd1.Server.ReadyNotify() @@ -314,6 +347,9 @@ func checkEtcdWithHangLeader(t *testing.T) error { urls, err := types.NewURLs([]string{proxyAddr}) re.NoError(err) client1, err := createEtcdClientWithMultiEndpoint(nil, urls) + defer func() { + client1.Close() + }() re.NoError(err) // Add a new member and set the client endpoints to etcd1 and etcd2. From 82efaae59ed674ecd1abfbc676307cdb402020f7 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Sun, 7 May 2023 01:18:44 +0800 Subject: [PATCH 24/24] fix test again Signed-off-by: lhy1024 --- pkg/utils/etcdutil/etcdutil.go | 7 +++++++ pkg/utils/etcdutil/etcdutil_test.go | 14 +++++++++++--- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index e1f7a539ae6..32f32bd6087 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -394,6 +394,9 @@ type LoopWatcher struct { loadBatchSize int64 // watchChangeRetryInterval is used to set the retry interval for watching etcd change. watchChangeRetryInterval time.Duration + // updateClientCh is used to update the etcd client. + // It's only used for testing. + updateClientCh chan *clientv3.Client } // NewLoopWatcher creates a new LoopWatcher. @@ -407,6 +410,7 @@ func NewLoopWatcher(ctx context.Context, wg *sync.WaitGroup, client *clientv3.Cl wg: wg, forceLoadCh: make(chan struct{}, 1), isLoadedCh: make(chan error, 1), + updateClientCh: make(chan *clientv3.Client, 1), putFn: putFn, deleteFn: deleteFn, postEventFn: postEventFn, @@ -445,6 +449,9 @@ func (lw *LoopWatcher) StartWatchLoop() { zap.Error(err)) watchStartRevision = nextRevision time.Sleep(lw.watchChangeRetryInterval) + failpoint.Inject("updateClient", func() { + lw.client = <-lw.updateClientCh + }) } } } diff --git a/pkg/utils/etcdutil/etcdutil_test.go b/pkg/utils/etcdutil/etcdutil_test.go index 0ab5d40b624..6bf63db79c9 100644 --- a/pkg/utils/etcdutil/etcdutil_test.go +++ b/pkg/utils/etcdutil/etcdutil_test.go @@ -318,6 +318,9 @@ func TestEtcdScaleInAndOutWithoutMultiPoint(t *testing.T) { // Add a new member and check members etcd2 := checkAddEtcdMember(t, cfg1, client1) + defer func() { + etcd2.Close() + }() checkMembers(re, client2, []*embed.Etcd{etcd1, etcd2}) // scale in etcd1 @@ -671,6 +674,9 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { suite.NoError(err) checkCache("") + // we use close client and update client in failpoint to simulate the network error and recover + failpoint.Enable("github.com/tikv/pd/pkg/utils/etcdutil/updateClient", "return(true)") + // Case1: restart the etcd server suite.etcd.Close() suite.startEtcd() @@ -681,7 +687,7 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { suite.client.Close() suite.client, err = createEtcdClient(nil, suite.config.LCUrls[0]) suite.NoError(err) - watcher.client = suite.client + watcher.updateClientCh <- suite.client suite.put("TestWatcherBreak", "2") checkCache("2") @@ -690,7 +696,7 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { suite.client, err = createEtcdClient(nil, suite.config.LCUrls[0]) suite.NoError(err) suite.put("TestWatcherBreak", "3") - watcher.client = suite.client + watcher.updateClientCh <- suite.client checkCache("3") // Case4: close the etcd client and put a new value with compact @@ -704,7 +710,7 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { resp2, err := suite.etcd.Server.Compact(suite.ctx, &etcdserverpb.CompactionRequest{Revision: revision}) suite.NoError(err) suite.Equal(revision, resp2.Header.Revision) - watcher.client = suite.client + watcher.updateClientCh <- suite.client checkCache("4") // Case5: there is an error data in cache @@ -713,6 +719,8 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { cache.Unlock() watcher.ForceLoad() checkCache("4") + + failpoint.Disable("github.com/tikv/pd/pkg/utils/etcdutil/updateClient") } func (suite *loopWatcherTestSuite) startEtcd() {