// Copyright 2018 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. package infosync import ( "context" "encoding/json" "fmt" "net/http" "os" "path" "strconv" "strings" "sync/atomic" "time" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/oracle" util2 "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/versioninfo" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3/concurrency" "go.uber.org/zap" ) const ( // ServerInformationPath store server information such as IP, port and so on. ServerInformationPath = "/tidb/server/info" // ServerMinStartTSPath store the server min start timestamp. ServerMinStartTSPath = "/tidb/server/minstartts" // TiFlashTableSyncProgressPath store the tiflash table replica sync progress. TiFlashTableSyncProgressPath = "/tiflash/table/sync" // keyOpDefaultRetryCnt is the default retry count for etcd store. keyOpDefaultRetryCnt = 5 // keyOpDefaultTimeout is the default time out for etcd store. keyOpDefaultTimeout = 1 * time.Second // InfoSessionTTL is the ETCD session's TTL in seconds. InfoSessionTTL = 10 * 60 // ReportInterval is interval of infoSyncerKeeper reporting min startTS. ReportInterval = 30 * time.Second // TopologyInformationPath means etcd path for storing topology info. TopologyInformationPath = "/topology/tidb" // TopologySessionTTL is ttl for topology, ant it's the ETCD session's TTL in seconds. TopologySessionTTL = 45 // TopologyTimeToRefresh means time to refresh etcd. TopologyTimeToRefresh = 30 * time.Second // TopologyPrometheus means address of prometheus. TopologyPrometheus = "/topology/prometheus" // TablePrometheusCacheExpiry is the expiry time for prometheus address cache. TablePrometheusCacheExpiry = 10 * time.Second ) // ErrPrometheusAddrIsNotSet is the error that Prometheus address is not set in PD and etcd var ErrPrometheusAddrIsNotSet = dbterror.ClassDomain.NewStd(errno.ErrPrometheusAddrIsNotSet) // InfoSyncer stores server info to etcd when the tidb-server starts and delete when tidb-server shuts down. type InfoSyncer struct { etcdCli *clientv3.Client info *ServerInfo serverInfoPath string minStartTS uint64 minStartTSPath string manager util2.SessionManager session *concurrency.Session topologySession *concurrency.Session prometheusAddr string modifyTime time.Time } // ServerInfo is server static information. // It will not be updated when tidb-server running. So please only put static information in ServerInfo struct. type ServerInfo struct { ServerVersionInfo ID string `json:"ddl_id"` IP string `json:"ip"` Port uint `json:"listening_port"` StatusPort uint `json:"status_port"` Lease string `json:"lease"` BinlogStatus string `json:"binlog_status"` StartTimestamp int64 `json:"start_timestamp"` } // ServerVersionInfo is the server version and git_hash. type ServerVersionInfo struct { Version string `json:"version"` GitHash string `json:"git_hash"` } // globalInfoSyncer stores the global infoSyncer. // Use a global variable for simply the code, use the domain.infoSyncer will have circle import problem in some pkg. // Use atomic.Value to avoid data race in the test. var globalInfoSyncer atomic.Value func getGlobalInfoSyncer() (*InfoSyncer, error) { v := globalInfoSyncer.Load() if v == nil { return nil, errors.New("infoSyncer is not initialized") } return v.(*InfoSyncer), nil } func setGlobalInfoSyncer(is *InfoSyncer) { globalInfoSyncer.Store(is) } // GlobalInfoSyncerInit return a new InfoSyncer. It is exported for testing. func GlobalInfoSyncerInit(ctx context.Context, id string, etcdCli *clientv3.Client, skipRegisterToDashBoard bool) (*InfoSyncer, error) { is := &InfoSyncer{ etcdCli: etcdCli, info: getServerInfo(id), serverInfoPath: fmt.Sprintf("%s/%s", ServerInformationPath, id), minStartTSPath: fmt.Sprintf("%s/%s", ServerMinStartTSPath, id), } err := is.init(ctx, skipRegisterToDashBoard) if err != nil { return nil, err } setGlobalInfoSyncer(is) return is, nil } // Init creates a new etcd session and stores server info to etcd. func (is *InfoSyncer) init(ctx context.Context, skipRegisterToDashboard bool) error { err := is.newSessionAndStoreServerInfo(ctx, owner.NewSessionDefaultRetryCnt) if err != nil { return err } if skipRegisterToDashboard { return nil } return is.newTopologySessionAndStoreServerInfo(ctx, owner.NewSessionDefaultRetryCnt) } // SetSessionManager set the session manager for InfoSyncer. func (is *InfoSyncer) SetSessionManager(manager util2.SessionManager) { is.manager = manager } // GetServerInfo gets self server static information. func GetServerInfo() (*ServerInfo, error) { is, err := getGlobalInfoSyncer() if err != nil { return nil, err } return is.info, nil } // GetServerInfoByID gets specified server static information from etcd. func GetServerInfoByID(ctx context.Context, id string) (*ServerInfo, error) { is, err := getGlobalInfoSyncer() if err != nil { return nil, err } return is.getServerInfoByID(ctx, id) } func (is *InfoSyncer) getServerInfoByID(ctx context.Context, id string) (*ServerInfo, error) { if is.etcdCli == nil || id == is.info.ID { return is.info, nil } key := fmt.Sprintf("%s/%s", ServerInformationPath, id) infoMap, err := getInfo(ctx, is.etcdCli, key, keyOpDefaultRetryCnt, keyOpDefaultTimeout) if err != nil { return nil, err } info, ok := infoMap[id] if !ok { return nil, errors.Errorf("[info-syncer] get %s failed", key) } return info, nil } // GetAllServerInfo gets all servers static information from etcd. func GetAllServerInfo(ctx context.Context) (map[string]*ServerInfo, error) { is, err := getGlobalInfoSyncer() if err != nil { return nil, err } return is.getAllServerInfo(ctx) } // UpdateTiFlashTableSyncProgress is used to update the tiflash table replica sync progress. func UpdateTiFlashTableSyncProgress(ctx context.Context, tid int64, progress float64) error { is, err := getGlobalInfoSyncer() if err != nil { return err } if is.etcdCli == nil { return nil } key := fmt.Sprintf("%s/%v", TiFlashTableSyncProgressPath, tid) return util.PutKVToEtcd(ctx, is.etcdCli, keyOpDefaultRetryCnt, key, strconv.FormatFloat(progress, 'f', 2, 64)) } // DeleteTiFlashTableSyncProgress is used to delete the tiflash table replica sync progress. func DeleteTiFlashTableSyncProgress(tid int64) error { is, err := getGlobalInfoSyncer() if err != nil { return err } if is.etcdCli == nil { return nil } key := fmt.Sprintf("%s/%v", TiFlashTableSyncProgressPath, tid) return util.DeleteKeyFromEtcd(key, is.etcdCli, keyOpDefaultRetryCnt, keyOpDefaultTimeout) } // GetTiFlashTableSyncProgress uses to get all the tiflash table replica sync progress. func GetTiFlashTableSyncProgress(ctx context.Context) (map[int64]float64, error) { is, err := getGlobalInfoSyncer() if err != nil { return nil, err } progressMap := make(map[int64]float64) if is.etcdCli == nil { return progressMap, nil } for i := 0; i < keyOpDefaultRetryCnt; i++ { resp, err := is.etcdCli.Get(ctx, TiFlashTableSyncProgressPath+"/", clientv3.WithPrefix()) if err != nil { logutil.BgLogger().Info("get tiflash table replica sync progress failed, continue checking.", zap.Error(err)) continue } for _, kv := range resp.Kvs { tid, err := strconv.ParseInt(string(kv.Key[len(TiFlashTableSyncProgressPath)+1:]), 10, 64) if err != nil { logutil.BgLogger().Info("invalid tiflash table replica sync progress key.", zap.String("key", string(kv.Key))) continue } progress, err := strconv.ParseFloat(string(kv.Value), 64) if err != nil { logutil.BgLogger().Info("invalid tiflash table replica sync progress value.", zap.String("key", string(kv.Key)), zap.String("value", string(kv.Value))) continue } progressMap[tid] = progress } break } return progressMap, nil } func (is *InfoSyncer) getAllServerInfo(ctx context.Context) (map[string]*ServerInfo, error) { allInfo := make(map[string]*ServerInfo) if is.etcdCli == nil { allInfo[is.info.ID] = getServerInfo(is.info.ID) return allInfo, nil } allInfo, err := getInfo(ctx, is.etcdCli, ServerInformationPath, keyOpDefaultRetryCnt, keyOpDefaultTimeout, clientv3.WithPrefix()) if err != nil { return nil, err } return allInfo, nil } // storeServerInfo stores self server static information to etcd. func (is *InfoSyncer) storeServerInfo(ctx context.Context) error { if is.etcdCli == nil { return nil } infoBuf, err := json.Marshal(is.info) if err != nil { return errors.Trace(err) } str := string(hack.String(infoBuf)) err = util.PutKVToEtcd(ctx, is.etcdCli, keyOpDefaultRetryCnt, is.serverInfoPath, str, clientv3.WithLease(is.session.Lease())) return err } // RemoveServerInfo remove self server static information from etcd. func (is *InfoSyncer) RemoveServerInfo() { if is.etcdCli == nil { return } err := util.DeleteKeyFromEtcd(is.serverInfoPath, is.etcdCli, keyOpDefaultRetryCnt, keyOpDefaultTimeout) if err != nil { logutil.BgLogger().Error("remove server info failed", zap.Error(err)) } } type topologyInfo struct { ServerVersionInfo StatusPort uint `json:"status_port"` DeployPath string `json:"deploy_path"` StartTimestamp int64 `json:"start_timestamp"` } func (is *InfoSyncer) getTopologyInfo() topologyInfo { s, err := os.Executable() if err != nil { s = "" } dir := path.Dir(s) return topologyInfo{ ServerVersionInfo: ServerVersionInfo{ Version: mysql.TiDBReleaseVersion, GitHash: is.info.ServerVersionInfo.GitHash, }, StatusPort: is.info.StatusPort, DeployPath: dir, StartTimestamp: is.info.StartTimestamp, } } // StoreTopologyInfo stores the topology of tidb to etcd. func (is *InfoSyncer) StoreTopologyInfo(ctx context.Context) error { if is.etcdCli == nil { return nil } topologyInfo := is.getTopologyInfo() infoBuf, err := json.Marshal(topologyInfo) if err != nil { return errors.Trace(err) } str := string(hack.String(infoBuf)) key := fmt.Sprintf("%s/%s:%v/info", TopologyInformationPath, is.info.IP, is.info.Port) // Note: no lease is required here. err = util.PutKVToEtcd(ctx, is.etcdCli, keyOpDefaultRetryCnt, key, str) if err != nil { return err } // Initialize ttl. return is.updateTopologyAliveness(ctx) } // GetMinStartTS get min start timestamp. // Export for testing. func (is *InfoSyncer) GetMinStartTS() uint64 { return is.minStartTS } // storeMinStartTS stores self server min start timestamp to etcd. func (is *InfoSyncer) storeMinStartTS(ctx context.Context) error { if is.etcdCli == nil { return nil } return util.PutKVToEtcd(ctx, is.etcdCli, keyOpDefaultRetryCnt, is.minStartTSPath, strconv.FormatUint(is.minStartTS, 10), clientv3.WithLease(is.session.Lease())) } // RemoveMinStartTS removes self server min start timestamp from etcd. func (is *InfoSyncer) RemoveMinStartTS() { if is.etcdCli == nil { return } err := util.DeleteKeyFromEtcd(is.minStartTSPath, is.etcdCli, keyOpDefaultRetryCnt, keyOpDefaultTimeout) if err != nil { logutil.BgLogger().Error("remove minStartTS failed", zap.Error(err)) } } // ReportMinStartTS reports self server min start timestamp to ETCD. func (is *InfoSyncer) ReportMinStartTS(store kv.Storage) { if is.manager == nil { // Server may not start in time. return } pl := is.manager.ShowProcessList() // Calculate the lower limit of the start timestamp to avoid extremely old transaction delaying GC. currentVer, err := store.CurrentVersion() if err != nil { logutil.BgLogger().Error("update minStartTS failed", zap.Error(err)) return } now := time.Unix(0, oracle.ExtractPhysical(currentVer.Ver)*1e6) startTSLowerLimit := variable.GoTimeToTS(now.Add(-time.Duration(kv.MaxTxnTimeUse) * time.Millisecond)) minStartTS := variable.GoTimeToTS(now) for _, info := range pl { if info.CurTxnStartTS > startTSLowerLimit && info.CurTxnStartTS < minStartTS { minStartTS = info.CurTxnStartTS } } is.minStartTS = minStartTS err = is.storeMinStartTS(context.Background()) if err != nil { logutil.BgLogger().Error("update minStartTS failed", zap.Error(err)) } } // Done returns a channel that closes when the info syncer is no longer being refreshed. func (is *InfoSyncer) Done() <-chan struct{} { if is.etcdCli == nil { return make(chan struct{}, 1) } return is.session.Done() } // TopologyDone returns a channel that closes when the topology syncer is no longer being refreshed. func (is *InfoSyncer) TopologyDone() <-chan struct{} { if is.etcdCli == nil { return make(chan struct{}, 1) } return is.topologySession.Done() } // Restart restart the info syncer with new session leaseID and store server info to etcd again. func (is *InfoSyncer) Restart(ctx context.Context) error { return is.newSessionAndStoreServerInfo(ctx, owner.NewSessionDefaultRetryCnt) } // RestartTopology restart the topology syncer with new session leaseID and store server info to etcd again. func (is *InfoSyncer) RestartTopology(ctx context.Context) error { return is.newTopologySessionAndStoreServerInfo(ctx, owner.NewSessionDefaultRetryCnt) } // newSessionAndStoreServerInfo creates a new etcd session and stores server info to etcd. func (is *InfoSyncer) newSessionAndStoreServerInfo(ctx context.Context, retryCnt int) error { if is.etcdCli == nil { return nil } logPrefix := fmt.Sprintf("[Info-syncer] %s", is.serverInfoPath) session, err := owner.NewSession(ctx, logPrefix, is.etcdCli, retryCnt, InfoSessionTTL) if err != nil { return err } is.session = session binloginfo.RegisterStatusListener(func(status binloginfo.BinlogStatus) error { is.info.BinlogStatus = status.String() err := is.storeServerInfo(ctx) return errors.Trace(err) }) return is.storeServerInfo(ctx) } // newTopologySessionAndStoreServerInfo creates a new etcd session and stores server info to etcd. func (is *InfoSyncer) newTopologySessionAndStoreServerInfo(ctx context.Context, retryCnt int) error { if is.etcdCli == nil { return nil } logPrefix := fmt.Sprintf("[topology-syncer] %s/%s:%d", TopologyInformationPath, is.info.IP, is.info.Port) session, err := owner.NewSession(ctx, logPrefix, is.etcdCli, retryCnt, TopologySessionTTL) if err != nil { return err } is.topologySession = session return is.StoreTopologyInfo(ctx) } // refreshTopology refreshes etcd topology with ttl stored in "/topology/tidb/ip:port/ttl". func (is *InfoSyncer) updateTopologyAliveness(ctx context.Context) error { if is.etcdCli == nil { return nil } key := fmt.Sprintf("%s/%s:%v/ttl", TopologyInformationPath, is.info.IP, is.info.Port) return util.PutKVToEtcd(ctx, is.etcdCli, keyOpDefaultRetryCnt, key, fmt.Sprintf("%v", time.Now().UnixNano()), clientv3.WithLease(is.topologySession.Lease())) } // GetPrometheusAddr gets prometheus Address func GetPrometheusAddr() (string, error) { is, err := getGlobalInfoSyncer() if err != nil { return "", err } // if the cache of prometheusAddr is over 10s, update the prometheusAddr if time.Since(is.modifyTime) < TablePrometheusCacheExpiry { return is.prometheusAddr, nil } return is.getPrometheusAddr() } type prometheus struct { IP string `json:"ip"` BinaryPath string `json:"binary_path"` Port int `json:"port"` } type metricStorage struct { PDServer struct { MetricStorage string `json:"metric-storage"` } `json:"pd-server"` } func (is *InfoSyncer) getPrometheusAddr() (string, error) { // Get PD servers info. pdAddrs := is.etcdCli.Endpoints() if len(pdAddrs) == 0 { return "", errors.Errorf("pd unavailable") } // Get prometheus address from pdApi. var url, res string if strings.HasPrefix(pdAddrs[0], "http://") { url = fmt.Sprintf("%s%s", pdAddrs[0], pdapi.Config) } else { url = fmt.Sprintf("http://%s%s", pdAddrs[0], pdapi.Config) } resp, err := http.Get(url) if err != nil { return "", err } var metricStorage metricStorage dec := json.NewDecoder(resp.Body) err = dec.Decode(&metricStorage) if err != nil { return "", err } res = metricStorage.PDServer.MetricStorage // Get prometheus address from etcdApi. if res == "" { values, err := is.getPrometheusAddrFromEtcd(TopologyPrometheus) if err != nil { return "", errors.Trace(err) } if values == "" { return "", ErrPrometheusAddrIsNotSet } var prometheus prometheus err = json.Unmarshal([]byte(values), &prometheus) if err != nil { return "", errors.Trace(err) } res = fmt.Sprintf("http://%s:%v", prometheus.IP, prometheus.Port) } is.prometheusAddr = res is.modifyTime = time.Now() setGlobalInfoSyncer(is) return res, nil } func (is *InfoSyncer) getPrometheusAddrFromEtcd(k string) (string, error) { ctx, cancel := context.WithTimeout(context.Background(), keyOpDefaultTimeout) resp, err := is.etcdCli.Get(ctx, k) cancel() if err != nil { return "", errors.Trace(err) } if len(resp.Kvs) > 0 { return string(resp.Kvs[0].Value), nil } return "", nil } // getInfo gets server information from etcd according to the key and opts. func getInfo(ctx context.Context, etcdCli *clientv3.Client, key string, retryCnt int, timeout time.Duration, opts ...clientv3.OpOption) (map[string]*ServerInfo, error) { var err error var resp *clientv3.GetResponse allInfo := make(map[string]*ServerInfo) for i := 0; i < retryCnt; i++ { select { case <-ctx.Done(): err = errors.Trace(ctx.Err()) return nil, err default: } childCtx, cancel := context.WithTimeout(ctx, timeout) resp, err = etcdCli.Get(childCtx, key, opts...) cancel() if err != nil { logutil.BgLogger().Info("get key failed", zap.String("key", key), zap.Error(err)) time.Sleep(200 * time.Millisecond) continue } for _, kv := range resp.Kvs { info := &ServerInfo{ BinlogStatus: binloginfo.BinlogStatusUnknown.String(), } err = json.Unmarshal(kv.Value, info) if err != nil { logutil.BgLogger().Info("get key failed", zap.String("key", string(kv.Key)), zap.ByteString("value", kv.Value), zap.Error(err)) return nil, errors.Trace(err) } allInfo[info.ID] = info } return allInfo, nil } return nil, errors.Trace(err) } // getServerInfo gets self tidb server information. func getServerInfo(id string) *ServerInfo { cfg := config.GetGlobalConfig() info := &ServerInfo{ ID: id, IP: cfg.AdvertiseAddress, Port: cfg.Port, StatusPort: cfg.Status.StatusPort, Lease: cfg.Lease, BinlogStatus: binloginfo.GetStatus().String(), StartTimestamp: time.Now().Unix(), } info.Version = mysql.ServerVersion info.GitHash = versioninfo.TiDBGitHash failpoint.Inject("mockServerInfo", func(val failpoint.Value) { if val.(bool) { info.StartTimestamp = 1282967700000 } }) return info }