From bee33423b02f8c0af56c72a5e79e312f693ab537 Mon Sep 17 00:00:00 2001 From: xjxia Date: Thu, 21 Nov 2024 16:53:59 +0800 Subject: [PATCH] fix(dbm-services): DBHA multi-GM may cause repeated switch close #8097 --- .../dbha/ha-module/agent/monitor_agent.go | 137 ++++++++++++------ .../common/dbha/ha-module/client/hadb.go | 2 +- .../dbmodule/dbmysql/MySQL_common_switch.go | 4 +- .../common/dbha/ha-module/dbutil/db_detect.go | 59 +++++--- dbm-services/common/dbha/ha-module/gm/gcm.go | 3 + dbm-services/common/dbha/ha-module/gm/gdm.go | 19 +-- dbm-services/common/dbha/ha-module/gm/gm.go | 8 +- .../common/dbha/ha-module/test/MySQL_test.go | 4 +- .../common/dbha/ha-module/test/agent_test.go | 2 +- .../common/dbha/ha-module/util/util.go | 26 +++- .../dbha/hadb-api/model/HASwitchQueue.go | 5 +- 11 files changed, 167 insertions(+), 102 deletions(-) diff --git a/dbm-services/common/dbha/ha-module/agent/monitor_agent.go b/dbm-services/common/dbha/ha-module/agent/monitor_agent.go index 8f86bd7e37..384187773b 100644 --- a/dbm-services/common/dbha/ha-module/agent/monitor_agent.go +++ b/dbm-services/common/dbha/ha-module/agent/monitor_agent.go @@ -1,8 +1,8 @@ package agent import ( - "dbm-services/common/dbha/hadb-api/model" "fmt" + "hash/crc32" "net" "sort" "strconv" @@ -16,8 +16,17 @@ import ( "dbm-services/common/dbha/ha-module/dbutil" "dbm-services/common/dbha/ha-module/log" "dbm-services/common/dbha/ha-module/monitor" + "dbm-services/common/dbha/hadb-api/model" ) +// CachedHostInfo instance had report to gm +type CachedHostInfo struct { + Ip string + ReporterGMTime time.Time + //interval seconds to expire + ExpireInterval int +} + // MonitorAgent agent work struct type MonitorAgent struct { CityID int @@ -31,9 +40,12 @@ type MonitorAgent struct { // mod for current agent HashMod int // mod value for current agent - HashValue int + HashValue int + //instances need to detect DBInstance map[string]dbutil.DataBaseDetect - GMInstance map[string]*GMConnection + //cache for reported GM instances + ReportGMCache map[string]*CachedHostInfo + GMInstance map[string]*GMConnection // config file Conf *config.Config // API client to access cmdb metadata @@ -42,6 +54,7 @@ type MonitorAgent struct { HaDBClient *client.HaDBClient heartbeat time.Time MaxConcurrency int // Add this field to store the max concurrency value + Mutex sync.RWMutex } // NewMonitorAgent new a new agent do detect @@ -54,6 +67,7 @@ func NewMonitorAgent(conf *config.Config, detectType string) (*MonitorAgent, err LastFetchInsTime: time.Now(), LastFetchGMTime: time.Now(), GMInstance: map[string]*GMConnection{}, + ReportGMCache: map[string]*CachedHostInfo{}, heartbeat: time.Now(), Conf: conf, CmDBClient: client.NewCmDBClient(&conf.DBConf.CMDB, conf.GetCloudId()), @@ -122,6 +136,11 @@ func (a *MonitorAgent) RefreshInstanceCache() { return } a.flushInsFetchTime() + + // delete reported gm cache + for k, _ := range a.ReportGMCache { + delete(a.ReportGMCache, k) + } } } @@ -136,14 +155,24 @@ func (a *MonitorAgent) DoDetectSingle(ins dbutil.DataBaseDetect) { } a.reportMonitor(ins, err) - if ins.NeedReporter() { - // reporter detect result to hadb - if err = a.ReporterDetectInfo(ins); err != nil { + if ins.NeedReportAgent() { + //only report to HADB's ha_agent_logs + if err = a.HaDBClient.ReportDBStatus(ins.GetApp(), a.MonIp, ip, port, + string(ins.GetDBType()), string(ins.GetStatus()), "N/A"); err != nil { log.Logger.Errorf( "reporter hadb instance status failed. err:%s, ip:%s, port:%d, db_type:%s, status:%s", err.Error(), ip, port, ins.GetDBType(), ins.GetStatus()) } - ins.UpdateReporterTime() + ins.UpdateReportTime() + } + + if a.NeedReportGM(ins) { + // reporter detect result to GM + if err = a.ReportDetectInfoToGM(ins); err != nil { + log.Logger.Errorf( + "reporter instance info to gm failed. err:%s, ip:%s, port:%d, db_type:%s, status:%s", + err.Error(), ip, port, ins.GetDBType(), ins.GetStatus()) + } } } @@ -165,16 +194,16 @@ func (a *MonitorAgent) RefreshGMCache() { a.flushGmFetchTime() } + a.Mutex.Lock() + defer a.Mutex.Unlock() for ip, ins := range a.GMInstance { - ins.Mutex.Lock() - anHour := time.Now().Add(-60 * time.Minute) + anHour := time.Now().Add(-30 * time.Minute) // connect leak? if ins.LastFetchTime.Before(anHour) { ins.IsClose = true log.Logger.Infof("gm:%s de-cached", ip) delete(a.GMInstance, ip) } - ins.Mutex.Unlock() } // we not return error here, next refresh, new added gm maybe available. @@ -261,6 +290,9 @@ func (a *MonitorAgent) FetchGMInstance() error { return err } + a.Mutex.Lock() + defer a.Mutex.Unlock() + for _, info := range gmInfo { if info.CityID == a.CityID || info.CloudID != a.Conf.AgentConf.CloudID { continue @@ -290,32 +322,48 @@ func (a *MonitorAgent) FetchGMInstance() error { return nil } -// ReporterDetectInfo report detect info to gm -func (a *MonitorAgent) ReporterDetectInfo(reporterInstance dbutil.DataBaseDetect) error { - var err error - isReporter := false - ip, port := reporterInstance.GetAddress() - if reporterInstance.GetStatus() == constvar.DBCheckSuccess || - reporterInstance.GetStatus() == constvar.SSHCheckSuccess { - if err = a.HaDBClient.ReportDBStatus(reporterInstance.GetApp(), a.MonIp, ip, port, - string(reporterInstance.GetDBType()), string(reporterInstance.GetStatus()), "N/A"); err != nil { - log.Logger.Errorf( - "reporter hadb instance status failed. err:%s, ip:%s, port:%d, db_type:%s, status:%s", - err.Error(), ip, port, reporterInstance.GetDBType(), reporterInstance.GetStatus()) +// NeedReportGM report detect info to GM module +func (a *MonitorAgent) NeedReportGM(ins dbutil.DataBaseDetect) bool { + ip, _ := ins.GetAddress() + if ins.GetStatus() == constvar.DBCheckSuccess || + ins.GetStatus() == constvar.SSHCheckSuccess { + return false + } + + if _, ok := a.ReportGMCache[ip]; ok { + cachedIns := a.ReportGMCache[ip] + now := time.Now() + if now.Before(cachedIns.ReporterGMTime.Add(time.Second * time.Duration(cachedIns.ExpireInterval))) { + return false } - return nil } + return true +} + +// ReportDetectInfoToGM report detect info to HADB, maybe gm also +func (a *MonitorAgent) ReportDetectInfoToGM(reporterInstance dbutil.DataBaseDetect) error { + var err error + isReported := false + ip, port := reporterInstance.GetAddress() + // 提取 GMInstance 的 IP 列表 var gmIPs []string + a.Mutex.RLock() for gmIP := range a.GMInstance { gmIPs = append(gmIPs, gmIP) } + a.Mutex.RUnlock() // 按照 IP 字典顺序排序 sort.Strings(gmIPs) + hashValue := crc32.ChecksumIEEE([]byte(ip)) + hashIndex := int(hashValue) % len(gmIPs) - for _, sortedIp := range gmIPs { + for i := 0; i < len(gmIPs); i++ { + //每次遍历时,都是从下一个偏移开始,确保在最差情况下能遍历完所有的gm + checkIndex := (hashIndex + i) % len(gmIPs) + sortedIp := gmIPs[checkIndex] gmIns := a.GMInstance[sortedIp] gmIns.Mutex.Lock() if !gmIns.IsConnection { @@ -334,30 +382,34 @@ func (a *MonitorAgent) ReporterDetectInfo(reporterInstance dbutil.DataBaseDetect if err != nil { log.Logger.Warnf("reporter gm failed. gm_ip:%s, gm_port:%d, err:%s", gmIns.Ip, gmIns.Port, err.Error()) gmIns.IsConnection = false - err = a.RepairGM(gmIns) - if err != nil { - log.Logger.Errorf("Repair gm failed:%s", err.Error()) - return err - } + gmIns.Mutex.Unlock() + a.RepairGMConnection(gmIns) + //do retry + continue } else { + isReported = true + gmIns.Mutex.Unlock() + a.ReportGMCache[ip] = &CachedHostInfo{ + ReporterGMTime: time.Now(), + Ip: ip, + ExpireInterval: a.Conf.AgentConf.ReportInterval, + } + //report bind gmInfo to ha_agent_logs if err = a.HaDBClient.ReportDBStatus(reporterInstance.GetApp(), a.MonIp, ip, port, string(reporterInstance.GetDBType()), string(reporterInstance.GetStatus()), gmInfo); err != nil { - log.Logger.Errorf( - "reporter hadb instance status failed. err:%s, ip:%s, port:%d, db_type:%s, status:%s", - err.Error(), ip, port, reporterInstance.GetDBType(), reporterInstance.GetStatus()) + log.Logger.Warnf( + "instance[%s#%d] reporter bind gm info[%s] failed:%s", ip, port, gmInfo, err.Error()) } - isReporter = true - gmIns.Mutex.Unlock() break } - gmIns.Mutex.Unlock() } - if !isReporter { + if !isReported { err = fmt.Errorf("get report GM failed: all gm disconnect") log.Logger.Error(err.Error()) return err } + return nil } @@ -381,12 +433,12 @@ func (a *MonitorAgent) flushGmFetchTime() { a.LastFetchGMTime = time.Now() } -// RepairGM if conn break, do reconnect -func (a *MonitorAgent) RepairGM(gmIns *GMConnection) error { +// RepairGMConnection if conn break, do reconnect +func (a *MonitorAgent) RepairGMConnection(gmIns *GMConnection) { go func(gmIns *GMConnection) { for { gmIns.Mutex.Lock() - if gmIns.IsClose { + if gmIns.IsClose || gmIns.IsConnection { gmIns.Mutex.Unlock() return } @@ -394,19 +446,18 @@ func (a *MonitorAgent) RepairGM(gmIns *GMConnection) error { conn, err := net.Dial("tcp", address) if err != nil { log.Logger.Warn( - "RepairGM: ip:", gmIns.Ip, " port:", gmIns.Port, " connect failed, err:", err.Error()) + "RepairGMConnection: ip:", gmIns.Ip, " port:", gmIns.Port, " connect failed, err:", err.Error()) } else { gmIns.NetConnection = conn gmIns.IsConnection = true - log.Logger.Info("RepairGM: ip:", gmIns.Ip, " port:", gmIns.Port, " connect success.") + log.Logger.Info("RepairGMConnection: ip:", gmIns.Ip, " port:", gmIns.Port, " connect success.") gmIns.Mutex.Unlock() return } gmIns.Mutex.Unlock() - time.Sleep(10 * time.Second) + time.Sleep(5 * time.Second) } }(gmIns) - return nil } // registerAgentInfoToHaDB register current agent info diff --git a/dbm-services/common/dbha/ha-module/client/hadb.go b/dbm-services/common/dbha/ha-module/client/hadb.go index f208f54666..936c6dbbfa 100644 --- a/dbm-services/common/dbha/ha-module/client/hadb.go +++ b/dbm-services/common/dbha/ha-module/client/hadb.go @@ -239,7 +239,7 @@ func (c *HaDBClient) ReportDBStatus(app, agentIp, ip string, port int, dbType, s return nil } -// ReportHaLogRough report ha logs +// ReportHaLogRough report ha logs without return func (c *HaDBClient) ReportHaLogRough(monIP, app, ip string, port int, module, comment string) { _, _ = c.ReportHaLog(monIP, app, ip, port, module, comment) } diff --git a/dbm-services/common/dbha/ha-module/dbmodule/dbmysql/MySQL_common_switch.go b/dbm-services/common/dbha/ha-module/dbmodule/dbmysql/MySQL_common_switch.go index 6cb55dfe5d..784b430359 100644 --- a/dbm-services/common/dbha/ha-module/dbmodule/dbmysql/MySQL_common_switch.go +++ b/dbm-services/common/dbha/ha-module/dbmodule/dbmysql/MySQL_common_switch.go @@ -359,7 +359,7 @@ func (ins *MySQLCommonSwitch) CheckSlaveStatus() error { slaveDelay, gmConf.GCM.AllowedSlaveDelayMax)) if timeDelay >= gmConf.GCM.AllowedTimeDelayMax { - return fmt.Errorf("IO_Thread delay on slave too large than master(%d >= %d)", timeDelay, + return fmt.Errorf("heartbeat delay on slave too large than master(%d >= %d)", timeDelay, gmConf.GCM.AllowedTimeDelayMax) } ins.ReportLogs(constvar.InfoResult, fmt.Sprintf("IO_Thread delay [%d] in allowed range[%d]", @@ -526,7 +526,7 @@ func (ins *MySQLCommonSwitch) CheckSlaveSlow(ignoreDelay bool) error { } binlogSizeMByte := maxBinlogSize.VariableValue / (1024 * 1024) - log.Logger.Infof("the slave max_binlog_size value is %d M!", binlogSizeMByte) + log.Logger.Infof("the slave max_binlog_size value is %dM!", binlogSizeMByte) status, err := GetSlaveStatus(db) if err != nil { diff --git a/dbm-services/common/dbha/ha-module/dbutil/db_detect.go b/dbm-services/common/dbha/ha-module/dbutil/db_detect.go index 560d8e3f26..9c2483e237 100644 --- a/dbm-services/common/dbha/ha-module/dbutil/db_detect.go +++ b/dbm-services/common/dbha/ha-module/dbutil/db_detect.go @@ -3,6 +3,7 @@ package dbutil import ( "bytes" "fmt" + "math/rand" "strconv" "strings" "time" @@ -58,7 +59,8 @@ type DataBaseDetect interface { // Serialization agent call this to serializa instance info, and then send to gdm Serialization() ([]byte, error) - NeedReporter() bool + //NeedReportAgent detect info need report to ha_agent_logs + NeedReportAgent() bool GetDBType() types.DBType // GetDetectType agent send detect type to gm, gm use this key to find callback func GetDetectType() string @@ -68,15 +70,16 @@ type DataBaseDetect interface { GetCluster() string GetClusterType() string GetClusterId() int - UpdateReporterTime() + UpdateReportTime() } // BaseDetectDB db detect base struct type BaseDetectDB struct { - Ip string - Port int - App string - DBType types.DBType + Ip string + Port int + App string + DBType types.DBType + //time for report ha_agent_logs ReporterTime time.Time ReportInterval int Status types.CheckStatus @@ -261,22 +264,32 @@ func (b *BaseDetectDB) DoSSHForWindows(shellStr string) error { return nil } -// NeedReporter decide whether need report detect result to HADB -func (b *BaseDetectDB) NeedReporter() bool { - var need bool - if b.Status == constvar.DBCheckSuccess { - now := time.Now() - if now.After(b.ReporterTime.Add(time.Second * time.Duration(b.ReportInterval))) { - need = true - } else { - need = false - } - // log.Logger.Debugf("now time:%s, reporter time:%s, reporter interval:%d, need:%s", - // now.String(), b.ReporterTime.String(), b.ReportInterval, need) - } else { - need = true +// NeedReportAgent decides whether to report the detect result to ha_agent_logs +func (b *BaseDetectDB) NeedReportAgent() bool { + // 获取当前时间 + now := time.Now() + + // 如果状态不是成功,则立即上报 + if b.Status != constvar.DBCheckSuccess { + return true + } + + // 如果距离上次上报已经超过1分钟,则必须上报 + if now.After(b.ReporterTime.Add(time.Minute)) { + return true } - return need + + // 否则,计算30秒内的随机延迟 + randomDelay := time.Duration(rand.Intn(30)) * time.Second + randomReportTime := b.ReporterTime.Add(time.Second * time.Duration(b.ReportInterval)).Add(randomDelay) + + // 如果当前时间超过随机的上报时间,触发上报 + if now.After(randomReportTime) { + return true + } + + // 否则不需要上报 + return false } // GetAddress return instance's ip, port @@ -320,8 +333,8 @@ func (b *BaseDetectDB) GetClusterType() string { return b.ClusterType } -// UpdateReporterTime update report info -func (b *BaseDetectDB) UpdateReporterTime() { +// UpdateReportTime update report ha_agent_logs time +func (b *BaseDetectDB) UpdateReportTime() { b.ReporterTime = time.Now() } diff --git a/dbm-services/common/dbha/ha-module/gm/gcm.go b/dbm-services/common/dbha/ha-module/gm/gcm.go index 20900e0220..423f62e00b 100644 --- a/dbm-services/common/dbha/ha-module/gm/gcm.go +++ b/dbm-services/common/dbha/ha-module/gm/gcm.go @@ -1,6 +1,7 @@ package gm import ( + "dbm-services/common/dbha/ha-module/util" "dbm-services/common/dbha/hadb-api/model" "fmt" "time" @@ -205,6 +206,8 @@ func (gcm *GCM) InsertSwitchQueue(instance dbutil.DataBaseSwitch) error { SwitchStartTime: ¤tTime, DbRole: instance.GetRole(), ConfirmResult: doubleCheckInfo, + SwitchHashID: util.GenerateHash(fmt.Sprintf("%#%d", ip, port), + int64(max(300, gcm.Conf.GMConf.ReportInterval))), }, } diff --git a/dbm-services/common/dbha/ha-module/gm/gdm.go b/dbm-services/common/dbha/ha-module/gm/gdm.go index 0c8250c20a..0dad231296 100644 --- a/dbm-services/common/dbha/ha-module/gm/gdm.go +++ b/dbm-services/common/dbha/ha-module/gm/gdm.go @@ -16,7 +16,7 @@ type GDM struct { AgentChan chan DoubleCheckInstanceInfo GMMChan chan DoubleCheckInstanceInfo ListenPort int - ReporterCache map[InstanceKey]*DoubleCheckInstanceInfo + ReporterCache map[string]*DoubleCheckInstanceInfo cacheMutex sync.Mutex DupExpire int ScanInterval int @@ -31,7 +31,7 @@ func NewGDM(conf *config.Config, ch chan DoubleCheckInstanceInfo, AgentChan: make(chan DoubleCheckInstanceInfo, 10), GMMChan: ch, ListenPort: conf.GMConf.ListenPort, - ReporterCache: map[InstanceKey]*DoubleCheckInstanceInfo{}, + ReporterCache: map[string]*DoubleCheckInstanceInfo{}, cacheMutex: sync.Mutex{}, DupExpire: conf.GMConf.GDM.DupExpire, ScanInterval: conf.GMConf.GDM.ScanInterval, @@ -127,19 +127,13 @@ func (gdm *GDM) isReporterRecently(ins *DoubleCheckInstanceInfo) bool { ip, port := ins.db.GetAddress() gdm.cacheMutex.Lock() defer gdm.cacheMutex.Unlock() - cache, ok := gdm.ReporterCache[InstanceKey{ - ip, - port, - }] + cache, ok := gdm.ReporterCache[ip] if ok && cache.db.GetStatus() == ins.db.GetStatus() { log.Logger.Infof("instance[%s#%d] cached, skip report", ip, port) return true } // 刷新缓存 - gdm.ReporterCache[InstanceKey{ - ip, - port, - }] = ins + gdm.ReporterCache[ip] = ins return false } @@ -161,10 +155,7 @@ func (gdm *GDM) flushCache() { func (gdm *GDM) InstanceSwitchDone(ip string, port int, dbType string) { gdm.cacheMutex.Lock() defer gdm.cacheMutex.Unlock() - cache, ok := gdm.ReporterCache[InstanceKey{ - ip, - port, - }] + cache, ok := gdm.ReporterCache[ip] if !ok { log.Logger.Warnf( "ip:%s, port:%d, dbtype:%s switch done, but cache not exist", ip, port, dbType) diff --git a/dbm-services/common/dbha/ha-module/gm/gm.go b/dbm-services/common/dbha/ha-module/gm/gm.go index bc4143b1dc..fb76dac801 100644 --- a/dbm-services/common/dbha/ha-module/gm/gm.go +++ b/dbm-services/common/dbha/ha-module/gm/gm.go @@ -11,12 +11,6 @@ import ( "dbm-services/common/dbha/ha-module/log" ) -// InstanceKey instance key info -type InstanceKey struct { - Ip string - Port int -} - // DoubleCheckInstanceInfo double check instance info type DoubleCheckInstanceInfo struct { AgentIp string @@ -26,7 +20,7 @@ type DoubleCheckInstanceInfo struct { ConfirmTime time.Time //double check result ResultInfo string - //gmm double check id + //gmm double check id, ha_gm_logs's uid CheckID int64 } diff --git a/dbm-services/common/dbha/ha-module/test/MySQL_test.go b/dbm-services/common/dbha/ha-module/test/MySQL_test.go index 22a09cdfbd..8bae0f979d 100644 --- a/dbm-services/common/dbha/ha-module/test/MySQL_test.go +++ b/dbm-services/common/dbha/ha-module/test/MySQL_test.go @@ -52,9 +52,9 @@ func TestDetectionSuccess(t *testing.T) { t.Errorf("detection failed.err:%s", err.Error()) } fmt.Printf("status: %s\n", d.GetStatus()) - if d.NeedReporter() { + if d.NeedReportAgent() { fmt.Println("need reporter") - d.UpdateReporterTime() + d.UpdateReportTime() } else { fmt.Println("needn't reporter") } diff --git a/dbm-services/common/dbha/ha-module/test/agent_test.go b/dbm-services/common/dbha/ha-module/test/agent_test.go index 0b0c7810dc..6bb91a4c0d 100644 --- a/dbm-services/common/dbha/ha-module/test/agent_test.go +++ b/dbm-services/common/dbha/ha-module/test/agent_test.go @@ -68,7 +68,7 @@ func TestAgentNetTransfor(t *testing.T) { case 3: dbIns.App = "APP4444" } - err = agentIns.ReporterDetectInfo(d) + err = agentIns.ReportDetectInfoToGM(d) if err != nil { t.Errorf("reporter gmInfo failed.err:%s", err.Error()) return diff --git a/dbm-services/common/dbha/ha-module/util/util.go b/dbm-services/common/dbha/ha-module/util/util.go index 2d0efd7708..d22513765d 100644 --- a/dbm-services/common/dbha/ha-module/util/util.go +++ b/dbm-services/common/dbha/ha-module/util/util.go @@ -5,7 +5,7 @@ import ( "bytes" "encoding/json" "fmt" - "hash/crc32" + "hash/fnv" "net" "os/exec" "reflect" @@ -73,11 +73,6 @@ func HostCheck(host string) bool { return true } -// CRC32 TODO -func CRC32(str string) uint32 { - return crc32.ChecksumIEEE([]byte(str)) -} - // CheckRedisErrIsAuthFail check if the return error of // // redis api is authentication failure, @@ -96,7 +91,7 @@ func CheckRedisErrIsAuthFail(err error) bool { return false } -// CheckSSHErrIsAuthFail check if the the return error of ssh api +// CheckSSHErrIsAuthFail check if the ssh return error of ssh api // // is authentication failure. func CheckSSHErrIsAuthFail(err error) bool { @@ -157,3 +152,20 @@ func GraceStructString(v interface{}) string { } return string(data) } + +// GenerateHash generates a consistent hash value for a given factor within a specified time window (in seconds). +func GenerateHash(factor string, timeWindow int64) uint32 { + // Get the current Unix timestamp + now := time.Now().Unix() + + // Calculate the start of the time window + windowStart := now - (now % timeWindow) + + // Combine the factor and windowStart into a single input string + input := fmt.Sprintf("%s:%d", factor, windowStart) + + // Use FNV-1a to hash the input string + hasher := fnv.New32a() + _, _ = hasher.Write([]byte(input)) + return hasher.Sum32() +} diff --git a/dbm-services/common/dbha/hadb-api/model/HASwitchQueue.go b/dbm-services/common/dbha/hadb-api/model/HASwitchQueue.go index 97de7b02c2..de42984833 100644 --- a/dbm-services/common/dbha/hadb-api/model/HASwitchQueue.go +++ b/dbm-services/common/dbha/hadb-api/model/HASwitchQueue.go @@ -19,8 +19,8 @@ type HASwitchQueue struct { Uid int64 `gorm:"column:uid;type:bigint;primary_key;AUTO_INCREMENT" json:"uid,omitempty"` CheckID int64 `gorm:"column:check_id;type:bigint;" json:"check_id,omitempty"` App string `gorm:"column:app;type:varchar(32);index:idx_app_ip_port" json:"app,omitempty"` - IP string `gorm:"column:ip;type:varchar(32);index:idx_app_ip_port;NOT NULL" json:"ip,omitempty"` - Port int `gorm:"column:port;type:int(11);index:idx_app_ip_port;NOT NULL" json:"port,omitempty"` + IP string `gorm:"column:ip;type:varchar(32);uniqueIndex:uniq_ip_port_hashid;index:idx_app_ip_port;NOT NULL" json:"ip,omitempty"` + Port int `gorm:"column:port;type:int(11);uniqueIndex:uniq_ip_port_hashid;index:idx_app_ip_port;NOT NULL" json:"port,omitempty"` ConfirmCheckTime *time.Time `gorm:"column:confirm_check_time;type:datetime;default:CURRENT_TIMESTAMP" json:"confirm_check_time,omitempty"` DbRole string `gorm:"column:db_role;type:varchar(32);NOT NULL" json:"db_role,omitempty"` SlaveIP string `gorm:"column:slave_ip;type:varchar(32)" json:"slave_ip,omitempty"` @@ -35,6 +35,7 @@ type HASwitchQueue struct { IdcID int `gorm:"column:idc_id;type:int(11)" json:"idc_id,omitempty"` CloudID int `gorm:"column:cloud_id;type:int(11);default:0" json:"cloud_id,omitempty"` Cluster string `gorm:"column:cluster;type:varchar(64)" json:"cluster,omitempty"` + SwitchHashID uint32 `gorm:"column:switch_hash_id;type:bigint;uniqueIndex:uniq_ip_port_hashid" json:"switch_hash_id,omitempty"` } // TableName TODO