diff --git a/cluster/cluster.go b/cluster/cluster.go
index 61230a831..a89534404 100644
--- a/cluster/cluster.go
+++ b/cluster/cluster.go
@@ -712,17 +712,22 @@ func (cluster *Cluster) StateProcessing() {
if s.ErrKey == "WARN0074" {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlInfo, "Sending master physical backup to reseed %s", s.ServerUrl)
if master != nil {
- backupext := ".xbtream"
- task := "reseed" + cluster.Conf.BackupPhysicalType
+ if servertoreseed.IsReseeding {
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Cancel backup reseeding, %s is already reseeding", s.ServerUrl)
+ } else {
+ servertoreseed.SetInReseedBackup(true)
+ backupext := ".xbtream"
+ task := "reseed" + cluster.Conf.BackupPhysicalType
- if cluster.Conf.CompressBackups {
- backupext = backupext + ".gz"
- }
+ if cluster.Conf.CompressBackups {
+ backupext = backupext + ".gz"
+ }
- if mybcksrv != nil {
- go cluster.SSTRunSender(mybcksrv.GetMyBackupDirectory()+cluster.Conf.BackupPhysicalType+backupext, servertoreseed, task)
- } else {
- go cluster.SSTRunSender(master.GetMasterBackupDirectory()+cluster.Conf.BackupPhysicalType+backupext, servertoreseed, task)
+ if mybcksrv != nil {
+ go cluster.SSTRunSender(mybcksrv.GetMyBackupDirectory()+cluster.Conf.BackupPhysicalType+backupext, servertoreseed, task)
+ } else {
+ go cluster.SSTRunSender(master.GetMasterBackupDirectory()+cluster.Conf.BackupPhysicalType+backupext, servertoreseed, task)
+ }
}
} else {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "No master cancel backup reseeding %s", s.ServerUrl)
@@ -746,12 +751,18 @@ func (cluster *Cluster) StateProcessing() {
// }
}
if s.ErrKey == "WARN0076" {
- task := "flashback" + cluster.Conf.BackupPhysicalType
- cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlInfo, "Sending server physical backup to flashback reseed %s", s.ServerUrl)
- if mybcksrv != nil {
- go cluster.SSTRunSender(mybcksrv.GetMyBackupDirectory()+cluster.Conf.BackupPhysicalType+".xbtream", servertoreseed, task)
+ if servertoreseed.IsReseeding {
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Cancel backup reseeding, %s is already reseeding", s.ServerUrl)
} else {
- go cluster.SSTRunSender(servertoreseed.GetMyBackupDirectory()+cluster.Conf.BackupPhysicalType+".xbtream", servertoreseed, task)
+ servertoreseed.SetInReseedBackup(true)
+ task := "flashback" + cluster.Conf.BackupPhysicalType
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlInfo, "Sending server physical backup to flashback reseed %s", s.ServerUrl)
+
+ if mybcksrv != nil {
+ go cluster.SSTRunSender(mybcksrv.GetMyBackupDirectory()+cluster.Conf.BackupPhysicalType+".xbtream", servertoreseed, task)
+ } else {
+ go cluster.SSTRunSender(servertoreseed.GetMyBackupDirectory()+cluster.Conf.BackupPhysicalType+".xbtream", servertoreseed, task)
+ }
}
}
if s.ErrKey == "WARN0077" {
diff --git a/cluster/cluster_acl.go b/cluster/cluster_acl.go
index c4e30db6f..7e6794e8e 100644
--- a/cluster/cluster_acl.go
+++ b/cluster/cluster_acl.go
@@ -482,6 +482,11 @@ func (cluster *Cluster) IsURLPassACL(strUser string, URL string) bool {
return true
}
}
+ if cluster.APIUsers[strUser].Grants[config.GrantClusterDropMonitor] {
+ if strings.Contains(URL, "/api/clusters/"+cluster.Name+"/actions/dropserver") {
+ return true
+ }
+ }
if cluster.APIUsers[strUser].Grants[config.GrantClusterSwitchover] {
if strings.Contains(URL, "/api/clusters/"+cluster.Name+"/actions/switchover") {
return true
diff --git a/cluster/cluster_chk.go b/cluster/cluster_chk.go
index 121db07ee..90d245be1 100644
--- a/cluster/cluster_chk.go
+++ b/cluster/cluster_chk.go
@@ -148,7 +148,7 @@ func (cluster *Cluster) isMasterFailed() bool {
func (cluster *Cluster) isMaxMasterFailedCountReached() bool {
// no illimited failed count
- if cluster.GetMaster().FailCount >= cluster.Conf.MaxFail {
+ if cluster.GetMaster() != nil && cluster.GetMaster().FailCount >= cluster.Conf.MaxFail {
cluster.SetState("WARN0023", state.State{ErrType: "WARNING", ErrDesc: fmt.Sprintf(clusterError["WARN0023"]), ErrFrom: "CHECK"})
return true
} else {
@@ -274,6 +274,7 @@ func (cluster *Cluster) isMaxscaleSupectRunning() bool {
}
func (cluster *Cluster) isFoundCandidateMaster() bool {
+
if cluster.GetTopology() == topoActivePassive {
return true
}
diff --git a/cluster/cluster_del.go b/cluster/cluster_del.go
index 50c9e301e..1f8f0e1ff 100644
--- a/cluster/cluster_del.go
+++ b/cluster/cluster_del.go
@@ -7,6 +7,8 @@
package cluster
import (
+ "errors"
+ "fmt"
"strings"
"github.com/signal18/replication-manager/config"
@@ -19,6 +21,33 @@ func (cluster *Cluster) RemoveServerFromIndex(index int) {
cluster.Servers = newServers
}
+func (cluster *Cluster) RemoveServerMonitor(host string, port string) error {
+ newServers := make([]*ServerMonitor, 0)
+ index := -1
+ //Find the index
+ for i, srv := range cluster.Servers {
+ //Skip the server
+ if srv.Host == host && srv.Port == port {
+ index = i
+ }
+
+ }
+
+ if index >= 0 {
+ cluster.Conf.Hosts = strings.ReplaceAll(strings.Replace(cluster.Conf.Hosts, host+":"+port, "", 1), ",,", ",")
+ cluster.StateMachine.SetFailoverState()
+ cluster.Lock()
+ newServers = append(newServers, cluster.Servers[:index]...)
+ newServers = append(newServers, cluster.Servers[index+1:]...)
+ cluster.Servers = newServers
+ cluster.Unlock()
+ cluster.StateMachine.RemoveFailoverState()
+ } else {
+ return errors.New(fmt.Sprintf("Host with address %s:%s not found in cluster!", host, port))
+ }
+ return nil
+}
+
func (cluster *Cluster) CancelRollingRestart() error {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlInfo, "API receive cancel rolling restart")
for _, pr := range cluster.Proxies {
@@ -79,3 +108,41 @@ func (cluster *Cluster) DropProxyTag(dtag string) {
cluster.SetClusterCredentialsFromConfig()
cluster.SetProxiesRestartCookie()
}
+
+func (cluster *Cluster) RemoveProxyMonitor(prx string, host string, port string) error {
+ newProxies := make([]DatabaseProxy, 0)
+ index := -1
+ for i, pr := range cluster.Proxies {
+ if pr.GetHost() == host && pr.GetPort() == port {
+ index = i
+ }
+ }
+ if index >= 0 {
+ cluster.StateMachine.SetFailoverState()
+ cluster.Lock()
+ if len(cluster.Proxies) == 1 {
+ cluster.Proxies = newProxies
+ } else {
+ newProxies = append(newProxies, cluster.Proxies[:index]...)
+ newProxies = append(newProxies, cluster.Proxies[index+1:]...)
+ cluster.Proxies = newProxies
+ }
+
+ switch prx {
+ case config.ConstProxyHaproxy:
+ cluster.Conf.HaproxyHosts = strings.ReplaceAll(strings.Replace(cluster.Conf.HaproxyHosts, host, "", 1), ",,", ",")
+ case config.ConstProxyMaxscale:
+ cluster.Conf.MxsHost = strings.ReplaceAll(strings.Replace(cluster.Conf.MxsHost, host, "", 1), ",,", ",")
+ case config.ConstProxySqlproxy:
+ cluster.Conf.ProxysqlHosts = strings.ReplaceAll(strings.Replace(cluster.Conf.ProxysqlHosts, host, "", 1), ",,", ",")
+ case config.ConstProxySpider:
+ cluster.Conf.MdbsProxyHosts = strings.ReplaceAll(strings.Replace(cluster.Conf.MdbsProxyHosts, host, "", 1), ",,", ",")
+ }
+ cluster.Unlock()
+ cluster.StateMachine.RemoveFailoverState()
+ } else {
+ return errors.New(fmt.Sprintf("Proxy host with address %s:%s not found in cluster!", host, port))
+ }
+
+ return nil
+}
diff --git a/cluster/cluster_topo.go b/cluster/cluster_topo.go
index 76105a9fe..ecb813ecd 100644
--- a/cluster/cluster_topo.go
+++ b/cluster/cluster_topo.go
@@ -259,6 +259,12 @@ func (cluster *Cluster) TopologyDiscover(wcg *sync.WaitGroup) error {
cluster.master = cluster.Servers[k]
cluster.master.SetMaster()
}
+
+ // Set master when master reconnect after become suspect
+ if cluster.master == cluster.Servers[k] && cluster.master.State == stateSuspect {
+ cluster.master.SetMaster()
+ }
+
if cluster.master.IsReadOnly() && !cluster.master.IsRelay {
cluster.master.SetReadWrite()
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModTopology, config.LvlInfo, "Server %s disable read only as last non slave", cluster.master.URL)
diff --git a/cluster/prx.go b/cluster/prx.go
index 8903122b5..7c1204d56 100644
--- a/cluster/prx.go
+++ b/cluster/prx.go
@@ -182,31 +182,35 @@ func (cluster *Cluster) newProxyList() error {
for k, proxyHost := range strings.Split(cluster.Conf.MxsHost, ",") {
prx := NewMaxscaleProxy(k, cluster, proxyHost)
cluster.AddProxy(prx)
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModProxy, config.LvlDbg, "New Maxscale proxy created: %s %s", prx.GetHost(), prx.GetPort())
}
}
- if cluster.Conf.HaproxyOn {
+ if cluster.Conf.HaproxyHosts != "" && cluster.Conf.HaproxyOn {
for k, proxyHost := range strings.Split(cluster.Conf.HaproxyHosts, ",") {
prx := NewHaproxyProxy(k, cluster, proxyHost)
cluster.AddProxy(prx)
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModProxy, config.LvlDbg, "New HA Proxy created: %s %s", prx.GetHost(), prx.GetPort())
}
}
- if cluster.Conf.ExtProxyOn {
+ if cluster.Conf.ExtProxyVIP != "" && cluster.Conf.ExtProxyOn {
for k, proxyHost := range strings.Split(cluster.Conf.ExtProxyVIP, ",") {
prx := NewExternalProxy(k, cluster, proxyHost)
cluster.AddProxy(prx)
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModProxy, config.LvlDbg, "New external proxy created: %s %s", prx.GetHost(), prx.GetPort())
}
-
}
- if cluster.Conf.ProxysqlOn {
+ if cluster.Conf.ProxysqlHosts != "" && cluster.Conf.ProxysqlOn {
for k, proxyHost := range strings.Split(cluster.Conf.ProxysqlHosts, ",") {
prx := NewProxySQLProxy(k, cluster, proxyHost)
cluster.AddProxy(prx)
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModProxy, config.LvlDbg, "New ProxySQL proxy created: %s %s", prx.GetHost(), prx.GetPort())
}
}
if cluster.Conf.ProxyJanitorHosts != "" {
for k, proxyHost := range strings.Split(cluster.Conf.ProxyJanitorHosts, ",") {
prx := NewProxyJanitor(k, cluster, proxyHost)
cluster.AddProxy(prx)
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModProxy, config.LvlDbg, "New ProxyJanitor proxy created: %s %s", prx.GetHost(), prx.GetPort())
}
}
if cluster.Conf.MdbsProxyHosts != "" && cluster.Conf.MdbsProxyOn {
@@ -301,7 +305,7 @@ func (cluster *Cluster) IsProxyEqualMaster() bool {
// if cluster.IsVerbose() {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModProxy, config.LvlInfo, "Proxy compare master: %d %d", cluster.GetMaster().ServerID, uint(sid))
// }
- if cluster.GetMaster().ServerID == uint64(sid) || pr.GetType() == config.ConstProxySpider {
+ if cluster.GetMaster() != nil && cluster.GetMaster().ServerID == uint64(sid) || pr.GetType() == config.ConstProxySpider {
return true
}
}
diff --git a/cluster/prx_maxscale.go b/cluster/prx_maxscale.go
index 49df49ceb..ebf5d5095 100644
--- a/cluster/prx_maxscale.go
+++ b/cluster/prx_maxscale.go
@@ -168,7 +168,7 @@ func (proxy *MaxscaleProxy) Init() {
if master == nil {
return
}
- if cluster.GetMaster().MxsServerName == "" {
+ if cluster.GetMaster() != nil && cluster.GetMaster().MxsServerName == "" {
return
}
diff --git a/cluster/prx_set.go b/cluster/prx_set.go
index ad37cf3fc..97a1ebda1 100644
--- a/cluster/prx_set.go
+++ b/cluster/prx_set.go
@@ -57,14 +57,15 @@ func (proxy *Proxy) SetPlacement(k int, ProvAgents string, SlapOSDBPartitions st
}
func (proxy *Proxy) SetDataDir() {
-
- proxy.Datadir = proxy.ClusterGroup.Conf.WorkingDir + "/" + proxy.ClusterGroup.Name + "/" + proxy.Host + "_" + proxy.Port
- if _, err := os.Stat(proxy.Datadir); os.IsNotExist(err) {
- os.MkdirAll(proxy.Datadir, os.ModePerm)
- os.MkdirAll(proxy.Datadir+"/log", os.ModePerm)
- os.MkdirAll(proxy.Datadir+"/var", os.ModePerm)
- os.MkdirAll(proxy.Datadir+"/init", os.ModePerm)
- os.MkdirAll(proxy.Datadir+"/bck", os.ModePerm)
+ if proxy.Host != "" {
+ proxy.Datadir = proxy.ClusterGroup.Conf.WorkingDir + "/" + proxy.ClusterGroup.Name + "/" + proxy.Host + "_" + proxy.Port
+ if _, err := os.Stat(proxy.Datadir); os.IsNotExist(err) {
+ os.MkdirAll(proxy.Datadir, os.ModePerm)
+ os.MkdirAll(proxy.Datadir+"/log", os.ModePerm)
+ os.MkdirAll(proxy.Datadir+"/var", os.ModePerm)
+ os.MkdirAll(proxy.Datadir+"/init", os.ModePerm)
+ os.MkdirAll(proxy.Datadir+"/bck", os.ModePerm)
+ }
}
}
diff --git a/cluster/srv.go b/cluster/srv.go
index 7dcca45fb..3352e6c6f 100644
--- a/cluster/srv.go
+++ b/cluster/srv.go
@@ -186,6 +186,7 @@ type ServerMonitor struct {
WorkLoad *config.WorkLoadsMap `json:"workLoad"`
DelayStat *ServerDelayStat `json:"delayStat"`
SlaveVariables SlaveVariables `json:"slaveVariables"`
+ IsReseeding bool `json:"isReseeding"`
MDevIssues ServerBug `json:"mdevIssues"`
IsCheckedForMDevIssues bool `json:"isCheckedForMdevIssues"`
IsInSlowQueryCapture bool
@@ -426,7 +427,7 @@ func (server *ServerMonitor) Ping(wg *sync.WaitGroup) {
}
if cluster.GetMaster() != nil && server.URL == cluster.GetMaster().URL && server.GetCluster().GetTopology() != topoUnknown {
server.FailSuspectHeartbeat = cluster.StateMachine.GetHeartbeats()
- if cluster.GetMaster().FailCount <= cluster.Conf.MaxFail {
+ if cluster.GetMaster() != nil && cluster.GetMaster().FailCount <= cluster.Conf.MaxFail {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "INFO", "Master Failure detected! Retry %d/%d", cluster.GetMaster().FailCount, cluster.Conf.MaxFail)
}
if server.FailCount >= cluster.Conf.MaxFail {
diff --git a/cluster/srv_job.go b/cluster/srv_job.go
index 2eeb703ae..0ee143372 100644
--- a/cluster/srv_job.go
+++ b/cluster/srv_job.go
@@ -228,6 +228,14 @@ func (server *ServerMonitor) JobReseedLogicalBackup() (int64, error) {
return 0, errors.New("No Logical Backup")
}
+ if server.IsReseeding {
+ err := errors.New("Server is in reseeding state")
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModTask, config.LvlErr, err.Error())
+ return 0, err
+ }
+
+ server.SetInReseedBackup(true)
+
//Delete wait logical backup cookie
server.DelWaitLogicalBackupCookie()
@@ -238,6 +246,7 @@ func (server *ServerMonitor) JobReseedLogicalBackup() (int64, error) {
jobid, err := server.JobInsertTaks(task, server.SSTPort, cluster.Conf.MonitorAddress)
if err != nil {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModTask, config.LvlErr, "Receive reseed logical backup %s request for server: %s %s", cluster.Conf.BackupLogicalType, server.URL, err)
+ server.SetInReseedBackup(false)
return jobid, err
} else {
dt.ct++
@@ -261,6 +270,7 @@ func (server *ServerMonitor) JobReseedLogicalBackup() (int64, error) {
}, server.DBVersion)
cluster.LogSQL(logs, err, server.URL, "Rejoin", config.LvlErr, "Reseed can't changing master for logical backup %s request for server: %s %s", cluster.Conf.BackupPhysicalType, server.URL, err)
if err != nil {
+ server.SetInReseedBackup(false)
return jobid, err
}
@@ -302,6 +312,15 @@ func (server *ServerMonitor) JobFlashbackLogicalBackup() (int64, error) {
server.SetWaitLogicalBackupCookie()
return 0, errors.New("No Logical Backup")
}
+
+ if server.IsReseeding {
+ err := errors.New("Server is in reseeding state")
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModTask, config.LvlErr, err.Error())
+ return 0, err
+ }
+
+ server.SetInReseedBackup(true)
+
if v, ok := server.ActiveTasks.Load(task); ok {
dt = v.(DBTask)
}
@@ -444,6 +463,8 @@ func (server *ServerMonitor) JobReseedMyLoader() {
cluster := server.ClusterGroup
threads := strconv.Itoa(cluster.Conf.BackupLogicalLoadThreads)
+ defer server.SetInReseedBackup(false)
+
myargs := strings.Split(strings.ReplaceAll(cluster.Conf.BackupMyLoaderOptions, " ", " "), " ")
if server.URL == cluster.GetMaster().URL {
myargs = append(myargs, "--enable-binlog")
@@ -495,6 +516,7 @@ func (server *ServerMonitor) JobReseedMyLoader() {
func (server *ServerMonitor) JobReseedMysqldump(task string) {
cluster := server.ClusterGroup
master := cluster.GetMaster()
+ defer server.SetInReseedBackup(false)
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModTask, config.LvlInfo, "Sending logical backup to reseed %s", server.URL)
if master != nil {
@@ -560,6 +582,8 @@ func (server *ServerMonitor) JobReseedMysqldump(task string) {
func (server *ServerMonitor) JobReseedBackupScript() {
cluster := server.ClusterGroup
+ defer server.SetInReseedBackup(false)
+
cmd := exec.Command(cluster.Conf.BackupLoadScript, misc.Unbracket(server.Host), misc.Unbracket(cluster.master.Host))
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModTask, config.LvlInfo, "Command backup load script: %s", strings.Replace(cmd.String(), cluster.GetDbPass(), "XXXX", 1))
@@ -937,17 +961,18 @@ func (server *ServerMonitor) JobBackupMyDumper() error {
dumpCmd.Start()
var wg sync.WaitGroup
+ var valid bool = true
wg.Add(2)
go func() {
defer wg.Done()
- server.copyLogs(stdoutIn, config.ConstLogModBackupStream, config.LvlDbg)
+ server.myDumperCopyLogs(stdoutIn, config.ConstLogModBackupStream, config.LvlDbg)
}()
go func() {
defer wg.Done()
- server.copyLogs(stderrIn, config.ConstLogModBackupStream, config.LvlDbg)
+ valid = server.myDumperCopyLogs(stderrIn, config.ConstLogModBackupStream, config.LvlDbg)
}()
wg.Wait()
- if err = dumpCmd.Wait(); err != nil {
+ if err = dumpCmd.Wait(); err != nil && !valid {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModTask, config.LvlErr, "MyDumper: %s", err)
} else {
server.SetBackupLogicalCookie()
@@ -1082,6 +1107,29 @@ func (server *ServerMonitor) copyLogs(r io.Reader, module int, level string) {
}
}
+func (server *ServerMonitor) myDumperCopyLogs(r io.Reader, module int, level string) bool {
+ cluster := server.ClusterGroup
+ valid := true
+ // buf := make([]byte, 1024)
+ s := bufio.NewScanner(r)
+ for {
+ if !s.Scan() {
+ break
+ } else {
+ stream := s.Text()
+ if strings.Contains(stream, "Error") {
+ if !strings.Contains(stream, "#mysql50#") {
+ valid = false
+ }
+ cluster.LogModulePrintf(cluster.Conf.Verbose, module, config.LvlErr, "[%s] %s", server.Name, stream)
+ } else {
+ cluster.LogModulePrintf(cluster.Conf.Verbose, module, level, "[%s] %s", server.Name, stream)
+ }
+ }
+ }
+ return valid
+}
+
func (server *ServerMonitor) BackupRestic(tags ...string) error {
cluster := server.ClusterGroup
var stdout, stderr []byte
@@ -1218,11 +1266,16 @@ func (server *ServerMonitor) JobRunViaSSH() error {
res := new(JobResult)
val := reflect.ValueOf(res).Elem()
for i := 0; i < val.NumField(); i++ {
- if strings.Contains(strings.ToLower(string(out)), strings.ToLower("no "+val.Type().Field(i).Name)) {
+ jobname := val.Type().Field(i).Name
+ if strings.Contains(strings.ToLower(string(out)), strings.ToLower("no "+jobname)) {
val.Field(i).SetBool(false)
} else {
val.Field(i).SetBool(true)
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModTask, config.LvlInfo, "Database jobs run via SSH: %s", val.Type().Field(i).Name)
+ lower := strings.ToLower(jobname)
+ if strings.HasPrefix(lower, "reseed") || strings.HasPrefix(lower, "flashback") {
+ server.SetInReseedBackup(false)
+ }
}
}
@@ -1251,6 +1304,11 @@ func (server *ServerMonitor) JobBackupBinlog(binlogfile string, isPurge bool) er
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModPurge, config.LvlDbg, "%s", err.Error())
return err
}
+ if server.IsReseeding {
+ err = errors.New("Cancel job copy binlog during reseed")
+ cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModPurge, config.LvlDbg, "%s", err.Error())
+ return err
+ }
//Skip setting in backup state due to batch purging
if !isPurge {
@@ -1402,6 +1460,7 @@ func (cluster *Cluster) CreateTmpClientConfFile() (string, error) {
}
func (cluster *Cluster) JobRejoinMysqldumpFromSource(source *ServerMonitor, dest *ServerMonitor) error {
+ defer dest.SetInReseedBackup(false)
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModTask, config.LvlInfo, "Rejoining from direct mysqldump from %s", source.URL)
dest.StopSlave()
usegtid := dest.JobGetDumpGtidParameter()
diff --git a/cluster/srv_rejoin.go b/cluster/srv_rejoin.go
index e7ab160d0..ab7c5c5c2 100644
--- a/cluster/srv_rejoin.go
+++ b/cluster/srv_rejoin.go
@@ -339,12 +339,19 @@ func (server *ServerMonitor) RejoinDirectDump() error {
cluster := server.ClusterGroup
var err3 error
+ if server.IsReseeding {
+ return errors.New("Server is in reseeding state")
+ }
+
+ server.SetInReseedBackup(true)
+
realmaster := cluster.master
if cluster.Conf.MxsBinlogOn || cluster.Conf.MultiTierSlave {
realmaster = cluster.GetRelayServer()
}
if realmaster == nil {
+ server.SetInReseedBackup(false)
return errors.New("No master defined exiting rejoin direct dump ")
}
// done change master just to set the host and port before dump
@@ -369,6 +376,7 @@ func (server *ServerMonitor) RejoinDirectDump() error {
cluster.LogSQL(logs, err3, server.URL, "Rejoin", config.LvlErr, "Failed change master maxscale on %s: %s", server.URL, err3)
}
if err3 != nil {
+ server.SetInReseedBackup(false)
return err3
}
// dump here
diff --git a/cluster/srv_set.go b/cluster/srv_set.go
index b247c4bdc..ed1c34c87 100644
--- a/cluster/srv_set.go
+++ b/cluster/srv_set.go
@@ -444,3 +444,7 @@ func (server *ServerMonitor) SetInCaptureMode(value bool) {
func (server *ServerMonitor) SetInRefreshBinlog(value bool) {
server.IsRefreshingBinlog = value
}
+
+func (server *ServerMonitor) SetInReseedBackup(value bool) {
+ server.IsReseeding = value
+}
diff --git a/regtest/test_failover_assync_autorejoin_flashback.go b/regtest/test_failover_assync_autorejoin_flashback.go
index 95a52e403..b798df9f1 100644
--- a/regtest/test_failover_assync_autorejoin_flashback.go
+++ b/regtest/test_failover_assync_autorejoin_flashback.go
@@ -34,7 +34,7 @@ func (regtest *RegTest) TestFailoverAssyncAutoRejoinFlashback(cluster *cluster.C
wg.Wait()
/// give time to start the failover
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", " Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_failover_assync_autorejoin_mysqldump.go b/regtest/test_failover_assync_autorejoin_mysqldump.go
index 0499aad8c..6fe3753bb 100644
--- a/regtest/test_failover_assync_autorejoin_mysqldump.go
+++ b/regtest/test_failover_assync_autorejoin_mysqldump.go
@@ -38,7 +38,7 @@ func (regtest *RegTest) TestFailoverAssyncAutoRejoinDump(cluster *cluster.Cluste
wg.Wait()
/// give time to start the failover
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", " Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_failover_assync_autorejoin_nogtid.go b/regtest/test_failover_assync_autorejoin_nogtid.go
index edaabf989..304494162 100644
--- a/regtest/test_failover_assync_autorejoin_nogtid.go
+++ b/regtest/test_failover_assync_autorejoin_nogtid.go
@@ -33,7 +33,7 @@ func (regtest *RegTest) TestFailoverAssyncAutoRejoinNoGtid(cluster *cluster.Clus
cluster.FailoverAndWait()
/// give time to start the failover
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", " Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_failover_assync_autorejoin_nowrite.go b/regtest/test_failover_assync_autorejoin_nowrite.go
index 26d1b5d92..f68cbd4c1 100644
--- a/regtest/test_failover_assync_autorejoin_nowrite.go
+++ b/regtest/test_failover_assync_autorejoin_nowrite.go
@@ -34,7 +34,7 @@ func (regtest *RegTest) TestFailoverAssyncAutoRejoinNowrites(cluster *cluster.Cl
wg.Wait()
/// give time to start the failover
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", " Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_failover_assync_autorejoin_relay.go b/regtest/test_failover_assync_autorejoin_relay.go
index 1c47f1119..8637005f3 100644
--- a/regtest/test_failover_assync_autorejoin_relay.go
+++ b/regtest/test_failover_assync_autorejoin_relay.go
@@ -35,7 +35,7 @@ func (regtest *RegTest) TestFailoverAssyncAutoRejoinRelay(cluster *cluster.Clust
wg.Wait()
/// give time to start the failover
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", " Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_failover_assync_countfailoverreach.go b/regtest/test_failover_assync_countfailoverreach.go
index 10c183dc1..2e1a9b9d3 100644
--- a/regtest/test_failover_assync_countfailoverreach.go
+++ b/regtest/test_failover_assync_countfailoverreach.go
@@ -37,7 +37,7 @@ func (regtest *RegTest) TestFailoverNumberFailureLimitReach(cluster *cluster.Clu
cluster.WaitFailoverEnd()
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "New Master %s ", cluster.GetMaster().URL)
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
SaveMaster.FailCount = 0
diff --git a/regtest/test_failover_assync_norplchecks.go b/regtest/test_failover_assync_norplchecks.go
index 629232a0c..f4df71e5b 100644
--- a/regtest/test_failover_assync_norplchecks.go
+++ b/regtest/test_failover_assync_norplchecks.go
@@ -27,7 +27,7 @@ func (regtest *RegTest) TestFailoverNoRplChecksNoSemiSync(cluster *cluster.Clust
cluster.SetRplChecks(false)
cluster.FailoverAndWait()
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "New Master %s ", cluster.GetMaster().URL)
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_failover_assync_norplchecks_masterhearbeat.go b/regtest/test_failover_assync_norplchecks_masterhearbeat.go
index e8cc6a97a..4dd3b4994 100644
--- a/regtest/test_failover_assync_norplchecks_masterhearbeat.go
+++ b/regtest/test_failover_assync_norplchecks_masterhearbeat.go
@@ -32,7 +32,7 @@ func (regtest *RegTest) TestFailoverNoRplChecksNoSemiSyncMasterHeartbeat(cluster
cluster.CheckFailed()
cluster.FailoverNow()
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "New Master %s ", cluster.GetMaster().URL)
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_failover_assync_slavesdelay_norplchecks.go b/regtest/test_failover_assync_slavesdelay_norplchecks.go
index 3e3eaeda9..234cc37b9 100644
--- a/regtest/test_failover_assync_slavesdelay_norplchecks.go
+++ b/regtest/test_failover_assync_slavesdelay_norplchecks.go
@@ -33,7 +33,7 @@ func (regtest *RegTest) TestFailoverAllSlavesDelayNoRplChecksNoSemiSync(cluster
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "New Master %s ", cluster.GetMaster().URL)
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Old master %s == New master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_failover_assync_slavesdelay_rplchecks.go b/regtest/test_failover_assync_slavesdelay_rplchecks.go
index f2cda6b12..6ee07cd51 100644
--- a/regtest/test_failover_assync_slavesdelay_rplchecks.go
+++ b/regtest/test_failover_assync_slavesdelay_rplchecks.go
@@ -29,7 +29,7 @@ func (regtest *RegTest) TestFailoverAllSlavesDelayRplChecksNoSemiSync(cluster *c
cluster.DelayAllSlaves()
cluster.FailoverNow()
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", " New Master %s ", cluster.GetMaster().URL)
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Old master %s != New master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_failover_assync_timenotreach.go b/regtest/test_failover_assync_timenotreach.go
index 62cda2cb9..ab6470860 100644
--- a/regtest/test_failover_assync_timenotreach.go
+++ b/regtest/test_failover_assync_timenotreach.go
@@ -36,7 +36,7 @@ func (regtest *RegTest) TestFailoverTimeNotReach(cluster *cluster.Cluster, conf
time.Sleep(4 * time.Second)
cluster.FailoverAndWait()
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "New Master %s ", cluster.GetMaster().URL)
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_failover_manual.go b/regtest/test_failover_manual.go
index c0d2f60fc..77f02abe6 100644
--- a/regtest/test_failover_manual.go
+++ b/regtest/test_failover_manual.go
@@ -19,7 +19,7 @@ func (regtest *RegTest) TestFailoverManual(cluster *cluster.Cluster, conf string
SaveMaster := cluster.GetMaster()
SaveMasterURL := SaveMaster.URL
cluster.FailoverNow()
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", " Old master %s != Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_failover_semisync_MSS_XMS_XXM_SXM_SSM.go b/regtest/test_failover_semisync_MSS_XMS_XXM_SXM_SSM.go
index 13a033c7c..96e24f59c 100644
--- a/regtest/test_failover_semisync_MSS_XMS_XXM_SXM_SSM.go
+++ b/regtest/test_failover_semisync_MSS_XMS_XXM_SXM_SSM.go
@@ -37,7 +37,7 @@ func (regtest *RegTest) TestFailoverSemisyncAutoRejoinMSSXMSXXMSXMSSM(cluster *c
SaveMaster2 := cluster.GetMaster()
cluster.RunBench()
cluster.FailoverAndWait()
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_failover_semisync_MSS_XMS_XXM_XSM_SSM.go b/regtest/test_failover_semisync_MSS_XMS_XXM_XSM_SSM.go
index d96c6f54e..225759602 100644
--- a/regtest/test_failover_semisync_MSS_XMS_XXM_XSM_SSM.go
+++ b/regtest/test_failover_semisync_MSS_XMS_XXM_XSM_SSM.go
@@ -37,7 +37,7 @@ func (regtest *RegTest) TestFailoverSemisyncAutoRejoinMSSXMSXXMXSMSSM(cluster *c
SaveMaster2 := cluster.GetMaster()
cluster.RunBench()
cluster.FailoverAndWait()
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_failover_semisync_autorejoin_flashback.go b/regtest/test_failover_semisync_autorejoin_flashback.go
index eaf93981d..fb3703892 100644
--- a/regtest/test_failover_semisync_autorejoin_flashback.go
+++ b/regtest/test_failover_semisync_autorejoin_flashback.go
@@ -33,7 +33,7 @@ func (regtest *RegTest) TestFailoverSemisyncAutoRejoinFlashback(cluster *cluster
cluster.FailoverAndWait()
/// give time to start the failover
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_failover_semisync_autorejoin_safe_MS_XM_SM.go b/regtest/test_failover_semisync_autorejoin_safe_MS_XM_SM.go
index cf7027612..640c141ec 100644
--- a/regtest/test_failover_semisync_autorejoin_safe_MS_XM_SM.go
+++ b/regtest/test_failover_semisync_autorejoin_safe_MS_XM_SM.go
@@ -42,7 +42,7 @@ func (regtest *RegTest) TestFailoverSemisyncAutoRejoinSafeMSXMSM(cluster *cluste
cluster.RunBench()
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_SM.go b/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_SM.go
index 63559cd70..53716d46c 100644
--- a/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_SM.go
+++ b/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_SM.go
@@ -42,7 +42,7 @@ func (regtest *RegTest) TestFailoverSemisyncAutoRejoinUnsafeMSXMSM(cluster *clus
cluster.RunBench()
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_XX_MX.go b/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_XX_MX.go
index ded606c9b..f235d67cc 100644
--- a/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_XX_MX.go
+++ b/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_XX_MX.go
@@ -48,7 +48,7 @@ func (regtest *RegTest) TestFailoverSemisyncAutoRejoinUnsafeMSXMXXMXMS(cluster *
cluster.StopDatabaseService(cluster.GetMaster())
wg.Wait()
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_XX_XM.go b/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_XX_XM.go
index c62b9153b..a901cc556 100644
--- a/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_XX_XM.go
+++ b/regtest/test_failover_semisync_autorejoin_unsafe_MS_XM_XX_XM.go
@@ -48,7 +48,7 @@ func (regtest *RegTest) TestFailoverSemisyncAutoRejoinUnsafeMSXMXXXMSM(cluster *
cluster.StopDatabaseService(cluster.GetMaster())
wg.Wait()
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_failover_semisync_slavekilled_autorejoin.go b/regtest/test_failover_semisync_slavekilled_autorejoin.go
index 037b5c23a..201f48dae 100644
--- a/regtest/test_failover_semisync_slavekilled_autorejoin.go
+++ b/regtest/test_failover_semisync_slavekilled_autorejoin.go
@@ -32,7 +32,7 @@ func (regtest *RegTest) TestFailoverSemisyncSlavekilledAutoRejoin(cluster *clust
time.Sleep(5 * time.Second)
cluster.FailoverAndWait()
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "Old master %s == Next master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
diff --git a/regtest/test_switchover_assync_longquery_norplcheck.go b/regtest/test_switchover_assync_longquery_norplcheck.go
index 811c55736..3bd2a86cc 100644
--- a/regtest/test_switchover_assync_longquery_norplcheck.go
+++ b/regtest/test_switchover_assync_longquery_norplcheck.go
@@ -38,7 +38,7 @@ func (regtest *RegTest) TestSwitchoverLongQueryNoRplCheckNoSemiSync(cluster *clu
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "%s", err)
return false
}
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Saved Prefered master %s <> from saved %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_switchover_assync_longtrx_nocommit_norplcheck.go b/regtest/test_switchover_assync_longtrx_nocommit_norplcheck.go
index 968aba71d..bceaaf0ae 100644
--- a/regtest/test_switchover_assync_longtrx_nocommit_norplcheck.go
+++ b/regtest/test_switchover_assync_longtrx_nocommit_norplcheck.go
@@ -41,7 +41,7 @@ func (regtest *RegTest) TestSwitchoverLongTrxWithoutCommitNoRplCheckNoSemiSync(c
return false
}
time.Sleep(2 * time.Second)
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Saved Prefered master %s <> from saved %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_switchover_assync_longtrx_norplcheck.go b/regtest/test_switchover_assync_longtrx_norplcheck.go
index c1e77fb5b..4bfa76d48 100644
--- a/regtest/test_switchover_assync_longtrx_norplcheck.go
+++ b/regtest/test_switchover_assync_longtrx_norplcheck.go
@@ -44,7 +44,7 @@ func (regtest *RegTest) TestSwitchoverLongTransactionNoRplCheckNoSemiSync(cluste
return false
}
time.Sleep(2 * time.Second)
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Saved master %s <> from master %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_switchover_assync_multimaster_slavesdelay_norplchecks.go b/regtest/test_switchover_assync_multimaster_slavesdelay_norplchecks.go
index b7ecd64f4..4ed616f79 100644
--- a/regtest/test_switchover_assync_multimaster_slavesdelay_norplchecks.go
+++ b/regtest/test_switchover_assync_multimaster_slavesdelay_norplchecks.go
@@ -43,7 +43,7 @@ func (regtest *RegTest) TestSwitchoverAllSlavesDelayMultimasterNoRplChecksNoSemi
return false
}
time.Sleep(2 * time.Second)
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Saved Prefered master %s <> from saved %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_switchover_assync_slavesdelay_norplchecks.go b/regtest/test_switchover_assync_slavesdelay_norplchecks.go
index bdd6d6e3f..9c9aca425 100644
--- a/regtest/test_switchover_assync_slavesdelay_norplchecks.go
+++ b/regtest/test_switchover_assync_slavesdelay_norplchecks.go
@@ -44,7 +44,7 @@ func (regtest *RegTest) TestSwitchoverAllSlavesDelayNoRplChecksNoSemiSync(cluste
return false
}
time.Sleep(2 * time.Second)
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Saved Prefered master %s <> from saved %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_switchover_assync_slavesdelay_rplcheck.go b/regtest/test_switchover_assync_slavesdelay_rplcheck.go
index 4b3fc432f..c49d9de9a 100644
--- a/regtest/test_switchover_assync_slavesdelay_rplcheck.go
+++ b/regtest/test_switchover_assync_slavesdelay_rplcheck.go
@@ -38,7 +38,7 @@ func (regtest *RegTest) TestSwitchoverAllSlavesDelayRplCheckNoSemiSync(cluster *
return false
}
time.Sleep(2 * time.Second)
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Saved Prefered master %s <> from saved %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_switchover_assync_slavesstop_norplcheck.go b/regtest/test_switchover_assync_slavesstop_norplcheck.go
index 7ba9a32bf..85507e999 100644
--- a/regtest/test_switchover_assync_slavesstop_norplcheck.go
+++ b/regtest/test_switchover_assync_slavesstop_norplcheck.go
@@ -34,7 +34,7 @@ func (regtest *RegTest) TestSwitchoverAllSlavesStopNoSemiSyncNoRplCheck(cluster
cluster.SwitchoverWaitTest()
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "New Master %s ", cluster.GetMaster().URL)
time.Sleep(2 * time.Second)
- if cluster.GetMaster().URL == SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL == SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Saved Prefered master %s <> from saved %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_switchover_assync_slavesstop_rplcheck.go b/regtest/test_switchover_assync_slavesstop_rplcheck.go
index d8cec9c91..7ca485898 100644
--- a/regtest/test_switchover_assync_slavesstop_rplcheck.go
+++ b/regtest/test_switchover_assync_slavesstop_rplcheck.go
@@ -43,7 +43,7 @@ func (regtest *RegTest) TestSwitchoverAllSlavesStopRplCheckNoSemiSync(cluster *c
return false
}
time.Sleep(2 * time.Second)
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Saved Prefered master %s <> from saved %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/regtest/test_switchover_semisync_switchback_prefmaster_norplcheck.go b/regtest/test_switchover_semisync_switchback_prefmaster_norplcheck.go
index db3b1f22a..f8a04e641 100644
--- a/regtest/test_switchover_semisync_switchback_prefmaster_norplcheck.go
+++ b/regtest/test_switchover_semisync_switchback_prefmaster_norplcheck.go
@@ -31,7 +31,7 @@ func (regtest *RegTest) TestSwitchoverBackPreferedMasterNoRplCheckSemiSync(clust
cluster.SwitchoverWaitTest()
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, "TEST", "New Master %s ", cluster.GetMaster().URL)
}
- if cluster.GetMaster().URL != SaveMasterURL {
+ if cluster.GetMaster() != nil && cluster.GetMaster().URL != SaveMasterURL {
cluster.LogModulePrintf(cluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlErr, "Saved Prefered master %s <> from saved %s ", SaveMasterURL, cluster.GetMaster().URL)
return false
}
diff --git a/server/api_cluster.go b/server/api_cluster.go
index d7c4c11da..07e86010e 100644
--- a/server/api_cluster.go
+++ b/server/api_cluster.go
@@ -202,6 +202,16 @@ func (repman *ReplicationManager) apiClusterProtectedHandler(router *mux.Router)
negroni.Wrap(http.HandlerFunc(repman.handlerMuxServerAdd)),
))
+ router.Handle("/api/clusters/{clusterName}/actions/dropserver/{host}/{port}", negroni.New(
+ negroni.HandlerFunc(repman.validateTokenMiddleware),
+ negroni.Wrap(http.HandlerFunc(repman.handlerMuxServerDrop)),
+ ))
+
+ router.Handle("/api/clusters/{clusterName}/actions/dropserver/{host}/{port}/{type}", negroni.New(
+ negroni.HandlerFunc(repman.validateTokenMiddleware),
+ negroni.Wrap(http.HandlerFunc(repman.handlerMuxServerDrop)),
+ ))
+
router.Handle("/api/clusters/{clusterName}/actions/rolling", negroni.New(
negroni.HandlerFunc(repman.validateTokenMiddleware),
negroni.Wrap(http.HandlerFunc(repman.handlerMuxRolling)),
@@ -1730,6 +1740,33 @@ func (repman *ReplicationManager) handlerMuxServerAdd(w http.ResponseWriter, r *
}
+func (repman *ReplicationManager) handlerMuxServerDrop(w http.ResponseWriter, r *http.Request) {
+ fmt.Printf("HANDLER MUX SERVER ADD\n")
+ w.Header().Set("Access-Control-Allow-Origin", "*")
+ vars := mux.Vars(r)
+ mycluster := repman.getClusterByName(vars["clusterName"])
+ if mycluster != nil {
+ if !repman.IsValidClusterACL(r, mycluster) {
+ http.Error(w, "No valid ACL", 403)
+ return
+ }
+ mycluster.LogModulePrintf(mycluster.Conf.Verbose, config.ConstLogModGeneral, config.LvlInfo, "Rest API receive drop %s monitor command for %s", vars["type"], vars["host"]+":"+vars["port"])
+ if vars["type"] == "" {
+ mycluster.RemoveServerMonitor(vars["host"], vars["port"])
+ } else {
+ if mycluster.MonitorType[vars["type"]] == "proxy" {
+ mycluster.RemoveProxyMonitor(vars["type"], vars["host"], vars["port"])
+ } else if mycluster.MonitorType[vars["type"]] == "database" {
+ mycluster.RemoveServerMonitor(vars["host"], vars["port"])
+ }
+ }
+ } else {
+ http.Error(w, "Cluster Not Found", 500)
+ return
+ }
+
+}
+
// swagger:operation GET /api/clusters/{clusterName}/status clusterStatus
// Shows the status for that specific named cluster
//
diff --git a/server/api_database.go b/server/api_database.go
index d907eadcb..fb43c0d82 100644
--- a/server/api_database.go
+++ b/server/api_database.go
@@ -563,10 +563,16 @@ func (repman *ReplicationManager) handlerMuxServerReseed(w http.ResponseWriter,
node := mycluster.GetServerFromName(vars["serverName"])
if node != nil {
if vars["backupMethod"] == "logicalbackup" {
- node.JobReseedLogicalBackup()
+ _, err := node.JobReseedLogicalBackup()
+ if err != nil {
+ mycluster.LogModulePrintf(mycluster.Conf.Verbose, config.ConstLogModGeneral, "ERROR", "logical flashback restore failed %s", err)
+ }
}
if vars["backupMethod"] == "logicalmaster" {
- node.RejoinDirectDump()
+ err := node.RejoinDirectDump()
+ if err != nil {
+ mycluster.LogModulePrintf(mycluster.Conf.Verbose, config.ConstLogModGeneral, "ERROR", "mysqldump flashback restore failed %s", err)
+ }
}
if vars["backupMethod"] == "physicalbackup" {
node.JobReseedPhysicalBackup()
diff --git a/share/dashboard/app/dashboard.js b/share/dashboard/app/dashboard.js
index 9843e2605..1c954a194 100644
--- a/share/dashboard/app/dashboard.js
+++ b/share/dashboard/app/dashboard.js
@@ -1245,6 +1245,14 @@ app.controller('DashboardController', function (
if (confirm("Unprovision operation will destroy your existing data. \n Are you really sure?")) httpGetWithoutResponse(getClusterUrl() + '/services/actions/unprovision');
};
+ $scope.dropmonitor = function(host, port, type="") {
+ let url = getClusterUrl() + '/actions/dropserver/' + host + '/' + port
+ if (type != "") {
+ url = url + "/" + type
+ }
+ if (confirm("Confirm remove "+ type +"monitor on server " + host + ":" + port)) httpGetWithoutResponse(url);
+ }
+
$scope.clusterRotateCredentials = function () {
if (confirm("Rotate database and replication monitoring user credentials. \n Are you really sure?")) httpGetWithoutResponse(getClusterUrl() + '/actions/rotate-passwords');
};
diff --git a/share/dashboard/static/menu-proxy.html b/share/dashboard/static/menu-proxy.html
index 0b5bdf023..ecdbad7cc 100644
--- a/share/dashboard/static/menu-proxy.html
+++ b/share/dashboard/static/menu-proxy.html
@@ -40,5 +40,11 @@
+
+
+
+ Remove From Monitor
+
+
diff --git a/share/dashboard/static/menu-server.html b/share/dashboard/static/menu-server.html
index a7867f6f8..8b8cc0500 100644
--- a/share/dashboard/static/menu-server.html
+++ b/share/dashboard/static/menu-server.html
@@ -56,37 +56,37 @@
-
+
Physical Backup
-
+
Logical Backup
-
+
Reseed Logical From Backup
-
+
Reseed Logical From Master
-
+
Reseed Physical From Backup
-
+
Flush logs
@@ -201,5 +201,11 @@
+
+
+
+ Remove Monitor
+
+