2021-05-18 11:07:27 +00:00
|
|
|
package sessionutil
|
2021-05-18 03:39:21 +00:00
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"encoding/json"
|
|
|
|
"errors"
|
|
|
|
"fmt"
|
2021-05-21 11:28:52 +00:00
|
|
|
"path"
|
2021-05-18 03:39:21 +00:00
|
|
|
"strconv"
|
|
|
|
"time"
|
|
|
|
|
|
|
|
"github.com/coreos/etcd/mvcc/mvccpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/retry"
|
|
|
|
"go.etcd.io/etcd/clientv3"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
)
|
|
|
|
|
2021-05-24 01:44:49 +00:00
|
|
|
const DefaultServiceRoot = "/session/"
|
|
|
|
const DefaultIDKey = "id"
|
|
|
|
const DefaultRetryTimes = 30
|
|
|
|
const DefaultTTL = 10
|
2021-05-18 03:39:21 +00:00
|
|
|
|
|
|
|
// Session is a struct to store service's session, including ServerID, ServerName,
|
|
|
|
// Address.
|
|
|
|
// LeaseID will be assigned after registered in etcd.
|
|
|
|
type Session struct {
|
2021-05-21 11:28:52 +00:00
|
|
|
ctx context.Context
|
|
|
|
ServerID int64 `json:"ServerID,omitempty"`
|
|
|
|
ServerName string `json:"ServerName,omitempty"`
|
|
|
|
Address string `json:"Address,omitempty"`
|
|
|
|
Exclusive bool `json:"Exclusive,omitempty"`
|
2021-05-18 03:39:21 +00:00
|
|
|
|
2021-05-21 11:28:52 +00:00
|
|
|
etcdCli *clientv3.Client
|
|
|
|
leaseID clientv3.LeaseID
|
|
|
|
cancel context.CancelFunc
|
2021-05-19 10:36:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// NewSession is a helper to build Session object.LeaseID will be assigned after
|
|
|
|
// registeration.
|
2021-05-24 01:44:49 +00:00
|
|
|
func NewSession(ctx context.Context, etcdAddress []string) *Session {
|
2021-05-21 11:28:52 +00:00
|
|
|
ctx, cancel := context.WithCancel(ctx)
|
|
|
|
session := &Session{
|
2021-05-24 01:44:49 +00:00
|
|
|
ctx: ctx,
|
|
|
|
cancel: cancel,
|
2021-05-19 10:36:05 +00:00
|
|
|
}
|
|
|
|
|
2021-05-21 11:28:52 +00:00
|
|
|
connectEtcdFn := func() error {
|
|
|
|
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: etcdAddress, DialTimeout: 5 * time.Second})
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
session.etcdCli = etcdCli
|
2021-05-19 10:36:05 +00:00
|
|
|
return nil
|
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
err := retry.Retry(100000, time.Millisecond*200, connectEtcdFn)
|
|
|
|
if err != nil {
|
|
|
|
return nil
|
2021-05-19 10:36:05 +00:00
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
return session
|
2021-05-19 10:36:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Init will initialize base struct in the SessionManager, including getServerID,
|
|
|
|
// and process keepAliveResponse
|
2021-05-24 01:44:49 +00:00
|
|
|
func (s *Session) Init(serverName, address string, exclusive bool) {
|
|
|
|
s.ServerName = serverName
|
|
|
|
s.Address = address
|
|
|
|
s.Exclusive = exclusive
|
2021-05-21 11:28:52 +00:00
|
|
|
s.checkIDExist()
|
|
|
|
serverID, err := s.getServerID()
|
2021-05-19 10:36:05 +00:00
|
|
|
if err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
s.ServerID = serverID
|
|
|
|
ch, err := s.registerService()
|
2021-05-19 10:36:05 +00:00
|
|
|
if err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
s.processKeepAliveResponse(ch)
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// GetServerID gets id from etcd with key: metaRootPath + "/services/id"
|
|
|
|
// Each server get ServerID and add one to id.
|
2021-05-21 11:28:52 +00:00
|
|
|
func (s *Session) getServerID() (int64, error) {
|
2021-05-24 01:44:49 +00:00
|
|
|
return s.getServerIDWithKey(DefaultIDKey, DefaultRetryTimes)
|
2021-05-19 10:36:05 +00:00
|
|
|
}
|
|
|
|
|
2021-05-21 11:28:52 +00:00
|
|
|
func (s *Session) checkIDExist() {
|
|
|
|
s.etcdCli.Txn(s.ctx).If(
|
|
|
|
clientv3.Compare(
|
2021-05-24 01:44:49 +00:00
|
|
|
clientv3.Version(path.Join(DefaultServiceRoot, DefaultIDKey)),
|
2021-05-21 11:28:52 +00:00
|
|
|
"=",
|
|
|
|
0)).
|
2021-05-24 01:44:49 +00:00
|
|
|
Then(clientv3.OpPut(path.Join(DefaultServiceRoot, DefaultIDKey), "1")).Commit()
|
2021-05-21 11:28:52 +00:00
|
|
|
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|
|
|
|
|
2021-05-21 11:28:52 +00:00
|
|
|
func (s *Session) getServerIDWithKey(key string, retryTimes int) (int64, error) {
|
2021-05-19 10:36:05 +00:00
|
|
|
res := int64(0)
|
2021-05-18 03:39:21 +00:00
|
|
|
getServerIDWithKeyFn := func() error {
|
2021-05-24 01:44:49 +00:00
|
|
|
getResp, err := s.etcdCli.Get(s.ctx, path.Join(DefaultServiceRoot, key))
|
2021-05-18 03:39:21 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil
|
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
if getResp.Count <= 0 {
|
|
|
|
return fmt.Errorf("there is no value on key = %s", key)
|
|
|
|
}
|
|
|
|
value := string(getResp.Kvs[0].Value)
|
2021-05-18 03:39:21 +00:00
|
|
|
valueInt, err := strconv.ParseInt(value, 10, 64)
|
|
|
|
if err != nil {
|
|
|
|
log.Debug("session", zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
txnResp, err := s.etcdCli.Txn(s.ctx).If(
|
|
|
|
clientv3.Compare(
|
2021-05-24 01:44:49 +00:00
|
|
|
clientv3.Value(path.Join(DefaultServiceRoot, DefaultIDKey)),
|
2021-05-21 11:28:52 +00:00
|
|
|
"=",
|
|
|
|
value)).
|
2021-05-24 01:44:49 +00:00
|
|
|
Then(clientv3.OpPut(path.Join(DefaultServiceRoot, DefaultIDKey), strconv.FormatInt(valueInt+1, 10))).Commit()
|
2021-05-18 03:39:21 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
|
|
|
|
if !txnResp.Succeeded {
|
|
|
|
return fmt.Errorf("function CompareAndSwap error for compare is false for key: %s", key)
|
|
|
|
}
|
2021-05-18 03:39:21 +00:00
|
|
|
res = valueInt
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
err := retry.Retry(retryTimes, time.Millisecond*200, getServerIDWithKeyFn)
|
|
|
|
return res, err
|
|
|
|
}
|
|
|
|
|
|
|
|
// RegisterService registers the service to etcd so that other services
|
|
|
|
// can find that the service is online and issue subsequent operations
|
|
|
|
// RegisterService will save a key-value in etcd
|
|
|
|
// key: metaRootPath + "/services" + "/ServerName-ServerID"
|
|
|
|
// value: json format
|
|
|
|
// {
|
2021-05-18 11:07:27 +00:00
|
|
|
// "ServerID": "ServerID",
|
|
|
|
// "ServerName": "ServerName",
|
|
|
|
// "Address": "ip:port",
|
|
|
|
// "LeaseID": "LeaseID",
|
2021-05-18 03:39:21 +00:00
|
|
|
// }
|
|
|
|
// MetaRootPath is configurable in the config file.
|
2021-05-18 11:07:27 +00:00
|
|
|
// Exclusive means whether this service can exist two at the same time, if so,
|
|
|
|
// it is false. Otherwise, set it to true.
|
2021-05-21 11:28:52 +00:00
|
|
|
func (s *Session) registerService() (<-chan *clientv3.LeaseKeepAliveResponse, error) {
|
|
|
|
var ch <-chan *clientv3.LeaseKeepAliveResponse
|
|
|
|
registerFn := func() error {
|
2021-05-24 01:44:49 +00:00
|
|
|
resp, err := s.etcdCli.Grant(s.ctx, DefaultTTL)
|
2021-05-21 11:28:52 +00:00
|
|
|
if err != nil {
|
|
|
|
log.Error("register service", zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
s.leaseID = resp.ID
|
2021-05-18 03:39:21 +00:00
|
|
|
|
2021-05-21 11:28:52 +00:00
|
|
|
sessionJSON, err := json.Marshal(s)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-05-18 03:39:21 +00:00
|
|
|
|
2021-05-21 11:28:52 +00:00
|
|
|
key := s.ServerName
|
|
|
|
if !s.Exclusive {
|
|
|
|
key = key + "-" + strconv.FormatInt(s.ServerID, 10)
|
|
|
|
}
|
|
|
|
txnResp, err := s.etcdCli.Txn(s.ctx).If(
|
|
|
|
clientv3.Compare(
|
2021-05-24 01:44:49 +00:00
|
|
|
clientv3.Version(path.Join(DefaultServiceRoot, key)),
|
2021-05-21 11:28:52 +00:00
|
|
|
"=",
|
|
|
|
0)).
|
2021-05-24 01:44:49 +00:00
|
|
|
Then(clientv3.OpPut(path.Join(DefaultServiceRoot, key), string(sessionJSON), clientv3.WithLease(resp.ID))).Commit()
|
2021-05-21 11:28:52 +00:00
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
fmt.Printf("compare and swap error %s\n. maybe the key has registered", err)
|
|
|
|
return err
|
|
|
|
}
|
2021-05-18 03:39:21 +00:00
|
|
|
|
2021-05-21 11:28:52 +00:00
|
|
|
if !txnResp.Succeeded {
|
|
|
|
return fmt.Errorf("function CompareAndSwap error for compare is false for key: %s", key)
|
|
|
|
}
|
|
|
|
|
|
|
|
ch, err = s.etcdCli.KeepAlive(s.ctx, resp.ID)
|
|
|
|
if err != nil {
|
|
|
|
fmt.Printf("keep alive error %s\n", err)
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
2021-05-24 01:44:49 +00:00
|
|
|
err := retry.Retry(DefaultRetryTimes, time.Millisecond*200, registerFn)
|
2021-05-18 03:39:21 +00:00
|
|
|
if err != nil {
|
2021-05-21 11:28:52 +00:00
|
|
|
return ch, nil
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|
|
|
|
return ch, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// ProcessKeepAliveResponse processes the response of etcd keepAlive interface
|
|
|
|
// If keepAlive fails for unexpected error, it will send a signal to the channel.
|
2021-05-21 11:28:52 +00:00
|
|
|
func (s *Session) processKeepAliveResponse(ch <-chan *clientv3.LeaseKeepAliveResponse) (failChannel <-chan bool) {
|
|
|
|
failCh := make(chan bool)
|
2021-05-18 03:39:21 +00:00
|
|
|
go func() {
|
|
|
|
for {
|
|
|
|
select {
|
2021-05-21 11:28:52 +00:00
|
|
|
case <-s.ctx.Done():
|
2021-05-18 03:39:21 +00:00
|
|
|
log.Error("keep alive", zap.Error(errors.New("context done")))
|
|
|
|
return
|
|
|
|
case resp, ok := <-ch:
|
|
|
|
if !ok {
|
2021-05-21 11:28:52 +00:00
|
|
|
failCh <- true
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|
2021-05-19 10:36:05 +00:00
|
|
|
if resp == nil {
|
2021-05-21 11:28:52 +00:00
|
|
|
failCh <- true
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
2021-05-21 11:28:52 +00:00
|
|
|
return failCh
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|
|
|
|
|
2021-05-21 11:28:52 +00:00
|
|
|
// GetSessions will get all sessions registered in etcd.
|
|
|
|
func (s *Session) GetSessions(prefix string) (map[string]*Session, error) {
|
|
|
|
res := make(map[string]*Session)
|
2021-05-24 01:44:49 +00:00
|
|
|
key := path.Join(DefaultServiceRoot, prefix)
|
2021-05-21 11:28:52 +00:00
|
|
|
resp, err := s.etcdCli.Get(s.ctx, key, clientv3.WithPrefix(),
|
|
|
|
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
|
2021-05-18 03:39:21 +00:00
|
|
|
if err != nil {
|
2021-05-21 11:28:52 +00:00
|
|
|
return nil, err
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
for _, kv := range resp.Kvs {
|
2021-05-18 03:39:21 +00:00
|
|
|
session := &Session{}
|
2021-05-21 11:28:52 +00:00
|
|
|
err = json.Unmarshal([]byte(kv.Value), session)
|
2021-05-18 03:39:21 +00:00
|
|
|
if err != nil {
|
2021-05-21 11:28:52 +00:00
|
|
|
return nil, err
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
res[string(kv.Key)] = session
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
return res, nil
|
2021-05-19 10:36:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// WatchServices watch the service's up and down in etcd, and saves it into local
|
2021-05-21 11:28:52 +00:00
|
|
|
// sessions.
|
|
|
|
// If a server up, it will be add to addChannel.
|
|
|
|
// If a server is offline, it will be add to delChannel.
|
|
|
|
func (s *Session) WatchServices(prefix string) (addChannel <-chan *Session, delChannel <-chan *Session) {
|
2021-05-20 07:07:25 +00:00
|
|
|
addCh := make(chan *Session, 10)
|
|
|
|
delCh := make(chan *Session, 10)
|
2021-05-24 01:44:49 +00:00
|
|
|
rch := s.etcdCli.Watch(s.ctx, path.Join(DefaultServiceRoot, prefix), clientv3.WithPrefix(), clientv3.WithPrevKV())
|
2021-05-18 03:39:21 +00:00
|
|
|
go func() {
|
|
|
|
for {
|
|
|
|
select {
|
2021-05-21 11:28:52 +00:00
|
|
|
case <-s.ctx.Done():
|
2021-05-18 03:39:21 +00:00
|
|
|
return
|
|
|
|
case wresp, ok := <-rch:
|
|
|
|
if !ok {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
for _, ev := range wresp.Events {
|
|
|
|
switch ev.Type {
|
|
|
|
case mvccpb.PUT:
|
|
|
|
log.Debug("watch services",
|
2021-05-19 10:36:05 +00:00
|
|
|
zap.Any("add kv", ev.Kv))
|
|
|
|
session := &Session{}
|
|
|
|
err := json.Unmarshal([]byte(ev.Kv.Value), session)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("watch services", zap.Error(err))
|
|
|
|
continue
|
|
|
|
}
|
2021-05-20 07:07:25 +00:00
|
|
|
addCh <- session
|
2021-05-18 03:39:21 +00:00
|
|
|
case mvccpb.DELETE:
|
|
|
|
log.Debug("watch services",
|
2021-05-21 11:28:52 +00:00
|
|
|
zap.Any("delete kv", ev.PrevKv))
|
|
|
|
session := &Session{}
|
|
|
|
err := json.Unmarshal([]byte(ev.PrevKv.Value), session)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("watch services", zap.Error(err))
|
|
|
|
continue
|
2021-05-20 07:07:25 +00:00
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
delCh <- session
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
2021-05-20 07:07:25 +00:00
|
|
|
return addCh, delCh
|
2021-05-19 10:36:05 +00:00
|
|
|
}
|
|
|
|
|
2021-05-21 11:28:52 +00:00
|
|
|
func initEtcd(etcdAddress string) (*clientv3.Client, error) {
|
|
|
|
var etcdCli *clientv3.Client
|
2021-05-19 10:36:05 +00:00
|
|
|
connectEtcdFn := func() error {
|
2021-05-21 11:28:52 +00:00
|
|
|
etcd, err := clientv3.New(clientv3.Config{Endpoints: []string{etcdAddress}, DialTimeout: 5 * time.Second})
|
2021-05-19 10:36:05 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
etcdCli = etcd
|
2021-05-19 10:36:05 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
err := retry.Retry(100000, time.Millisecond*200, connectEtcdFn)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2021-05-21 11:28:52 +00:00
|
|
|
return etcdCli, nil
|
2021-05-18 03:39:21 +00:00
|
|
|
}
|