From 2d801e2c788402e190279b0da4b759bc04bb30f8 Mon Sep 17 00:00:00 2001 From: andrewshan Date: Fri, 29 Oct 2021 14:46:33 +0800 Subject: [PATCH 1/5] #104: fix health status not changed in some cases --- .github/workflows/testing.yml | 2 +- apiserver/grpcserver/client_access.go | 2 +- apiserver/grpcserver/server.go | 28 +- apiserver/httpserver/client_access.go | 2 +- apiserver/httpserver/console_access.go | 16 +- apiserver/httpserver/handler.go | 35 +- apiserver/httpserver/maintain_access.go | 4 +- apiserver/httpserver/server.go | 19 +- bootstrap/server.go | 38 +- common/api/v1/codeinfo.go | 1 + common/model/metadata.go | 26 + common/redispool/redis_pool.go | 437 ++++++++-------- common/srand/scalable_rand.go | 93 ++++ common/utils/funcs.go | 2 - config/config.go | 14 +- go.mod | 1 + healthcheck/cache.go | 239 +++++++++ healthcheck/check.go | 326 ++++++++++++ healthcheck/config.go | 29 ++ healthcheck/dispatch.go | 184 +++++++ healthcheck/ketama.go | 123 +++++ healthcheck/report.go | 79 +++ healthcheck/server.go | 179 +++++++ healthcheck/time_adjust.go | 69 +++ naming/batch/instance.go | 4 +- naming/cache/cache.go | 78 +-- naming/cache/cache_test.go | 4 +- naming/cache/instance.go | 72 +-- naming/cache/instance_test.go | 2 +- naming/cache/listener.go | 74 +++ naming/cache/service.go | 7 +- naming/client.go | 14 - naming/healthcheck.go | 478 ------------------ naming/instance.go | 80 +-- naming/server.go | 58 +-- naming/test/main_test.go | 2 +- naming/test/run.sh | 2 +- naming/testauthplugin/main_test.go | 5 +- plugin.go | 7 +- .../discoverlocal/README.md | 0 .../discoverlocal/discovercall.go | 0 .../discoverlocal/local.go | 0 .../discoverlocal/local_test.go | 0 .../discoverlocal/logger.go | 0 plugin/healthchecker.go | 100 ++++ .../heartbeatmemory/checker_memory.go | 112 ++++ .../heartbeatredis/checker_redis.go | 189 +++++++ plugin/{parsePassword.go => password.go} | 0 .../{tokenbucket => token}/api_limit.go | 2 +- .../{tokenbucket => token}/api_limit_test.go | 2 +- .../{tokenbucket => token}/config.go | 2 +- .../{tokenbucket => token}/implement.go | 2 +- .../{tokenbucket => token}/invoke.go | 2 +- .../{tokenbucket => token}/invoke_test.go | 2 +- .../{tokenbucket => token}/limiter.go | 2 +- .../{tokenbucket => token}/register.go | 2 +- .../resource_limiter.go | 2 +- .../resource_limiter_test.go | 2 +- polaris-server.yaml | 33 +- store/api.go | 9 + store/boltdb/default.go | 1 + store/boltdb/tool.go | 29 ++ store/mock/api_mock.go | 7 +- store/sqldb/default.go | 4 + store/sqldb/polaris_server.sql | 4 +- store/sqldb/tool.go | 45 ++ version | 2 +- 67 files changed, 2372 insertions(+), 1018 deletions(-) create mode 100644 common/model/metadata.go create mode 100644 common/srand/scalable_rand.go create mode 100644 healthcheck/cache.go create mode 100644 healthcheck/check.go create mode 100644 healthcheck/config.go create mode 100644 healthcheck/dispatch.go create mode 100644 healthcheck/ketama.go create mode 100644 healthcheck/report.go create mode 100644 healthcheck/server.go create mode 100644 healthcheck/time_adjust.go create mode 100644 naming/cache/listener.go delete mode 100644 naming/healthcheck.go rename plugin/{discoverstatis => discoverstat}/discoverlocal/README.md (100%) rename plugin/{discoverstatis => discoverstat}/discoverlocal/discovercall.go (100%) rename plugin/{discoverstatis => discoverstat}/discoverlocal/local.go (100%) rename plugin/{discoverstatis => discoverstat}/discoverlocal/local_test.go (100%) rename plugin/{discoverstatis => discoverstat}/discoverlocal/logger.go (100%) create mode 100644 plugin/healthchecker.go create mode 100644 plugin/healthchecker/heartbeatmemory/checker_memory.go create mode 100644 plugin/healthchecker/heartbeatredis/checker_redis.go rename plugin/{parsePassword.go => password.go} (100%) rename plugin/ratelimit/{tokenbucket => token}/api_limit.go (99%) rename plugin/ratelimit/{tokenbucket => token}/api_limit_test.go (99%) rename plugin/ratelimit/{tokenbucket => token}/config.go (99%) rename plugin/ratelimit/{tokenbucket => token}/implement.go (98%) rename plugin/ratelimit/{tokenbucket => token}/invoke.go (98%) rename plugin/ratelimit/{tokenbucket => token}/invoke_test.go (99%) rename plugin/ratelimit/{tokenbucket => token}/limiter.go (97%) rename plugin/ratelimit/{tokenbucket => token}/register.go (97%) rename plugin/ratelimit/{tokenbucket => token}/resource_limiter.go (99%) rename plugin/ratelimit/{tokenbucket => token}/resource_limiter_test.go (99%) create mode 100644 store/boltdb/tool.go create mode 100644 store/sqldb/tool.go diff --git a/.github/workflows/testing.yml b/.github/workflows/testing.yml index a9aab5877..4923083ef 100644 --- a/.github/workflows/testing.yml +++ b/.github/workflows/testing.yml @@ -41,7 +41,7 @@ jobs: pushd ./naming/cache go test -v popd - pushd ./plugin/ratelimit/tokenbucket + pushd ./plugin/ratelimit/token go test -v popd pushd ./store/sqldb diff --git a/apiserver/grpcserver/client_access.go b/apiserver/grpcserver/client_access.go index 5313e316d..8c02fffa5 100644 --- a/apiserver/grpcserver/client_access.go +++ b/apiserver/grpcserver/client_access.go @@ -138,7 +138,7 @@ func (g *GRPCServer) Discover(server api.PolarisGRPC_DiscoverServer) error { * @brief 上报心跳 */ func (g *GRPCServer) Heartbeat(ctx context.Context, in *api.Instance) (*api.Response, error) { - out := g.namingServer.Heartbeat(convertContext(ctx), in) + out := g.healthCheckServer.Report(convertContext(ctx), in) return out, nil } diff --git a/apiserver/grpcserver/server.go b/apiserver/grpcserver/server.go index e30e409bc..4c8b981a1 100644 --- a/apiserver/grpcserver/server.go +++ b/apiserver/grpcserver/server.go @@ -20,6 +20,7 @@ package grpcserver import ( "context" "fmt" + "github.com/polarismesh/polaris-server/healthcheck" "io" "net" "net/http" @@ -49,32 +50,27 @@ type GRPCServer struct { restart bool exitCh chan struct{} - server *grpc.Server - namingServer *naming.Server - statis plugin.Statis - ratelimit plugin.Ratelimit + server *grpc.Server + namingServer *naming.Server + healthCheckServer *healthcheck.Server + statis plugin.Statis + ratelimit plugin.Ratelimit openAPI map[string]apiserver.APIConfig openMethod map[string]bool } -/** - * @brief 获取端口 - */ +// GetPort 获取端口 func (g *GRPCServer) GetPort() uint32 { return g.listenPort } -/** - * @brief 获取Server的协议 - */ +// GetProtocol 获取Server的协议 func (g *GRPCServer) GetProtocol() string { return "grpc" } -/** - * Initialize 初始化GRPC API服务器 - */ +// Initialize 初始化GRPC API服务器 func (g *GRPCServer) Initialize(_ context.Context, option map[string]interface{}, api map[string]apiserver.APIConfig) error { g.listenIP = option["listenIP"].(string) @@ -110,6 +106,12 @@ func (g *GRPCServer) Run(errCh chan error) { var err error // 引入功能模块和插件 + g.healthCheckServer, err = healthcheck.GetServer() + if err != nil { + log.Errorf("%v", err) + errCh <- err + return + } g.namingServer, err = naming.GetServer() if err != nil { log.Errorf("%v", err) diff --git a/apiserver/httpserver/client_access.go b/apiserver/httpserver/client_access.go index b47d97880..2d7d9b0ae 100644 --- a/apiserver/httpserver/client_access.go +++ b/apiserver/httpserver/client_access.go @@ -184,5 +184,5 @@ func (h *HTTPServer) Heartbeat(req *restful.Request, rsp *restful.Response) { return } - handler.WriteHeaderAndProto(h.namingServer.Heartbeat(ctx, instance)) + handler.WriteHeaderAndProto(h.healthCheckServer.Report(ctx, instance)) } diff --git a/apiserver/httpserver/console_access.go b/apiserver/httpserver/console_access.go index ff9954ace..456d45f2d 100644 --- a/apiserver/httpserver/console_access.go +++ b/apiserver/httpserver/console_access.go @@ -20,12 +20,12 @@ package httpserver import ( "context" "fmt" + proto "github.com/golang/protobuf/proto" + "github.com/polarismesh/polaris-server/common/log" "net/http" "github.com/emicklei/go-restful" - "github.com/golang/protobuf/proto" api "github.com/polarismesh/polaris-server/common/api/v1" - "github.com/polarismesh/polaris-server/common/log" "github.com/polarismesh/polaris-server/common/utils" ) @@ -34,9 +34,7 @@ const ( defaultAccess string = "default" ) -/** - * GetConsoleAccessServer 注册管理端接口 - */ +// GetConsoleAccessServer 注册管理端接口 func (h *HTTPServer) GetConsoleAccessServer(include []string) (*restful.WebService, error) { consoleAccess := []string{defaultAccess} @@ -72,9 +70,7 @@ func (h *HTTPServer) GetConsoleAccessServer(include []string) (*restful.WebServi return ws, nil } -/** - * addDefaultReadAccess 增加默认读接口 - */ +// addDefaultReadAccess 增加默认读接口 func (h *HTTPServer) addDefaultReadAccess(ws *restful.WebService) { // 管理端接口:只包含读接口 ws.Route(ws.GET("/namespaces").To(h.GetNamespaces)) @@ -107,9 +103,7 @@ func (h *HTTPServer) addDefaultReadAccess(ws *restful.WebService) { ws.Route(ws.GET("/platform/token").To(h.GetPlatformToken)) } -/** - * addDefaultAccess 增加默认接口 - */ +// addDefaultAccess 增加默认接口 func (h *HTTPServer) addDefaultAccess(ws *restful.WebService) { // 管理端接口:增删改查请求全部操作存储层 ws.Route(ws.POST("/namespaces").To(h.CreateNamespaces)) diff --git a/apiserver/httpserver/handler.go b/apiserver/httpserver/handler.go index 346bd2098..52f8dce6f 100644 --- a/apiserver/httpserver/handler.go +++ b/apiserver/httpserver/handler.go @@ -33,26 +33,13 @@ import ( "go.uber.org/zap" ) -/** - * Handler HTTP请求/回复处理器 - */ +// Handler HTTP请求/回复处理器 type Handler struct { *restful.Request *restful.Response } -/** - * Parse 解析请求 - */ -func (h *Handler) Parse(message proto.Message) (context.Context, error) { - requestID := h.Request.HeaderParameter("Request-Id") - if err := jsonpb.Unmarshal(h.Request.Request.Body, message); err != nil { - log.Error(err.Error(), zap.String("request-id", requestID)) - return nil, err - } - return h.postParseMessage(requestID) -} - +// ParseArray 解析PB数组对象 func (h *Handler) ParseArray(createMessage func() proto.Message) (context.Context, error) { requestID := h.Request.HeaderParameter("Request-Id") @@ -102,9 +89,17 @@ func (h *Handler) postParseMessage(requestID string) (context.Context, error) { return ctx, nil } -/** - * WriteHeader 仅返回Code - */ +// Parse 解析请求 +func (h *Handler) Parse(message proto.Message) (context.Context, error) { + requestID := h.Request.HeaderParameter("Request-Id") + if err := jsonpb.Unmarshal(h.Request.Request.Body, message); err != nil { + log.Error(err.Error(), zap.String("request-id", requestID)) + return nil, err + } + return h.postParseMessage(requestID) +} + +// WriteHeader 仅返回Code func (h *Handler) WriteHeader(polarisCode uint32, httpStatus int) { requestID := h.Request.HeaderParameter(utils.PolarisRequestID) h.Request.SetAttribute(utils.PolarisCode, polarisCode) // api统计的时候,用该code @@ -118,9 +113,7 @@ func (h *Handler) WriteHeader(polarisCode uint32, httpStatus int) { h.Response.WriteHeader(httpStatus) } -/** - * WriteHeaderAndProto 返回Code和Proto - */ +// WriteHeaderAndProto 返回Code和Proto func (h *Handler) WriteHeaderAndProto(obj api.ResponseMessage) { requestID := h.Request.HeaderParameter(utils.PolarisRequestID) h.Request.SetAttribute(utils.PolarisCode, obj.GetCode().GetValue()) diff --git a/apiserver/httpserver/maintain_access.go b/apiserver/httpserver/maintain_access.go index 38bc8f5f9..93f1f26e8 100644 --- a/apiserver/httpserver/maintain_access.go +++ b/apiserver/httpserver/maintain_access.go @@ -215,7 +215,7 @@ func (h *HTTPServer) GetLastHeartbeat(req *restful.Request, rsp *restful.Respons instance := &api.Instance{} if id, ok := params["id"]; ok && id != "" { instance.Id = utils.NewStringValue(id) - ret := h.namingServer.GetLastHeartbeat(instance) + ret := h.healthCheckServer.GetLastHeartbeat(instance) handler.WriteHeaderAndProto(ret) return } @@ -227,6 +227,6 @@ func (h *HTTPServer) GetLastHeartbeat(req *restful.Request, rsp *restful.Respons port, _ := strconv.Atoi(params["port"]) instance.Port = utils.NewUInt32Value(uint32(port)) - ret := h.namingServer.GetLastHeartbeat(instance) + ret := h.healthCheckServer.GetLastHeartbeat(instance) handler.WriteHeaderAndProto(ret) } diff --git a/apiserver/httpserver/server.go b/apiserver/httpserver/server.go index 447dee678..74ce56778 100644 --- a/apiserver/httpserver/server.go +++ b/apiserver/httpserver/server.go @@ -20,6 +20,7 @@ package httpserver import ( "context" "fmt" + "github.com/polarismesh/polaris-server/healthcheck" "net" "net/http" "net/http/pprof" @@ -58,11 +59,12 @@ type HTTPServer struct { freeMemMu *sync.Mutex - server *http.Server - namingServer *naming.Server - rateLimit plugin.Ratelimit - statis plugin.Statis - auth plugin.Auth + server *http.Server + namingServer *naming.Server + healthCheckServer *healthcheck.Server + rateLimit plugin.Ratelimit + statis plugin.Statis + auth plugin.Auth } const ( @@ -137,6 +139,12 @@ func (h *HTTPServer) Run(errCh chan error) { errCh <- err return } + h.healthCheckServer, err = healthcheck.GetServer() + if err != nil { + log.Errorf("%v", err) + errCh <- err + return + } h.statis = plugin.GetStatis() // 初始化http server @@ -159,7 +167,6 @@ func (h *HTTPServer) Run(errCh chan error) { } ln = &tcpKeepAliveListener{ln.(*net.TCPListener)} - // 开启最大连接数限制 if h.connLimitConfig != nil && h.connLimitConfig.OpenConnLimit { log.Infof("http server use max connection limit per ip: %d, http max limit: %d", diff --git a/bootstrap/server.go b/bootstrap/server.go index c8cce0631..43984c78f 100644 --- a/bootstrap/server.go +++ b/bootstrap/server.go @@ -21,6 +21,8 @@ import ( "context" "errors" "fmt" + "github.com/polarismesh/polaris-server/common/model" + "github.com/polarismesh/polaris-server/healthcheck" "net" "strings" "time" @@ -105,15 +107,11 @@ func Start(configFilePath string) { fmt.Printf("[ERROR] %v\n", err) return } - - cfg.Naming.HealthCheck.LocalHost = LocalHost // 补充healthCheck的配置 - naming.SetHealthCheckConfig(&cfg.Naming.HealthCheck) - err = naming.Initialize(ctx, &cfg.Naming, &cfg.Cache) + err = StartComponents(ctx, cfg) if err != nil { fmt.Printf("[ERROR] %v\n", err) return } - errCh := make(chan error, len(cfg.APIServers)) servers, err := StartServers(ctx, cfg, errCh) if err != nil { @@ -130,7 +128,32 @@ func Start(configFilePath string) { RunMainLoop(servers, errCh) } -// 启动server +// StartComponents start healthcheck and naming components +func StartComponents(ctx context.Context, cfg *config.Config) error { + var err error + if len(cfg.HealthChecks.LocalHost) == 0 { + cfg.HealthChecks.LocalHost = LocalHost // 补充healthCheck的配置 + } + err = healthcheck.Initialize(ctx, &cfg.HealthChecks, cfg.Cache.Open) + if err != nil { + return err + } + healthCheckServer, err := healthcheck.GetServer() + if err != nil { + return err + } + cacheProvider, err := healthCheckServer.CacheProvider() + if err != nil { + return err + } + err = naming.Initialize(ctx, &cfg.Naming, &cfg.Cache, cacheProvider) + if err != nil { + return err + } + return nil +} + +// StartServers 启动server func StartServers(ctx context.Context, cfg *config.Config, errCh chan error) ( []apiserver.Apiserver, error) { // 启动API服务器 @@ -362,7 +385,8 @@ func selfRegister(host string, port uint32, protocol string, isolated bool, pola Version: utils.NewStringValue(version.Get()), Isolate: utils.NewBoolValue(isolated), // 自注册,默认是隔离的 Metadata: map[string]string{ - "build-revision": version.GetRevision(), + model.MetaKeyBuildRevision: version.GetRevision(), + model.MetaKeyPolarisService: name, }, } diff --git a/common/api/v1/codeinfo.go b/common/api/v1/codeinfo.go index 5fd1ba861..a9ef55e40 100644 --- a/common/api/v1/codeinfo.go +++ b/common/api/v1/codeinfo.go @@ -61,6 +61,7 @@ const ( HealthCheckNotOpen = 400140 HeartbeatOnDisabledIns = 400141 HeartbeatExceedLimit = 400142 + HeartbeatTypeNotFound = 400143 InvalidMetadata = 400150 InvalidRateLimitID = 400151 InvalidRateLimitLabels = 400152 diff --git a/common/model/metadata.go b/common/model/metadata.go new file mode 100644 index 000000000..2f4cd7221 --- /dev/null +++ b/common/model/metadata.go @@ -0,0 +1,26 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package model + +const ( + // MetaKeyPolarisService service identifier by self registration + MetaKeyPolarisService = "polaris_service" + + // MetaKeyBuildRevision build revision for server + MetaKeyBuildRevision = "build-revision" +) diff --git a/common/redispool/redis_pool.go b/common/redispool/redis_pool.go index 031a3e691..fa932f70c 100644 --- a/common/redispool/redis_pool.go +++ b/common/redispool/redis_pool.go @@ -18,17 +18,16 @@ package redispool import ( + "context" + "encoding/json" + "errors" "fmt" - "hash/crc32" - "math/rand" - "strconv" - "sync" + "hash/fnv" "sync/atomic" "time" "github.com/gomodule/redigo/redis" "github.com/polarismesh/polaris-server/common/log" - "github.com/polarismesh/polaris-server/common/model" ) const ( @@ -40,14 +39,22 @@ const ( Del ) -/** - * Task ckv任务请求结构体 - */ +const ( + // keyPrefix the prefix for hb key + keyPrefix = "hb_" + // eventSep the separator to split event + eventSep = ":" +) + +func toRedisKey(instanceId string) string { + return fmt.Sprintf("%s%s", keyPrefix, instanceId) +} + +// Task ckv任务请求结构体 type Task struct { taskType int id string - status int - beatTime int64 + value string respCh chan *Resp } @@ -55,296 +62,310 @@ type Task struct { * Resp ckv任务结果 */ type Resp struct { - Value string - Err error - Local bool + Value string + Err error + Exists bool } -/** - * MetaData ckv连接池元数据 - */ +// MetaData ckv连接池元数据 type MetaData struct { insConnNum int - kvPasswd string - localHost string - MaxIdle int - IdleTimeout int + password string + maxIdle int + idleTimeout int + address string } -/** - * Instance ckv节点结构体 - */ +// Instance ckv节点结构体 type Instance struct { index uint32 // 节点在连接池中的序号 addr string redisPool *redis.Pool ch []chan *Task - stopCh chan struct{} + stopCtx chan struct{} } -/** - * Pool ckv连接池结构体 - */ -type Pool struct { - mu sync.Mutex - meta *MetaData - instances []*Instance - instanceNum int32 +// Config redis pool configuration +type Config struct { + KvAddr string `json:"kvAddr"` + KvPasswd string `json:"kvPasswd"` + SlotNum int `json:"slotNum"` + MaxIdle int `json:"maxIdle"` + IdleTimeout Duration `json:"idleTimeout"` + ConnectTimeout Duration `json:"connectTimeout"` + MsgTimeout Duration `json:"msgTimeout"` + Concurrency int `json:"concurrency"` } -/** - * NewPool 初始化一个redis连接池实例 - */ -func NewPool(insConnNum int, kvPasswd, localHost string, redisInstances []*model.Instance, - maxIdle, idleTimeout int) (*Pool, error) { - var instances []*Instance - if len(redisInstances) > 0 { - for _, instance := range redisInstances { - instance := &Instance{ - redisPool: genRedisPool(insConnNum, kvPasswd, instance, maxIdle, idleTimeout), - stopCh: make(chan struct{}, 1), - } - instance.ch = make([]chan *Task, 0, 100*insConnNum) - for i := 0; i < 100*insConnNum; i++ { - instance.ch = append(instance.ch, make(chan *Task)) - } - rand.Seed(time.Now().Unix()) - // 从一个随机位置开始,防止所有server都从一个ckv开始 - instance.index = uint32(rand.Intn(100 * insConnNum)) - instances = append(instances, instance) +// DefaultConfig redis pool configuration with default values +func DefaultConfig() *Config { + return &Config{ + SlotNum: 30, + MaxIdle: 200, + IdleTimeout: Duration(120 * time.Second), + ConnectTimeout: Duration(500 * time.Millisecond), + MsgTimeout: Duration(200 * time.Millisecond), + Concurrency: 200, + } +} + +// Duration duration alias +type Duration time.Duration + +// MarshalJSON marshal duration to json +func (d Duration) MarshalJSON() ([]byte, error) { + return json.Marshal(time.Duration(d).String()) +} + +// UnmarshalJSON unmarshal json text to struct +func (d *Duration) UnmarshalJSON(b []byte) error { + var v interface{} + if err := json.Unmarshal(b, &v); err != nil { + return err + } + switch value := v.(type) { + case float64: + *d = Duration(time.Duration(value)) + return nil + case string: + tmp, err := time.ParseDuration(value) + if err != nil { + return err } + *d = Duration(tmp) + return nil + default: + return errors.New("invalid duration") } +} - pool := &Pool{ - meta: &MetaData{ - insConnNum: insConnNum, - kvPasswd: kvPasswd, - localHost: localHost, - MaxIdle: maxIdle, - IdleTimeout: idleTimeout, - }, - instances: instances, - instanceNum: int32(len(redisInstances)), +// Validate validate config params +func (c *Config) Validate() error { + if len(c.KvAddr) == 0 { + return errors.New("kvAddr is empty") + } + if len(c.KvPasswd) == 0 { + return errors.New("KvPasswd is empty") + } + if c.SlotNum <= 0 { + return errors.New("slotNum is empty") + } + if c.MaxIdle <= 0 { + return errors.New("maxIdle is empty") + } + if c.IdleTimeout == 0 { + return errors.New("idleTimeout is empty") } + if c.ConnectTimeout == 0 { + return errors.New("connectTimeout is empty") + } + if c.MsgTimeout == 0 { + return errors.New("msgTimeout is empty") + } + if c.Concurrency <= 0 { + return errors.New("concurrency is empty") + } + return nil +} - return pool, nil +// Pool ckv连接池结构体 +type Pool struct { + config *Config + ctx context.Context + ch []chan *Task + redisPool *redis.Pool + redisDead uint32 + recoverTimeSec int64 } -func genRedisPool(insConnNum int, kvPasswd string, instance *model.Instance, maxIdle, idleTimeout int) *redis.Pool { - pool := &redis.Pool{ - MaxIdle: maxIdle, - MaxActive: 0, - IdleTimeout: time.Duration(idleTimeout), +// NewPool init a redis connection pool instance +func NewPool(ctx context.Context, config *Config) *Pool { + redisPool := &redis.Pool{ + MaxIdle: config.MaxIdle, + MaxActive: config.Concurrency, + IdleTimeout: time.Duration(config.IdleTimeout), Dial: func() (redis.Conn, error) { - conn, err := redis.Dial("tcp", instance.Host()+":"+ - strconv.Itoa(int(instance.Port())), redis.DialPassword(kvPasswd)) + conn, err := redis.Dial("tcp", config.KvAddr, redis.DialPassword(config.KvPasswd), + redis.DialConnectTimeout(time.Duration(config.ConnectTimeout)), + redis.DialReadTimeout(time.Duration(config.MsgTimeout)), + redis.DialWriteTimeout(time.Duration(config.MsgTimeout))) if err != nil { log.Infof("ERROR: fail init redis: %s", err.Error()) return nil, err } - return conn, nil + return conn, err }, TestOnBorrow: func(c redis.Conn, t time.Time) error { _, err := c.Do("PING") return err }, } - return pool -} - -/** - * Update 更新ckv连接池中的节点 - * 重新建立ckv连接 - * 对业务无影响 - */ -func (p *Pool) Update(newKvInstances []*model.Instance) error { - p.mu.Lock() - defer p.mu.Unlock() - - change := len(newKvInstances) - int(atomic.LoadInt32(&p.instanceNum)) - log.Infof("[ckv] update, old ins num:%d, new ins num:%d, change:%d", p.instanceNum, len(newKvInstances), change) - - // 新建一个pool.instances数组 - var instances []*Instance - for _, instance := range newKvInstances { - instance := &Instance{ - redisPool: genRedisPool(p.meta.insConnNum, p.meta.kvPasswd, instance, p.meta.MaxIdle, p.meta.IdleTimeout), - stopCh: make(chan struct{}, 1), - } - instance.ch = make([]chan *Task, 0, 100*p.meta.insConnNum) - for i := 0; i < 100*p.meta.insConnNum; i++ { - instance.ch = append(instance.ch, make(chan *Task)) - } - instance.index = uint32(rand.Intn(100 * p.meta.insConnNum)) - instances = append(instances, instance) + pool := &Pool{ + config: config, + ctx: ctx, + redisPool: redisPool, + recoverTimeSec: time.Now().Unix(), + ch: make([]chan *Task, 0, config.Concurrency), } - - // 关闭前一个连接池 - for i := 0; i < len(p.instances); i++ { - close(p.instances[i].stopCh) - time.Sleep(10 * time.Millisecond) - for j := 0; j < len(p.instances[i].ch); j++ { - close(p.instances[i].ch[j]) - } - err := p.instances[i].redisPool.Close() - if err != nil { - log.Errorf("close redis pool :%s", err) - } + for i := 0; i < config.Concurrency; i++ { + pool.ch = append(pool.ch, make(chan *Task, 100)) } - - time.Sleep(10 * time.Millisecond) - // 结构体属性重新赋值,并重新开始消费 - p.instances = instances - atomic.StoreInt32(&p.instanceNum, int32(len(p.instances))) - - for i := 0; i < len(p.instances); i++ { - for k := 0; k < len(p.instances[i].ch); k++ { - go p.worker(i, k) - } - } - - log.Infof("[redis] update success, node num:%d", len(p.instances)) - - return nil + return pool } -func (p *Pool) checkHasKvInstances(ch chan *Resp) bool { - if atomic.LoadInt32(&p.instanceNum) == 0 { - go func() { - ch <- &Resp{ - Local: true, - } - }() - return true - } - return false +func hashValue(s string) int { + h := fnv.New32a() + h.Write([]byte(s)) + return int(h.Sum32()) } -/** - * Get 使用连接池,向redis发起Get请求 - */ -func (p *Pool) Get(id string, ch chan *Resp) { // nolint - if p.checkHasKvInstances(ch) { - return +// Get 使用连接池,向redis发起Get请求 +func (p *Pool) Get(id string, ch chan *Resp) error { // nolint + if err := p.checkRedisDead(); nil != err { + return err } task := &Task{ taskType: Get, id: id, respCh: ch, } - - insIndex, chIndex := p.genInsChIndex(id) - p.instances[insIndex].ch[chIndex] <- task + p.ch[hashValue(id)%p.config.Concurrency] <- task + return nil } -/** - * Set 使用连接池,向redis发起Set请求 - */ -func (p *Pool) Set(id string, status int, beatTime int64, ch chan *Resp) { // nolint - if p.checkHasKvInstances(ch) { - return +// Set 使用连接池,向redis发起Set请求 +func (p *Pool) Set(id string, value string, ch chan *Resp) error { // nolint + if err := p.checkRedisDead(); nil != err { + return err } task := &Task{ taskType: Set, id: id, - status: status, - beatTime: beatTime, + value: value, respCh: ch, } - - insIndex, chIndex := p.genInsChIndex(id) - p.instances[insIndex].ch[chIndex] <- task + p.ch[hashValue(id)%p.config.Concurrency] <- task + return nil } -/** - * Del 使用连接池,向redis发起Del请求 - */ -func (p *Pool) Del(id string, ch chan *Resp) { // nolint +// Del 使用连接池,向redis发起Del请求 +func (p *Pool) Del(id string, ch chan *Resp) error { // nolint + if err := p.checkRedisDead(); nil != err { + return err + } task := &Task{ taskType: Del, id: id, respCh: ch, } - - insIndex, chIndex := p.genInsChIndex(id) - p.instances[insIndex].ch[chIndex] <- task + p.ch[hashValue(id)%p.config.Concurrency] <- task + return nil } -/** - * genInsChIndex 生成index公共方法 - */ -func (p *Pool) genInsChIndex(id string) (int, uint32) { - insIndex := String(id) % int(atomic.LoadInt32(&p.instanceNum)) - - chIndex := atomic.AddUint32(&p.instances[insIndex].index, 1) % uint32(p.meta.insConnNum*100) - return insIndex, chIndex +func (p *Pool) checkRedisDead() error { + if atomic.LoadUint32(&p.redisDead) == 1 { + return errors.New(fmt.Sprintf("redis %s is dead", p.config.KvAddr)) + } + return nil } -/** - * Start 启动ckv连接池工作 - */ +// Start 启动ckv连接池工作 func (p *Pool) Start() { - p.mu.Lock() - defer p.mu.Unlock() + for i := 0; i < p.config.Concurrency; i++ { + go p.worker(i) + } + log.Infof("[RedisPool]redis pool started") +} - for i := 0; i < len(p.instances); i++ { - for k := 0; k < len(p.instances[i].ch); k++ { - go p.worker(i, k) +func (p *Pool) worker(idx int) { + log.Infof("[Health Check]start redis pool %d", idx) + for { + select { + case task := <-p.ch[idx]: + p.handleTask(task) + case <-p.ctx.Done(): + return } } - log.Infof("[redis] redis pool start") } -/** - * worker 接收任务worker - */ -func (p *Pool) worker(instanceIndex, chIndex int) { +const ( + redisCheckInterval = 1 * time.Second + errCountThreshold = 2 + maxCheckCount = 3 +) + +// checkRedis check redis alive +func (p *Pool) checkRedis() { + ticker := time.NewTicker(redisCheckInterval) + defer ticker.Stop() for { select { - case task := <-p.instances[instanceIndex].ch[chIndex]: - p.handleTask(task, instanceIndex) - case <-p.instances[instanceIndex].stopCh: + case <-ticker.C: + var errCount int + for i := 0; i < maxCheckCount; i++ { + if !p.doCheckRedis() { + errCount++ + } + } + if errCount >= errCountThreshold { + if atomic.CompareAndSwapUint32(&p.redisDead, 0, 1) { + atomic.StoreInt64(&p.recoverTimeSec, 0) + } + } else { + if atomic.CompareAndSwapUint32(&p.redisDead, 1, 0) { + atomic.StoreInt64(&p.recoverTimeSec, time.Now().Unix()) + } + } + case <-p.ctx.Done(): return } } } -/** - * handleTask 任务处理函数 - */ -func (p *Pool) handleTask(task *Task, index int) { +// RecoverTimeSec the time second record when recover +func (p *Pool) RecoverTimeSec() int64 { + return atomic.LoadInt64(&p.recoverTimeSec) +} + +// doCheckRedis test the connection +func (p *Pool) doCheckRedis() bool { + conn := p.redisPool.Get() + defer conn.Close() + _, err := conn.Do("PING") + if err != nil { + return false + } + atomic.StoreUint32(&p.redisDead, 0) + return true +} + +// handleTask 任务处理函数 +func (p *Pool) handleTask(task *Task) { if task == nil { log.Errorf("receive nil task") return } - con := p.instances[index].redisPool.Get() + con := p.redisPool.Get() defer con.Close() var resp Resp switch task.taskType { case Get: - resp.Value, resp.Err = redis.String(con.Do("GET", task.id)) + resp.Value, resp.Err = redis.String(con.Do("GET", toRedisKey(task.id))) + resp.Exists = true + if resp.Err == redis.ErrNil { + resp.Err = nil + resp.Exists = false + } task.respCh <- &resp case Set: - value := fmt.Sprintf("%d:%d:%s", task.status, task.beatTime, p.meta.localHost) - _, resp.Err = con.Do("SET", task.id, value) + _, resp.Err = con.Do("SET", toRedisKey(task.id), task.value) task.respCh <- &resp case Del: - _, resp.Err = con.Do("DEL", task.id) + _, resp.Err = con.Do("DEL", toRedisKey(task.id)) task.respCh <- &resp default: log.Errorf("[ckv] set key:%s type:%d wrong", task.id, task.taskType) } } - -// String 字符串转hash值 -func String(s string) int { - v := int(crc32.ChecksumIEEE([]byte(s))) - if v >= 0 { - return v - } - if -v >= 0 { - return -v - } - return 0 -} diff --git a/common/srand/scalable_rand.go b/common/srand/scalable_rand.go new file mode 100644 index 000000000..496d284bc --- /dev/null +++ b/common/srand/scalable_rand.go @@ -0,0 +1,93 @@ +/** + * Tencent is pleased to support the open source community by making polaris-go available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package srand + +import ( + "math/rand" + "runtime" + "sync" + "sync/atomic" + "time" +) + +// ScalableRand 可水平扩展的随机数发生器 +type ScalableRand struct { + initSeed int64 + randPool *sync.Pool +} + +// NewScalableRand 初始化随机数发生器 +func NewScalableRand() *ScalableRand { + scalableRand := &ScalableRand{ + randPool: &sync.Pool{}, + } + cpuCount := runtime.NumCPU() + for i := 0; i < cpuCount; i++ { + scalableRand.randPool.Put(rand.New(rand.NewSource(scalableRand.getRandSeed()))) + } + return scalableRand +} + +// getRandSeed 循环并获取唯一的随机数种子 +func (s *ScalableRand) getRandSeed() int64 { + var seed int64 + for { + seed = time.Now().UnixNano() + if s.getAndSetInitSeed(seed) { + break + } + time.Sleep(1) + } + return seed +} + +// getAndSetInitSeed 获取并比较种子数 +func (s *ScalableRand) getAndSetInitSeed(seed int64) bool { + initSeed := atomic.LoadInt64(&s.initSeed) + if initSeed == seed { + return false + } + return atomic.CompareAndSwapInt64(&s.initSeed, initSeed, seed) +} + +// Intn 获取随机数 +func (s *ScalableRand) Intn(n int) int { + var randSeed *rand.Rand + value := s.randPool.Get() + if nil != value { + randSeed = value.(*rand.Rand) + } else { + randSeed = rand.New(rand.NewSource(s.getRandSeed())) + } + randValue := randSeed.Intn(n) + s.randPool.Put(randSeed) + return randValue +} + +//全局随机种子 +var globalRand *ScalableRand + +// Intn 返回全局随机数 +func Intn(n int) int { + return globalRand.Intn(n) +} + +//初始化全局随机种子 +func init() { + globalRand = NewScalableRand() +} diff --git a/common/utils/funcs.go b/common/utils/funcs.go index 2c120e30f..4e67288e8 100644 --- a/common/utils/funcs.go +++ b/common/utils/funcs.go @@ -82,8 +82,6 @@ func CreateInstanceModel(serviceID string, req *api.Instance) *model.Instance { } protoIns.HealthCheck.Heartbeat.Ttl.Value = 5 } - // 开启健康检查,且没有代入健康状态,则健康状态默认都是false - protoIns.Healthy.Value = false } instance.Proto = protoIns diff --git a/config/config.go b/config/config.go index 48207c79f..cb12784c8 100644 --- a/config/config.go +++ b/config/config.go @@ -20,6 +20,7 @@ package config import ( "errors" "fmt" + "github.com/polarismesh/polaris-server/healthcheck" "os" "github.com/polarismesh/polaris-server/apiserver" @@ -33,12 +34,13 @@ import ( // Config 配置 type Config struct { - Bootstrap Bootstrap `yaml:"bootstrap"` - APIServers []apiserver.Config `yaml:"apiservers"` - Cache cache.Config `yaml:"cache"` - Naming naming.Config `yaml:"naming"` - Store store.Config `yaml:"store"` - Plugin plugin.Config `yaml:"plugin"` + Bootstrap Bootstrap `yaml:"bootstrap"` + APIServers []apiserver.Config `yaml:"apiservers"` + Cache cache.Config `yaml:"cache"` + Naming naming.Config `yaml:"naming"` + HealthChecks healthcheck.Config `yaml:"healthcheck"` + Store store.Config `yaml:"store"` + Plugin plugin.Config `yaml:"plugin"` } // Bootstrap 启动引导配置 diff --git a/go.mod b/go.mod index 5dfd43f62..1fb600681 100644 --- a/go.mod +++ b/go.mod @@ -14,6 +14,7 @@ require ( github.com/google/uuid v1.2.0 github.com/hashicorp/golang-lru v0.5.3 github.com/mitchellh/mapstructure v1.1.2 + github.com/modern-go/reflect2 v1.0.1 github.com/natefinch/lumberjack v2.0.0+incompatible github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.11.0 diff --git a/healthcheck/cache.go b/healthcheck/cache.go new file mode 100644 index 000000000..e9b91ff87 --- /dev/null +++ b/healthcheck/cache.go @@ -0,0 +1,239 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package healthcheck + +import ( + api "github.com/polarismesh/polaris-server/common/api/v1" + "github.com/polarismesh/polaris-server/common/log" + "github.com/polarismesh/polaris-server/common/model" + "github.com/polarismesh/polaris-server/plugin" + "sync" +) + +// CacheProvider provider health check objects for service cache +type CacheProvider struct { + healthCheckInstances *sync.Map + healthCheckMutex *sync.Mutex + selfService string + selfServiceInstances *sync.Map + selfServiceMutex *sync.Mutex + cacheEvents chan CacheEvent +} + +// CacheEvent provide the event for cache changes +type CacheEvent struct { + healthCheckInstancesChanged bool + selfServiceInstancesChanged bool +} + +func newCacheProvider(selfService string) *CacheProvider { + return &CacheProvider{ + healthCheckInstances: &sync.Map{}, + healthCheckMutex: &sync.Mutex{}, + selfServiceInstances: &sync.Map{}, + selfServiceMutex: &sync.Mutex{}, + selfService: selfService, + cacheEvents: make(chan CacheEvent, 100)} +} + +func (c *CacheProvider) isSelfServiceInstance(instance *api.Instance) bool { + metadata := instance.GetMetadata() + if svcName, ok := metadata[model.MetaKeyPolarisService]; ok { + return svcName == c.selfService + } + return false +} + +func (c *CacheProvider) sendEvent(event CacheEvent) { + select { + case c.cacheEvents <- event: + default: + return + } +} + +// CacheEvents return a channel to receive cache events +func (c *CacheProvider) CacheEvents() <-chan CacheEvent { + return c.cacheEvents +} + +func compareAndStoreServiceInstance( + instanceWithChecker *InstanceWithChecker, mutex *sync.Mutex, values *sync.Map) bool { + mutex.Lock() + defer mutex.Unlock() + instanceId := instanceWithChecker.instance.GetId().GetValue() + value, ok := values.Load(instanceId) + if !ok { + log.Infof("[Health Check][Cache]create service instance is %s:%d", + instanceWithChecker.instance.GetHost().GetValue(), instanceWithChecker.instance.GetPort().GetValue()) + values.Store(instanceId, instanceWithChecker) + return true + } + lastInstance := value.(*InstanceWithChecker).instance + if lastInstance.GetRevision().GetValue() == instanceWithChecker.instance.GetRevision().GetValue() { + return false + } + log.Infof("[Health Check][Cache]update service instance is %s:%d", + instanceWithChecker.instance.GetHost().GetValue(), instanceWithChecker.instance.GetPort().GetValue()) + values.Store(instanceId, instanceWithChecker) + return true +} + +func storeServiceInstance(instanceWithChecker *InstanceWithChecker, mutex *sync.Mutex, values *sync.Map) bool { + mutex.Lock() + defer mutex.Unlock() + log.Infof("[Health Check][Cache]create service instance is %s:%d", + instanceWithChecker.instance.GetHost().GetValue(), instanceWithChecker.instance.GetPort().GetValue()) + instanceId := instanceWithChecker.instance.GetId().GetValue() + values.Store(instanceId, instanceWithChecker) + return true +} + +func deleteServiceInstance(instance *api.Instance, mutex *sync.Mutex, values *sync.Map) bool { + mutex.Lock() + defer mutex.Unlock() + instanceId := instance.GetId().GetValue() + _, ok := values.Load(instanceId) + if ok { + log.Infof("[Health Check][Cache]delete service instance is %s:%d", + instance.GetHost().GetValue(), instance.GetPort().GetValue()) + values.Delete(instanceId) + } + return true +} + +// InstanceWithChecker instance and checker combine +type InstanceWithChecker struct { + instance *api.Instance + checker plugin.HealthChecker +} + +// OnCreated callback when cache value created +func (c *CacheProvider) OnCreated(value interface{}) { + if instance, ok := value.(*api.Instance); ok { + if c.isSelfServiceInstance(instance) { + storeServiceInstance(&InstanceWithChecker{ + instance: instance}, c.selfServiceMutex, c.selfServiceInstances) + c.sendEvent(CacheEvent{selfServiceInstancesChanged: true}) + return + } + hcEnable, checker := isHealthCheckEnable(instance) + if !hcEnable { + return + } + storeServiceInstance(&InstanceWithChecker{ + instance: instance, + checker: checker, + }, c.healthCheckMutex, c.healthCheckInstances) + c.sendEvent(CacheEvent{healthCheckInstancesChanged: true}) + } +} + +func isHealthCheckEnable(instance *api.Instance) (bool, plugin.HealthChecker) { + if !instance.GetEnableHealthCheck().GetValue() || nil == instance.GetHealthCheck() { + return false, nil + } + checker, ok := server.checkers[int32(instance.GetHealthCheck().GetType())] + if !ok { + return false, nil + } + return true, checker +} + +// OnUpdated callback when cache value updated +func (c *CacheProvider) OnUpdated(value interface{}) { + if instance, ok := value.(*api.Instance); ok { + if c.isSelfServiceInstance(instance) { + if compareAndStoreServiceInstance(&InstanceWithChecker{ + instance: instance}, c.selfServiceMutex, c.selfServiceInstances) { + c.sendEvent(CacheEvent{selfServiceInstancesChanged: true}) + } + return + } + //check exists + c.healthCheckMutex.Lock() + defer c.healthCheckMutex.Unlock() + healthCheckInstanceValue, exists := c.healthCheckInstances.Load(instance.GetId().GetValue()) + hcEnable, checker := isHealthCheckEnable(instance) + if !hcEnable { + if !exists { + return + } + log.Infof("[Health Check][Cache]delete service instance is %s:%d for health check disabled", + instance.GetHost().GetValue(), instance.GetPort().GetValue()) + c.healthCheckInstances.Delete(instance.GetId().GetValue()) + c.sendEvent(CacheEvent{healthCheckInstancesChanged: true}) + return + } + var noChanged bool + if exists { + healthCheckInstance := healthCheckInstanceValue.(*InstanceWithChecker).instance + noChanged = healthCheckInstance.GetRevision().GetValue() == instance.GetRevision().GetValue() + } + if !noChanged { + log.Infof("[Health Check][Cache]update service instance is %s:%d", + instance.GetHost().GetValue(), instance.GetPort().GetValue()) + c.healthCheckInstances.Store(instance.GetId().GetValue(), &InstanceWithChecker{ + instance: instance, + checker: checker, + }) + c.sendEvent(CacheEvent{healthCheckInstancesChanged: true}) + } + } +} + +// OnDeleted callback when cache value deleted +func (c *CacheProvider) OnDeleted(value interface{}) { + if instance, ok := value.(*api.Instance); ok { + if c.isSelfServiceInstance(instance) { + deleteServiceInstance(instance, c.selfServiceMutex, c.selfServiceInstances) + c.sendEvent(CacheEvent{selfServiceInstancesChanged: true}) + return + } + if !instance.GetEnableHealthCheck().GetValue() || nil == instance.GetHealthCheck() { + return + } + deleteServiceInstance(instance, c.healthCheckMutex, c.healthCheckInstances) + c.sendEvent(CacheEvent{healthCheckInstancesChanged: true}) + } +} + +// RangeHealthCheckInstances range loop healthCheckInstances +func (c *CacheProvider) RangeHealthCheckInstances(check func(instance *InstanceWithChecker)) { + c.healthCheckInstances.Range(func(key, value interface{}) bool { + check(value.(*InstanceWithChecker)) + return true + }) +} + +// RangeSelfServiceInstances range loop selfServiceInstances +func (c *CacheProvider) RangeSelfServiceInstances(check func(instance *api.Instance)) { + c.selfServiceInstances.Range(func(key, value interface{}) bool { + check(value.(*InstanceWithChecker).instance) + return true + }) +} + +// GetInstance get instance by id +func (c *CacheProvider) GetInstance(id string) *api.Instance { + value, ok := c.healthCheckInstances.Load(id) + if !ok { + return nil + } + return value.(*InstanceWithChecker).instance +} diff --git a/healthcheck/check.go b/healthcheck/check.go new file mode 100644 index 000000000..6d7d1bdb7 --- /dev/null +++ b/healthcheck/check.go @@ -0,0 +1,326 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package healthcheck + +import ( + "context" + "fmt" + api "github.com/polarismesh/polaris-server/common/api/v1" + "github.com/polarismesh/polaris-server/common/log" + "github.com/polarismesh/polaris-server/common/model" + "github.com/polarismesh/polaris-server/common/srand" + "github.com/polarismesh/polaris-server/common/timewheel" + "github.com/polarismesh/polaris-server/common/utils" + "github.com/polarismesh/polaris-server/naming" + "github.com/polarismesh/polaris-server/plugin" + "sync" + "sync/atomic" + "time" +) + +const ( + NotHealthy = 0 + Healthy = 1 +) + +const ( + expireTtlCount = 3 +) + +// CheckScheduler schedule and run check actions +type CheckScheduler struct { + rwMutex *sync.RWMutex + scheduledInstances map[string]*instanceValue + + timeWheel *timewheel.TimeWheel +} + +type instanceValue struct { + mutex *sync.Mutex + id string + host string + port uint32 + scheduled uint32 + lastSetEventTimeSec int64 + lastCheckTimeSec int64 + ttlDurationSec uint32 + expireDurationSec int64 + checker plugin.HealthChecker +} + +func (i *instanceValue) eventExpired() (int64, bool) { + curTimeSec := time.Now().Unix() + return curTimeSec, curTimeSec-i.lastSetEventTimeSec >= i.expireDurationSec +} + +func newCheckScheduler(ctx context.Context, slotNum int) *CheckScheduler { + scheduler := &CheckScheduler{ + rwMutex: &sync.RWMutex{}, + scheduledInstances: make(map[string]*instanceValue), + timeWheel: timewheel.New(time.Second, slotNum, "[Health Check]interval-check"), + } + go scheduler.startRoutines(ctx) + return scheduler +} + +func (c *CheckScheduler) startRoutines(ctx context.Context) { + c.timeWheel.Start() + log.Infof("[Health Check][Check]timeWheel has been started") + for { + select { + case <-ctx.Done(): + c.timeWheel.Stop() + log.Infof("[Health Check][Check]timeWheel has been stopped") + return + } + } +} + +func (c *CheckScheduler) putIfAbsent(instanceWithChecker *InstanceWithChecker) (bool, *instanceValue) { + c.rwMutex.Lock() + defer c.rwMutex.Unlock() + instance := instanceWithChecker.instance + var instValue *instanceValue + var ok bool + if instValue, ok = c.scheduledInstances[instance.GetId().GetValue()]; ok { + return true, instValue + } + instValue = &instanceValue{ + mutex: &sync.Mutex{}, + host: instance.GetHost().GetValue(), + port: instance.GetPort().GetValue(), + id: instance.GetId().GetValue(), + expireDurationSec: int64(getExpireDurationSec(instance)), + checker: instanceWithChecker.checker, + ttlDurationSec: instance.GetHealthCheck().GetHeartbeat().GetTtl().GetValue(), + } + c.scheduledInstances[instance.GetId().GetValue()] = instValue + return false, instValue +} + +func (c *CheckScheduler) checkExistsReadOnly(instance *instanceValue) bool { + c.rwMutex.RLock() + defer c.rwMutex.RUnlock() + _, ok := c.scheduledInstances[instance.id] + log.Debugf("[Health Check][Check]check exists ro for id is %s, host is %s:%d, result is %v", + instance.id, instance.host, instance.port, ok) + return ok +} + +func (c *CheckScheduler) getInstanceValue(instanceId string) *instanceValue { + c.rwMutex.RLock() + defer c.rwMutex.RUnlock() + return c.scheduledInstances[instanceId] +} + +// AddInstance add instance to check +func (c *CheckScheduler) AddInstance(instanceWithChecker *InstanceWithChecker) { + exists, instValue := c.putIfAbsent(instanceWithChecker) + if exists { + return + } + instance := instanceWithChecker.instance + log.Debugf("[Health Check][Check]add for id is %s, host is %s:%d", + instance.GetId().GetValue(), instance.GetHost().GetValue(), instance.GetPort().GetValue()) + c.addUnHealthyCallback(instValue) +} + +func getExpireDurationSec(instance *api.Instance) uint32 { + ttlValue := instance.GetHealthCheck().GetHeartbeat().GetTtl().GetValue() + return expireTtlCount * ttlValue +} + +func getRandDelayMilli() time.Duration { + delayMilli := srand.Intn(1000) + return time.Duration(delayMilli) * time.Millisecond +} + +func (c *CheckScheduler) addHealthyCallback(instance *instanceValue, lastHeartbeatTimeSec int64) { + delay := time.Duration(instance.expireDurationSec) * time.Second + var nextDelay time.Duration + if lastHeartbeatTimeSec > 0 { + curTimeSec := currentTimeSec() + timePassed := curTimeSec - lastHeartbeatTimeSec + if timePassed > 0 { + nextDelay = delay - time.Duration(timePassed)*time.Second - getRandDelayMilli() + } + } + if nextDelay > 0 { + delay = nextDelay + } + host := instance.host + port := instance.port + instanceId := instance.id + log.Debugf("[Health Check][Check]add healthy callback, instance is %s:%d, id is %s, delay is %v", + host, port, instanceId, delay) + _ = c.timeWheel.AddTask(delay, instanceId, c.checkCallback) +} + +func (c *CheckScheduler) addUnHealthyCallback(instance *instanceValue) { + delay := time.Duration(instance.expireDurationSec-int64(instance.ttlDurationSec)) * time.Second + delay = delay - getRandDelayMilli() + host := instance.host + port := instance.port + instanceId := instance.id + log.Debugf("[Health Check][Check]add first/unhealthy callback, instance is %s:%d, id is %s, delay is %v", + host, port, instanceId, delay) + _ = c.timeWheel.AddTask(delay, instanceId, c.checkCallback) +} + +func (c *CheckScheduler) addInstCallback(value *instanceValue) { + log.Debugf("[Health Check][Check]add instant callback, instance is %s:%d, id is %s", + value.host, value.port, value.id) + _ = c.timeWheel.AddTask(1*time.Second, value, c.instCheckCallback) +} + +func (c *CheckScheduler) instCheckCallback(value interface{}) { + instValue := value.(*instanceValue) + c.checkCallback(instValue.id) + atomic.StoreUint32(&instValue.scheduled, 0) +} + +func (c *CheckScheduler) checkCallback(value interface{}) { + instanceId := value.(string) + instanceValue := c.getInstanceValue(instanceId) + if nil == instanceValue { + log.Infof("[Health Check][Check]instance %s has been deleted", instanceId) + return + } + instanceValue.mutex.Lock() + defer instanceValue.mutex.Unlock() + curTimeSec := time.Now().Unix() + + if instanceValue.lastCheckTimeSec == curTimeSec { + return + } + instanceValue.lastCheckTimeSec = curTimeSec + + log.Debugf("[Health Check][Check]start to check instance %s:%d, id is %s", + instanceValue.host, instanceValue.port, instanceValue.id) + cachedInstance := server.cacheProvider.GetInstance(instanceValue.id) + request := &plugin.CheckRequest{ + QueryRequest: plugin.QueryRequest{ + InstanceId: instanceValue.id, + Host: instanceValue.host, + Port: instanceValue.port, + Healthy: cachedInstance.GetHealthy().GetValue(), + }, + CurTimeSec: currentTimeSec(), + ExpireDurationSec: uint32(instanceValue.expireDurationSec), + } + checkResp, err := instanceValue.checker.Check(request) + if nil != err { + log.Errorf("[Health Check][Check]fail to check instance %s:%d, id is %s, err is %v", + instanceValue.host, instanceValue.port, instanceValue.id, err) + if c.checkExistsReadOnly(instanceValue) { + c.addUnHealthyCallback(instanceValue) + } + return + } + if checkResp.Healthy && !cachedInstance.GetHealthy().GetValue() { + //from unhealthy to healthy + log.Infof("[Health Check][Check]instance change from unhealthy to healthy, id is %s, address is %s:%d", + instanceValue.id, instanceValue.host, instanceValue.port) + err = setInsDbStatus(cachedInstance, Healthy) + } + if !checkResp.Healthy && cachedInstance.GetHealthy().GetValue() && !checkResp.OnRecover { + //from healthy to unhealthy + log.Infof("[Health Check][Check]instance change from healthy to unhealthy, id is %s, address is %s:%d", + instanceValue.id, instanceValue.host, instanceValue.port) + err = setInsDbStatus(cachedInstance, NotHealthy) + } + if nil != err { + log.Errorf("[Health Check][Check]fail to update instance, id is %s, address is %s:%d, err is %v", + instanceValue.id, instanceValue.host, instanceValue.port, err) + if c.checkExistsReadOnly(instanceValue) { + c.addHealthyCallback(instanceValue, checkResp.LastHeartbeatTimeSec) + } + return + } + if c.checkExistsReadOnly(instanceValue) && + (checkResp.Healthy || (checkResp.OnRecover && cachedInstance.GetHealthy().GetValue())) { + c.addHealthyCallback(instanceValue, checkResp.LastHeartbeatTimeSec) + } else { + c.addUnHealthyCallback(instanceValue) + } +} + +// DelInstance del instance from check +func (c *CheckScheduler) DelInstance(instanceWithChecker *InstanceWithChecker) { + c.rwMutex.Lock() + defer c.rwMutex.Unlock() + instance := instanceWithChecker.instance + log.Infof("[Health Check][Check]remove check instance is %s:%d, id is %s", + instance.GetHost().GetValue(), instance.GetPort().GetValue(), instance.GetId().GetValue()) + delete(c.scheduledInstances, instance.GetId().GetValue()) +} + +// setInsDbStatus 修改实例状态, 需要打印操作记录 +func setInsDbStatus(instance *api.Instance, status int) error { + id := instance.GetId().GetValue() + host := instance.GetHost().GetValue() + port := instance.GetPort().GetValue() + log.Infof("[Health Check][Check]addr:%s:%d id:%s set db status %d", host, port, id, status) + err := server.storage.SetInstanceHealthStatus(id, status, naming.NewUUID()) + if err != nil { + log.Errorf("[Health Check][Check]id: %s set db status err:%s", id, err) + return err + } + healthStatus := true + if status == 0 { + healthStatus = false + } + recordInstance := &model.Instance{ + Proto: &api.Instance{ + Host: instance.GetHost(), + Port: instance.GetPort(), + Priority: instance.GetPriority(), + Weight: instance.GetWeight(), + Healthy: utils.NewBoolValue(healthStatus), + Isolate: instance.GetIsolate(), + }, + } + + server.RecordHistory(instanceRecordEntry(recordInstance, model.OUpdate)) + return nil +} + +// instanceRecordEntry generate instance record entry +func instanceRecordEntry(ins *model.Instance, opt model.OperationType) *model.RecordEntry { + if ins == nil { + return nil + } + entry := &model.RecordEntry{ + ResourceType: model.RInstance, + OperationType: opt, + Namespace: ins.Proto.GetNamespace().GetValue(), + Service: ins.Proto.GetService().GetValue(), + Operator: "Polaris", + CreateTime: time.Now(), + } + if opt == model.OCreate || opt == model.OUpdate { + entry.Context = fmt.Sprintf("host:%s,port:%d,weight:%d,healthy:%v,isolate:%v,priority:%d,meta:%+v", + ins.Host(), ins.Port(), ins.Weight(), ins.Healthy(), ins.Isolate(), + ins.Priority(), ins.Metadata()) + } else if opt == model.OUpdateIsolate { + entry.Context = fmt.Sprintf("host:%s,port=%d,isolate:%v", ins.Host(), ins.Port(), ins.Isolate()) + } else { + entry.Context = fmt.Sprintf("host:%s,port:%d", ins.Host(), ins.Port()) + } + return entry +} diff --git a/healthcheck/config.go b/healthcheck/config.go new file mode 100644 index 000000000..b657669cb --- /dev/null +++ b/healthcheck/config.go @@ -0,0 +1,29 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package healthcheck + +import "github.com/polarismesh/polaris-server/plugin" + +// Config 健康检查配置 +type Config struct { + Open bool `yaml:"open"` + Service string `yaml:"service"` + SlotNum int `yaml:"slotNum"` + LocalHost string `yaml:"localHost"` + Checkers []plugin.ConfigEntry `yaml:"checkers"` +} diff --git a/healthcheck/dispatch.go b/healthcheck/dispatch.go new file mode 100644 index 000000000..439f71169 --- /dev/null +++ b/healthcheck/dispatch.go @@ -0,0 +1,184 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package healthcheck + +import ( + "context" + api "github.com/polarismesh/polaris-server/common/api/v1" + "github.com/polarismesh/polaris-server/common/log" + "sync/atomic" + "time" +) + +const ( + // eventInterval, trigger after instance change event + eventInterval = 5 * time.Second + // ensureInterval, trigger when timeout + ensureInterval = 61 * time.Second +) + +// Dispatcher dispatch all instances using consistent hash ring +type Dispatcher struct { + healthCheckInstancesChanged uint32 + selfServiceInstancesChanged uint32 + managedInstances map[string]*InstanceWithChecker + + selfServiceBuckets map[Bucket]bool + continuum *Continuum +} + +func newDispatcher(ctx context.Context) *Dispatcher { + dispatcher := &Dispatcher{} + dispatcher.startCacheEventJob(ctx) + dispatcher.startDispatchingJob(ctx) + return dispatcher +} + +// startCacheEventJob receive cache events and process +func (d *Dispatcher) startCacheEventJob(ctx context.Context) { + go func() { + for { + select { + case event := <-server.cacheProvider.CacheEvents(): + if event.selfServiceInstancesChanged { + atomic.StoreUint32(&d.selfServiceInstancesChanged, 1) + } + if event.healthCheckInstancesChanged { + atomic.StoreUint32(&d.healthCheckInstancesChanged, 1) + } + case <-ctx.Done(): + return + } + } + }() +} + +// startDispatchingJob start job to dispatch instances +func (d *Dispatcher) startDispatchingJob(ctx context.Context) { + go func() { + eventTicker := time.NewTicker(eventInterval) + defer eventTicker.Stop() + ensureTicker := time.NewTicker(ensureInterval) + defer ensureTicker.Stop() + + for { + select { + case <-eventTicker.C: + d.processEvent() + case <-ensureTicker.C: + d.processEnsure() + case <-ctx.Done(): + return + } + } + }() +} + +const weight = 100 + +func compareBuckets(src map[Bucket]bool, dst map[Bucket]bool) bool { + if len(src) != len(dst) { + return false + } + if len(src) == 0 { + return false + } + for bucket := range dst { + if _, ok := src[bucket]; !ok { + return false + } + } + return true +} + +func (d *Dispatcher) reloadSelfContinuum() bool { + nextBuckets := make(map[Bucket]bool) + server.cacheProvider.RangeSelfServiceInstances(func(instance *api.Instance) { + if instance.GetIsolate().GetValue() || !instance.GetHealthy().GetValue() { + return + } + nextBuckets[Bucket{ + Host: instance.GetHost().GetValue(), + Weight: weight, + }] = true + }) + originBucket := d.selfServiceBuckets + log.Infof("[Health Check][Dispatcher]reload continuum by %v, origin is %v", nextBuckets, originBucket) + if compareBuckets(originBucket, nextBuckets) { + return false + } + d.selfServiceBuckets = nextBuckets + d.continuum = New(d.selfServiceBuckets) + return true +} + +func (d *Dispatcher) reloadManagedInstances() { + nextInstances := make(map[string]*InstanceWithChecker) + var totalCount int + if nil != d.continuum { + server.cacheProvider.RangeHealthCheckInstances(func(instance *InstanceWithChecker) { + instanceId := instance.instance.GetId().GetValue() + host := d.continuum.Hash(instanceId) + if host == server.localHost { + nextInstances[instanceId] = instance + } + totalCount++ + }) + } + log.Infof("[Health Check][Dispatcher]count %d instances has been dispatched to %s, total is %d", + len(nextInstances), server.localHost, totalCount) + originInstances := d.managedInstances + d.managedInstances = nextInstances + if len(nextInstances) > 0 { + for id, instance := range nextInstances { + if len(originInstances) == 0 { + server.checkScheduler.AddInstance(instance) + continue + } + if _, ok := originInstances[id]; !ok { + server.checkScheduler.AddInstance(instance) + } + } + } + if len(originInstances) > 0 { + for id, instance := range originInstances { + if len(nextInstances) == 0 { + server.checkScheduler.DelInstance(instance) + continue + } + if _, ok := nextInstances[id]; !ok { + server.checkScheduler.DelInstance(instance) + } + } + } +} + +func (d *Dispatcher) processEvent() { + var selfContinuumReloaded bool + if atomic.CompareAndSwapUint32(&d.selfServiceInstancesChanged, 1, 0) { + selfContinuumReloaded = d.reloadSelfContinuum() + } + if selfContinuumReloaded || atomic.CompareAndSwapUint32(&d.healthCheckInstancesChanged, 1, 0) { + d.reloadManagedInstances() + } +} + +func (d *Dispatcher) processEnsure() { + d.reloadSelfContinuum() + d.reloadManagedInstances() +} diff --git a/healthcheck/ketama.go b/healthcheck/ketama.go new file mode 100644 index 000000000..88e5c23e6 --- /dev/null +++ b/healthcheck/ketama.go @@ -0,0 +1,123 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package healthcheck + +import ( + "crypto/sha1" + "fmt" + "sort" +) + +// Bucket single bucket of hash ring +type Bucket struct { + Host string + Weight uint32 +} + +type continuumPoint struct { + bucket Bucket + point uint +} + +// Continuum consistent hash ring +type Continuum struct { + ring points +} + +type points []continuumPoint + +// Less +func (c points) Less(i, j int) bool { return c[i].point < c[j].point } + +// Len +func (c points) Len() int { return len(c) } + +// Swap +func (c points) Swap(i, j int) { c[i], c[j] = c[j], c[i] } + +func sha1Digest(in string) []byte { + h := sha1.New() + h.Write([]byte(in)) + return h.Sum(nil) +} + +func hashString(in string) uint { + digest := sha1Digest(in) + return uint(digest[3])<<24 | uint(digest[2])<<16 | uint(digest[1])<<8 | uint(digest[0]) +} + +// New new hash ring +func New(buckets map[Bucket]bool) *Continuum { + numBuckets := len(buckets) + + if numBuckets == 0 { + return nil + } + + ring := make(points, 0, numBuckets*160) + + var totalWeight uint32 + for bucket := range buckets { + totalWeight += bucket.Weight + } + + for bucket := range buckets { + pct := float64(bucket.Weight) / float64(totalWeight) + + // this is the equivalent of C's promotion rules, but in Go, to maintain exact compatibility with the C library + limit := int(pct * 40.0 * float64(numBuckets)) + + for k := 0; k < limit; k++ { + /* 40 hashes, 4 numbers per hash = 160 points per bucket */ + ss := fmt.Sprintf("%s-%d", bucket.Host, k) + digest := sha1Digest(ss) + + for h := 0; h < 4; h++ { + point := continuumPoint{ + point: uint(digest[3+h*4])<<24 | uint(digest[2+h*4])<<16 | uint(digest[1+h*4])<<8 | uint(digest[h*4]), + bucket: bucket, + } + ring = append(ring, point) + } + } + } + + sort.Sort(ring) + + return &Continuum{ + ring: ring, + } +} + +// Hash hash string to lookup node +func (c *Continuum) Hash(thing string) string { + if len(c.ring) == 0 { + return "" + } + + h := hashString(thing) + + // the above md5 is way more expensive than this branch + var i uint + i = uint(sort.Search(len(c.ring), func(i int) bool { return c.ring[i].point >= h })) + if i >= uint(len(c.ring)) { + i = 0 + } + + return c.ring[i].bucket.Host +} diff --git a/healthcheck/report.go b/healthcheck/report.go new file mode 100644 index 000000000..128b97785 --- /dev/null +++ b/healthcheck/report.go @@ -0,0 +1,79 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package healthcheck + +import ( + "context" + api "github.com/polarismesh/polaris-server/common/api/v1" + "github.com/polarismesh/polaris-server/common/log" + "github.com/polarismesh/polaris-server/common/utils" + "github.com/polarismesh/polaris-server/naming" + "github.com/polarismesh/polaris-server/plugin" + "time" +) + +// checkHeartbeatInstance 检查心跳实例请求参数 +// 检查是否存在token,以及 id或者四元组 +// 注意:心跳上报只允许从client上报,因此token只会存在req中 +func checkHeartbeatInstance(req *api.Instance) (string, *api.Response) { + if req == nil { + return "", api.NewInstanceResponse(api.EmptyRequest, req) + } + if req.GetId() != nil { + if req.GetId().GetValue() == "" { + return "", api.NewInstanceResponse(api.InvalidInstanceID, req) + } + return req.GetId().GetValue(), nil + } + return naming.CheckInstanceTetrad(req) +} + +func (s *Server) doReport(ctx context.Context, instance *api.Instance) *api.Response { + if len(s.checkers) == 0 { + return api.NewResponse(api.HealthCheckNotOpen) + } + id, errRsp := checkHeartbeatInstance(instance) + if errRsp != nil { + return errRsp + } + instance.Id = utils.NewStringValue(id) + insCache := s.cacheProvider.GetInstance(id) + if insCache == nil { + return api.NewInstanceResponse(api.HeartbeatOnDisabledIns, instance) + } + checker, ok := s.checkers[int32(insCache.GetHealthCheck().GetType())] + if !ok { + return api.NewInstanceResponse(api.HeartbeatTypeNotFound, instance) + } + request := &plugin.ReportRequest{ + QueryRequest: plugin.QueryRequest{ + InstanceId: id, + Host: instance.GetHost().GetValue(), + Port: instance.GetPort().GetValue(), + }, + LocalHost: s.localHost, + CurTimeSec: time.Now().Unix() - s.timeAdjuster.GetDiff(), + } + err := checker.Report(request) + if nil != err { + log.Errorf("[Heartbeat][Server]fail to do report for %s:%d, id is %s, err is %v", + instance.GetHost().GetValue(), instance.GetPort().GetValue(), id, err) + return api.NewInstanceResponse(api.HeartbeatException, instance) + } + return api.NewInstanceResponse(api.ExecuteSuccess, instance) +} diff --git a/healthcheck/server.go b/healthcheck/server.go new file mode 100644 index 000000000..53884a0a7 --- /dev/null +++ b/healthcheck/server.go @@ -0,0 +1,179 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package healthcheck + +import ( + "context" + "errors" + "fmt" + api "github.com/polarismesh/polaris-server/common/api/v1" + "github.com/polarismesh/polaris-server/common/model" + "github.com/polarismesh/polaris-server/common/utils" + "github.com/polarismesh/polaris-server/plugin" + "github.com/polarismesh/polaris-server/store" + "strconv" + "sync" + "time" +) + +var ( + server = new(Server) + once = sync.Once{} + finishInit = false +) + +// Server health check main server +type Server struct { + storage store.Store + checkers map[int32]plugin.HealthChecker + cacheProvider *CacheProvider + timeAdjuster *TimeAdjuster + dispatcher *Dispatcher + checkScheduler *CheckScheduler + history plugin.History + localHost string +} + +// Initialize 初始化 +func Initialize(ctx context.Context, hcOpt *Config, cacheOpen bool) error { + var err error + once.Do(func() { + err = initialize(ctx, hcOpt, cacheOpen) + }) + + if err != nil { + return err + } + + finishInit = true + return nil +} + +func initialize(ctx context.Context, hcOpt *Config, cacheOpen bool) error { + if !hcOpt.Open { + return nil + } + if !cacheOpen { + return fmt.Errorf("[healthcheck]cache not open") + } + if len(hcOpt.Checkers) > 0 { + server.checkers = make(map[int32]plugin.HealthChecker, len(hcOpt.Checkers)) + for _, entry := range hcOpt.Checkers { + checker := plugin.GetHealthChecker(entry.Name, &entry) + if nil == checker { + return fmt.Errorf("[healthcheck]unknown healthchecker %s", entry.Name) + } + server.checkers[int32(checker.Type())] = checker + } + } + var err error + if server.storage, err = store.GetStore(); nil != err { + return err + } + server.localHost = hcOpt.LocalHost + server.history = plugin.GetHistory() + server.cacheProvider = newCacheProvider(hcOpt.Service) + server.timeAdjuster = newTimeAdjuster(ctx) + server.checkScheduler = newCheckScheduler(ctx, hcOpt.SlotNum) + server.dispatcher = newDispatcher(ctx) + return nil +} + +// Report report heartbeat request +func (s *Server) Report(ctx context.Context, req *api.Instance) *api.Response { + return s.doReport(ctx, req) +} + +// GetServer 获取已经初始化好的Server +func GetServer() (*Server, error) { + if !finishInit { + return nil, errors.New("server has not done InitializeServer") + } + + return server, nil +} + +// CacheProvider get cache provider +func (s *Server) CacheProvider() (*CacheProvider, error) { + if !finishInit { + return nil, errors.New("cache provider has not done InitializeServer") + } + return s.cacheProvider, nil +} + +// RecordHistory server对外提供history插件的简单封装 +func (s *Server) RecordHistory(entry *model.RecordEntry) { + // 如果插件没有初始化,那么不记录history + if s.history == nil { + return + } + // 如果数据为空,则不需要打印了 + if entry == nil { + return + } + + // 调用插件记录history + s.history.Record(entry) +} + +// GetLastHeartbeat 获取上一次心跳的时间 +func (s *Server) GetLastHeartbeat(req *api.Instance) *api.Response { + if len(s.checkers) == 0 { + return api.NewResponse(api.HealthCheckNotOpen) + } + id, errRsp := checkHeartbeatInstance(req) + if errRsp != nil { + return errRsp + } + req.Id = utils.NewStringValue(id) + insCache := s.cacheProvider.GetInstance(id) + if insCache == nil { + return api.NewInstanceResponse(api.NotFoundResource, req) + } + checker, ok := s.checkers[int32(insCache.GetHealthCheck().GetType())] + if !ok { + return api.NewInstanceResponse(api.HeartbeatTypeNotFound, req) + } + queryResp, err := checker.Query(&plugin.QueryRequest{ + InstanceId: insCache.GetId().GetValue(), + Host: insCache.GetHost().GetValue(), + Port: insCache.GetPort().GetValue(), + }) + if err != nil { + return api.NewInstanceRespWithError(api.ExecuteException, err, req) + } + req.Service = insCache.GetService() + req.Namespace = insCache.GetNamespace() + req.Host = insCache.GetHost() + req.Port = insCache.GetPort() + req.VpcId = insCache.GetVpcId() + req.HealthCheck = insCache.GetHealthCheck() + req.Metadata["last-heartbeat-timestamp"] = strconv.Itoa(int(queryResp.LastHeartbeatSec)) + req.Metadata["last-heartbeat-time"] = time2String(time.Unix(queryResp.LastHeartbeatSec, 0)) + req.Metadata["system-time"] = time2String(time.Unix(currentTimeSec(), 0)) + return api.NewInstanceResponse(api.ExecuteSuccess, req) +} + +// time2String time.Time转为字符串时间 +func time2String(t time.Time) string { + return t.Format("2006-01-02 15:04:05") +} + +func currentTimeSec() int64 { + return time.Now().Unix() - server.timeAdjuster.GetDiff() +} diff --git a/healthcheck/time_adjust.go b/healthcheck/time_adjust.go new file mode 100644 index 000000000..78275f7f0 --- /dev/null +++ b/healthcheck/time_adjust.go @@ -0,0 +1,69 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package healthcheck + +import ( + "context" + "github.com/polarismesh/polaris-server/common/log" + "sync/atomic" + "time" +) + +const adjustInterval = 10 * time.Second + +// TimeAdjuster adjust the seconds from database +type TimeAdjuster struct { + diff int64 +} + +func newTimeAdjuster(ctx context.Context) *TimeAdjuster { + adjuster := &TimeAdjuster{} + go adjuster.doTimeAdjust(ctx) + return adjuster +} + +func (t *TimeAdjuster) doTimeAdjust(ctx context.Context) { + t.calcDiff() + ticker := time.NewTicker(adjustInterval) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + log.Infof("[healthcheck]time adjuster has been stopped") + return + case <-ticker.C: + t.calcDiff() + } + } +} + +func (t *TimeAdjuster) calcDiff() { + curTimeSecond, err := server.storage.GetNow() + if nil != err { + log.Errorf("[healthcheck]fail to get now from store, err is %s", err.Error()) + return + } + sysNow := time.Now().Unix() + diff := sysNow - curTimeSecond + atomic.StoreInt64(&t.diff, diff) +} + +// GetDiff get diff time between store and current PC +func (t *TimeAdjuster) GetDiff() int64 { + return atomic.LoadInt64(&t.diff) +} diff --git a/naming/batch/instance.go b/naming/batch/instance.go index c3e788cd2..5c3c701d5 100644 --- a/naming/batch/instance.go +++ b/naming/batch/instance.go @@ -115,7 +115,7 @@ func newBatchInstanceCtrl(storage store.Store, authority auth.Authority, auth pl return nil, errors.New("config waitTime is invalid") } - instanceCtrl := &InstanceCtrl{ + instance := &InstanceCtrl{ config: config, storage: storage, authority: authority, @@ -125,7 +125,7 @@ func newBatchInstanceCtrl(storage store.Store, authority auth.Authority, auth pl queue: make(chan *InstanceFuture, config.QueueSize), waitDuration: duration, } - return instanceCtrl, nil + return instance, nil } // 注册主协程 diff --git a/naming/cache/cache.go b/naming/cache/cache.go index 23cf89d37..b55f2741d 100644 --- a/naming/cache/cache.go +++ b/naming/cache/cache.go @@ -52,9 +52,7 @@ const ( DefaultTimeDiff = -1 * time.Second * 5 ) -/** - * Cache 缓存接口 - */ +// Cache 缓存接口 type Cache interface { initialize(c map[string]interface{}) error update() error @@ -88,9 +86,7 @@ func newRevisionNotify(serviceID string, valid bool) *revisionNotify { } } -/** - * NamingCache 名字服务缓存 - */ +// NamingCache 名字服务缓存 type NamingCache struct { storage store.Store caches []Cache @@ -99,10 +95,8 @@ type NamingCache struct { revisions *sync.Map // service id -> reversion (所有instance reversion 的累计计算值) } -/** - * NewNamingCache 新建一个缓存对象 - */ -func NewNamingCache(storage store.Store) (*NamingCache, error) { +// NewNamingCache 新建一个缓存对象 +func NewNamingCache(storage store.Store, listeners []Listener) (*NamingCache, error) { nc := &NamingCache{ storage: storage, caches: make([]Cache, CacheLast), @@ -111,7 +105,7 @@ func NewNamingCache(storage store.Store) (*NamingCache, error) { } sc := newServiceCache(storage, nc.comRevisionCh) - ic := newInstanceCache(storage, nc.comRevisionCh) + ic := newInstanceCache(storage, nc.comRevisionCh, listeners) nc.caches[CacheService] = sc nc.caches[CacheInstance] = ic @@ -131,9 +125,7 @@ func NewNamingCache(storage store.Store) (*NamingCache, error) { return nc, nil } -/** - * initialize 缓存对象初始化 - */ +// initialize 缓存对象初始化 func (nc *NamingCache) initialize() error { for _, obj := range nc.caches { var option map[string]interface{} @@ -151,9 +143,7 @@ func (nc *NamingCache) initialize() error { return nil } -/** - * update 缓存更新 - */ +// update 缓存更新 func (nc *NamingCache) update() error { var wg sync.WaitGroup for _, entry := range config.Resources { @@ -172,9 +162,7 @@ func (nc *NamingCache) update() error { return nil } -/** - * clear 清除caches的所有缓存数据 - */ +// clear 清除caches的所有缓存数据 func (nc *NamingCache) clear() error { for _, obj := range nc.caches { if err := obj.clear(); err != nil { @@ -185,9 +173,7 @@ func (nc *NamingCache) clear() error { return nil } -/** - * Start 缓存对象启动协程,定时更新缓存 - */ +// Start 缓存对象启动协程,定时更新缓存 func (nc *NamingCache) Start(ctx context.Context) error { log.Infof("[Cache] cache goroutine start") // 先启动revision计算协程 @@ -218,17 +204,13 @@ func (nc *NamingCache) Start(ctx context.Context) error { return nil } -/** - * Clear 主动清除缓存数据 - */ +// Clear 主动清除缓存数据 func (nc *NamingCache) Clear() error { nc.revisions = new(sync.Map) return nc.clear() } -/** - * revisionWorker Cache中计算服务实例revision的worker - */ +// revisionWorker Cache中计算服务实例revision的worker func (nc *NamingCache) revisionWorker(ctx context.Context) { log.Infof("[Cache] compute revision worker start") defer log.Infof("[Cache] compute revision worker done") @@ -294,9 +276,7 @@ func (nc *NamingCache) GetUpdateCacheInterval() time.Duration { return UpdateCacheInterval } -/** - * GetServiceInstanceRevision 获取服务实例计算之后的revision - */ +// GetServiceInstanceRevision 获取服务实例计算之后的revision func (nc *NamingCache) GetServiceInstanceRevision(serviceID string) string { value, ok := nc.revisions.Load(serviceID) if !ok { @@ -306,9 +286,7 @@ func (nc *NamingCache) GetServiceInstanceRevision(serviceID string) string { return value.(string) } -/** - * GetServiceRevisionCount 计算一下缓存中的revision的个数 - */ +// GetServiceRevisionCount 计算一下缓存中的revision的个数 func (nc *NamingCache) GetServiceRevisionCount() int { count := 0 nc.revisions.Range(func(key, value interface{}) bool { @@ -319,49 +297,37 @@ func (nc *NamingCache) GetServiceRevisionCount() int { return count } -/** - * Service 获取Service缓存信息 - */ +// Service 获取Service缓存信息 func (nc *NamingCache) Service() ServiceCache { return nc.caches[CacheService].(ServiceCache) } -/** - * Instance 获取Instance缓存信息 - */ +// Instance 获取Instance缓存信息 func (nc *NamingCache) Instance() InstanceCache { return nc.caches[CacheInstance].(InstanceCache) } -/** - * RoutingConfig 获取路由配置的缓存信息 - */ +// RoutingConfig 获取路由配置的缓存信息 func (nc *NamingCache) RoutingConfig() RoutingConfigCache { return nc.caches[CacheRoutingConfig].(RoutingConfigCache) } -/** - * CL5 获取l5缓存信息 - */ +// CL5 获取l5缓存信息 func (nc *NamingCache) CL5() L5Cache { return nc.caches[CacheCL5].(L5Cache) } -/** - * RateLimit 获取限流规则缓存信息 - */ +// RateLimit 获取限流规则缓存信息 func (nc *NamingCache) RateLimit() RateLimitCache { return nc.caches[CacheRateLimit].(RateLimitCache) } -/** - * CircuitBreaker 获取熔断规则缓存信息 - */ +// CircuitBreaker 获取熔断规则缓存信息 func (nc *NamingCache) CircuitBreaker() CircuitBreakerCache { return nc.caches[CacheCircuitBreaker].(CircuitBreakerCache) } -// ComputeRevision +// ComputeRevision 计算唯一的版本标识 func ComputeRevision(serviceRevision string, instances []*model.Instance) (string, error) { h := sha1.New() if _, err := h.Write([]byte(serviceRevision)); err != nil { @@ -382,9 +348,7 @@ func ComputeRevision(serviceRevision string, instances []*model.Instance) (strin return hex.EncodeToString(h.Sum(nil)), nil } -/** - * RegisterCache 注册缓存资源 - */ +// RegisterCache 注册缓存资源 func RegisterCache(name string, index int) { if _, exist := cacheSet[name]; exist { panic(fmt.Sprintf("existed cache resource: name = %s", name)) diff --git a/naming/cache/cache_test.go b/naming/cache/cache_test.go index ab1161f8d..4eb6d6589 100644 --- a/naming/cache/cache_test.go +++ b/naming/cache/cache_test.go @@ -63,7 +63,7 @@ func TestNamingCache_Start(t *testing.T) { SetCacheConfig(conf) Convey("测试正常的更新缓存逻辑", t, func() { - c, err := NewNamingCache(storage) + c, err := NewNamingCache(storage, nil) So(err, ShouldBeNil) So(c, ShouldNotBeNil) @@ -104,7 +104,7 @@ func TestRevisionWorker(t *testing.T) { defer ctl.Finish() Convey("revision计算,chan可以正常收发", t, func() { - nc, err := NewNamingCache(storage) + nc, err := NewNamingCache(storage, nil) defer func() { _ = nc.Clear() }() So(err, ShouldBeNil) diff --git a/naming/cache/instance.go b/naming/cache/instance.go index d671ef4b9..4298e234a 100644 --- a/naming/cache/instance.go +++ b/naming/cache/instance.go @@ -18,13 +18,12 @@ package cache import ( - "sync" - "time" - "github.com/polarismesh/polaris-server/common/log" "github.com/polarismesh/polaris-server/common/model" "github.com/polarismesh/polaris-server/store" "go.uber.org/zap" + "sync" + "time" ) const ( @@ -35,55 +34,50 @@ const ( // InstanceIterProc instance iter proc func type InstanceIterProc func(key string, value *model.Instance) (bool, error) -/** - * InstanceCache 实例相关的缓存接口 - */ +// InstanceCache 实例相关的缓存接口 type InstanceCache interface { + // Cache 公共缓存接口 Cache + // GetInstance 根据实例ID获取实例数据 GetInstance(instanceID string) *model.Instance - // 根据服务名获取实例,先查找服务名对应的服务ID,再找实例列表 + // GetInstancesByServiceID 根据服务名获取实例,先查找服务名对应的服务ID,再找实例列表 GetInstancesByServiceID(serviceID string) []*model.Instance - // 迭代 + // IteratorInstances 迭代 IteratorInstances(iterProc InstanceIterProc) error - // 根据服务ID进行迭代 + // IteratorInstancesWithService 根据服务ID进行迭代 IteratorInstancesWithService(serviceID string, iterProc InstanceIterProc) error - // 获取instance的个数 + // GetInstancesCount 获取instance的个数 GetInstancesCount() int } -/** - * @brief 实例缓存的类 - */ +// instanceCache 实例缓存的类 type instanceCache struct { storage store.Store lastMtime time.Time firstUpdate bool ids *sync.Map // id -> instance - services *sync.Map // service id -> [instance id -> instance] + services *sync.Map // service id -> [instances] revisionCh chan *revisionNotify disableBusiness bool needMeta bool systemServiceID []string + manager *listenerManager } -/** - * @brief 自注册到缓存列表 - */ func init() { RegisterCache(InstanceName, CacheInstance) } -// 新建一个instanceCache -func newInstanceCache(storage store.Store, ch chan *revisionNotify) *instanceCache { +// newInstanceCache 新建一个instanceCache +func newInstanceCache(storage store.Store, ch chan *revisionNotify, listeners []Listener) *instanceCache { return &instanceCache{ storage: storage, revisionCh: ch, + manager: newListenerManager(listeners), } } -/** - * @brief 初始化函数 - */ +// initialize 初始化函数 func (ic *instanceCache) initialize(opt map[string]interface{}) error { ic.ids = new(sync.Map) ic.services = new(sync.Map) @@ -111,9 +105,7 @@ func (ic *instanceCache) initialize(opt map[string]interface{}) error { return nil } -/** - * @brief 更新缓存函数 - */ +// update 更新缓存函数 func (ic *instanceCache) update() error { // 拉取diff前的所有数据 start := time.Now() @@ -126,14 +118,12 @@ func (ic *instanceCache) update() error { ic.firstUpdate = false update, del := ic.setInstances(instances) - log.Info("[Cache][Instance] get more instances", zap.Int("update", update), zap.Int("delete", del), + log.Debug("[Cache][Instance] get more instances", zap.Int("update", update), zap.Int("delete", del), zap.Time("last", ic.lastMtime), zap.Duration("used", time.Now().Sub(start))) return nil } -/** - * @brief 清理内部缓存数据 - */ +// clear 清理内部缓存数据 func (ic *instanceCache) clear() error { ic.ids = new(sync.Map) ic.services = new(sync.Map) @@ -141,16 +131,12 @@ func (ic *instanceCache) clear() error { return nil } -/** - * @brief 获取资源名称 - */ +// name 获取资源名称 func (ic *instanceCache) name() string { return InstanceName } -/** - * @brief 获取系统服务ID - */ +// getSystemServices 获取系统服务ID func (ic *instanceCache) getSystemServices() ([]*model.Service, error) { services, err := ic.storage.GetSystemServices() if err != nil { @@ -187,6 +173,7 @@ func (ic *instanceCache) setInstances(ins map[string]*model.Instance) (int, int) if !item.Valid { del++ ic.ids.Delete(item.ID()) + ic.manager.onEvent(item.Proto, EventDeleted) value, ok := ic.services.Load(item.ServiceID) if !ok { continue @@ -209,6 +196,9 @@ func (ic *instanceCache) setInstances(ins map[string]*model.Instance) (int, int) if !ok { value = new(sync.Map) ic.services.Store(item.ServiceID, value) + ic.manager.onEvent(item.Proto, EventCreated) + } else { + ic.manager.onEvent(item.Proto, EventUpdated) } value.(*sync.Map).Store(item.ID(), item) } @@ -229,9 +219,7 @@ func (ic *instanceCache) setInstances(ins map[string]*model.Instance) (int, int) return update, del } -/** - * GetInstance 根据实例ID获取实例数据 - */ +// GetInstance 根据实例ID获取实例数据 func (ic *instanceCache) GetInstance(instanceID string) *model.Instance { if instanceID == "" { return nil @@ -245,9 +233,7 @@ func (ic *instanceCache) GetInstance(instanceID string) *model.Instance { return value.(*model.Instance) } -/** - * GetInstancesByServiceID 根据ServiceID获取实例数据 - */ +// GetInstancesByServiceID 根据ServiceID获取实例数据 func (ic *instanceCache) GetInstancesByServiceID(serviceID string) []*model.Instance { if serviceID == "" { return nil @@ -267,9 +253,7 @@ func (ic *instanceCache) GetInstancesByServiceID(serviceID string) []*model.Inst return out } -/** - * IteratorInstances 迭代所有的instance的函数 - */ +// IteratorInstances 迭代所有的instance的函数 func (ic *instanceCache) IteratorInstances(iterProc InstanceIterProc) error { return iteratorInstancesProc(ic.ids, iterProc) } diff --git a/naming/cache/instance_test.go b/naming/cache/instance_test.go index 29fbdf7e1..496ce9272 100644 --- a/naming/cache/instance_test.go +++ b/naming/cache/instance_test.go @@ -34,7 +34,7 @@ func newTestInstanceCache(t *testing.T) (*gomock.Controller, *mock.MockStore, *i ctl := gomock.NewController(t) storage := mock.NewMockStore(ctl) - ic := newInstanceCache(storage, make(chan *revisionNotify, 1024)) + ic := newInstanceCache(storage, make(chan *revisionNotify, 1024), nil) opt := map[string]interface{}{ "disableBusiness": false, "needMeta": true, diff --git a/naming/cache/listener.go b/naming/cache/listener.go new file mode 100644 index 000000000..651934af6 --- /dev/null +++ b/naming/cache/listener.go @@ -0,0 +1,74 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package cache + +import ( + "sync" +) + +// Listener listener for value changes +type Listener interface { + // OnCreated callback when cache value created + OnCreated(value interface{}) + // OnUpdated callback when cache value updated + OnUpdated(value interface{}) + // OnDeleted callback when cache value deleted + OnDeleted(value interface{}) +} + +// EventType common event type +type EventType int + +const ( + // EventCreated value create event + EventCreated EventType = iota + // EventUpdated value update event + EventUpdated + // EventDeleted value delete event + EventDeleted +) + +type listenerManager struct { + rwMutex *sync.RWMutex + listeners []Listener +} + +func newListenerManager(listeners []Listener) *listenerManager { + return &listenerManager{ + rwMutex: &sync.RWMutex{}, + listeners: listeners, + } +} + +func (l *listenerManager) onEvent(value interface{}, event EventType) { + l.rwMutex.RLock() + defer l.rwMutex.RUnlock() + if len(l.listeners) == 0 { + return + } + for _, listener := range l.listeners { + switch event { + case EventCreated: + listener.OnCreated(value) + case EventUpdated: + listener.OnUpdated(value) + case EventDeleted: + listener.OnDeleted(value) + } + } +} diff --git a/naming/cache/service.go b/naming/cache/service.go index 57e8d2d7c..f3c7fd4b8 100644 --- a/naming/cache/service.go +++ b/naming/cache/service.go @@ -18,13 +18,12 @@ package cache import ( - "sync" - "time" - "github.com/polarismesh/polaris-server/common/log" "github.com/polarismesh/polaris-server/common/model" "github.com/polarismesh/polaris-server/store" "go.uber.org/zap" + "sync" + "time" ) const ( @@ -126,7 +125,7 @@ func (sc *serviceCache) update() error { sc.firstUpdate = false update, del := sc.setServices(services) - log.Info("[Cache][Service] get more services", zap.Int("update", update), zap.Int("delete", del), + log.Debug("[Cache][Service] get more services", zap.Int("update", update), zap.Int("delete", del), zap.Time("last", sc.lastMtime), zap.Duration("used", time.Now().Sub(start))) return nil } diff --git a/naming/client.go b/naming/client.go index e833c83b6..f4fc8c0e1 100644 --- a/naming/client.go +++ b/naming/client.go @@ -59,20 +59,6 @@ func (s *Server) ReportClient(ctx context.Context, req *api.Client) *api.Respons return api.NewClientResponse(api.ExecuteSuccess, out) } -/** - * Heartbeat 服务实例上报心跳 - */ -func (s *Server) Heartbeat(ctx context.Context, req *api.Instance) *api.Response { - if s.caches == nil { - return api.NewResponse(api.ClientAPINotOpen) - } - - if s.hbMgr == nil { - return api.NewInstanceResponse(api.HealthCheckNotOpen, req) - } - return s.hbMgr.healthCheck(ctx, req) -} - /** * GetServiceWithCache 根据元数据查询服务 */ diff --git a/naming/healthcheck.go b/naming/healthcheck.go deleted file mode 100644 index f8a05fd5e..000000000 --- a/naming/healthcheck.go +++ /dev/null @@ -1,478 +0,0 @@ -/* - * Tencent is pleased to support the open source community by making Polaris available. - * - * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. - * - * Licensed under the BSD 3-Clause License (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * https://opensource.org/licenses/BSD-3-Clause - * - * Unless required by applicable law or agreed to in writing, software distributed - * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR - * CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package naming - -import ( - "context" - "errors" - "strconv" - "strings" - "sync" - "time" - - api "github.com/polarismesh/polaris-server/common/api/v1" - "github.com/polarismesh/polaris-server/common/log" - "github.com/polarismesh/polaris-server/common/model" - "github.com/polarismesh/polaris-server/common/redispool" - "github.com/polarismesh/polaris-server/common/timewheel" - "github.com/polarismesh/polaris-server/common/utils" - "go.uber.org/zap" -) - -/** - * HealthCheckConfig 健康检查配置 - */ -type HealthCheckConfig struct { - Open bool `yaml:"open"` - KvConnNum int `yaml:"kvConnNum"` - KvServiceName string `yaml:"kvServiceName"` - KvNamespace string `yaml:"kvNamespace"` - KvPasswd string `yaml:"kvPasswd"` - SlotNum int `yaml:"slotNum"` - LocalHost string `yaml:"localHost"` - MaxIdle int `yaml:"maxIdle"` - IdleTimeout int `yaml:"idleTimeout"` -} - -/** - * HbInfo 记录实例心跳信息 - */ -type HbInfo struct { - id string - addr string - beatTime int64 - ttl uint32 -} - -/** - * HeartBeatMgr 心跳管理器结构体 - * 包括时间轮、ckv连接池、存储实例心跳信息的map - */ -type HeartBeatMgr struct { - ctx context.Context - mu sync.Mutex - hbMap map[string]*HbInfo - ckvTw *timewheel.TimeWheel - dbTw *timewheel.TimeWheel - // ckvPool *ckv.Pool - redisPool *redispool.Pool -} - -/** - * TimeWheelTask 时间轮任务结构体 - */ -type TimeWheelTask struct { - lastBeatTime int64 - hbInfo *HbInfo -} - -const ( - NotHealthy = 0 - Healthy = 1 - RedisNoKeyErr = "redigo: nil returned" -) - -var ( - healthCheckConf *HealthCheckConfig - hbMgr *HeartBeatMgr - - /** - * @brief 时间轮回调函数1:实例上报心跳1 ttl后执行,发现这 - * 段时间内该实例没有再次上报心跳则改写ckv中的实例状态为不健康 - */ - ckvCallback timewheel.Callback = func(data interface{}) { - task := data.(*TimeWheelTask) - lastBeatTime := task.lastBeatTime - // 说明本机收到了心跳,实例状态正常,不做任何处理 - if lastBeatTime < task.hbInfo.beatTime { - return - } - - // 更新上报心跳时间 - now := time.Now().Unix() - task.lastBeatTime = now - - // 从ckv获取实例状态,看其他server这段时间有没有收到心跳 - respCh := make(chan *redispool.Resp) - hbMgr.redisPool.Get(task.hbInfo.id, respCh) - resp := <-respCh - if !resp.Local { - if resp.Err != nil { - // 获取ckv失败,不能退出,直接进入set ckv unhealthy & dbCallback流程 - log.Errorf("[health check] addr:%s id:%s 1ttl get redis err:%s", - task.hbInfo.addr, task.hbInfo.id, resp.Err) - } else { - // ckv中value格式 健康状态(1健康 0不健康):心跳时间戳:写者ip - // 如: 1:timestamp:10.60.31.22 - // 基于解析性能考虑,没有使用json - res := strings.Split(resp.Value, ":") - kvBeatTime, err := strconv.ParseInt(res[1], 10, 64) - if err != nil { - log.Errorf("[health check] addr:%s id:%s redis beat time parse err:%s", - task.hbInfo.addr, task.hbInfo.id, err) - } - - // ckv中的实例状态为已经被改为不健康 - // 或ckv心跳时间 > 本地上次心跳时间,说明其他server收到了心跳,不做任何处理 - if res[0] == "0" || kvBeatTime > lastBeatTime { - return - } - } - - // 将ckv中状态改为不健康 - log.Infof("[health check] addr:%s id:%s 1ttl overtime, set redis not healthy", task.hbInfo.addr, task.hbInfo.id) - hbMgr.redisPool.Set(task.hbInfo.id, NotHealthy, now, respCh) - resp = <-respCh - if resp.Err != nil { - log.Errorf("[health check] addr:%s id:%s set redis err:%s", task.hbInfo.addr, task.hbInfo.id, resp.Err) - } - } - // 添加时间轮任务:再过2ttl后仍未收到心跳,改写db实例状态为不健康 - _ = hbMgr.dbTw.AddTask(time.Duration(2*task.hbInfo.ttl-1)*time.Second, task, dbCallback) - } - - /** - * @brief 时间轮回调函数2:实例上报心跳1 ttl后若 - * 未上报心跳,则再过2ttl后执行此函数,负责判定实例死亡、改写db状态 - */ - dbCallback timewheel.Callback = func(data interface{}) { - task := data.(*TimeWheelTask) - // 说明本机收到了心跳,实例状态正常,不做任何处理 - if task.lastBeatTime < task.hbInfo.beatTime { - return - } - - // 从ckv获取下key状态,看其他server有没有收到心跳 - respCh := make(chan *redispool.Resp) - hbMgr.redisPool.Get(task.hbInfo.id, respCh) - resp := <-respCh - if !resp.Local { - if resp.Err != nil { - log.Errorf("[healthCheck] dbCallback get addr(%s) id(%s) from redis err: %s", - task.hbInfo.addr, task.hbInfo.id, resp.Err) - } else { - res := strings.Split(resp.Value, ":") - if res[0] == "1" { - log.Infof("[health check] addr: %s id: %s redis status is healthy, ignore set db unhealthy", - task.hbInfo.addr, task.hbInfo.id) - return - } - } - - // 删除kv - log.Infof("[health check] del redis id:%s", task.hbInfo.id) - hbMgr.redisPool.Del(task.hbInfo.id, respCh) - resp = <-respCh - if resp.Err != nil { - log.Errorf("[health check] addr:%s id:%s del redis err:%s", task.hbInfo.addr, task.hbInfo.id, resp.Err) - } - } - - insCache := server.caches.Instance().GetInstance(task.hbInfo.id) - if insCache == nil { - log.Errorf(`[health check] addr:%s id:%s ready to set db status - not health, but not found instance`, task.hbInfo.addr, task.hbInfo.id) - return - } - - // 修改db状态为不健康 - // 如果用户关闭了健康检查功能,则不做任何处理 - if insCache.EnableHealthCheck() && insCache.Healthy() != false { - setInsDbStatus(task.hbInfo.id, task.hbInfo.addr, NotHealthy) - } - - // 从本机map中删除 - hbMgr.mu.Lock() - delete(hbMgr.hbMap, task.hbInfo.id) - hbMgr.mu.Unlock() - } -) - -/** -* SetHealthCheckConfig 设置健康检查配置 - */ -func SetHealthCheckConfig(conf *HealthCheckConfig) { - healthCheckConf = conf -} - -/** - * NewHeartBeatMgr 初始化心跳管理器 - */ -func NewHeartBeatMgr(ctx context.Context) (*HeartBeatMgr, error) { - kvService := server.caches.Service(). - GetServiceByName(healthCheckConf.KvServiceName, healthCheckConf.KvNamespace) - var kvInstances []*model.Instance - - if kvService != nil { - kvInstances = server.caches.Instance().GetInstancesByServiceID(kvService.ID) - } - // if len(kvInstances) == 0 { - // return nil, fmt.Errorf("no available ckv instance, serviceId:%s", kvService.ID) - // } - - redisPool, err := redispool.NewPool(healthCheckConf.KvConnNum, healthCheckConf.KvPasswd, - healthCheckConf.LocalHost, kvInstances, healthCheckConf.MaxIdle, healthCheckConf.IdleTimeout) - if err != nil { - return nil, err - } - - mgr := &HeartBeatMgr{ - ctx: ctx, - hbMap: make(map[string]*HbInfo), - ckvTw: timewheel.New(time.Second, healthCheckConf.SlotNum, "ckv task timewheel"), - dbTw: timewheel.New(time.Second, healthCheckConf.SlotNum, "db task timewheel"), - redisPool: redisPool, - } - if kvService != nil { - go mgr.watchCkvService(kvService.ID) - } - return mgr, nil -} - -/** - * Start 启动心跳管理器,启动健康检查功能 - */ -func (hb *HeartBeatMgr) Start() { - hb.redisPool.Start() - hb.ckvTw.Start() - hb.dbTw.Start() -} - -/** - * @brief 监控ckv实例有没有变化 - */ -func (hb *HeartBeatMgr) watchCkvService(id string) { - kvInstances := server.caches.Instance().GetInstancesByServiceID(id) - lastRevision, err := server.GetServiceInstanceRevision(id, kvInstances) - if err != nil { - log.Errorf("[health check] get redis revision err:%s", err) - } - - for range time.NewTicker(10 * time.Second).C { - kvInstances = server.caches.Instance().GetInstancesByServiceID(id) - if len(kvInstances) == 0 { - // need alert - log.Errorf("[health check] get redis ins nil") - continue - } - newRevision, err := server.GetServiceInstanceRevision(id, kvInstances) - if err != nil { - log.Errorf("[health check] get redis revision err:%s", err) - continue - } - if lastRevision != newRevision { - err := hb.redisPool.Update(kvInstances) - if err != nil { - // need alert - log.Errorf("[health check] update redis pool err:%s", err) - continue - } - lastRevision = newRevision - } - } -} - -/** -* @brief 心跳处理函数 - */ -func (hb *HeartBeatMgr) healthCheck(ctx context.Context, instance *api.Instance) *api.Response { - id, errRsp := checkHeartbeatInstance(instance) - if errRsp != nil { - return errRsp - } - instance.Id = utils.NewStringValue(id) - insCache := server.caches.Instance().GetInstance(id) - if insCache == nil { - return api.NewInstanceResponse(api.NotFoundResource, instance) - } - - service := server.caches.Service().GetServiceByID(insCache.ServiceID) - if service == nil { - return api.NewInstanceResponse(api.NotFoundResource, instance) - } - // 鉴权 - token := instance.GetServiceToken().GetValue() - if !server.authority.VerifyToken(token) { - return api.NewInstanceResponse(api.InvalidServiceToken, instance) - } - ok := server.authority.VerifyInstance(service.Token, token) - if !ok { - return api.NewInstanceResponse(api.Unauthorized, instance) - } - - // 如果实例未开启健康检查,返回 - if !insCache.EnableHealthCheck() || insCache.HealthCheck() == nil { - return api.NewInstanceResponse(api.HeartbeatOnDisabledIns, instance) - } - - // 记录收到心跳的instance日志,方便定位实例是否上报心跳 - log.Info("receive heartbeat", ZapRequestID(ParseRequestID(ctx)), zap.String("id", id), - zap.String("service", service.Namespace+":"+service.Name), - zap.String("host", insCache.Host()), zap.Uint32("port", insCache.Port())) - addr := insCache.Host() + ":" + strconv.Itoa(int(insCache.Port())) - ttl := insCache.HealthCheck().GetHeartbeat().GetTtl().GetValue() - now := time.Now().Unix() - var hbInfo *HbInfo - respCh := make(chan *redispool.Resp) - - hb.mu.Lock() - hbInfo, ok = hb.hbMap[id] - if !ok { - hbInfo = &HbInfo{id, addr, now, ttl} - hb.hbMap[id] = hbInfo - hb.mu.Unlock() - - // hbMap中没有找到该实例,说明这是实例近期第一次上报心跳,set ckv中实例状态为健康 - log.Infof("[health check] addr:%s id:%s ttl:%d heartbeat first time, set redis", addr, id, ttl) - hbMgr.redisPool.Set(id, Healthy, now, respCh) - resp := <-respCh - if resp.Err != nil { - log.Errorf("[health check] addr:%s id:%s set redis err:%s", addr, id, resp.Err) - return api.NewInstanceResponse(api.HeartbeatException, instance) - } - } else { - hb.mu.Unlock() - lastBeatTime := hbInfo.beatTime - if now == lastBeatTime { - log.Debugf("[health check] addr:%s id:%s ins heartbeat exceed 1 time/s", addr, id) - return api.NewInstanceResponse(api.HeartbeatExceedLimit, instance) - } - - // 修改实例心跳上报时间 - hbInfo.beatTime = now - hbInfo.ttl = ttl - - // 本机超过1 ttl + 1s未收到心跳,set一次ckv状态 - if now-lastBeatTime >= int64(ttl+1) { - log.Infof("[health check] addr:%s, id:%s receive heart beat after ttl + 1s, set redis healthy", addr, id) - hbMgr.redisPool.Set(id, Healthy, now, respCh) - resp := <-respCh - - if resp.Err != nil { - log.Errorf("[health check] addr:%s id:%s set redis err:%s", addr, id, resp.Err) - return api.NewInstanceResponse(api.HeartbeatException, instance) - } - } - } - - // db中实例状态若为不健康,设为健康 - if insCache.Healthy() != true { - setInsDbStatus(id, addr, Healthy) - } - - // 将超时检查任务放入时间轮 - task := &TimeWheelTask{now, hbInfo} - _ = hb.ckvTw.AddTask(time.Duration(ttl+1)*time.Second, task, ckvCallback) - - return api.NewInstanceResponse(api.ExecuteSuccess, instance) -} - -// 获取上一次的心跳时间 -func (hb *HeartBeatMgr) acquireLastHeartbeat(instance *api.Instance) error { - id := instance.GetId().GetValue() - if instance.Metadata == nil { - instance.Metadata = make(map[string]string) - } - - // 先获取本地记录的时间,这里可能为空的 - // (该实例不是上报到这台server,可以根据ckv信息获取其上报的server) - hb.mu.Lock() - info, ok := hb.hbMap[id] - hb.mu.Unlock() - if ok { - instance.Metadata["last-heartbeat-time"] = time2String(time.Unix(info.beatTime, 0)) - instance.Metadata["system-time"] = time2String(time.Now()) - } - - // 获取ckv记录的时间 - respCh := make(chan *redispool.Resp) - hbMgr.redisPool.Get(id, respCh) - resp := <-respCh - if resp.Err != nil { - if resp.Err.Error() == RedisNoKeyErr { - return nil - } - - log.Errorf("[health check] get id(%s) from redis err: %s", id, resp.Err.Error()) - return resp.Err - } - if resp.Local { - return nil - } - - res := strings.Split(resp.Value, ":") - if len(res) != 3 { - log.Errorf("[health check] id(%s) redis record invalid(%s)", id, resp.Value) - return errors.New("invalid ckv record") - } - tm, err := strconv.ParseInt(res[1], 10, 64) - if err != nil { - log.Errorf("[health check] id(%s) redis record heartbeat time(%s) is invalid", id, res[1]) - return err - } - - // ckv记录的心跳时间与心跳server, - // 根据这个心跳server可以获取到实例上报到哪台心跳server - instance.Metadata["ckv-record-healthy"] = res[0] - instance.Metadata["ckv-record-heartbeat-time"] = time2String(time.Unix(tm, 0)) - instance.Metadata["ckv-record-heartbeat-server"] = res[2] - return nil -} - -/** -* @brief 修改实例状态 - 需要打印操作记录 - server 是当前package的全局变量 -*/ -func setInsDbStatus(id, addr string, status int) { - log.Infof("[health check] addr:%s id:%s set db status %d", addr, id, status) - err := server.storage.SetInstanceHealthStatus(id, status, NewUUID()) - if err != nil { - log.Errorf("[health check] id: %s set db status err:%s", id, err) - return - } - - instance := server.caches.Instance().GetInstance(id) - if instance == nil { - log.Errorf("[HealthCheck] not found instance(%s)", id) - return - } - service := server.caches.Service().GetServiceByID(instance.ServiceID) - if service == nil { - log.Errorf("[HealthCheck] not found serviceID(%s) for instance(%s)", - instance.ServiceID, id) - return - } - - healthStatus := true - if status == 0 { - healthStatus = false - } - recordInstance := &model.Instance{ - Proto: &api.Instance{ - Host: instance.Proto.GetHost(), - Port: instance.Proto.GetPort(), - Priority: instance.Proto.GetPriority(), - Weight: instance.Proto.GetWeight(), - Healthy: utils.NewBoolValue(healthStatus), - Isolate: instance.Proto.GetIsolate(), - }, - } - - server.RecordHistory(instanceRecordEntry(nil, service, recordInstance, model.OUpdate)) -} diff --git a/naming/instance.go b/naming/instance.go index e9ba6fa33..dd68449ec 100644 --- a/naming/instance.go +++ b/naming/instance.go @@ -130,7 +130,6 @@ func (s *Server) createInstance(ctx context.Context, req *api.Instance, ins *api if server.bc == nil || !server.bc.CreateInstanceOpen() { return s.serialCreateInstance(ctx, req, ins) // 单个同步 } - return s.asyncCreateInstance(ctx, req, ins) // 批量异步 } @@ -711,7 +710,7 @@ func (s *Server) CleanInstance(ctx context.Context, req *api.Instance) *api.Resp } return req.GetId().GetValue(), nil } - return checkInstanceTetrad(req) + return CheckInstanceTetrad(req) } instanceID, resp := getInstanceID() @@ -727,49 +726,6 @@ func (s *Server) CleanInstance(ctx context.Context, req *api.Instance) *api.Resp return api.NewInstanceResponse(api.ExecuteSuccess, req) } -// GetLastHeartbeat 获取上一次心跳的时间 -func (s *Server) GetLastHeartbeat(req *api.Instance) *api.Response { - if s.hbMgr == nil { - return api.NewInstanceResponse(api.HealthCheckNotOpen, req) - } - if s.caches == nil || s.caches.Instance() == nil || s.caches.Service() == nil { - return api.NewInstanceResponse(api.ClientAPINotOpen, req) - } - - id := req.GetId().GetValue() - if id == "" { - tmpID, resp := checkInstanceTetrad(req) - if resp != nil { - return resp - } - id = tmpID - } - if id == "" { - return api.NewInstanceResponse(api.InvalidInstanceID, req) - } - req.Id = utils.NewStringValue(id) - - instance := s.caches.Instance().GetInstance(id) - if instance == nil { - return api.NewInstanceResponse(api.NotFoundInstance, req) - } - service := s.caches.Service().GetServiceByID(instance.ServiceID) - if service == nil { - return api.NewInstanceResponse(api.NotFoundService, req) - } - - if err := s.hbMgr.acquireLastHeartbeat(req); err != nil { - return api.NewInstanceRespWithError(api.ExecuteException, err, req) - } - req.Service = utils.NewStringValue(service.Name) - req.Namespace = utils.NewStringValue(service.Namespace) - req.Host = instance.Proto.GetHost() - req.Port = instance.Proto.GetPort() - req.VpcId = instance.Proto.GetVpcId() - req.HealthCheck = instance.HealthCheck() - return api.NewInstanceResponse(api.ExecuteSuccess, req) -} - // update/delete instance前置条件 func (s *Server) execInstancePreStep(ctx context.Context, req *api.Instance) ( *model.Service, *model.Instance, *api.Response) { @@ -915,7 +871,7 @@ func checkCreateInstance(req *api.Instance) (string, *api.Response) { return "", err } - return checkInstanceTetrad(req) + return CheckInstanceTetrad(req) } /* @@ -939,7 +895,7 @@ func checkReviseInstance(req *api.Instance) (string, *api.Response) { return "", err } - return checkInstanceTetrad(req) + return CheckInstanceTetrad(req) } /* @@ -957,13 +913,11 @@ func checkHeartbeatInstance(req *api.Instance) (string, *api.Response) { } return req.GetId().GetValue(), nil } - return checkInstanceTetrad(req) + return CheckInstanceTetrad(req) } -/* - * @brief 根据服务实例四元组计算ID - */ -func checkInstanceTetrad(req *api.Instance) (string, *api.Response) { +// CheckInstanceTetrad 根据服务实例四元组计算ID +func CheckInstanceTetrad(req *api.Instance) (string, *api.Response) { if err := checkResourceName(req.GetService()); err != nil { return "", api.NewInstanceResponse(api.InvalidServiceName, req) } @@ -980,16 +934,20 @@ func checkInstanceTetrad(req *api.Instance) (string, *api.Response) { return "", api.NewInstanceResponse(api.InvalidInstancePort, req) } - id, err := CalculateInstanceID(req.GetNamespace().GetValue(), req.GetService().GetValue(), - req.GetVpcId().GetValue(), - req.GetHost().GetValue(), - req.GetPort().GetValue(), - ) - if err != nil { - return "", api.NewInstanceResponse(api.ExecuteException, req) + var instId string + instId = req.GetId().GetValue() + if len(instId) == 0 { + id, err := CalculateInstanceID(req.GetNamespace().GetValue(), req.GetService().GetValue(), + req.GetVpcId().GetValue(), + req.GetHost().GetValue(), + req.GetPort().GetValue(), + ) + if err != nil { + return "", api.NewInstanceResponse(api.ExecuteException, req) + } + instId = id } - - return id, nil + return instId, nil } // 获取instance请求的token信息 diff --git a/naming/server.go b/naming/server.go index af381d28a..5cfc410e3 100644 --- a/naming/server.go +++ b/naming/server.go @@ -21,6 +21,7 @@ import ( "context" "encoding/hex" "errors" + "github.com/modern-go/reflect2" "sync" "time" @@ -57,24 +58,18 @@ var ( finishInit = false ) -/** - * Config 核心逻辑层配置 - */ +// Config 核心逻辑层配置 type Config struct { - Auth map[string]interface{} `yaml:"auth"` - HealthCheck HealthCheckConfig `yaml:"healthcheck"` - Batch map[string]interface{} `yaml:"batch"` + Auth map[string]interface{} `yaml:"auth"` + Batch map[string]interface{} `yaml:"batch"` } -/** - * Server 对接API层的server层,用以处理业务逻辑 - */ +// Server 对接API层的server层,用以处理业务逻辑 type Server struct { storage store.Store caches *cache.NamingCache authority auth.Authority - hbMgr *HeartBeatMgr bc *batch.Controller cmdb plugin.CMDB @@ -86,13 +81,11 @@ type Server struct { l5service *l5service } -/** - * Initialize 初始化 - */ -func Initialize(ctx context.Context, namingOpt *Config, cacheOpt *cache.Config) error { +// Initialize 初始化 +func Initialize(ctx context.Context, namingOpt *Config, cacheOpt *cache.Config, listener cache.Listener) error { var err error once.Do(func() { - err = initialize(ctx, namingOpt, cacheOpt) + err = initialize(ctx, namingOpt, cacheOpt, listener) }) if err != nil { @@ -112,16 +105,12 @@ func GetServer() (*Server, error) { return server, nil } -/** - * Authority 返回鉴权对象,获取鉴权信息 - */ +// Authority 返回鉴权对象,获取鉴权信息 func (s *Server) Authority() auth.Authority { return s.authority } -/** - * Cache 返回Cache - */ +// Cache 返回Cache func (s *Server) Cache() *cache.NamingCache { return s.caches } @@ -141,9 +130,7 @@ func (s *Server) RecordHistory(entry *model.RecordEntry) { s.history.Record(entry) } -/** - * RecordDiscoverStatis 打印服务发现统计 - */ +// RecordDiscoverStatis 打印服务发现统计 func (s *Server) RecordDiscoverStatis(service, namespace string) { if s.discoverStatis == nil { return @@ -197,7 +184,7 @@ func (s *Server) allowInstanceAccess(instanceID string) bool { } // 内部初始化函数 -func initialize(ctx context.Context, namingOpt *Config, cacheOpt *cache.Config) error { +func initialize(ctx context.Context, namingOpt *Config, cacheOpt *cache.Config, listener cache.Listener) error { // 获取存储层对象 s, err := store.GetStore() if err != nil { @@ -223,7 +210,11 @@ func initialize(ctx context.Context, namingOpt *Config, cacheOpt *cache.Config) if cacheOpt.Open { cache.SetCacheConfig(cacheOpt) log.Infof("cache is open, can access the client api function") - caches, cacheErr := cache.NewNamingCache(s) + var listeners []cache.Listener + if !reflect2.IsNil(listener) { + listeners = append(listeners, listener) + } + caches, cacheErr := cache.NewNamingCache(s, listeners) if cacheErr != nil { log.Errorf("[Naming][Server] new naming cache err: %s", cacheErr.Error()) return cacheErr @@ -235,17 +226,6 @@ func initialize(ctx context.Context, namingOpt *Config, cacheOpt *cache.Config) } } - // 启动健康检查 - if healthCheckConf.Open { - hbMgr, err = NewHeartBeatMgr(ctx) - if err != nil { - log.Errorf("[Naming][Server] new heartbeat mgr failed: %s", err.Error()) - return err - } - - server.hbMgr = hbMgr - server.hbMgr.Start() - } // 批量控制器 batchConfig, err := batch.ParseBatchConfig(namingOpt.Batch) if err != nil { @@ -303,9 +283,7 @@ func pluginInitialize() { } } -/** - * NewUUID 返回一个随机的UUID - */ +// NewUUID 返回一个随机的UUID func NewUUID() string { uuidBytes := uuid.New() return hex.EncodeToString(uuidBytes[:]) diff --git a/naming/test/main_test.go b/naming/test/main_test.go index 2dff6b6ae..bb3d5f6b6 100644 --- a/naming/test/main_test.go +++ b/naming/test/main_test.go @@ -42,7 +42,7 @@ import ( _ "github.com/go-sql-driver/mysql" _ "github.com/polarismesh/polaris-server/plugin/history/logger" - _ "github.com/polarismesh/polaris-server/plugin/ratelimit/tokenbucket" + _ "github.com/polarismesh/polaris-server/plugin/ratelimit/token" _ "github.com/polarismesh/polaris-server/store/sqldb" "gopkg.in/yaml.v2" ) diff --git a/naming/test/run.sh b/naming/test/run.sh index 159a0e7ea..cac4fc88d 100644 --- a/naming/test/run.sh +++ b/naming/test/run.sh @@ -1,5 +1,5 @@ #!/bin/bash -go test -v -mod=vendor -cover -timeout=3600s -coverprofile=cover.out -coverpkg=github.com/polarismesh/polaris-server/naming,github.com/polarismesh/polaris-server/naming/batch,github.com/polarismesh/polaris-server/naming/cache,github.com/polarismesh/polaris-server/naming/auth,github.com/polarismesh/polaris-server/store/defaultStore,github.com/polarismesh/polaris-server/plugin/ratelimit/tokenBucket,github.com/polarismesh/polaris-server/common/model | tee test.log +go test -v -mod=vendor -cover -timeout=3600s -coverprofile=cover.out -coverpkg=github.com/polarismesh/polaris-server/naming,github.com/polarismesh/polaris-server/naming/batch,github.com/polarismesh/polaris-server/naming/cache,github.com/polarismesh/polaris-server/naming/auth,github.com/polarismesh/polaris-server/store/defaultStore,github.com/polarismesh/polaris-server/plugin/ratelimit/token,github.com/polarismesh/polaris-server/common/model | tee test.log go tool cover -html=cover.out -o index.html coverage=$(cat test.log | grep "coverage:" | awk '{print $2}' | cut -d '%' -f 1) diff --git a/naming/testauthplugin/main_test.go b/naming/testauthplugin/main_test.go index 964c8d9e7..1d4e9460c 100644 --- a/naming/testauthplugin/main_test.go +++ b/naming/testauthplugin/main_test.go @@ -36,7 +36,7 @@ import ( _ "github.com/go-sql-driver/mysql" _ "github.com/polarismesh/polaris-server/plugin/auth/platform" _ "github.com/polarismesh/polaris-server/plugin/history/logger" - _ "github.com/polarismesh/polaris-server/plugin/ratelimit/tokenbucket" + _ "github.com/polarismesh/polaris-server/plugin/ratelimit/token" _ "github.com/polarismesh/polaris-server/store/sqldb" ) @@ -79,8 +79,7 @@ func initialize() error { // 初始化naming server ctx := context.Background() - naming.SetHealthCheckConfig(&cfg.Naming.HealthCheck) - if err := naming.Initialize(ctx, &cfg.Naming, &cfg.Cache); err != nil { + if err := naming.Initialize(ctx, &cfg.Naming, &cfg.Cache, nil); err != nil { panic(err) } diff --git a/plugin.go b/plugin.go index d3eead03b..c50013971 100644 --- a/plugin.go +++ b/plugin.go @@ -30,10 +30,13 @@ import ( _ "github.com/polarismesh/polaris-server/plugin/cmdb/memory" _ "github.com/polarismesh/polaris-server/plugin/auth/platform" - _ "github.com/polarismesh/polaris-server/plugin/discoverstatis/discoverlocal" + _ "github.com/polarismesh/polaris-server/plugin/discoverstat/discoverlocal" _ "github.com/polarismesh/polaris-server/plugin/history/logger" _ "github.com/polarismesh/polaris-server/plugin/password" _ "github.com/polarismesh/polaris-server/plugin/ratelimit/lrurate" - _ "github.com/polarismesh/polaris-server/plugin/ratelimit/tokenbucket" + _ "github.com/polarismesh/polaris-server/plugin/ratelimit/token" _ "github.com/polarismesh/polaris-server/plugin/statis/local" + + _ "github.com/polarismesh/polaris-server/plugin/healthchecker/heartbeatmemory" + _ "github.com/polarismesh/polaris-server/plugin/healthchecker/heartbeatredis" ) diff --git a/plugin/discoverstatis/discoverlocal/README.md b/plugin/discoverstat/discoverlocal/README.md similarity index 100% rename from plugin/discoverstatis/discoverlocal/README.md rename to plugin/discoverstat/discoverlocal/README.md diff --git a/plugin/discoverstatis/discoverlocal/discovercall.go b/plugin/discoverstat/discoverlocal/discovercall.go similarity index 100% rename from plugin/discoverstatis/discoverlocal/discovercall.go rename to plugin/discoverstat/discoverlocal/discovercall.go diff --git a/plugin/discoverstatis/discoverlocal/local.go b/plugin/discoverstat/discoverlocal/local.go similarity index 100% rename from plugin/discoverstatis/discoverlocal/local.go rename to plugin/discoverstat/discoverlocal/local.go diff --git a/plugin/discoverstatis/discoverlocal/local_test.go b/plugin/discoverstat/discoverlocal/local_test.go similarity index 100% rename from plugin/discoverstatis/discoverlocal/local_test.go rename to plugin/discoverstat/discoverlocal/local_test.go diff --git a/plugin/discoverstatis/discoverlocal/logger.go b/plugin/discoverstat/discoverlocal/logger.go similarity index 100% rename from plugin/discoverstatis/discoverlocal/logger.go rename to plugin/discoverstat/discoverlocal/logger.go diff --git a/plugin/healthchecker.go b/plugin/healthchecker.go new file mode 100644 index 000000000..0e877720e --- /dev/null +++ b/plugin/healthchecker.go @@ -0,0 +1,100 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package plugin + +import ( + "github.com/polarismesh/polaris-server/common/log" + "os" + "sync" +) + +// ReportRequest report heartbeat request +type ReportRequest struct { + QueryRequest + LocalHost string + CurTimeSec int64 +} + +// CheckRequest +type CheckRequest struct { + QueryRequest + ExpireDurationSec uint32 + CurTimeSec int64 +} + +// CheckResponse +type CheckResponse struct { + Healthy bool + LastHeartbeatTimeSec int64 + OnRecover bool +} + +// QueryRequest +type QueryRequest struct { + InstanceId string + Host string + Port uint32 + Healthy bool +} + +// QueryResponse +type QueryResponse struct { + Server string + Exists bool + LastHeartbeatSec int64 +} + +type HealthCheckType int32 + +const ( + HealthCheckerHeartbeat HealthCheckType = iota + 1 +) + +var ( + healthCheckOnce = &sync.Once{} +) + +// HealthChecker health checker plugin interface +type HealthChecker interface { + Plugin + // Type type for health check plugin, only one same type plugin is allowed + Type() HealthCheckType + // Report process heartbeat info report + Report(request *ReportRequest) error + // Report process the instance check + Check(request *CheckRequest) (*CheckResponse, error) + // Query query the heartbeat time + Query(request *QueryRequest) (*QueryResponse, error) +} + +// GetHealthChecker get the health checker by name +func GetHealthChecker(name string, cfg *ConfigEntry) HealthChecker { + plugin, exist := pluginSet[name] + if !exist { + return nil + } + + healthCheckOnce.Do(func() { + if err := plugin.Initialize(cfg); err != nil { + log.Errorf("plugin init err: %s", err.Error()) + os.Exit(-1) + } + }) + + return plugin.(HealthChecker) +} diff --git a/plugin/healthchecker/heartbeatmemory/checker_memory.go b/plugin/healthchecker/heartbeatmemory/checker_memory.go new file mode 100644 index 000000000..821fb0419 --- /dev/null +++ b/plugin/healthchecker/heartbeatmemory/checker_memory.go @@ -0,0 +1,112 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package heartbeatmemory + +import ( + "github.com/polarismesh/polaris-server/plugin" + "sync" +) + +// 把操作记录记录到日志文件中 +const ( + // PluginName plugin name + PluginName = "heartbeatMemory" +) + +// HeartbeatRecord record for heartbeat +type HeartbeatRecord struct { + Server string + CurTimeSec int64 +} + +// MemoryHealthChecker +type MemoryHealthChecker struct { + hbRecords *sync.Map +} + +// Name +func (r *MemoryHealthChecker) Name() string { + return PluginName +} + +// Initialize +func (r *MemoryHealthChecker) Initialize(c *plugin.ConfigEntry) error { + r.hbRecords = &sync.Map{} + return nil +} + +// Destroy +func (r *MemoryHealthChecker) Destroy() error { + return nil +} + +// Type type for health check plugin, only one same type plugin is allowed +func (r *MemoryHealthChecker) Type() plugin.HealthCheckType { + return plugin.HealthCheckerHeartbeat +} + +// Report process heartbeat info report +func (r *MemoryHealthChecker) Report(request *plugin.ReportRequest) error { + record := HeartbeatRecord{ + Server: request.LocalHost, + CurTimeSec: request.CurTimeSec, + } + r.hbRecords.Store(request.InstanceId, record) + return nil +} + +// Query query the heartbeat time +func (r *MemoryHealthChecker) Query(request *plugin.QueryRequest) (*plugin.QueryResponse, error) { + recordValue, ok := r.hbRecords.Load(request.InstanceId) + if !ok { + return &plugin.QueryResponse{ + LastHeartbeatSec: 0, + }, nil + } + record := recordValue.(*HeartbeatRecord) + return &plugin.QueryResponse{ + Server: record.Server, + LastHeartbeatSec: record.CurTimeSec, + }, nil +} + +// Report process the instance check +func (r *MemoryHealthChecker) Check(request *plugin.CheckRequest) (*plugin.CheckResponse, error) { + queryResp, err := r.Query(&request.QueryRequest) + if nil != err { + return nil, err + } + lastHeartbeatTime := queryResp.LastHeartbeatSec + checkResp := &plugin.CheckResponse{ + LastHeartbeatTimeSec: lastHeartbeatTime, + } + if request.CurTimeSec > lastHeartbeatTime { + if request.CurTimeSec-lastHeartbeatTime >= int64(request.ExpireDurationSec) { + //心跳超时 + checkResp.Healthy = false + return checkResp, nil + } + } + checkResp.Healthy = true + return checkResp, nil +} + +func init() { + d := &MemoryHealthChecker{} + plugin.RegisterPlugin(d.Name(), d) +} diff --git a/plugin/healthchecker/heartbeatredis/checker_redis.go b/plugin/healthchecker/heartbeatredis/checker_redis.go new file mode 100644 index 000000000..d67dd0c12 --- /dev/null +++ b/plugin/healthchecker/heartbeatredis/checker_redis.go @@ -0,0 +1,189 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package heartbeatredis + +import ( + "context" + "encoding/json" + "fmt" + "github.com/polarismesh/polaris-server/common/log" + "github.com/polarismesh/polaris-server/common/redispool" + "github.com/polarismesh/polaris-server/plugin" + "strconv" + "strings" + "time" +) + +// 把操作记录记录到日志文件中 +const ( + // PluginName plugin name + PluginName = "heartbeatRedis" + // Sep separator to divide id and timestamp + Sep = ":" +) + +// RedisHealthChecker +type RedisHealthChecker struct { + redisPool *redispool.Pool + cancel context.CancelFunc + respChan chan *redispool.Resp +} + +// Name +func (r *RedisHealthChecker) Name() string { + return PluginName +} + +func (r *RedisHealthChecker) processRedisResp(ctx context.Context) { + for { + select { + case <-ctx.Done(): + return + case resp := <-r.respChan: + if resp.Err != nil { + log.Errorf("[Health Check][RedisCheck]id:%s set redis err:%s", + resp.Value, resp.Err) + } + } + } +} + +// Initialize +func (r *RedisHealthChecker) Initialize(c *plugin.ConfigEntry) error { + redisBytes, err := json.Marshal(c.Option) + if nil != err { + return fmt.Errorf("fail to marshal %s config entry, err is %v", PluginName, err) + } + config := redispool.DefaultConfig() + if err = json.Unmarshal(redisBytes, config); nil != err { + return fmt.Errorf("fail to unmarshal %s config entry, err is %v", PluginName, err) + } + r.respChan = make(chan *redispool.Resp) + var ctx context.Context + ctx, r.cancel = context.WithCancel(context.Background()) + go r.processRedisResp(ctx) + r.redisPool = redispool.NewPool(ctx, config) + r.redisPool.Start() + return nil +} + +// Destroy +func (r *RedisHealthChecker) Destroy() error { + r.cancel() + return nil +} + +// Type type for health check plugin, only one same type plugin is allowed +func (r *RedisHealthChecker) Type() plugin.HealthCheckType { + return plugin.HealthCheckerHeartbeat +} + +// Report process heartbeat info report +func (r *RedisHealthChecker) Report(request *plugin.ReportRequest) error { + value := fmt.Sprintf("%d%s%s", request.CurTimeSec, Sep, request.LocalHost) + log.Debugf("[Health Check][RedisCheck]redis set key is %s, value is %s", request.InstanceId, value) + return r.redisPool.Set(request.InstanceId, value, r.respChan) +} + +// Query query the heartbeat time +func (r *RedisHealthChecker) Query(request *plugin.QueryRequest) (*plugin.QueryResponse, error) { + respCh := make(chan *redispool.Resp) + err := r.redisPool.Get(request.InstanceId, respCh) + if nil != err { + return nil, err + } + resp := <-respCh + if resp.Err != nil { + log.Errorf("[Health Check][RedisCheck]addr:%s:%d, id:%s, get redis err:%s", + request.Host, request.Port, request.InstanceId, resp.Err) + return nil, resp.Err + } + value := resp.Value + queryResp := &plugin.QueryResponse{ + Exists: resp.Exists, + } + if len(value) == 0 { + return queryResp, nil + } + tokens := strings.Split(value, Sep) + if len(tokens) != 2 { + log.Errorf("[Health Check][RedisCheck]addr:%s:%d, id:%s, invalid redis value:%s", + request.Host, request.Port, request.InstanceId, value) + return nil, fmt.Errorf("invalid redis value %s", value) + } + lastHeartbeatTimeStr := tokens[0] + lastHeartbeatTime, err := strconv.ParseInt(lastHeartbeatTimeStr, 10, 64) + if resp.Err != nil { + log.Errorf("[Health Check][RedisCheck]addr is %s:%d, id is %s, parse heartbeatTime %s err:%v", + request.Host, request.Port, request.InstanceId, lastHeartbeatTimeStr, err) + return nil, resp.Err + } + queryResp.Server = tokens[1] + queryResp.LastHeartbeatSec = lastHeartbeatTime + return queryResp, nil +} + +// Report process the instance check +func (r *RedisHealthChecker) Check(request *plugin.CheckRequest) (*plugin.CheckResponse, error) { + queryResp, err := r.Query(&request.QueryRequest) + if nil != err { + return nil, err + } + lastHeartbeatTime := queryResp.LastHeartbeatSec + checkResp := &plugin.CheckResponse{ + LastHeartbeatTimeSec: lastHeartbeatTime, + } + recoverTimeSec := r.redisPool.RecoverTimeSec() + localCurTimeSec := time.Now().Unix() + if localCurTimeSec >= recoverTimeSec && localCurTimeSec-recoverTimeSec < int64(request.ExpireDurationSec) { + checkResp.OnRecover = true + } + if request.CurTimeSec > lastHeartbeatTime { + if request.CurTimeSec-lastHeartbeatTime >= int64(request.ExpireDurationSec) { + //心跳超时 + checkResp.Healthy = false + if request.Healthy { + log.Infof("[Health Check][RedisCheck]health check expired, "+ + "last hb timestamp is %d, curTimeSec is %d, expireDurationSec is %d instanceId %s", + lastHeartbeatTime, request.CurTimeSec, request.ExpireDurationSec, request.InstanceId) + } + if queryResp.Exists { + respCh := make(chan *redispool.Resp) + err = r.redisPool.Del(request.InstanceId, respCh) + if nil != err { + return nil, err + } + resp := <-respCh + if resp.Err != nil { + log.Errorf("[Health Check][RedisCheck]addr is %s:%d, id is %s, delete redis err is %s", + request.Host, request.Port, request.InstanceId, resp.Err) + return nil, resp.Err + } + } + return checkResp, nil + } + } + checkResp.Healthy = true + log.Debugf("[Health Check][RedisCheck]instanceId is %s, healthy is %v", request.InstanceId, checkResp.Healthy) + return checkResp, nil +} + +func init() { + d := &RedisHealthChecker{} + plugin.RegisterPlugin(d.Name(), d) +} diff --git a/plugin/parsePassword.go b/plugin/password.go similarity index 100% rename from plugin/parsePassword.go rename to plugin/password.go diff --git a/plugin/ratelimit/tokenbucket/api_limit.go b/plugin/ratelimit/token/api_limit.go similarity index 99% rename from plugin/ratelimit/tokenbucket/api_limit.go rename to plugin/ratelimit/token/api_limit.go index 943885bbf..62e6f78df 100644 --- a/plugin/ratelimit/tokenbucket/api_limit.go +++ b/plugin/ratelimit/token/api_limit.go @@ -15,7 +15,7 @@ * specific language governing permissions and limitations under the License. */ -package tokenbucket +package token import ( "errors" diff --git a/plugin/ratelimit/tokenbucket/api_limit_test.go b/plugin/ratelimit/token/api_limit_test.go similarity index 99% rename from plugin/ratelimit/tokenbucket/api_limit_test.go rename to plugin/ratelimit/token/api_limit_test.go index 8d50756bd..8165ecd47 100644 --- a/plugin/ratelimit/tokenbucket/api_limit_test.go +++ b/plugin/ratelimit/token/api_limit_test.go @@ -15,7 +15,7 @@ * specific language governing permissions and limitations under the License. */ -package tokenbucket +package token import ( . "github.com/smartystreets/goconvey/convey" diff --git a/plugin/ratelimit/tokenbucket/config.go b/plugin/ratelimit/token/config.go similarity index 99% rename from plugin/ratelimit/tokenbucket/config.go rename to plugin/ratelimit/token/config.go index 82603e7a5..a663b60e4 100644 --- a/plugin/ratelimit/tokenbucket/config.go +++ b/plugin/ratelimit/token/config.go @@ -15,7 +15,7 @@ * specific language governing permissions and limitations under the License. */ -package tokenbucket +package token import ( "fmt" diff --git a/plugin/ratelimit/tokenbucket/implement.go b/plugin/ratelimit/token/implement.go similarity index 98% rename from plugin/ratelimit/tokenbucket/implement.go rename to plugin/ratelimit/token/implement.go index cabb2bd6f..b5cfefb27 100644 --- a/plugin/ratelimit/tokenbucket/implement.go +++ b/plugin/ratelimit/token/implement.go @@ -15,7 +15,7 @@ * specific language governing permissions and limitations under the License. */ -package tokenbucket +package token import ( "github.com/polarismesh/polaris-server/common/log" diff --git a/plugin/ratelimit/tokenbucket/invoke.go b/plugin/ratelimit/token/invoke.go similarity index 98% rename from plugin/ratelimit/tokenbucket/invoke.go rename to plugin/ratelimit/token/invoke.go index 2ce6ca630..c67617af9 100644 --- a/plugin/ratelimit/tokenbucket/invoke.go +++ b/plugin/ratelimit/token/invoke.go @@ -15,7 +15,7 @@ * specific language governing permissions and limitations under the License. */ -package tokenbucket +package token import ( "github.com/polarismesh/polaris-server/plugin" diff --git a/plugin/ratelimit/tokenbucket/invoke_test.go b/plugin/ratelimit/token/invoke_test.go similarity index 99% rename from plugin/ratelimit/tokenbucket/invoke_test.go rename to plugin/ratelimit/token/invoke_test.go index df2e8c233..879af9023 100644 --- a/plugin/ratelimit/tokenbucket/invoke_test.go +++ b/plugin/ratelimit/token/invoke_test.go @@ -15,7 +15,7 @@ * specific language governing permissions and limitations under the License. */ -package tokenbucket +package token import ( "github.com/polarismesh/polaris-server/plugin" diff --git a/plugin/ratelimit/tokenbucket/limiter.go b/plugin/ratelimit/token/limiter.go similarity index 97% rename from plugin/ratelimit/tokenbucket/limiter.go rename to plugin/ratelimit/token/limiter.go index 5140b22b3..114241990 100644 --- a/plugin/ratelimit/tokenbucket/limiter.go +++ b/plugin/ratelimit/token/limiter.go @@ -15,7 +15,7 @@ * specific language governing permissions and limitations under the License. */ -package tokenbucket +package token // 限制器 type limiter interface { diff --git a/plugin/ratelimit/tokenbucket/register.go b/plugin/ratelimit/token/register.go similarity index 97% rename from plugin/ratelimit/tokenbucket/register.go rename to plugin/ratelimit/token/register.go index 6f11afc62..614ae00bb 100644 --- a/plugin/ratelimit/tokenbucket/register.go +++ b/plugin/ratelimit/token/register.go @@ -15,7 +15,7 @@ * specific language governing permissions and limitations under the License. */ -package tokenbucket +package token import "github.com/polarismesh/polaris-server/plugin" diff --git a/plugin/ratelimit/tokenbucket/resource_limiter.go b/plugin/ratelimit/token/resource_limiter.go similarity index 99% rename from plugin/ratelimit/tokenbucket/resource_limiter.go rename to plugin/ratelimit/token/resource_limiter.go index 93644ef73..888cf230c 100644 --- a/plugin/ratelimit/tokenbucket/resource_limiter.go +++ b/plugin/ratelimit/token/resource_limiter.go @@ -15,7 +15,7 @@ * specific language governing permissions and limitations under the License. */ -package tokenbucket +package token import ( "fmt" diff --git a/plugin/ratelimit/tokenbucket/resource_limiter_test.go b/plugin/ratelimit/token/resource_limiter_test.go similarity index 99% rename from plugin/ratelimit/tokenbucket/resource_limiter_test.go rename to plugin/ratelimit/token/resource_limiter_test.go index aab4d33a2..38652f686 100644 --- a/plugin/ratelimit/tokenbucket/resource_limiter_test.go +++ b/plugin/ratelimit/token/resource_limiter_test.go @@ -15,7 +15,7 @@ * specific language governing permissions and limitations under the License. */ -package tokenbucket +package token import ( "fmt" diff --git a/polaris-server.yaml b/polaris-server.yaml index 5a140e0dd..9998b5942 100644 --- a/polaris-server.yaml +++ b/polaris-server.yaml @@ -79,20 +79,8 @@ apiservers: # clusterName: cl5.discover # 核心逻辑的配置 naming: - # 鉴权配置 auth: - # 是否开启鉴权 open: false - # 健康检查 - healthcheck: - open: true -# kvConnNum: 50 -# kvServiceName: polaris.redis -# kvNamespace: Polaris -# kvPasswd: ##REDIS_PWD## - slotNum: 30 - maxIdle: 20 - idleTimeout: 120 # 批量控制器 batch: register: @@ -107,6 +95,23 @@ naming: waitTime: 32ms maxBatchCount: 32 concurrency: 64 +# 健康检查的配置 +healthcheck: + open: true + service: polaris.checker + slotNum: 30 + checkers: + - name: heartbeatMemory +# - name: heartbeatRedis +# option: +# kvAddr: ##REDIS_ADDR## +# kvPasswd: ##REDIS_PWD## +# slotNum: 30 +# maxIdle: 200 +# idleTimeout: 120s +# connectTimeout: 20s +# msgTimeout: 5s +# concurrency: 200 # 缓存配置 cache: open: true @@ -175,8 +180,8 @@ plugin: instance-limit: open: true global: - bucket: 2 - rate: 2 + bucket: 200 + rate: 100 resource-cache-amount: 1024 api-limit: # 接口级限流 open: false # 是否开启接口限流,全局开关,只有为true,才代表系统的限流开启。默认关闭 diff --git a/store/api.go b/store/api.go index 9988c81af..5de8c15b3 100644 --- a/store/api.go +++ b/store/api.go @@ -63,6 +63,9 @@ type Store interface { // PlatformStore 平台信息接口 PlatformStore + + // ToolStore 函数及工具接口 + ToolStore } // NamespaceStore 命名空间存储接口 @@ -371,3 +374,9 @@ type Transaction interface { // RLockService 共享锁service RLockService(name string, namespace string) (*model.Service, error) } + +// ToolStore 存储相关的函数及工具接口 +type ToolStore interface { + // GetNow 获取当前时间 + GetNow() (int64, error) +} diff --git a/store/boltdb/default.go b/store/boltdb/default.go index 245419715..5ca9d82f1 100644 --- a/store/boltdb/default.go +++ b/store/boltdb/default.go @@ -36,6 +36,7 @@ type boltStore struct { *rateLimitStore *platformStore *circuitBreakerStore + *toolStore handler BoltHandler start bool diff --git a/store/boltdb/tool.go b/store/boltdb/tool.go new file mode 100644 index 000000000..8efebee06 --- /dev/null +++ b/store/boltdb/tool.go @@ -0,0 +1,29 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package boltdb + +import "time" + +// toolStore 实现了ToolStoreStore +type toolStore struct { +} + +// GetNow 获取当前时间,单位秒 +func (t *toolStore) GetNow() (int64, error) { + return time.Now().Unix(), nil +} diff --git a/store/mock/api_mock.go b/store/mock/api_mock.go index 4ff52c1c0..2c37f418b 100644 --- a/store/mock/api_mock.go +++ b/store/mock/api_mock.go @@ -5,9 +5,9 @@ package mock import ( + gomock "github.com/golang/mock/gomock" model "github.com/polarismesh/polaris-server/common/model" store "github.com/polarismesh/polaris-server/store" - gomock "github.com/golang/mock/gomock" reflect "reflect" time "time" ) @@ -114,6 +114,11 @@ func (m *MockStore) UpdateNamespace(namespace *model.Namespace) error { return ret0 } +// GetNow ȡǰʱ +func (m *MockStore) GetNow() (int64, error) { + return time.Now().Unix(), nil +} + // UpdateNamespace indicates an expected call of UpdateNamespace func (mr *MockStoreMockRecorder) UpdateNamespace(namespace interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() diff --git a/store/sqldb/default.go b/store/sqldb/default.go index 425a0e322..1d494f431 100644 --- a/store/sqldb/default.go +++ b/store/sqldb/default.go @@ -58,6 +58,7 @@ type stableStore struct { *rateLimitStore *circuitBreakerStore *platformStore + *toolStore // 主数据库,可以进行读写 master *BaseDB @@ -222,6 +223,7 @@ func (s *stableStore) CreateTransaction() (store.Transaction, error) { // 初始化子类 func (s *stableStore) newStore() { s.namespaceStore = &namespaceStore{db: s.master} + s.businessStore = &businessStore{db: s.master} s.serviceStore = &serviceStore{master: s.master, slave: s.slave} @@ -237,6 +239,8 @@ func (s *stableStore) newStore() { s.circuitBreakerStore = &circuitBreakerStore{master: s.master, slave: s.slave} s.platformStore = &platformStore{master: s.master} + + s.toolStore = &toolStore{db: s.master} } // time.Time转为字符串时间 diff --git a/store/sqldb/polaris_server.sql b/store/sqldb/polaris_server.sql index 00afaf3d6..b720f35f1 100644 --- a/store/sqldb/polaris_server.sql +++ b/store/sqldb/polaris_server.sql @@ -224,9 +224,7 @@ CREATE TABLE `service` ( -- INSERT INTO `service` (`id`, `name`, `namespace`, `comment`, `business`, `token`, `revision`, `owner`, `flag`, `ctime`, `mtime`) VALUES -('5d0e7ed4c771483a88541529333127c0', 'polaris.discover', 'Polaris', 'polaris discover service', 'polaris', '40c3acb5dbc6472982fbe91b011eddc0', '1c9b5059adf04f709370a827d3fa5290', 'polaris', 0, '2021-09-06 07:55:07', '2021-09-06 07:55:07'), -('fe7cac0a2fb84fce85940125c9c5a32f', 'polaris.healthcheck', 'Polaris', 'polaris healthcheck service', 'polaris', '40c3acb5dbc6472982fbe91b011eddc0', '3fa6ef85feba48a48c6acc091a1d8e9e', 'polaris', 0, '2021-09-06 07:55:07', '2021-09-06 07:55:09'), -('3987abdde91e4f19b482b40a2c5e66bd', 'polaris.redis', 'Polaris', 'polaris redis service', 'polaris', '1d622250c8cd49e996522bdf16b4eaf8', 'e0d3293b5f3749818358033f1c879a62', 'polaris', 0, '2021-09-06 07:55:07', '2021-09-06 07:55:10'), +('fbca9bfa04ae4ead86e1ecf5811e32a9', 'polaris.checker', 'Polaris', 'polaris checker service', 'polaris', '7d19c46de327408d8709ee7392b7700b', '301b1e9f0bbd47a6b697e26e99dfe012', 'polaris', 0, '2021-09-06 07:55:07', '2021-09-06 07:55:09'), ('bbfdda174ea64e11ac862adf14593c03', 'polaris.monitor', 'Polaris', 'polaris monitor service', 'polaris', '50b4e7d8affa4634b52523d398d1a369', '3649b17283d94d7baee5fb5d8160a225', 'polaris', 0, '2021-09-06 07:55:07', '2021-09-06 07:55:11'); -- -------------------------------------------------------- diff --git a/store/sqldb/tool.go b/store/sqldb/tool.go new file mode 100644 index 000000000..88962ef3f --- /dev/null +++ b/store/sqldb/tool.go @@ -0,0 +1,45 @@ +/** + * Tencent is pleased to support the open source community by making Polaris available. + * + * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. + * + * Licensed under the BSD 3-Clause License (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://opensource.org/licenses/BSD-3-Clause + * + * Unless required by applicable law or agreed to in writing, software distributed + * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR + * CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package sqldb + +import "github.com/polarismesh/polaris-server/common/log" + +// toolStore 实现了ToolStoreStore +type toolStore struct { + db *BaseDB +} + +const nowSql = `select UNIX_TIMESTAMP(SYSDATE())` + +// GetNow 获取当前时间,单位秒 +func (t *toolStore) GetNow() (int64, error) { + rows, err := t.db.Query(nowSql) + if err != nil { + log.Errorf("[Store][database] query now err: %s", err.Error()) + return 0, err + } + defer rows.Close() + var value int64 + for rows.Next() { + if err := rows.Scan(&value); err != nil { + log.Errorf("[Store][database] get now rows scan err: %s", err.Error()) + return 0, err + } + } + return value, nil +} diff --git a/version b/version index cc904638a..8b3a0227b 100644 --- a/version +++ b/version @@ -1 +1 @@ -v1.2.2 +v1.3.0 \ No newline at end of file From 9bc9000275e5d9391111768a9f9b6cb529c4de7e Mon Sep 17 00:00:00 2001 From: andrewshan Date: Fri, 29 Oct 2021 15:01:28 +0800 Subject: [PATCH 2/5] feat: remove useless test --- naming/test/healthcheck_test.go | 636 -------------------------------- 1 file changed, 636 deletions(-) delete mode 100644 naming/test/healthcheck_test.go diff --git a/naming/test/healthcheck_test.go b/naming/test/healthcheck_test.go deleted file mode 100644 index 475cdddfb..000000000 --- a/naming/test/healthcheck_test.go +++ /dev/null @@ -1,636 +0,0 @@ -/** - * Tencent is pleased to support the open source community by making Polaris available. - * - * Copyright (C) 2019 THL A29 Limited, a Tencent company. All rights reserved. - * - * Licensed under the BSD 3-Clause License (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * https://opensource.org/licenses/BSD-3-Clause - * - * Unless required by applicable law or agreed to in writing, software distributed - * under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR - * CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package test - -import ( - "context" - "fmt" - "github.com/polarismesh/polaris-server/common/model" - "github.com/polarismesh/polaris-server/naming" - "math/rand" - "strconv" - "sync" - "testing" - "time" - - "github.com/polarismesh/polaris-server/common/utils" - - "github.com/polarismesh/polaris-server/naming/cache" - - api "github.com/polarismesh/polaris-server/common/api/v1" - - // 使用mysql库 - - _ "github.com/go-sql-driver/mysql" - "github.com/golang/protobuf/ptypes/wrappers" - _ "github.com/polarismesh/polaris-server/store/sqldb" - "github.com/stretchr/testify/assert" -) - -// 测试样例结构体 -type Case struct { - field string - req *api.Instance - expect uint32 -} - -const ( - timeoutTimes = 2 -) - -// 测试健康检查功能未开启的情况 -func TestHealthCheckNotOpen(t *testing.T) { - _, serviceResp := createCommonService(t, 131) - defer cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - _, req := createInstanceNotOpenHealthCheck(t, serviceResp, 131) - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - defer cleanInstance(req.GetId().GetValue()) - - var heartbeatOnDisabledIns uint32 = 400141 - wait4Cache() - rsp := server.Heartbeat(context.Background(), req) - fmt.Printf("actiual:%v", rsp.GetCode().GetValue()) - assert.EqualValues(t, heartbeatOnDisabledIns, rsp.GetCode().GetValue()) -} - -// 测试错误输入的情况 -func TestWrongInput(t *testing.T) { - // 输入样例 - cases := []Case{ - {"req", nil, api.EmptyRequest}, - {"service_token", &api.Instance{}, api.InvalidServiceToken}, - {"service", &api.Instance{ - Namespace: &wrappers.StringValue{Value: "n"}, - Host: &wrappers.StringValue{Value: "h"}, - Port: &wrappers.UInt32Value{Value: 1}, - HealthCheck: &api.HealthCheck{ - Heartbeat: &api.HeartbeatHealthCheck{Ttl: &wrappers.UInt32Value{Value: 1}}, - }, - ServiceToken: &wrappers.StringValue{Value: "t"}, - }, api.InvalidServiceName}, - {"namespace", &api.Instance{ - Service: &wrappers.StringValue{Value: "s"}, - Host: &wrappers.StringValue{Value: "h"}, - Port: &wrappers.UInt32Value{Value: 1}, - HealthCheck: &api.HealthCheck{ - Heartbeat: &api.HeartbeatHealthCheck{Ttl: &wrappers.UInt32Value{Value: 1}}, - }, - ServiceToken: &wrappers.StringValue{Value: "t"}, - }, api.InvalidNamespaceName}, - {"host", &api.Instance{ - Service: &wrappers.StringValue{Value: "s"}, - Namespace: &wrappers.StringValue{Value: "n"}, - Port: &wrappers.UInt32Value{Value: 1}, - HealthCheck: &api.HealthCheck{ - Heartbeat: &api.HeartbeatHealthCheck{Ttl: &wrappers.UInt32Value{Value: 1}}, - }, - ServiceToken: &wrappers.StringValue{Value: "t"}, - }, api.InvalidInstanceHost}, - {"port", &api.Instance{ - Service: &wrappers.StringValue{Value: "s"}, - Namespace: &wrappers.StringValue{Value: "n"}, - Host: &wrappers.StringValue{Value: "h"}, - HealthCheck: &api.HealthCheck{ - Heartbeat: &api.HeartbeatHealthCheck{Ttl: &wrappers.UInt32Value{Value: 1}}, - }, - ServiceToken: &wrappers.StringValue{Value: "t"}, - }, api.InvalidInstancePort}, - } - - for _, c := range cases { - func(c Case) { - t.Run(fmt.Sprintf("测试输入缺少%v的情况", c.field), func(t *testing.T) { - t.Parallel() - rsp := server.Heartbeat(context.Background(), c.req) - assert.EqualValues(t, c.expect, rsp.GetCode().GetValue()) - }) - }(c) - } - - t.Run("测试传入非法token的情况", func(t *testing.T) { - t.Parallel() - var ( - req *api.Instance - rsp *api.Response - index = 1006 - ) - _, serviceResp := createCommonService(t, index) - defer cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - _, req = createCommonInstance(t, serviceResp, index) - defer cleanInstance(req.GetId().GetValue()) - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - wait4Cache() - req.ServiceToken = &wrappers.StringValue{Value: "err token"} - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.Unauthorized, rsp.GetCode().GetValue()) - }) -} - -// 测试输入正确的情况 -func TestHealthCheckInputRight(t *testing.T) { - t.Run("测试输入正确,使用id为key的情况", func(t *testing.T) { - var ( - req *api.Instance - rsp *api.Response - index = 15 - ) - _, serviceResp := createCommonService(t, index) - defer cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - _, req = createCommonInstance(t, serviceResp, index) - defer cleanInstance(req.GetId().GetValue()) - wait4Cache() - req.Service = nil - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.ExecuteSuccess, rsp.GetCode().GetValue()) - assert.True(t, getHealthStatus(req.GetHost().GetValue(), int(req.GetPort().GetValue()))) - }) - - t.Run("测试输入正确,使用四元组为key的情况", func(t *testing.T) { - var ( - req *api.Instance - rsp *api.Response - index = 16 - ) - _, serviceResp := createCommonService(t, index) - defer cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - _, req = createCommonInstance(t, serviceResp, index) - defer cleanInstance(req.GetId().GetValue()) - wait4Cache() - req.Id = nil - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.ExecuteSuccess, rsp.GetCode().GetValue()) - assert.True(t, getHealthStatus(req.GetHost().GetValue(), int(req.GetPort().GetValue()))) - }) -} - -// 测试健康状态的变化 -func TestTurnUnhealthy(t *testing.T) { - t.Run("测试从健康变为不健康", turnUnhealthy) -} - -// 测试ttl增加 -func TestTtlIncrease(t *testing.T) { - t.Run("测试ttl增加", ttlIncrease) -} - -// 测试ttl减少 -func TestTtlDecrease(t *testing.T) { - t.Run("测试ttl减少", ttlDecrease) -} - -// 从健康转变为不健康 -func turnUnhealthy(t *testing.T) { - //t.Parallel() - var ( - req *api.Instance - rsp *api.Response - ttl uint32 = 1 - index = 6001 - ) - _, serviceResp := createCommonService(t, index) - defer cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - _, req = createCommonInstance(t, serviceResp, index) - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - defer cleanInstance(req.GetId().GetValue()) - updateTTL(t, req, ttl) - wait4Cache() - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.ExecuteSuccess, rsp.GetCode().GetValue()) - time.Sleep(time.Duration(ttl) * time.Second) - assert.True(t, getHealthStatus(req.GetHost().GetValue(), int(req.GetPort().GetValue()))) - - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.ExecuteSuccess, rsp.GetCode().GetValue()) - time.Sleep(time.Duration(ttl) * time.Second) - assert.True(t, getHealthStatus(req.GetHost().GetValue(), int(req.GetPort().GetValue()))) - - // 停止发送心跳,等到超时时间到了后,该实例变为不健康 - time.Sleep(time.Duration((timeoutTimes+1)*ttl+3) * time.Second) - assert.False(t, getHealthStatus(req.GetHost().GetValue(), int(req.GetPort().GetValue()))) -} - -// 多次心跳,ttl增加 -func ttlIncrease(t *testing.T) { - //t.Parallel() - var ( - req *api.Instance - rsp *api.Response - ttl uint32 = 1 - anotherTTL uint32 = 3 - index = 50002 - ) - - _, serviceResp := createCommonService(t, index) - defer cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - _, req = createCommonInstance(t, serviceResp, index) - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - defer cleanInstance(req.GetId().GetValue()) - updateTTL(t, req, ttl) - wait4Cache() - - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.ExecuteSuccess, rsp.GetCode().GetValue()) - // 更新ttl - updateTTL(t, req, anotherTTL) - - time.Sleep(time.Second) - rsp = server.Heartbeat(context.Background(), req) - if rsp.GetCode().GetValue() != api.ExecuteSuccess { - t.Errorf("heartBeat err:%s", rsp.GetInfo().GetValue()) - } - assert.EqualValues(t, api.ExecuteSuccess, rsp.GetCode().GetValue()) - - timeoutSec := timeoutTimes*anotherTTL + 1 - oldTimeoutSec := timeoutTimes*ttl + 1 - // 确保旧超时时间后,按照新的ttl来计算还未超时 - assert.Greater(t, timeoutSec, oldTimeoutSec) - - // 等待旧超时时间过去,此时实例应该还未超时 - time.Sleep(time.Duration(oldTimeoutSec) * time.Second) - assert.True(t, getHealthStatus(req.GetHost().GetValue(), int(req.GetPort().GetValue()))) - - // 再等待达到新超时时间,此时实例应该超时 - time.Sleep(time.Duration(timeoutSec+5) * time.Second) - assert.False(t, getHealthStatus(req.GetHost().GetValue(), int(req.GetPort().GetValue()))) -} - -// 多次心跳,ttl减少 -func ttlDecrease(t *testing.T) { - //t.Parallel() - var ( - req *api.Instance - // instance *api.Instance - rsp *api.Response - ttl uint32 = 3 - anotherTTL uint32 = 1 - index = 50003 - ) - _, serviceResp := createCommonService(t, index) - defer cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - _, req = createCommonInstance(t, serviceResp, index) - defer cleanInstance(req.GetId().GetValue()) - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - updateTTL(t, req, ttl) - wait4Cache() - - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.ExecuteSuccess, rsp.GetCode().GetValue()) - - // 更新ttl - updateTTL(t, req, anotherTTL) - wait4Cache() - - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.ExecuteSuccess, rsp.GetCode().GetValue()) - - timeoutSec := timeoutTimes*anotherTTL + 2 - oldTimeoutSec := timeoutTimes * ttl - // 确保超时时间后,按照旧的ttl来计算还未超时 - assert.Less(t, timeoutSec, oldTimeoutSec) - - // 等待旧超时时间过去,此时实例应该就已经超时 - time.Sleep(time.Duration(oldTimeoutSec) * time.Second) - assert.False(t, getHealthStatus(req.GetHost().GetValue(), int(req.GetPort().GetValue()))) -} - -// 获取实例健康状态 -func getHealthStatus(host string, port int) bool { - query := map[string]string{"limit": "20", "host": host, "port": strconv.Itoa(port)} - rsp := server.GetInstances(query) - if !respSuccess(rsp) { - panic("寻找实例失败") - } - instances := rsp.GetInstances() - if len(instances) != 1 { - panic(fmt.Sprintf("找到的实例不唯一,已找到%v个实例", len(instances))) - } - return instances[0].GetHealthy().GetValue() -} - -// 获取实例健康状态 -func getHealthStatusByID(id string) bool { - ins := server.Cache().Instance().GetInstance(id) - if ins == nil { - panic("寻找实例失败") - } - - return ins.Proto.Healthy.Value -} - -// 等待cache加载数据,在创建实例后需要使用 -func wait4Cache() { - time.Sleep(2 * cache.UpdateCacheInterval) -} - -// 更新实例的ttl -func updateTTL(t *testing.T, instance *api.Instance, ttl uint32) { - instance.HealthCheck = &api.HealthCheck{Heartbeat: &api.HeartbeatHealthCheck{Ttl: utils.NewUInt32Value(ttl)}} - if resp := server.UpdateInstance(defaultCtx, instance); !respSuccess(resp) { - t.Fatalf("error: %s", resp.GetInfo().GetValue()) - } -} - -// 获取实例的ttl -func getTTL(t *testing.T, id string) uint32 { - insCache := server.Cache().Instance().GetInstance(id) - if insCache == nil { - return 0 - } - return insCache.HealthCheck().Heartbeat.Ttl.Value -} - -// 测试存在不合法的实例的情况 -func TestInvalidHealthInstance(t *testing.T) { - t.Run("测试不存在实例的情况", func(t *testing.T) { - t.Parallel() - var ( - req *api.Instance - rsp *api.Response - index = 1004 - ) - _, serviceResp := createCommonService(t, index) - defer cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - _, req = createCommonInstance(t, serviceResp, index) - // 创建一个实例,然后将其删除 - cleanInstance(req.GetId().GetValue()) - wait4Cache() - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.NotFoundResource, rsp.GetCode().GetValue()) - }) - - t.Run("测试不存在service的情况", func(t *testing.T) { - t.Parallel() - var ( - req *api.Instance - rsp *api.Response - index = 1007 - ) - _, serviceResp := createCommonService(t, index) - _, req = createCommonInstance(t, serviceResp, index) - defer cleanInstance(req.GetId().GetValue()) - // 删除服务 - cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - wait4Cache() - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.NotFoundResource, rsp.GetCode().GetValue()) - }) - - t.Run("测试存在ttl非法的实例的情况", func(t *testing.T) { - t.Parallel() - var ( - req *api.Instance - rsp *api.Response - index = 1005 - ) - _, serviceResp := createCommonService(t, index) - defer cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - _, req = createCommonInstance(t, serviceResp, index) - defer cleanInstance(req.GetId().GetValue()) - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - updateTTL(t, req, 123123) - wait4Cache() - - ttl := getTTL(t, req.GetId().GetValue()) - assert.EqualValues(t, ttl, 5) - - rsp = server.Heartbeat(context.Background(), req) - assert.EqualValues(t, api.ExecuteSuccess, rsp.GetCode().GetValue()) - }) -} - -// 测试大量heartbeat用时 -func TestHeartBeatUseTime(t *testing.T) { - t.Run("测试大量heartbeat用时", heartBeatUseTime) -} - -// 测试大量heartbeat用时 -func heartBeatUseTime(t *testing.T) { - heartBeatBatch(t, 10, 80) -} - -// 模拟正常多实例心跳上报情境 -func heartBeatBatch(t *testing.T, serviceNum, insNum int) { - var ( - req *api.Instance - index = 10000 - insArray = make([]*api.Instance, 0) - wg sync.WaitGroup - mu sync.Mutex - ) - - start := time.Now() - // 创建服务和实例 - for i := 0; i < serviceNum; i++ { - var wgt sync.WaitGroup - _, serviceResp := createCommonService(t, index+i) - defer cleanServiceName(serviceResp.GetName().GetValue(), serviceResp.GetNamespace().GetValue()) - for j := 0; j < insNum; j++ { - go func(index int, serviceResp *api.Service) { - wgt.Add(1) - defer wgt.Done() - _, req = createCommonInstance(t, serviceResp, index) - req.ServiceToken = &wrappers.StringValue{Value: serviceResp.GetToken().GetValue()} - - mu.Lock() - insArray = append(insArray, req) - mu.Unlock() - }(index+i+j, serviceResp) - //睡眠0.1毫秒,削峰 - time.Sleep(time.Microsecond * 100) - } - wgt.Wait() - time.Sleep(100 * time.Millisecond) - } - t.Logf("create use time:%+v", time.Now().Sub(start)) - time.Sleep(time.Second * 2) - - wg.Add(len(insArray)) - exceedNum := 0 - now := time.Now() - for _, ins := range insArray { - go func(ins *api.Instance) { - resp := server.Heartbeat(context.Background(), ins) - if resp.GetCode().GetValue() != api.ExecuteSuccess { - exceedNum++ - } - wg.Done() - }(ins) - time.Sleep(time.Microsecond) - } - wg.Wait() - t.Logf("first, use time:%v, exceedNum:%d", time.Now().Sub(now), exceedNum) - - time.Sleep(time.Second * 3) - wg.Add(len(insArray)) - exceedNum = 0 - now = time.Now() - for _, ins := range insArray { - go func(ins *api.Instance) { - resp := server.Heartbeat(context.Background(), ins) - if resp.GetCode().GetValue() != api.ExecuteSuccess { - exceedNum++ - } - wg.Done() - }(ins) - time.Sleep(time.Microsecond) - } - wg.Wait() - t.Logf("third, use time:%v, exceedNum:%d", time.Now().Sub(now), exceedNum) - t.Logf("len:%d", len(insArray)) - - for _, ins := range insArray { - assert.True(t, getHealthStatusByID(ins.GetId().GetValue())) - } - ttl := 5 - time.Sleep(time.Duration((timeoutTimes+2)*ttl+4) * time.Second) - wait4Cache() - time.Sleep(20 * time.Second) - for _, ins := range insArray { - assert.False(t, getHealthStatusByID(ins.GetId().GetValue())) - } - for _, ins := range insArray { - cleanInstance(ins.GetId().GetValue()) - } -} - -// 测试ckv+节点变更 -func TestCkvNodeChange(t *testing.T) { - t.Logf("第一次测试心跳") - heartBeatBatch(t, 8, 20) - - name := cfg.Naming.HealthCheck.KvServiceName - namespace := cfg.Naming.HealthCheck.KvNamespace - service := server.Cache().Service().GetServiceByName(name, namespace) - if service == nil { - t.Fatalf("cannot get service, name:%s, namespace:%s", name, namespace) - } - instances := server.Cache().Instance().GetInstancesByServiceID(service.ID) - if len(instances) == 0 { - t.Fatalf("cannot get instance, name:%s, namespace:%s", name, namespace) - } - t.Logf("len:%d, instaces:%+v", len(instances), instances[0]) - - _ = server.Cache().Clear() // 为了防止影响,每个函数需要把缓存的内容清空 - creq := &api.Instance{ - ServiceToken: utils.NewStringValue(service.Token), - Id: utils.NewStringValue(instances[0].ID()), - } - // 节点增加 - t.Logf("ckv节点增加") - addInstance(t, ins2Api(instances[0], service.Token, service.Name, service.Namespace)) - time.Sleep(30 * time.Second) - t.Logf("再次测试心跳") - heartBeatBatch(t, 10, 20) - - // 节点减少 - t.Logf("ckv节点减少") - resp := server.DeleteInstance(defaultCtx, creq) - if !respSuccess(resp) { - t.Fatalf("error: %s", resp.GetInfo().GetValue()) - } - time.Sleep(30 * time.Second) - t.Logf("再次测试心跳") - heartBeatBatch(t, 10, 20) - - t.Logf("ok") -} - -func ins2Api(ins *model.Instance, token, name, namespace string) *api.Instance { - rand.Seed(time.Now().UnixNano()) - return &api.Instance{ - ServiceToken: utils.NewStringValue(token), - Service: utils.NewStringValue(name), - Namespace: utils.NewStringValue(namespace), - VpcId: utils.NewStringValue(strconv.Itoa(rand.Intn(10000))), - Host: utils.NewStringValue(ins.Host()), - Port: utils.NewUInt32Value(ins.Port()), - Protocol: utils.NewStringValue(ins.Protocol()), - Version: utils.NewStringValue(ins.Version()), - Priority: utils.NewUInt32Value(ins.Priority()), - Weight: utils.NewUInt32Value(ins.Weight()), - HealthCheck: ins.HealthCheck(), - Healthy: utils.NewBoolValue(ins.Healthy()), - Isolate: utils.NewBoolValue(ins.Isolate()), - LogicSet: utils.NewStringValue(ins.LogicSet()), - Metadata: ins.Metadata(), - } -} - -// 新增一个不开启健康检查的实例 -func createInstanceNotOpenHealthCheck(t *testing.T, service *api.Service, id int) ( - *api.Instance, *api.Instance) { - instanceReq := &api.Instance{ - ServiceToken: utils.NewStringValue(service.GetToken().GetValue()), - Service: utils.NewStringValue(service.GetName().GetValue()), - Namespace: utils.NewStringValue(service.GetNamespace().GetValue()), - VpcId: utils.NewStringValue(fmt.Sprintf("vpcid-%d", id)), - Host: utils.NewStringValue(fmt.Sprintf("10.10.10.%d", id)), - Port: utils.NewUInt32Value(8000 + uint32(id)), - Protocol: utils.NewStringValue(fmt.Sprintf("protocol-%d", id)), - Version: utils.NewStringValue(fmt.Sprintf("version-%d", id)), - Priority: utils.NewUInt32Value(1 + uint32(id)%10), - Weight: utils.NewUInt32Value(1 + uint32(id)%1000), - HealthCheck: nil, - Healthy: utils.NewBoolValue(false), // 默认是非健康,因为打开了healthCheck - Isolate: utils.NewBoolValue(false), - LogicSet: utils.NewStringValue(fmt.Sprintf("logic-set-%d", id)), - Metadata: map[string]string{ - "internal-personal-xxx": fmt.Sprintf("internal-personal-xxx_%d", id), - "2my-meta": fmt.Sprintf("my-meta-%d", id), - "my-meta-a1": "1111", - "smy-xmeta-h2": "2222", - "my-1meta-o3": "2222", - "my-2meta-4c": "2222", - "my-3meta-d5": "2222", - "dmy-meta-6p": "2222", - "1my-pmeta-d7": "2222", - "my-dmeta-8c": "2222", - "my-xmeta-9p": "2222", - "other-meta-x": "xxx", - "other-meta-1": "xx11", - "amy-instance": "my-instance", - "very-long-key-data-xxxxxxxxx": "Y", - "very-long-key-data-uuuuuuuuu": "P", - }, - } - - resp := server.CreateInstance(defaultCtx, instanceReq) - if respSuccess(resp) { - return instanceReq, resp.GetInstance() - } - - if resp.GetCode().GetValue() != api.ExistedResource { - t.Fatalf("error: %s", resp.GetInfo().GetValue()) - } - - // repeated - InstanceID, _ := naming.CalculateInstanceID(instanceReq.GetNamespace().GetValue(), instanceReq.GetService().GetValue(), - instanceReq.GetVpcId().GetValue(), instanceReq.GetHost().GetValue(), instanceReq.GetPort().GetValue()) - cleanInstance(InstanceID) - t.Logf("repeatd create instance(%s)", InstanceID) - resp = server.CreateInstance(defaultCtx, instanceReq) - if !respSuccess(resp) { - t.Fatalf("error: %s", resp.GetInfo().GetValue()) - } - - return instanceReq, resp.GetInstance() -} From 393b2193b66ffde4f6417af97544d1927fa3c71b Mon Sep 17 00:00:00 2001 From: andrewshan Date: Fri, 29 Oct 2021 15:08:50 +0800 Subject: [PATCH 3/5] feat: fix compile test error --- naming/test/main_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/naming/test/main_test.go b/naming/test/main_test.go index bb3d5f6b6..8fc418a41 100644 --- a/naming/test/main_test.go +++ b/naming/test/main_test.go @@ -111,8 +111,7 @@ func initialize() error { } }() - naming.SetHealthCheckConfig(&cfg.Naming.HealthCheck) - if err := naming.Initialize(ctx, &cfg.Naming, &cfg.Cache); err != nil { + if err := naming.Initialize(ctx, &cfg.Naming, &cfg.Cache, nil); err != nil { panic(err) } From b5d2791c74025d2a52dee7d782b0fa4cf020d111 Mon Sep 17 00:00:00 2001 From: andrewshan Date: Fri, 29 Oct 2021 15:15:10 +0800 Subject: [PATCH 4/5] fix: compile error --- store/mock/api_mock.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store/mock/api_mock.go b/store/mock/api_mock.go index 2c37f418b..405827aac 100644 --- a/store/mock/api_mock.go +++ b/store/mock/api_mock.go @@ -114,7 +114,7 @@ func (m *MockStore) UpdateNamespace(namespace *model.Namespace) error { return ret0 } -// GetNow ȡǰʱ +// GetNow func (m *MockStore) GetNow() (int64, error) { return time.Now().Unix(), nil } From 4ed3cf6107dea263713354a244eef7db64233d02 Mon Sep 17 00:00:00 2001 From: andrewshan Date: Fri, 29 Oct 2021 15:18:50 +0800 Subject: [PATCH 5/5] feat: update go.mod --- go.mod | 1 - 1 file changed, 1 deletion(-) diff --git a/go.mod b/go.mod index 1fb600681..7d129374d 100644 --- a/go.mod +++ b/go.mod @@ -21,7 +21,6 @@ require ( github.com/smartystreets/goconvey v0.0.0-20190710185942-9d28bd7c0945 github.com/spf13/cobra v0.0.5 github.com/spf13/pflag v1.0.5 // indirect - github.com/stretchr/testify v1.5.1 go.uber.org/atomic v1.5.1 go.uber.org/multierr v1.4.0 // indirect go.uber.org/zap v1.14.0