From 771dc7e151a231d30d1a96c5c727fb85bc4c922a Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Wed, 12 Feb 2025 20:30:56 +0530 Subject: [PATCH 01/27] feat: add the first iteration of semi-sync watcher Signed-off-by: Manan Gupta --- .../vttablet/tabletmanager/rpc_replication.go | 2 + .../tabletserver/semisyncwatcher/watcher.go | 221 ++++++++++++++++++ 2 files changed, 223 insertions(+) create mode 100644 go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go diff --git a/go/vt/vttablet/tabletmanager/rpc_replication.go b/go/vt/vttablet/tabletmanager/rpc_replication.go index 7ac37515b67..f2cffb322e8 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication.go @@ -589,6 +589,8 @@ func (tm *TabletManager) demotePrimary(ctx context.Context, revertPartialFailure // set MySQL to super_read_only mode. If we are already super_read_only because of a // previous demotion, or because we are not primary anyway, this should be // idempotent. + + // TODO(@GuptaManan100): Reject PR if not done. Check we have no writes blocked on semi-sync. if _, err := tm.MysqlDaemon.SetSuperReadOnly(ctx, true); err != nil { if sqlErr, ok := err.(*sqlerror.SQLError); ok && sqlErr.Number() == sqlerror.ERUnknownSystemVariable { log.Warningf("server does not know about super_read_only, continuing anyway...") diff --git a/go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go b/go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go new file mode 100644 index 00000000000..b27ebc3e06a --- /dev/null +++ b/go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go @@ -0,0 +1,221 @@ +/* +Copyright 2025 The Vitess Authors. + +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, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package semisyncwatcher + +import ( + "context" + "errors" + "math" + "sync" + "time" + + "vitess.io/vitess/go/timer" + "vitess.io/vitess/go/vt/dbconnpool" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/mysqlctl" + "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" +) + +const ( + semiSyncWaitSessionsRead = "SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'" +) + +// Watcher is a watcher that checks if the primary tablet +// is blocked on a semi-sync ack from the replica. +// If the semi-sync ACK is lost in the network, +// it is possible that the primary is indefinitely stuck, +// blocking PRS. The watcher looks for this situation and manufactures a write +// periodically to unblock the primary. +type Watcher struct { + ticks *timer.Timer + env tabletenv.Env + + mu sync.Mutex + appPool *dbconnpool.ConnectionPool + isOpen bool + isWriting bool + isBlocked bool +} + +// NewWatcher creates a new Watcher. +func NewWatcher(env tabletenv.Env) *Watcher { + // TODO (@GuptaManan100): Parameterize the watch interval. + watchInterval := 30 * time.Second + return &Watcher{ + env: env, + ticks: timer.NewTimer(watchInterval), + appPool: dbconnpool.NewConnectionPool("SemiSyncWatcherAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), + } +} + +// Open starts the watcher. +func (w *Watcher) Open() { + w.mu.Lock() + defer w.mu.Unlock() + if w.isOpen { + // If we are already open, then there is nothing to do + return + } + // Set the watcher to be open. + w.isOpen = true + // We reset the state when the watcher starts. + w.isBlocked = false + log.Info("SemiSync Watcher: opening") + + // This function could be running from within a unit test scope, in which case we use + // mock pools that are already open. This is why we test for the pool being open. + if !w.appPool.IsOpen() { + w.appPool.Open(w.env.Config().DB.AppWithDB()) + } + w.ticks.Start(w.checkAndFixSemiSyncBlocked) +} + +// Close stops the watcher. +func (w *Watcher) Close() { + w.mu.Lock() + defer w.mu.Unlock() + if !w.isOpen { + // If we are already closed, then there is nothing to do + return + } + w.isOpen = false + log.Info("SemiSync Watcher: closing") + w.ticks.Stop() + w.appPool.Close() +} + +// checkAndFixSemiSyncBlocked checks if the primary is blocked on semi-sync ack +// and manufactures a write to unblock the primary. +func (w *Watcher) checkAndFixSemiSyncBlocked() { + // Check if semi-sync is blocked or not + isBlocked, err := w.isSemiSyncBlocked(context.Background()) + if err != nil { + // If we are unable to determine whether the primary is blocked or not, + // then we can just abort the function and try again later. + log.Errorf("SemiSync Watcher: failed to check if primary is blocked on semi-sync: %v", err) + return + } + // Set the isBlocked state. + w.setIsBlocked(isBlocked) + if isBlocked { + // If we are blocked, then we want to start the writes. + // That function is re-entrant. If we are already writing, then it will just return. + w.startWrites() + } +} + +// isSemiSyncBlocked checks if the primary is blocked on semi-sync. +func (w *Watcher) isSemiSyncBlocked(ctx context.Context) (bool, error) { + // Get a connection from the pool + conn, err := w.appPool.Get(ctx) + if err != nil { + return false, err + } + defer conn.Recycle() + + // Execute the query to check if the primary is blocked on semi-sync. + res, err := conn.Conn.ExecuteFetch(semiSyncWaitSessionsRead, 1, false) + if err != nil { + return false, err + } + // If we have no rows, then the primary doesn't have semi-sync enabled. + // It then follows, that the primary isn't blocked :) + if len(res.Rows) == 0 { + return false, nil + } + + // Read the status value and check if it is non-zero. + if len(res.Rows) != 1 || len(res.Rows[0]) != 2 { + return false, errors.New("unexpected number of rows received") + } + value, err := res.Rows[0][1].ToInt() + return value != 0, err +} + +// waitUntilSemiSyncUnblocked waits until the primary is not blocked +// on semi-sync. +func (w *Watcher) waitUntilSemiSyncUnblocked() { + w.checkAndFixSemiSyncBlocked() + // TODO: Complete the function. +} + +// continueWrites returns true if the watcher should continue writing to the DB. +// It checks if the watcher is still open and if the primary is still blocked. +func (w *Watcher) continueWrites() bool { + w.mu.Lock() + defer w.mu.Unlock() + return w.isOpen && w.isBlocked +} + +// checkAndSetIsWriting checks if the watcher is already writing to the DB. +// If it is not, then it sets the isWriting field and signals the caller. +func (w *Watcher) checkAndSetIsWriting() bool { + w.mu.Lock() + defer w.mu.Unlock() + if w.isWriting { + return false + } + w.isWriting = true + return true +} + +// clearIsWriting clears the isWriting field. +func (w *Watcher) clearIsWriting() { + w.mu.Lock() + defer w.mu.Unlock() + w.isWriting = false +} + +// startWrites starts writing to the DB. +// It is re-entrant and will return if we are already writing. +func (w *Watcher) startWrites() { + // If we are already writing, then we can just return. + if !w.checkAndSetIsWriting() { + return + } + // We defer the clear of the isWriting field. + defer w.clearIsWriting() + + // We start writing to the DB with a backoff. + backoff := 1 * time.Second + maxBackoff := 1 * time.Minute + // Check if we need to continue writing or not. + for !w.continueWrites() { + go w.write() + <-time.After(backoff) + backoff = time.Duration(math.Min(float64(backoff*2), float64(maxBackoff))) + } +} + +// write writes a heartbeat to unblock semi-sync being stuck. +func (w *Watcher) write() { + // Get a connection from the pool + conn, err := w.appPool.Get(context.Background()) + if err != nil { + return + } + defer conn.Recycle() + // TODO: Fix the write in question. + _, _ = conn.Conn.ExecuteFetch("INSERT INTO heartbeat (timestamp) VALUES (NOW())", 1, false) +} + +// setIsBlocked sets the isBlocked field. +func (w *Watcher) setIsBlocked(val bool) { + w.mu.Lock() + defer w.mu.Unlock() + w.isBlocked = val +} From 4dcaf5422679cdcc712d0a9c5c596e7f5db22ee4 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Thu, 13 Feb 2025 16:07:20 +0530 Subject: [PATCH 02/27] feat: add semisync recovery sql and other missing pieces in the code Signed-off-by: Manan Gupta --- .../schema/misc/semisync_recover.sql | 21 ++++ .../tabletserver/semisyncwatcher/watcher.go | 95 +++++++++++++++---- 2 files changed, 99 insertions(+), 17 deletions(-) create mode 100644 go/vt/sidecardb/schema/misc/semisync_recover.sql diff --git a/go/vt/sidecardb/schema/misc/semisync_recover.sql b/go/vt/sidecardb/schema/misc/semisync_recover.sql new file mode 100644 index 00000000000..7f6f0a55ade --- /dev/null +++ b/go/vt/sidecardb/schema/misc/semisync_recover.sql @@ -0,0 +1,21 @@ +/* +Copyright 2025 The Vitess Authors. + +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, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +CREATE TABLE IF NOT EXISTS semisync_recover +( + ts BIGINT UNSIGNED NOT NULL, + PRIMARY KEY (`ts`) +) ENGINE = InnoDB CHARSET = utf8mb4 diff --git a/go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go b/go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go index b27ebc3e06a..dbf72e47733 100644 --- a/go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go +++ b/go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go @@ -32,6 +32,8 @@ import ( const ( semiSyncWaitSessionsRead = "SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'" + semiSyncRecoverWrite = "INSERT INTO semisync_recover (ts) VALUES (NOW())" + semiSyncRecoverClear = "DELETE FROM semisync_recover" ) // Watcher is a watcher that checks if the primary tablet @@ -41,14 +43,27 @@ const ( // blocking PRS. The watcher looks for this situation and manufactures a write // periodically to unblock the primary. type Watcher struct { + // env is used to get the connection parameters. + env tabletenv.Env + // ticks is the ticker on which we'll check + // if the primary is blocked on semi-sync ACKs or not. ticks *timer.Timer - env tabletenv.Env + // clearTicks is the ticker to clear the data in + // the semisync_recover table. + clearTicks *timer.Timer - mu sync.Mutex - appPool *dbconnpool.ConnectionPool - isOpen bool + // mu protects the fields below. + mu sync.Mutex + appPool *dbconnpool.ConnectionPool + isOpen bool + // isWriting stores if the watcher is currently writing to the DB. + // We don't want two different threads initiating writes, so we use this + // for synchronization. isWriting bool + // isBlocked stores if the primary is blocked on semi-sync ack. isBlocked bool + // waiters stores the list of waiters that are waiting for the primary to be unblocked. + waiters []chan any } // NewWatcher creates a new Watcher. @@ -56,9 +71,13 @@ func NewWatcher(env tabletenv.Env) *Watcher { // TODO (@GuptaManan100): Parameterize the watch interval. watchInterval := 30 * time.Second return &Watcher{ - env: env, - ticks: timer.NewTimer(watchInterval), - appPool: dbconnpool.NewConnectionPool("SemiSyncWatcherAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), + env: env, + ticks: timer.NewTimer(watchInterval), + // We clear the data every day. We can make it configurable in the future, + // but this seams fine for now. + clearTicks: timer.NewTimer(24 * time.Hour), + appPool: dbconnpool.NewConnectionPool("SemiSyncWatcherAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), + waiters: make([]chan any, 0), } } @@ -72,8 +91,6 @@ func (w *Watcher) Open() { } // Set the watcher to be open. w.isOpen = true - // We reset the state when the watcher starts. - w.isBlocked = false log.Info("SemiSync Watcher: opening") // This function could be running from within a unit test scope, in which case we use @@ -81,6 +98,7 @@ func (w *Watcher) Open() { if !w.appPool.IsOpen() { w.appPool.Open(w.env.Config().DB.AppWithDB()) } + w.clearTicks.Start(w.clearAllData) w.ticks.Start(w.checkAndFixSemiSyncBlocked) } @@ -94,12 +112,14 @@ func (w *Watcher) Close() { } w.isOpen = false log.Info("SemiSync Watcher: closing") + w.clearTicks.Stop() w.ticks.Stop() w.appPool.Close() } // checkAndFixSemiSyncBlocked checks if the primary is blocked on semi-sync ack -// and manufactures a write to unblock the primary. +// and manufactures a write to unblock the primary. This function is safe to +// be called multiple times in parallel. func (w *Watcher) checkAndFixSemiSyncBlocked() { // Check if semi-sync is blocked or not isBlocked, err := w.isSemiSyncBlocked(context.Background()) @@ -149,13 +169,21 @@ func (w *Watcher) isSemiSyncBlocked(ctx context.Context) (bool, error) { // waitUntilSemiSyncUnblocked waits until the primary is not blocked // on semi-sync. func (w *Watcher) waitUntilSemiSyncUnblocked() { + // run one iteration of checking if semi-sync is blocked or not. w.checkAndFixSemiSyncBlocked() - // TODO: Complete the function. + if !w.stillBlocked() { + // If we find that the primary isn't blocked, we're good, + // we don't need to wait for anything. + return + } + // The primary is blocked. We need to wait for it to be unblocked. + ch := w.addWaiter() + <-ch } -// continueWrites returns true if the watcher should continue writing to the DB. -// It checks if the watcher is still open and if the primary is still blocked. -func (w *Watcher) continueWrites() bool { +// stillBlocked returns true if the watcher should continue writing to the DB +// because the watcher is still open, and the primary is still blocked. +func (w *Watcher) stillBlocked() bool { w.mu.Lock() defer w.mu.Unlock() return w.isOpen && w.isBlocked @@ -194,7 +222,7 @@ func (w *Watcher) startWrites() { backoff := 1 * time.Second maxBackoff := 1 * time.Minute // Check if we need to continue writing or not. - for !w.continueWrites() { + for w.stillBlocked() { go w.write() <-time.After(backoff) backoff = time.Duration(math.Min(float64(backoff*2), float64(maxBackoff))) @@ -209,8 +237,7 @@ func (w *Watcher) write() { return } defer conn.Recycle() - // TODO: Fix the write in question. - _, _ = conn.Conn.ExecuteFetch("INSERT INTO heartbeat (timestamp) VALUES (NOW())", 1, false) + _, _ = conn.Conn.ExecuteFetch(semiSyncRecoverWrite, 0, false) } // setIsBlocked sets the isBlocked field. @@ -218,4 +245,38 @@ func (w *Watcher) setIsBlocked(val bool) { w.mu.Lock() defer w.mu.Unlock() w.isBlocked = val + if !val { + // If we are unblocked, then we need to signal all the waiters. + for _, ch := range w.waiters { + close(ch) + } + // We also empty the list of current waiters. + w.waiters = nil + } +} + +// clearAllData clears all the data in the table so that it never +// consumes too much space on the MySQL instance. +func (w *Watcher) clearAllData() { + // Get a connection from the pool + conn, err := w.appPool.Get(context.Background()) + if err != nil { + log.Errorf("SemiSync Watcher: failed to clear semisync_recovery table: %v", err) + return + } + defer conn.Recycle() + _, err = conn.Conn.ExecuteFetch(semiSyncRecoverClear, 0, false) + if err != nil { + log.Errorf("SemiSync Watcher: failed to clear semisync_recovery table: %v", err) + } +} + +// addWaiter adds a waiter to the list of waiters +// that will be unblocked when the primary is no longer blocked. +func (w *Watcher) addWaiter() chan any { + w.mu.Lock() + defer w.mu.Unlock() + ch := make(chan any, 1) + w.waiters = append(w.waiters, ch) + return ch } From 8a47cc08446a1c74ab8bb66fb66b00e74d89e659 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Fri, 14 Feb 2025 15:11:07 +0530 Subject: [PATCH 03/27] test: add a test to check semi-sync block issue Signed-off-by: Manan Gupta --- .../reparent/newfeaturetest/reparent_test.go | 75 +++++++++++++++++++ 1 file changed, 75 insertions(+) diff --git a/go/test/endtoend/reparent/newfeaturetest/reparent_test.go b/go/test/endtoend/reparent/newfeaturetest/reparent_test.go index fc5db965847..c9ef98b9e95 100644 --- a/go/test/endtoend/reparent/newfeaturetest/reparent_test.go +++ b/go/test/endtoend/reparent/newfeaturetest/reparent_test.go @@ -19,15 +19,18 @@ package newfeaturetest import ( "context" "fmt" + "os/exec" "sync" "testing" "time" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/test/endtoend/cluster" "vitess.io/vitess/go/test/endtoend/reparent/utils" + "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/vtctl/reparentutil/policy" ) @@ -234,3 +237,75 @@ func TestBufferingWithMultipleDisruptions(t *testing.T) { // Wait for all the writes to have succeeded. wg.Wait() } + +// TestSemiSyncBlockDueToDisruption tests that Vitess can recover from a situation +// where a primary is stuck waiting for semi-sync ACKs due to a network issue, +// even if no new writes from the user arrives. +func TestSemiSyncBlockDueToDisruption(t *testing.T) { + t.Skip("Test not meant to be run on CI") + clusterInstance := utils.SetupReparentCluster(t, policy.DurabilitySemiSync) + defer utils.TeardownCluster(clusterInstance) + tablets := clusterInstance.Keyspaces[0].Shards[0].Vttablets + utils.ConfirmReplication(t, tablets[0], []*cluster.Vttablet{tablets[1], tablets[2], tablets[3]}) + + // stop heartbeats on all the replicas + for idx, tablet := range tablets { + if idx == 0 { + continue + } + utils.RunSQLs(context.Background(), t, []string{ + "stop slave;", + "change master to MASTER_HEARTBEAT_PERIOD = 0;", + "start slave;", + }, tablet) + } + + // Take a backup of the pf.conf file + runCommandWithSudo(t, "cp", "/etc/pf.conf", "/etc/pf.conf.backup") + defer func() { + // Restore the file from backup + runCommandWithSudo(t, "mv", "/etc/pf.conf.backup", "/etc/pf.conf") + runCommandWithSudo(t, "pfctl", "-f", "/etc/pf.conf") + }() + // Disrupt the network between the primary and the replicas + runCommandWithSudo(t, "sh", "-c", fmt.Sprintf("echo 'block in proto tcp from any to any port %d' | sudo tee -a /etc/pf.conf > /dev/null", tablets[0].MySQLPort)) + + // This following command is only required if pfctl is not already enabled + //runCommandWithSudo(t, "pfctl", "-e") + runCommandWithSudo(t, "pfctl", "-f", "/etc/pf.conf") + rules := runCommandWithSudo(t, "pfctl", "-s", "rules") + log.Errorf("Rules enforced - %v", rules) + + // Start a write that will be blocked by the primary waiting for semi-sync ACKs + ch := make(chan any) + go func() { + defer func() { + ch <- true + }() + utils.ConfirmReplication(t, tablets[0], []*cluster.Vttablet{tablets[1], tablets[2], tablets[3]}) + }() + + time.Sleep(30 * time.Second) + + // Restore the network + runCommandWithSudo(t, "cp", "/etc/pf.conf.backup", "/etc/pf.conf") + runCommandWithSudo(t, "pfctl", "-f", "/etc/pf.conf") + + // We expect the problem to be resolved in less than 30 seconds. + select { + case <-time.After(30 * time.Second): + t.Errorf("Timed out waiting for semi-sync to be unblocked") + case <-ch: + log.Errorf("Woohoo, write finished!") + } +} + +// runCommandWithSudo runs the provided command with sudo privileges +// when the command is run, it prompts the user for the password, and it must be +// entered for the program to resume. +func runCommandWithSudo(t *testing.T, args ...string) string { + cmd := exec.Command("sudo", args...) + out, err := cmd.CombinedOutput() + assert.NoError(t, err, string(out)) + return string(out) +} From bf9b845ee8524fccdebcc2d775c5c0195146a4a2 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Fri, 14 Feb 2025 15:24:21 +0530 Subject: [PATCH 04/27] feat: address review comments Signed-off-by: Manan Gupta --- .../watcher.go => semisyncmonitor/monitor.go} | 175 ++++++++++-------- 1 file changed, 93 insertions(+), 82 deletions(-) rename go/vt/vttablet/tabletserver/{semisyncwatcher/watcher.go => semisyncmonitor/monitor.go} (63%) diff --git a/go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go similarity index 63% rename from go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go rename to go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go index dbf72e47733..5f3c6c6853a 100644 --- a/go/vt/vttablet/tabletserver/semisyncwatcher/watcher.go +++ b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package semisyncwatcher +package semisyncmonitor import ( "context" @@ -33,16 +33,16 @@ import ( const ( semiSyncWaitSessionsRead = "SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'" semiSyncRecoverWrite = "INSERT INTO semisync_recover (ts) VALUES (NOW())" - semiSyncRecoverClear = "DELETE FROM semisync_recover" + semiSyncRecoverClear = "TRUNCATE TABLE semisync_recover" ) -// Watcher is a watcher that checks if the primary tablet +// Monitor is a monitor that checks if the primary tablet // is blocked on a semi-sync ack from the replica. // If the semi-sync ACK is lost in the network, // it is possible that the primary is indefinitely stuck, -// blocking PRS. The watcher looks for this situation and manufactures a write +// blocking PRS. The monitor looks for this situation and manufactures a write // periodically to unblock the primary. -type Watcher struct { +type Monitor struct { // env is used to get the connection parameters. env tabletenv.Env // ticks is the ticker on which we'll check @@ -56,7 +56,7 @@ type Watcher struct { mu sync.Mutex appPool *dbconnpool.ConnectionPool isOpen bool - // isWriting stores if the watcher is currently writing to the DB. + // isWriting stores if the monitor is currently writing to the DB. // We don't want two different threads initiating writes, so we use this // for synchronization. isWriting bool @@ -66,82 +66,84 @@ type Watcher struct { waiters []chan any } -// NewWatcher creates a new Watcher. -func NewWatcher(env tabletenv.Env) *Watcher { +// NewMonitor creates a new Monitor. +func NewMonitor(env tabletenv.Env) *Monitor { // TODO (@GuptaManan100): Parameterize the watch interval. watchInterval := 30 * time.Second - return &Watcher{ + return &Monitor{ env: env, ticks: timer.NewTimer(watchInterval), // We clear the data every day. We can make it configurable in the future, // but this seams fine for now. clearTicks: timer.NewTimer(24 * time.Hour), - appPool: dbconnpool.NewConnectionPool("SemiSyncWatcherAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), + appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), waiters: make([]chan any, 0), } } -// Open starts the watcher. -func (w *Watcher) Open() { - w.mu.Lock() - defer w.mu.Unlock() - if w.isOpen { +// Open starts the monitor. +func (m *Monitor) Open() { + m.mu.Lock() + defer m.mu.Unlock() + if m.isOpen { // If we are already open, then there is nothing to do return } - // Set the watcher to be open. - w.isOpen = true - log.Info("SemiSync Watcher: opening") + // Set the monitor to be open. + m.isOpen = true + log.Info("SemiSync Monitor: opening") // This function could be running from within a unit test scope, in which case we use // mock pools that are already open. This is why we test for the pool being open. - if !w.appPool.IsOpen() { - w.appPool.Open(w.env.Config().DB.AppWithDB()) + if !m.appPool.IsOpen() { + m.appPool.Open(m.env.Config().DB.AppWithDB()) } - w.clearTicks.Start(w.clearAllData) - w.ticks.Start(w.checkAndFixSemiSyncBlocked) + m.clearTicks.Start(m.clearAllData) + m.ticks.Start(m.checkAndFixSemiSyncBlocked) } -// Close stops the watcher. -func (w *Watcher) Close() { - w.mu.Lock() - defer w.mu.Unlock() - if !w.isOpen { +// Close stops the monitor. +func (m *Monitor) Close() { + m.mu.Lock() + defer m.mu.Unlock() + if !m.isOpen { // If we are already closed, then there is nothing to do return } - w.isOpen = false - log.Info("SemiSync Watcher: closing") - w.clearTicks.Stop() - w.ticks.Stop() - w.appPool.Close() + m.isOpen = false + log.Info("SemiSync Monitor: closing") + m.clearTicks.Stop() + m.ticks.Stop() + m.appPool.Close() } // checkAndFixSemiSyncBlocked checks if the primary is blocked on semi-sync ack // and manufactures a write to unblock the primary. This function is safe to // be called multiple times in parallel. -func (w *Watcher) checkAndFixSemiSyncBlocked() { +func (m *Monitor) checkAndFixSemiSyncBlocked() { // Check if semi-sync is blocked or not - isBlocked, err := w.isSemiSyncBlocked(context.Background()) + ctx, cancel := context.WithTimeout(context.Background(), 15*time.Second) + defer cancel() + isBlocked, err := m.isSemiSyncBlocked(ctx) if err != nil { // If we are unable to determine whether the primary is blocked or not, // then we can just abort the function and try again later. - log.Errorf("SemiSync Watcher: failed to check if primary is blocked on semi-sync: %v", err) + log.Errorf("SemiSync Monitor: failed to check if primary is blocked on semi-sync: %v", err) return } // Set the isBlocked state. - w.setIsBlocked(isBlocked) + m.setIsBlocked(isBlocked) if isBlocked { // If we are blocked, then we want to start the writes. // That function is re-entrant. If we are already writing, then it will just return. - w.startWrites() + m.startWrites() } } // isSemiSyncBlocked checks if the primary is blocked on semi-sync. -func (w *Watcher) isSemiSyncBlocked(ctx context.Context) (bool, error) { +func (m *Monitor) isSemiSyncBlocked(ctx context.Context) (bool, error) { // Get a connection from the pool - conn, err := w.appPool.Get(ctx) + conn, err := m.appPool.Get(ctx) if err != nil { return false, err } @@ -168,115 +170,124 @@ func (w *Watcher) isSemiSyncBlocked(ctx context.Context) (bool, error) { // waitUntilSemiSyncUnblocked waits until the primary is not blocked // on semi-sync. -func (w *Watcher) waitUntilSemiSyncUnblocked() { +func (m *Monitor) waitUntilSemiSyncUnblocked() { // run one iteration of checking if semi-sync is blocked or not. - w.checkAndFixSemiSyncBlocked() - if !w.stillBlocked() { + m.checkAndFixSemiSyncBlocked() + if !m.stillBlocked() { // If we find that the primary isn't blocked, we're good, // we don't need to wait for anything. return } // The primary is blocked. We need to wait for it to be unblocked. - ch := w.addWaiter() + ch := m.addWaiter() <-ch } -// stillBlocked returns true if the watcher should continue writing to the DB -// because the watcher is still open, and the primary is still blocked. -func (w *Watcher) stillBlocked() bool { - w.mu.Lock() - defer w.mu.Unlock() - return w.isOpen && w.isBlocked +// stillBlocked returns true if the monitor should continue writing to the DB +// because the monitor is still open, and the primary is still blocked. +func (m *Monitor) stillBlocked() bool { + m.mu.Lock() + defer m.mu.Unlock() + return m.isOpen && m.isBlocked } -// checkAndSetIsWriting checks if the watcher is already writing to the DB. +// checkAndSetIsWriting checks if the monitor is already writing to the DB. // If it is not, then it sets the isWriting field and signals the caller. -func (w *Watcher) checkAndSetIsWriting() bool { - w.mu.Lock() - defer w.mu.Unlock() - if w.isWriting { +func (m *Monitor) checkAndSetIsWriting() bool { + m.mu.Lock() + defer m.mu.Unlock() + if m.isWriting { return false } - w.isWriting = true + m.isWriting = true return true } // clearIsWriting clears the isWriting field. -func (w *Watcher) clearIsWriting() { - w.mu.Lock() - defer w.mu.Unlock() - w.isWriting = false +func (m *Monitor) clearIsWriting() { + m.mu.Lock() + defer m.mu.Unlock() + m.isWriting = false } // startWrites starts writing to the DB. // It is re-entrant and will return if we are already writing. -func (w *Watcher) startWrites() { +func (m *Monitor) startWrites() { // If we are already writing, then we can just return. - if !w.checkAndSetIsWriting() { + if !m.checkAndSetIsWriting() { return } // We defer the clear of the isWriting field. - defer w.clearIsWriting() + defer m.clearIsWriting() // We start writing to the DB with a backoff. backoff := 1 * time.Second maxBackoff := 1 * time.Minute // Check if we need to continue writing or not. - for w.stillBlocked() { - go w.write() + for m.stillBlocked() { + // We do the writes in a go-routine because if the network disruption + // is somewhat long-lived, then the writes themselves can also block. + // By doing them in a go-routine we give the system more time to recover while + // exponentially backing off. We will eventually run out of the connections in the pool + // at which point, there would be nothing that we can do. + go m.write() <-time.After(backoff) backoff = time.Duration(math.Min(float64(backoff*2), float64(maxBackoff))) } } // write writes a heartbeat to unblock semi-sync being stuck. -func (w *Watcher) write() { +func (m *Monitor) write() { // Get a connection from the pool - conn, err := w.appPool.Get(context.Background()) + conn, err := m.appPool.Get(context.Background()) if err != nil { + log.Errorf("SemiSync Monitor: failed to write to semisync_recovery table: %v", err) return } defer conn.Recycle() - _, _ = conn.Conn.ExecuteFetch(semiSyncRecoverWrite, 0, false) + _, err = conn.Conn.ExecuteFetch(semiSyncRecoverWrite, 0, false) + if err != nil { + log.Errorf("SemiSync Monitor: failed to write to semisync_recovery table: %v", err) + } } // setIsBlocked sets the isBlocked field. -func (w *Watcher) setIsBlocked(val bool) { - w.mu.Lock() - defer w.mu.Unlock() - w.isBlocked = val +func (m *Monitor) setIsBlocked(val bool) { + m.mu.Lock() + defer m.mu.Unlock() + m.isBlocked = val if !val { // If we are unblocked, then we need to signal all the waiters. - for _, ch := range w.waiters { + for _, ch := range m.waiters { close(ch) } // We also empty the list of current waiters. - w.waiters = nil + m.waiters = nil } } // clearAllData clears all the data in the table so that it never // consumes too much space on the MySQL instance. -func (w *Watcher) clearAllData() { +func (m *Monitor) clearAllData() { // Get a connection from the pool - conn, err := w.appPool.Get(context.Background()) + conn, err := m.appPool.Get(context.Background()) if err != nil { - log.Errorf("SemiSync Watcher: failed to clear semisync_recovery table: %v", err) + log.Errorf("SemiSync Monitor: failed to clear semisync_recovery table: %v", err) return } defer conn.Recycle() _, err = conn.Conn.ExecuteFetch(semiSyncRecoverClear, 0, false) if err != nil { - log.Errorf("SemiSync Watcher: failed to clear semisync_recovery table: %v", err) + log.Errorf("SemiSync Monitor: failed to clear semisync_recovery table: %v", err) } } // addWaiter adds a waiter to the list of waiters // that will be unblocked when the primary is no longer blocked. -func (w *Watcher) addWaiter() chan any { - w.mu.Lock() - defer w.mu.Unlock() - ch := make(chan any, 1) - w.waiters = append(w.waiters, ch) +func (m *Monitor) addWaiter() chan any { + m.mu.Lock() + defer m.mu.Unlock() + ch := make(chan any) + m.waiters = append(m.waiters, ch) return ch } From 955d0c0e4c5e5f35f849b59b16b905f9d96f22f2 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Fri, 14 Feb 2025 16:27:13 +0530 Subject: [PATCH 05/27] feat: wire up the monitor to the state manager and the tablet server Signed-off-by: Manan Gupta --- .../vttablet/tabletmanager/rpc_replication.go | 12 ++-- go/vt/vttablet/tabletserver/controller.go | 3 + .../tabletserver/semisyncmonitor/monitor.go | 25 ++++--- go/vt/vttablet/tabletserver/state_manager.go | 31 +++++---- .../tabletserver/state_manager_test.go | 67 ++++++++++--------- go/vt/vttablet/tabletserver/tabletserver.go | 39 ++++++----- go/vt/vttablet/tabletservermock/controller.go | 6 ++ 7 files changed, 111 insertions(+), 72 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/rpc_replication.go b/go/vt/vttablet/tabletmanager/rpc_replication.go index f2cffb322e8..471b959e643 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication.go @@ -585,12 +585,16 @@ func (tm *TabletManager) demotePrimary(ctx context.Context, revertPartialFailure }() } - // Now that we know no writes are in-flight and no new writes can occur, - // set MySQL to super_read_only mode. If we are already super_read_only because of a + // Now we know no writes are in-flight and no new writes can occur. + // We just need to wait for no write being blocked on semi-sync ACKs. + err = tm.QueryServiceControl.WaitUntilSemiSyncBeingUnblocked(ctx) + if err != nil { + return nil, err + } + + // We can now set MySQL to super_read_only mode. If we are already super_read_only because of a // previous demotion, or because we are not primary anyway, this should be // idempotent. - - // TODO(@GuptaManan100): Reject PR if not done. Check we have no writes blocked on semi-sync. if _, err := tm.MysqlDaemon.SetSuperReadOnly(ctx, true); err != nil { if sqlErr, ok := err.(*sqlerror.SQLError); ok && sqlErr.Number() == sqlerror.ERUnknownSystemVariable { log.Warningf("server does not know about super_read_only, continuing anyway...") diff --git a/go/vt/vttablet/tabletserver/controller.go b/go/vt/vttablet/tabletserver/controller.go index ab2875ae27b..057eab3218a 100644 --- a/go/vt/vttablet/tabletserver/controller.go +++ b/go/vt/vttablet/tabletserver/controller.go @@ -125,6 +125,9 @@ type Controller interface { // IsDiskStalled returns if the disk is stalled. IsDiskStalled() bool + + // WaitUntilSemiSyncBeingUnblocked waits until the primary is not blocked on semi-sync. + WaitUntilSemiSyncBeingUnblocked(ctx context.Context) error } // Ensure TabletServer satisfies Controller interface. diff --git a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go index 5f3c6c6853a..171b68b3c42 100644 --- a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go @@ -69,15 +69,16 @@ type Monitor struct { // NewMonitor creates a new Monitor. func NewMonitor(env tabletenv.Env) *Monitor { // TODO (@GuptaManan100): Parameterize the watch interval. - watchInterval := 30 * time.Second + watchInterval := 10 * time.Second return &Monitor{ env: env, ticks: timer.NewTimer(watchInterval), // We clear the data every day. We can make it configurable in the future, // but this seams fine for now. clearTicks: timer.NewTimer(24 * time.Hour), - appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), - waiters: make([]chan any, 0), + // TODO: Add a metric on the number of writes that are blocked. + appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), + waiters: make([]chan any, 0), } } @@ -168,19 +169,25 @@ func (m *Monitor) isSemiSyncBlocked(ctx context.Context) (bool, error) { return value != 0, err } -// waitUntilSemiSyncUnblocked waits until the primary is not blocked -// on semi-sync. -func (m *Monitor) waitUntilSemiSyncUnblocked() { +// WaitUntilSemiSyncUnblocked waits until the primary is not blocked +// on semi-sync or until the context expires. +func (m *Monitor) WaitUntilSemiSyncUnblocked(ctx context.Context) error { // run one iteration of checking if semi-sync is blocked or not. m.checkAndFixSemiSyncBlocked() if !m.stillBlocked() { // If we find that the primary isn't blocked, we're good, // we don't need to wait for anything. - return + return nil } - // The primary is blocked. We need to wait for it to be unblocked. + // The primary is blocked. We need to wait for it to be unblocked + // or the context to expire. ch := m.addWaiter() - <-ch + select { + case <-ch: + return nil + case <-ctx.Done(): + return ctx.Err() + } } // stillBlocked returns true if the monitor should continue writing to the DB diff --git a/go/vt/vttablet/tabletserver/state_manager.go b/go/vt/vttablet/tabletserver/state_manager.go index 0ccd0e42735..1966c9e0375 100644 --- a/go/vt/vttablet/tabletserver/state_manager.go +++ b/go/vt/vttablet/tabletserver/state_manager.go @@ -112,19 +112,20 @@ type stateManager struct { // Open must be done in forward order. // Close must be done in reverse order. // All Close functions must be called before Open. - hs *healthStreamer - se schemaEngine - rt replTracker - vstreamer subComponent - tracker subComponent - watcher subComponent - qe queryEngine - txThrottler txThrottler - te txEngine - messager subComponent - ddle onlineDDLExecutor - throttler lagThrottler - tableGC tableGarbageCollector + hs *healthStreamer + se schemaEngine + rt replTracker + vstreamer subComponent + tracker subComponent + watcher subComponent + semiSyncMonitor subComponent + qe queryEngine + txThrottler txThrottler + te txEngine + messager subComponent + ddle onlineDDLExecutor + throttler lagThrottler + tableGC tableGarbageCollector // hcticks starts on initialization and runs forever. hcticks *timer.Timer @@ -469,6 +470,7 @@ func (sm *stateManager) servePrimary() error { sm.throttler.Open() sm.tableGC.Open() sm.ddle.Open() + sm.semiSyncMonitor.Open() sm.setState(topodatapb.TabletType_PRIMARY, StateServing) return nil } @@ -485,6 +487,7 @@ func (sm *stateManager) unservePrimary() error { sm.se.MakePrimary(false) sm.hs.MakePrimary(false) sm.rt.MakePrimary() + sm.semiSyncMonitor.Open() sm.setState(topodatapb.TabletType_PRIMARY, StateNotServing) return nil } @@ -495,6 +498,7 @@ func (sm *stateManager) serveNonPrimary(wantTabletType topodatapb.TabletType) er cancel := sm.terminateAllQueries(nil) defer cancel() + sm.semiSyncMonitor.Close() sm.ddle.Close() sm.tableGC.Close() sm.messager.Close() @@ -517,6 +521,7 @@ func (sm *stateManager) serveNonPrimary(wantTabletType topodatapb.TabletType) er func (sm *stateManager) unserveNonPrimary(wantTabletType topodatapb.TabletType) error { sm.unserveCommon() + sm.semiSyncMonitor.Close() sm.se.MakeNonPrimary() sm.hs.MakeNonPrimary() diff --git a/go/vt/vttablet/tabletserver/state_manager_test.go b/go/vt/vttablet/tabletserver/state_manager_test.go index 9af2e061502..1bc38117007 100644 --- a/go/vt/vttablet/tabletserver/state_manager_test.go +++ b/go/vt/vttablet/tabletserver/state_manager_test.go @@ -79,7 +79,6 @@ func TestStateManagerServePrimary(t *testing.T) { assert.Equal(t, testNow, sm.ptsTimestamp) verifySubcomponent(t, 1, sm.watcher, testStateClosed) - verifySubcomponent(t, 2, sm.se, testStateOpen) verifySubcomponent(t, 3, sm.vstreamer, testStateOpen) verifySubcomponent(t, 4, sm.qe, testStateOpen) @@ -91,6 +90,7 @@ func TestStateManagerServePrimary(t *testing.T) { verifySubcomponent(t, 10, sm.throttler, testStateOpen) verifySubcomponent(t, 11, sm.tableGC, testStateOpen) verifySubcomponent(t, 12, sm.ddle, testStateOpen) + verifySubcomponent(t, 13, sm.semiSyncMonitor, testStateOpen) assert.False(t, sm.se.(*testSchemaEngine).nonPrimary) assert.True(t, sm.se.(*testSchemaEngine).ensureCalled) @@ -105,20 +105,21 @@ func TestStateManagerServeNonPrimary(t *testing.T) { err := sm.SetServingType(topodatapb.TabletType_REPLICA, testNow, StateServing, "") require.NoError(t, err) - verifySubcomponent(t, 1, sm.ddle, testStateClosed) - verifySubcomponent(t, 2, sm.tableGC, testStateClosed) - verifySubcomponent(t, 3, sm.messager, testStateClosed) - verifySubcomponent(t, 4, sm.tracker, testStateClosed) + verifySubcomponent(t, 1, sm.semiSyncMonitor, testStateClosed) + verifySubcomponent(t, 2, sm.ddle, testStateClosed) + verifySubcomponent(t, 3, sm.tableGC, testStateClosed) + verifySubcomponent(t, 4, sm.messager, testStateClosed) + verifySubcomponent(t, 5, sm.tracker, testStateClosed) assert.True(t, sm.se.(*testSchemaEngine).nonPrimary) - verifySubcomponent(t, 5, sm.se, testStateOpen) - verifySubcomponent(t, 6, sm.vstreamer, testStateOpen) - verifySubcomponent(t, 7, sm.qe, testStateOpen) - verifySubcomponent(t, 8, sm.txThrottler, testStateOpen) - verifySubcomponent(t, 9, sm.te, testStateNonPrimary) - verifySubcomponent(t, 10, sm.rt, testStateNonPrimary) - verifySubcomponent(t, 11, sm.watcher, testStateOpen) - verifySubcomponent(t, 12, sm.throttler, testStateOpen) + verifySubcomponent(t, 6, sm.se, testStateOpen) + verifySubcomponent(t, 7, sm.vstreamer, testStateOpen) + verifySubcomponent(t, 8, sm.qe, testStateOpen) + verifySubcomponent(t, 9, sm.txThrottler, testStateOpen) + verifySubcomponent(t, 10, sm.te, testStateNonPrimary) + verifySubcomponent(t, 11, sm.rt, testStateNonPrimary) + verifySubcomponent(t, 12, sm.watcher, testStateOpen) + verifySubcomponent(t, 13, sm.throttler, testStateOpen) assert.Equal(t, topodatapb.TabletType_REPLICA, sm.target.TabletType) assert.Equal(t, StateServing, sm.state) @@ -144,6 +145,7 @@ func TestStateManagerUnservePrimary(t *testing.T) { verifySubcomponent(t, 11, sm.txThrottler, testStateOpen) verifySubcomponent(t, 12, sm.rt, testStatePrimary) + verifySubcomponent(t, 13, sm.semiSyncMonitor, testStateOpen) assert.Equal(t, topodatapb.TabletType_PRIMARY, sm.target.TabletType) assert.Equal(t, StateNotServing, sm.state) @@ -196,13 +198,14 @@ func TestStateManagerUnserveNonPrimary(t *testing.T) { verifySubcomponent(t, 6, sm.tracker, testStateClosed) assert.True(t, sm.se.(*testSchemaEngine).nonPrimary) - verifySubcomponent(t, 7, sm.se, testStateOpen) - verifySubcomponent(t, 8, sm.vstreamer, testStateOpen) - verifySubcomponent(t, 9, sm.qe, testStateOpen) - verifySubcomponent(t, 10, sm.txThrottler, testStateOpen) + verifySubcomponent(t, 7, sm.semiSyncMonitor, testStateClosed) + verifySubcomponent(t, 8, sm.se, testStateOpen) + verifySubcomponent(t, 9, sm.vstreamer, testStateOpen) + verifySubcomponent(t, 10, sm.qe, testStateOpen) + verifySubcomponent(t, 11, sm.txThrottler, testStateOpen) - verifySubcomponent(t, 11, sm.rt, testStateNonPrimary) - verifySubcomponent(t, 12, sm.watcher, testStateOpen) + verifySubcomponent(t, 12, sm.rt, testStateNonPrimary) + verifySubcomponent(t, 13, sm.watcher, testStateOpen) assert.Equal(t, topodatapb.TabletType_RDONLY, sm.target.TabletType) assert.Equal(t, StateNotServing, sm.state) @@ -328,20 +331,21 @@ func TestStateManagerSetServingTypeNoChange(t *testing.T) { err = sm.SetServingType(topodatapb.TabletType_REPLICA, testNow, StateServing, "") require.NoError(t, err) - verifySubcomponent(t, 1, sm.ddle, testStateClosed) - verifySubcomponent(t, 2, sm.tableGC, testStateClosed) - verifySubcomponent(t, 3, sm.messager, testStateClosed) - verifySubcomponent(t, 4, sm.tracker, testStateClosed) + verifySubcomponent(t, 1, sm.semiSyncMonitor, testStateClosed) + verifySubcomponent(t, 2, sm.ddle, testStateClosed) + verifySubcomponent(t, 3, sm.tableGC, testStateClosed) + verifySubcomponent(t, 4, sm.messager, testStateClosed) + verifySubcomponent(t, 5, sm.tracker, testStateClosed) assert.True(t, sm.se.(*testSchemaEngine).nonPrimary) - verifySubcomponent(t, 5, sm.se, testStateOpen) - verifySubcomponent(t, 6, sm.vstreamer, testStateOpen) - verifySubcomponent(t, 7, sm.qe, testStateOpen) - verifySubcomponent(t, 8, sm.txThrottler, testStateOpen) - verifySubcomponent(t, 9, sm.te, testStateNonPrimary) - verifySubcomponent(t, 10, sm.rt, testStateNonPrimary) - verifySubcomponent(t, 11, sm.watcher, testStateOpen) - verifySubcomponent(t, 12, sm.throttler, testStateOpen) + verifySubcomponent(t, 6, sm.se, testStateOpen) + verifySubcomponent(t, 7, sm.vstreamer, testStateOpen) + verifySubcomponent(t, 8, sm.qe, testStateOpen) + verifySubcomponent(t, 9, sm.txThrottler, testStateOpen) + verifySubcomponent(t, 10, sm.te, testStateNonPrimary) + verifySubcomponent(t, 11, sm.rt, testStateNonPrimary) + verifySubcomponent(t, 12, sm.watcher, testStateOpen) + verifySubcomponent(t, 13, sm.throttler, testStateOpen) assert.Equal(t, topodatapb.TabletType_REPLICA, sm.target.TabletType) assert.Equal(t, StateServing, sm.state) @@ -821,6 +825,7 @@ func newTestStateManager() *stateManager { vstreamer: &testSubcomponent{}, tracker: &testSubcomponent{}, watcher: &testSubcomponent{}, + semiSyncMonitor: &testSubcomponent{}, qe: &testQueryEngine{}, txThrottler: &testTxThrottler{}, te: &testTxEngine{}, diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index e09e04a9679..5d78ab05905 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -66,6 +66,7 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletserver/repltracker" "vitess.io/vitess/go/vt/vttablet/tabletserver/rules" "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" + "vitess.io/vitess/go/vt/vttablet/tabletserver/semisyncmonitor" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" "vitess.io/vitess/go/vt/vttablet/tabletserver/throttle" "vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/base" @@ -104,21 +105,22 @@ type TabletServer struct { topoServer *topo.Server // These are sub-components of TabletServer. - statelessql *QueryList - statefulql *QueryList - olapql *QueryList - se *schema.Engine - rt *repltracker.ReplTracker - vstreamer *vstreamer.Engine - tracker *schema.Tracker - watcher *BinlogWatcher - qe *QueryEngine - txThrottler txthrottler.TxThrottler - te *TxEngine - messager *messager.Engine - hs *healthStreamer - lagThrottler *throttle.Throttler - tableGC *gc.TableGC + statelessql *QueryList + statefulql *QueryList + olapql *QueryList + se *schema.Engine + rt *repltracker.ReplTracker + vstreamer *vstreamer.Engine + tracker *schema.Tracker + watcher *BinlogWatcher + semiSyncMonitor *semisyncmonitor.Monitor + qe *QueryEngine + txThrottler txthrottler.TxThrottler + te *TxEngine + messager *messager.Engine + hs *healthStreamer + lagThrottler *throttle.Throttler + tableGC *gc.TableGC // sm manages state transitions. sm *stateManager @@ -181,6 +183,7 @@ func NewTabletServer(ctx context.Context, env *vtenv.Environment, name string, c tsv.vstreamer = vstreamer.NewEngine(tsv, srvTopoServer, tsv.se, tsv.lagThrottler, alias.Cell) tsv.tracker = schema.NewTracker(tsv, tsv.vstreamer, tsv.se) tsv.watcher = NewBinlogWatcher(tsv, tsv.vstreamer, tsv.config) + tsv.semiSyncMonitor = semisyncmonitor.NewMonitor(tsv) tsv.qe = NewQueryEngine(tsv, tsv.se) tsv.txThrottler = txthrottler.NewTxThrottler(tsv, topoServer) tsv.te = NewTxEngine(tsv, tsv.hs.sendUnresolvedTransactionSignal) @@ -199,6 +202,7 @@ func NewTabletServer(ctx context.Context, env *vtenv.Environment, name string, c vstreamer: tsv.vstreamer, tracker: tsv.tracker, watcher: tsv.watcher, + semiSyncMonitor: tsv.semiSyncMonitor, qe: tsv.qe, txThrottler: tsv.txThrottler, te: tsv.te, @@ -774,6 +778,11 @@ func (tsv *TabletServer) IsDiskStalled() bool { return tsv.sm.diskHealthMonitor.IsDiskStalled() } +// WaitUntilSemiSyncBeingUnblocked waits until semi-sync is unblocked or if context has expired. +func (tsv *TabletServer) WaitUntilSemiSyncBeingUnblocked(ctx context.Context) error { + return tsv.semiSyncMonitor.WaitUntilSemiSyncUnblocked(ctx) +} + // CreateTransaction creates the metadata for a 2PC transaction. func (tsv *TabletServer) CreateTransaction(ctx context.Context, target *querypb.Target, dtid string, participants []*querypb.Target) (err error) { return tsv.execRequest( diff --git a/go/vt/vttablet/tabletservermock/controller.go b/go/vt/vttablet/tabletservermock/controller.go index 21b38755302..20fadc11bc8 100644 --- a/go/vt/vttablet/tabletservermock/controller.go +++ b/go/vt/vttablet/tabletservermock/controller.go @@ -285,6 +285,12 @@ func (tqsc *Controller) IsDiskStalled() bool { return false } +// WaitUntilSemiSyncBeingUnblocked is part of the tabletserver.Controller interface +func (tqsc *Controller) WaitUntilSemiSyncBeingUnblocked(context.Context) error { + tqsc.MethodCalled["WaitUntilSemiSyncBeingUnblocked"] = true + return nil +} + // EnterLameduck implements tabletserver.Controller. func (tqsc *Controller) EnterLameduck() { tqsc.mu.Lock() From 312433956ff3f7de1d5bb945a8b0d6b0a2b6f281 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Fri, 14 Feb 2025 18:34:22 +0530 Subject: [PATCH 06/27] feat: fix some bugs Signed-off-by: Manan Gupta --- .../tabletserver/semisyncmonitor/monitor.go | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go index 171b68b3c42..eaea178b7ca 100644 --- a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go @@ -23,17 +23,19 @@ import ( "sync" "time" + "vitess.io/vitess/go/constants/sidecar" "vitess.io/vitess/go/timer" "vitess.io/vitess/go/vt/dbconnpool" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/mysqlctl" + "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" ) const ( semiSyncWaitSessionsRead = "SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'" - semiSyncRecoverWrite = "INSERT INTO semisync_recover (ts) VALUES (NOW())" - semiSyncRecoverClear = "TRUNCATE TABLE semisync_recover" + semiSyncRecoverWrite = "INSERT INTO %s.semisync_recover (ts) VALUES (NOW())" + semiSyncRecoverClear = "TRUNCATE TABLE %s.semisync_recover" ) // Monitor is a monitor that checks if the primary tablet @@ -137,7 +139,9 @@ func (m *Monitor) checkAndFixSemiSyncBlocked() { if isBlocked { // If we are blocked, then we want to start the writes. // That function is re-entrant. If we are already writing, then it will just return. - m.startWrites() + // We start it in a go-routine, because we want to continue to check for when + // we get unblocked. + go m.startWrites() } } @@ -165,7 +169,7 @@ func (m *Monitor) isSemiSyncBlocked(ctx context.Context) (bool, error) { if len(res.Rows) != 1 || len(res.Rows[0]) != 2 { return false, errors.New("unexpected number of rows received") } - value, err := res.Rows[0][1].ToInt() + value, err := res.Rows[0][1].ToCastInt64() return value != 0, err } @@ -252,7 +256,7 @@ func (m *Monitor) write() { return } defer conn.Recycle() - _, err = conn.Conn.ExecuteFetch(semiSyncRecoverWrite, 0, false) + _, err = conn.Conn.ExecuteFetch(m.bindSideCarDBName(semiSyncRecoverWrite), 0, false) if err != nil { log.Errorf("SemiSync Monitor: failed to write to semisync_recovery table: %v", err) } @@ -283,7 +287,7 @@ func (m *Monitor) clearAllData() { return } defer conn.Recycle() - _, err = conn.Conn.ExecuteFetch(semiSyncRecoverClear, 0, false) + _, err = conn.Conn.ExecuteFetch(m.bindSideCarDBName(semiSyncRecoverClear), 0, false) if err != nil { log.Errorf("SemiSync Monitor: failed to clear semisync_recovery table: %v", err) } @@ -298,3 +302,8 @@ func (m *Monitor) addWaiter() chan any { m.waiters = append(m.waiters, ch) return ch } + +// bindSideCarDBName binds the sidecar db name to the query. +func (m *Monitor) bindSideCarDBName(query string) string { + return sqlparser.BuildParsedQuery(query, sidecar.GetIdentifier()).Query +} From 2b2708f48fe652fd53f4fe31271547aaa093df79 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Mon, 17 Feb 2025 11:16:57 +0530 Subject: [PATCH 07/27] feat: add the capability to the monitor to keep track of outstanding writes Signed-off-by: Manan Gupta --- .../tabletserver/semisyncmonitor/monitor.go | 51 +++++++++++++++---- 1 file changed, 42 insertions(+), 9 deletions(-) diff --git a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go index eaea178b7ca..03187a2c5ca 100644 --- a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go @@ -19,7 +19,6 @@ package semisyncmonitor import ( "context" "errors" - "math" "sync" "time" @@ -36,6 +35,8 @@ const ( semiSyncWaitSessionsRead = "SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'" semiSyncRecoverWrite = "INSERT INTO %s.semisync_recover (ts) VALUES (NOW())" semiSyncRecoverClear = "TRUNCATE TABLE %s.semisync_recover" + maxWritesPermitted = 15 + clearTimerDuration = 24 * time.Hour ) // Monitor is a monitor that checks if the primary tablet @@ -62,6 +63,10 @@ type Monitor struct { // We don't want two different threads initiating writes, so we use this // for synchronization. isWriting bool + // inProgressWriteCount is the number of writes currently in progress. + // The writes from the monitor themselves might get blocked and hence a count for them is required. + // After enough writes are blocked, we want to notify VTOrc to run an ERS. + inProgressWriteCount int // isBlocked stores if the primary is blocked on semi-sync ack. isBlocked bool // waiters stores the list of waiters that are waiting for the primary to be unblocked. @@ -77,7 +82,7 @@ func NewMonitor(env tabletenv.Env) *Monitor { ticks: timer.NewTimer(watchInterval), // We clear the data every day. We can make it configurable in the future, // but this seams fine for now. - clearTicks: timer.NewTimer(24 * time.Hour), + clearTicks: timer.NewTimer(clearTimerDuration), // TODO: Add a metric on the number of writes that are blocked. appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), waiters: make([]chan any, 0), @@ -231,24 +236,52 @@ func (m *Monitor) startWrites() { // We defer the clear of the isWriting field. defer m.clearIsWriting() - // We start writing to the DB with a backoff. - backoff := 1 * time.Second - maxBackoff := 1 * time.Minute // Check if we need to continue writing or not. for m.stillBlocked() { // We do the writes in a go-routine because if the network disruption // is somewhat long-lived, then the writes themselves can also block. // By doing them in a go-routine we give the system more time to recover while - // exponentially backing off. We will eventually run out of the connections in the pool - // at which point, there would be nothing that we can do. + // exponentially backing off. We will not do more than maxWritesPermitted writes and once + // all maxWritesPermitted writes are blocked, we'll wait for VTOrc to run an ERS. go m.write() - <-time.After(backoff) - backoff = time.Duration(math.Min(float64(backoff*2), float64(maxBackoff))) + <-time.After(1 * time.Second) } } +// incrementWriteCount tries to increment the write count. It +// also checks that the write count value should not exceed +// the maximum value configured. It returns whether it was able +// to increment the value or not. +func (m *Monitor) incrementWriteCount() bool { + m.mu.Lock() + defer m.mu.Unlock() + if m.inProgressWriteCount == maxWritesPermitted { + return false + } + m.inProgressWriteCount = m.inProgressWriteCount + 1 + return true +} + +// AllWritesBlocked returns if maxWritesPermitted number of writes +// are already outstanding. +func (m *Monitor) AllWritesBlocked() bool { + return m.inProgressWriteCount == maxWritesPermitted +} + +// decrementWriteCount decrements the write count. +func (m *Monitor) decrementWriteCount() { + m.mu.Lock() + defer m.mu.Unlock() + m.inProgressWriteCount = m.inProgressWriteCount - 1 +} + // write writes a heartbeat to unblock semi-sync being stuck. func (m *Monitor) write() { + shouldWrite := m.incrementWriteCount() + if !shouldWrite { + return + } + defer m.decrementWriteCount() // Get a connection from the pool conn, err := m.appPool.Get(context.Background()) if err != nil { From 0be07430745c2b0af5a0281fda55526a7fdaee26 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Mon, 17 Feb 2025 12:14:06 +0530 Subject: [PATCH 08/27] feat: add the capability to run ERS to VTOrc when it detects semi-sync is blocked on the primary Signed-off-by: Manan Gupta --- .../vtorc/readtopologyinstance/main_test.go | 2 + .../replicationdata/replicationdata.pb.go | 30 ++++++--- .../replicationdata_vtproto.pb.go | 36 +++++++++++ go/vt/vtorc/db/generate_base.go | 1 + go/vt/vtorc/inst/analysis.go | 2 + go/vt/vtorc/inst/analysis_dao.go | 7 +++ go/vt/vtorc/inst/analysis_dao_test.go | 62 +++++++++++++++++-- go/vt/vtorc/inst/instance.go | 1 + go/vt/vtorc/inst/instance_dao.go | 4 ++ go/vt/vtorc/inst/instance_dao_test.go | 20 +++--- go/vt/vtorc/logic/topology_recovery.go | 2 +- go/vt/vtorc/logic/topology_recovery_test.go | 15 +++++ go/vt/vtorc/test/recovery_analysis.go | 2 + .../vttablet/tabletmanager/rpc_replication.go | 1 + go/vt/vttablet/tabletserver/controller.go | 3 + .../tabletserver/semisyncmonitor/monitor.go | 4 +- go/vt/vttablet/tabletserver/tabletserver.go | 5 ++ go/vt/vttablet/tabletservermock/controller.go | 6 ++ proto/replicationdata.proto | 1 + web/vtadmin/src/proto/vtadmin.d.ts | 6 ++ web/vtadmin/src/proto/vtadmin.js | 23 +++++++ 21 files changed, 208 insertions(+), 25 deletions(-) diff --git a/go/test/endtoend/vtorc/readtopologyinstance/main_test.go b/go/test/endtoend/vtorc/readtopologyinstance/main_test.go index 6a565ac046f..a73afdc946d 100644 --- a/go/test/endtoend/vtorc/readtopologyinstance/main_test.go +++ b/go/test/endtoend/vtorc/readtopologyinstance/main_test.go @@ -91,6 +91,7 @@ func TestReadTopologyInstanceBufferable(t *testing.T) { assert.True(t, primaryInstance.SemiSyncReplicaEnabled) assert.True(t, primaryInstance.SemiSyncPrimaryStatus) assert.False(t, primaryInstance.SemiSyncReplicaStatus) + assert.False(t, primaryInstance.SemiSyncMonitorBlocked) assert.EqualValues(t, 2, primaryInstance.SemiSyncPrimaryClients) assert.EqualValues(t, 1, primaryInstance.SemiSyncPrimaryWaitForReplicaCount) assert.EqualValues(t, 1000000000000000000, primaryInstance.SemiSyncPrimaryTimeout) @@ -142,6 +143,7 @@ func TestReadTopologyInstanceBufferable(t *testing.T) { assert.False(t, replicaInstance.SemiSyncPrimaryEnabled) assert.True(t, replicaInstance.SemiSyncReplicaEnabled) assert.False(t, replicaInstance.SemiSyncPrimaryStatus) + assert.False(t, replicaInstance.SemiSyncMonitorBlocked) assert.True(t, replicaInstance.SemiSyncReplicaStatus) assert.EqualValues(t, 0, replicaInstance.SemiSyncPrimaryClients) assert.EqualValues(t, 1, replicaInstance.SemiSyncPrimaryWaitForReplicaCount) diff --git a/go/vt/proto/replicationdata/replicationdata.pb.go b/go/vt/proto/replicationdata/replicationdata.pb.go index d99881e2cad..e4cfdbd8ad6 100644 --- a/go/vt/proto/replicationdata/replicationdata.pb.go +++ b/go/vt/proto/replicationdata/replicationdata.pb.go @@ -504,6 +504,7 @@ type FullStatus struct { SuperReadOnly bool `protobuf:"varint,21,opt,name=super_read_only,json=superReadOnly,proto3" json:"super_read_only,omitempty"` ReplicationConfiguration *Configuration `protobuf:"bytes,22,opt,name=replication_configuration,json=replicationConfiguration,proto3" json:"replication_configuration,omitempty"` DiskStalled bool `protobuf:"varint,23,opt,name=disk_stalled,json=diskStalled,proto3" json:"disk_stalled,omitempty"` + SemiSyncMonitorBlocked bool `protobuf:"varint,24,opt,name=semi_sync_monitor_blocked,json=semiSyncMonitorBlocked,proto3" json:"semi_sync_monitor_blocked,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -699,6 +700,13 @@ func (x *FullStatus) GetDiskStalled() bool { return false } +func (x *FullStatus) GetSemiSyncMonitorBlocked() bool { + if x != nil { + return x.SemiSyncMonitorBlocked + } + return false +} + var File_replicationdata_proto protoreflect.FileDescriptor var file_replicationdata_proto_rawDesc = string([]byte{ @@ -786,7 +794,7 @@ var file_replicationdata_proto_rawDesc = string([]byte{ 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x69, 0x6c, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, - 0x55, 0x75, 0x69, 0x64, 0x22, 0xeb, 0x08, 0x0a, 0x0a, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, + 0x55, 0x75, 0x69, 0x64, 0x22, 0xa6, 0x09, 0x0a, 0x0a, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, @@ -857,14 +865,18 @@ var file_replicationdata_proto_rawDesc = string([]byte{ 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x65, 0x64, 0x18, 0x17, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x64, 0x69, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x6c, 0x6c, - 0x65, 0x64, 0x2a, 0x3b, 0x0a, 0x13, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x12, 0x0a, 0x0e, 0x49, 0x4f, 0x41, - 0x4e, 0x44, 0x53, 0x51, 0x4c, 0x54, 0x48, 0x52, 0x45, 0x41, 0x44, 0x10, 0x00, 0x12, 0x10, 0x0a, - 0x0c, 0x49, 0x4f, 0x54, 0x48, 0x52, 0x45, 0x41, 0x44, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0x01, 0x42, - 0x2e, 0x5a, 0x2c, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, - 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, - 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x64, 0x61, 0x74, 0x61, 0x62, - 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x65, 0x64, 0x12, 0x39, 0x0a, 0x19, 0x73, 0x65, 0x6d, 0x69, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, + 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x18, + 0x18, 0x20, 0x01, 0x28, 0x08, 0x52, 0x16, 0x73, 0x65, 0x6d, 0x69, 0x53, 0x79, 0x6e, 0x63, 0x4d, + 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x2a, 0x3b, 0x0a, + 0x13, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x12, 0x0a, 0x0e, 0x49, 0x4f, 0x41, 0x4e, 0x44, 0x53, 0x51, 0x4c, + 0x54, 0x48, 0x52, 0x45, 0x41, 0x44, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x49, 0x4f, 0x54, 0x48, + 0x52, 0x45, 0x41, 0x44, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0x01, 0x42, 0x2e, 0x5a, 0x2c, 0x76, 0x69, + 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, + 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, }) var ( diff --git a/go/vt/proto/replicationdata/replicationdata_vtproto.pb.go b/go/vt/proto/replicationdata/replicationdata_vtproto.pb.go index 92f8e3074c3..b7ae5481a33 100644 --- a/go/vt/proto/replicationdata/replicationdata_vtproto.pb.go +++ b/go/vt/proto/replicationdata/replicationdata_vtproto.pb.go @@ -143,6 +143,7 @@ func (m *FullStatus) CloneVT() *FullStatus { r.SuperReadOnly = m.SuperReadOnly r.ReplicationConfiguration = m.ReplicationConfiguration.CloneVT() r.DiskStalled = m.DiskStalled + r.SemiSyncMonitorBlocked = m.SemiSyncMonitorBlocked if len(m.unknownFields) > 0 { r.unknownFields = make([]byte, len(m.unknownFields)) copy(r.unknownFields, m.unknownFields) @@ -553,6 +554,18 @@ func (m *FullStatus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.SemiSyncMonitorBlocked { + i-- + if m.SemiSyncMonitorBlocked { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xc0 + } if m.DiskStalled { i-- if m.DiskStalled { @@ -991,6 +1004,9 @@ func (m *FullStatus) SizeVT() (n int) { if m.DiskStalled { n += 3 } + if m.SemiSyncMonitorBlocked { + n += 3 + } n += len(m.unknownFields) return n } @@ -2587,6 +2603,26 @@ func (m *FullStatus) UnmarshalVT(dAtA []byte) error { } } m.DiskStalled = bool(v != 0) + case 24: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SemiSyncMonitorBlocked", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.SemiSyncMonitorBlocked = bool(v != 0) default: iNdEx = preIndex skippy, err := protohelpers.Skip(dAtA[iNdEx:]) diff --git a/go/vt/vtorc/db/generate_base.go b/go/vt/vtorc/db/generate_base.go index 8baa9a12476..03e89147c00 100644 --- a/go/vt/vtorc/db/generate_base.go +++ b/go/vt/vtorc/db/generate_base.go @@ -105,6 +105,7 @@ CREATE TABLE database_instance ( semi_sync_primary_status TINYint NOT NULL DEFAULT 0, semi_sync_replica_status TINYint NOT NULL DEFAULT 0, semi_sync_primary_clients int NOT NULL DEFAULT 0, + semi_sync_monitor_blocked tinyint NOT NULL DEFAULT 0, is_disk_stalled TINYint NOT NULL DEFAULT 0, PRIMARY KEY (alias) )`, diff --git a/go/vt/vtorc/inst/analysis.go b/go/vt/vtorc/inst/analysis.go index 6a800e5ee0b..4c31389cd3c 100644 --- a/go/vt/vtorc/inst/analysis.go +++ b/go/vt/vtorc/inst/analysis.go @@ -55,6 +55,7 @@ const ( AllPrimaryReplicasNotReplicatingOrDead AnalysisCode = "AllPrimaryReplicasNotReplicatingOrDead" LockedSemiSyncPrimaryHypothesis AnalysisCode = "LockedSemiSyncPrimaryHypothesis" LockedSemiSyncPrimary AnalysisCode = "LockedSemiSyncPrimary" + PrimarySemiSyncBlocked AnalysisCode = "PrimarySemiSyncBlocked" ErrantGTIDDetected AnalysisCode = "ErrantGTIDDetected" PrimaryDiskStalled AnalysisCode = "PrimaryDiskStalled" ) @@ -115,6 +116,7 @@ type ReplicationAnalysis struct { SemiSyncPrimaryWaitForReplicaCount uint SemiSyncPrimaryClients uint SemiSyncReplicaEnabled bool + SemiSyncMonitorBlocked bool CountSemiSyncReplicasEnabled uint CountLoggingReplicas uint CountStatementBasedLoggingReplicas uint diff --git a/go/vt/vtorc/inst/analysis_dao.go b/go/vt/vtorc/inst/analysis_dao.go index d487973b0f0..70b6d98e671 100644 --- a/go/vt/vtorc/inst/analysis_dao.go +++ b/go/vt/vtorc/inst/analysis_dao.go @@ -152,6 +152,9 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna MIN( primary_instance.semi_sync_primary_status ) AS semi_sync_primary_status, + MIN( + primary_instance.semi_sync_monitor_blocked + ) AS semi_sync_monitor_blocked, MIN( primary_instance.semi_sync_replica_enabled ) AS semi_sync_replica_enabled, @@ -333,6 +336,7 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna a.BinlogServerImmediateTopology = countValidBinlogServerReplicas == a.CountValidReplicas && a.CountValidReplicas > 0 a.SemiSyncPrimaryEnabled = m.GetBool("semi_sync_primary_enabled") a.SemiSyncPrimaryStatus = m.GetBool("semi_sync_primary_status") + a.SemiSyncMonitorBlocked = m.GetBool("semi_sync_monitor_blocked") a.SemiSyncReplicaEnabled = m.GetBool("semi_sync_replica_enabled") a.CountSemiSyncReplicasEnabled = m.GetUint("count_semi_sync_replicas") // countValidSemiSyncReplicasEnabled := m.GetUint("count_valid_semi_sync_replicas") @@ -505,6 +509,9 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna a.Description = "Semi sync primary seems to be locked, more samplings needed to validate" } // + } else if a.IsPrimary && a.SemiSyncMonitorBlocked { + a.Analysis = PrimarySemiSyncBlocked + a.Description = "Writes seem to be blocked on semi-sync acks on the primary, even though sufficient replicas are configured to send ACKs" } else if a.IsPrimary && a.LastCheckValid && a.CountReplicas == 1 && a.CountValidReplicas == a.CountReplicas && a.CountValidReplicatingReplicas == 0 { a.Analysis = PrimarySingleReplicaNotReplicating a.Description = "Primary is reachable but its single replica is not replicating" diff --git a/go/vt/vtorc/inst/analysis_dao_test.go b/go/vt/vtorc/inst/analysis_dao_test.go index baa1121b776..a954cf7f3de 100644 --- a/go/vt/vtorc/inst/analysis_dao_test.go +++ b/go/vt/vtorc/inst/analysis_dao_test.go @@ -34,10 +34,10 @@ var ( // The initialSQL is a set of insert commands copied from a dump of an actual running VTOrc instances. The relevant insert commands are here. // This is a dump taken from a test running 4 tablets, zone1-101 is the primary, zone1-100 is a replica, zone1-112 is a rdonly and zone2-200 is a cross-cell replica. initialSQL = []string{ - `INSERT INTO database_instance VALUES('zone1-0000000112','localhost',6747,'2022-12-28 07:26:04','2022-12-28 07:26:04',213696377,'8.0.31','ROW',1,1,'vt-0000000112-bin.000001',15963,'localhost',6714,8,4.0,1,1,'vt-0000000101-bin.000001',15583,'vt-0000000101-bin.000001',15583,0,0,1,'','',1,'vt-0000000112-relay-bin.000002',15815,1,0,'zone1','',0,0,0,1,'729a4cc4-8680-11ed-a104-47706090afbd:1-54','729a5138-8680-11ed-9240-92a06c3be3c2','2022-12-28 07:26:04','',1,0,0,'Homebrew','8.0','FULL',10816929,0,0,'ON',1,'729a4cc4-8680-11ed-a104-47706090afbd','','729a4cc4-8680-11ed-a104-47706090afbd,729a5138-8680-11ed-9240-92a06c3be3c2',1,1,'',1000000000000000000,1,0,0,0,false);`, - `INSERT INTO database_instance VALUES('zone1-0000000100','localhost',6711,'2022-12-28 07:26:04','2022-12-28 07:26:04',1094500338,'8.0.31','ROW',1,1,'vt-0000000100-bin.000001',15963,'localhost',6714,8,4.0,1,1,'vt-0000000101-bin.000001',15583,'vt-0000000101-bin.000001',15583,0,0,1,'','',1,'vt-0000000100-relay-bin.000002',15815,1,0,'zone1','',0,0,0,1,'729a4cc4-8680-11ed-a104-47706090afbd:1-54','729a5138-8680-11ed-acf8-d6b0ef9f4eaa','2022-12-28 07:26:04','',1,0,0,'Homebrew','8.0','FULL',10103920,0,1,'ON',1,'729a4cc4-8680-11ed-a104-47706090afbd','','729a4cc4-8680-11ed-a104-47706090afbd,729a5138-8680-11ed-acf8-d6b0ef9f4eaa',1,1,'',1000000000000000000,1,0,1,0,false);`, - `INSERT INTO database_instance VALUES('zone1-0000000101','localhost',6714,'2022-12-28 07:26:04','2022-12-28 07:26:04',390954723,'8.0.31','ROW',1,1,'vt-0000000101-bin.000001',15583,'',0,0,0,0,0,'',0,'',0,NULL,NULL,0,'','',0,'',0,0,0,'zone1','',0,0,0,1,'729a4cc4-8680-11ed-a104-47706090afbd:1-54','729a4cc4-8680-11ed-a104-47706090afbd','2022-12-28 07:26:04','',0,0,0,'Homebrew','8.0','FULL',11366095,1,1,'ON',1,'','','729a4cc4-8680-11ed-a104-47706090afbd',-1,-1,'',1000000000000000000,1,1,0,2,false);`, - `INSERT INTO database_instance VALUES('zone2-0000000200','localhost',6756,'2022-12-28 07:26:05','2022-12-28 07:26:05',444286571,'8.0.31','ROW',1,1,'vt-0000000200-bin.000001',15963,'localhost',6714,8,4.0,1,1,'vt-0000000101-bin.000001',15583,'vt-0000000101-bin.000001',15583,0,0,1,'','',1,'vt-0000000200-relay-bin.000002',15815,1,0,'zone2','',0,0,0,1,'729a4cc4-8680-11ed-a104-47706090afbd:1-54','729a497c-8680-11ed-8ad4-3f51d747db75','2022-12-28 07:26:05','',1,0,0,'Homebrew','8.0','FULL',10443112,0,1,'ON',1,'729a4cc4-8680-11ed-a104-47706090afbd','','729a4cc4-8680-11ed-a104-47706090afbd,729a497c-8680-11ed-8ad4-3f51d747db75',1,1,'',1000000000000000000,1,0,1,0,false);`, + `INSERT INTO database_instance VALUES('zone1-0000000112','localhost',6747,'2022-12-28 07:26:04','2022-12-28 07:26:04',213696377,'8.0.31','ROW',1,1,'vt-0000000112-bin.000001',15963,'localhost',6714,8,4.0,1,1,'vt-0000000101-bin.000001',15583,'vt-0000000101-bin.000001',15583,0,0,1,'','',1,'vt-0000000112-relay-bin.000002',15815,1,0,'zone1','',0,0,0,1,'729a4cc4-8680-11ed-a104-47706090afbd:1-54','729a5138-8680-11ed-9240-92a06c3be3c2','2022-12-28 07:26:04','',1,0,0,'Homebrew','8.0','FULL',10816929,0,0,'ON',1,'729a4cc4-8680-11ed-a104-47706090afbd','','729a4cc4-8680-11ed-a104-47706090afbd,729a5138-8680-11ed-9240-92a06c3be3c2',1,1,'',1000000000000000000,1,0,0,0,false,false);`, + `INSERT INTO database_instance VALUES('zone1-0000000100','localhost',6711,'2022-12-28 07:26:04','2022-12-28 07:26:04',1094500338,'8.0.31','ROW',1,1,'vt-0000000100-bin.000001',15963,'localhost',6714,8,4.0,1,1,'vt-0000000101-bin.000001',15583,'vt-0000000101-bin.000001',15583,0,0,1,'','',1,'vt-0000000100-relay-bin.000002',15815,1,0,'zone1','',0,0,0,1,'729a4cc4-8680-11ed-a104-47706090afbd:1-54','729a5138-8680-11ed-acf8-d6b0ef9f4eaa','2022-12-28 07:26:04','',1,0,0,'Homebrew','8.0','FULL',10103920,0,1,'ON',1,'729a4cc4-8680-11ed-a104-47706090afbd','','729a4cc4-8680-11ed-a104-47706090afbd,729a5138-8680-11ed-acf8-d6b0ef9f4eaa',1,1,'',1000000000000000000,1,0,1,0,false,false);`, + `INSERT INTO database_instance VALUES('zone1-0000000101','localhost',6714,'2022-12-28 07:26:04','2022-12-28 07:26:04',390954723,'8.0.31','ROW',1,1,'vt-0000000101-bin.000001',15583,'',0,0,0,0,0,'',0,'',0,NULL,NULL,0,'','',0,'',0,0,0,'zone1','',0,0,0,1,'729a4cc4-8680-11ed-a104-47706090afbd:1-54','729a4cc4-8680-11ed-a104-47706090afbd','2022-12-28 07:26:04','',0,0,0,'Homebrew','8.0','FULL',11366095,1,1,'ON',1,'','','729a4cc4-8680-11ed-a104-47706090afbd',-1,-1,'',1000000000000000000,1,1,0,2,false,false);`, + `INSERT INTO database_instance VALUES('zone2-0000000200','localhost',6756,'2022-12-28 07:26:05','2022-12-28 07:26:05',444286571,'8.0.31','ROW',1,1,'vt-0000000200-bin.000001',15963,'localhost',6714,8,4.0,1,1,'vt-0000000101-bin.000001',15583,'vt-0000000101-bin.000001',15583,0,0,1,'','',1,'vt-0000000200-relay-bin.000002',15815,1,0,'zone2','',0,0,0,1,'729a4cc4-8680-11ed-a104-47706090afbd:1-54','729a497c-8680-11ed-8ad4-3f51d747db75','2022-12-28 07:26:05','',1,0,0,'Homebrew','8.0','FULL',10443112,0,1,'ON',1,'729a4cc4-8680-11ed-a104-47706090afbd','','729a4cc4-8680-11ed-a104-47706090afbd,729a497c-8680-11ed-8ad4-3f51d747db75',1,1,'',1000000000000000000,1,0,1,0,false,false);`, `INSERT INTO vitess_tablet VALUES('zone1-0000000100','localhost',6711,'ks','0','zone1',2,'0001-01-01 00:00:00+00:00',X'616c6961733a7b63656c6c3a227a6f6e653122207569643a3130307d20686f73746e616d653a226c6f63616c686f73742220706f72745f6d61703a7b6b65793a2267727063222076616c75653a363731307d20706f72745f6d61703a7b6b65793a227674222076616c75653a363730397d206b657973706163653a226b73222073686172643a22302220747970653a5245504c494341206d7973716c5f686f73746e616d653a226c6f63616c686f737422206d7973716c5f706f72743a363731312064625f7365727665725f76657273696f6e3a22382e302e3331222064656661756c745f636f6e6e5f636f6c6c6174696f6e3a3435');`, `INSERT INTO vitess_tablet VALUES('zone1-0000000101','localhost',6714,'ks','0','zone1',1,'2022-12-28 07:23:25.129898+00:00',X'616c6961733a7b63656c6c3a227a6f6e653122207569643a3130317d20686f73746e616d653a226c6f63616c686f73742220706f72745f6d61703a7b6b65793a2267727063222076616c75653a363731337d20706f72745f6d61703a7b6b65793a227674222076616c75653a363731327d206b657973706163653a226b73222073686172643a22302220747970653a5052494d415259206d7973716c5f686f73746e616d653a226c6f63616c686f737422206d7973716c5f706f72743a36373134207072696d6172795f7465726d5f73746172745f74696d653a7b7365636f6e64733a31363732323132323035206e616e6f7365636f6e64733a3132393839383030307d2064625f7365727665725f76657273696f6e3a22382e302e3331222064656661756c745f636f6e6e5f636f6c6c6174696f6e3a3435');`, `INSERT INTO vitess_tablet VALUES('zone1-0000000112','localhost',6747,'ks','0','zone1',3,'0001-01-01 00:00:00+00:00',X'616c6961733a7b63656c6c3a227a6f6e653122207569643a3131327d20686f73746e616d653a226c6f63616c686f73742220706f72745f6d61703a7b6b65793a2267727063222076616c75653a363734367d20706f72745f6d61703a7b6b65793a227674222076616c75653a363734357d206b657973706163653a226b73222073686172643a22302220747970653a52444f4e4c59206d7973716c5f686f73746e616d653a226c6f63616c686f737422206d7973716c5f706f72743a363734372064625f7365727665725f76657273696f6e3a22382e302e3331222064656661756c745f636f6e6e5f636f6c6c6174696f6e3a3435');`, @@ -119,6 +119,60 @@ func TestGetReplicationAnalysisDecision(t *testing.T) { keyspaceWanted: "ks", shardWanted: "0", codeWanted: PrimaryDiskStalled, + }, { + name: "PrimarySemiSyncBlocked", + info: []*test.InfoForRecoveryAnalysis{{ + TabletInfo: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{Cell: "zon1", Uid: 100}, + Hostname: "localhost", + Keyspace: "ks", + Shard: "0", + Type: topodatapb.TabletType_PRIMARY, + MysqlHostname: "localhost", + MysqlPort: 6709, + }, + DurabilityPolicy: "semi_sync", + LastCheckValid: 1, + CountReplicas: 4, + CountValidReplicas: 4, + CountValidReplicatingReplicas: 4, + IsPrimary: 1, + SemiSyncPrimaryEnabled: 1, + SemiSyncPrimaryStatus: 1, + SemiSyncPrimaryWaitForReplicaCount: 2, + SemiSyncPrimaryClients: 2, + SemiSyncMonitorBlocked: 1, + }}, + keyspaceWanted: "ks", + shardWanted: "0", + codeWanted: PrimarySemiSyncBlocked, + }, { + name: "LockedSemiSync", + info: []*test.InfoForRecoveryAnalysis{{ + TabletInfo: &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{Cell: "zon1", Uid: 100}, + Hostname: "localhost", + Keyspace: "ks", + Shard: "0", + Type: topodatapb.TabletType_PRIMARY, + MysqlHostname: "localhost", + MysqlPort: 6709, + }, + DurabilityPolicy: "semi_sync", + LastCheckValid: 1, + CountReplicas: 4, + CountValidReplicas: 4, + CountValidReplicatingReplicas: 4, + IsPrimary: 1, + SemiSyncPrimaryEnabled: 1, + SemiSyncPrimaryStatus: 1, + SemiSyncPrimaryWaitForReplicaCount: 2, + SemiSyncPrimaryClients: 1, + SemiSyncMonitorBlocked: 1, + }}, + keyspaceWanted: "ks", + shardWanted: "0", + codeWanted: LockedSemiSyncPrimaryHypothesis, }, { name: "DeadPrimary", info: []*test.InfoForRecoveryAnalysis{{ diff --git a/go/vt/vtorc/inst/instance.go b/go/vt/vtorc/inst/instance.go index b7b097bb14d..ba4a619d831 100644 --- a/go/vt/vtorc/inst/instance.go +++ b/go/vt/vtorc/inst/instance.go @@ -85,6 +85,7 @@ type Instance struct { SemiSyncPrimaryStatus bool SemiSyncPrimaryClients uint SemiSyncReplicaStatus bool + SemiSyncMonitorBlocked bool LastSeenTimestamp string IsLastCheckValid bool diff --git a/go/vt/vtorc/inst/instance_dao.go b/go/vt/vtorc/inst/instance_dao.go index 9e35e6e3e0b..e150a119d6d 100644 --- a/go/vt/vtorc/inst/instance_dao.go +++ b/go/vt/vtorc/inst/instance_dao.go @@ -240,6 +240,7 @@ func ReadTopologyInstanceBufferable(tabletAlias string, latency *stopwatch.Named instance.SemiSyncPrimaryClients = uint(fs.SemiSyncPrimaryClients) instance.SemiSyncPrimaryStatus = fs.SemiSyncPrimaryStatus instance.SemiSyncReplicaStatus = fs.SemiSyncReplicaStatus + instance.SemiSyncMonitorBlocked = fs.SemiSyncMonitorBlocked if instance.IsOracleMySQL() || instance.IsPercona() { // Stuff only supported on Oracle / Percona MySQL @@ -579,6 +580,7 @@ func readInstanceRow(m sqlutils.RowMap) *Instance { instance.SemiSyncPrimaryStatus = m.GetBool("semi_sync_primary_status") instance.SemiSyncPrimaryClients = m.GetUint("semi_sync_primary_clients") instance.SemiSyncReplicaStatus = m.GetBool("semi_sync_replica_status") + instance.SemiSyncMonitorBlocked = m.GetBool("semi_sync_monitor_blocked") instance.ReplicationDepth = m.GetUint("replication_depth") instance.IsCoPrimary = m.GetBool("is_co_primary") instance.HasReplicationCredentials = m.GetBool("has_replication_credentials") @@ -879,6 +881,7 @@ func mkInsertForInstances(instances []*Instance, instanceWasActuallyFound bool, "semi_sync_primary_status", "semi_sync_primary_clients", "semi_sync_replica_status", + "semi_sync_monitor_blocked", "last_discovery_latency", "is_disk_stalled", } @@ -959,6 +962,7 @@ func mkInsertForInstances(instances []*Instance, instanceWasActuallyFound bool, args = append(args, instance.SemiSyncPrimaryStatus) args = append(args, instance.SemiSyncPrimaryClients) args = append(args, instance.SemiSyncReplicaStatus) + args = append(args, instance.SemiSyncMonitorBlocked) args = append(args, instance.LastDiscoveryLatency.Nanoseconds()) args = append(args, instance.StalledDisk) } diff --git a/go/vt/vtorc/inst/instance_dao_test.go b/go/vt/vtorc/inst/instance_dao_test.go index c3b99455741..8276befe267 100644 --- a/go/vt/vtorc/inst/instance_dao_test.go +++ b/go/vt/vtorc/inst/instance_dao_test.go @@ -64,13 +64,13 @@ func TestMkInsertSingle(t *testing.T) { version, major_version, version_comment, binlog_server, read_only, binlog_format, binlog_row_image, log_bin, log_replica_updates, binary_log_file, binary_log_pos, source_host, source_port, replica_net_timeout, heartbeat_interval, replica_sql_running, replica_io_running, replication_sql_thread_state, replication_io_thread_state, has_replication_filters, supports_oracle_gtid, oracle_gtid, source_uuid, ancestry_uuid, executed_gtid_set, gtid_mode, gtid_purged, gtid_errant, - source_log_file, read_source_log_pos, relay_source_log_file, exec_source_log_pos, relay_log_file, relay_log_pos, last_sql_error, last_io_error, replication_lag_seconds, replica_lag_seconds, sql_delay, data_center, region, physical_environment, replication_depth, is_co_primary, has_replication_credentials, allow_tls, semi_sync_enforced, semi_sync_primary_enabled, semi_sync_primary_timeout, semi_sync_primary_wait_for_replica_count, semi_sync_replica_enabled, semi_sync_primary_status, semi_sync_primary_clients, semi_sync_replica_status, last_discovery_latency, is_disk_stalled, last_seen) + source_log_file, read_source_log_pos, relay_source_log_file, exec_source_log_pos, relay_log_file, relay_log_pos, last_sql_error, last_io_error, replication_lag_seconds, replica_lag_seconds, sql_delay, data_center, region, physical_environment, replication_depth, is_co_primary, has_replication_credentials, allow_tls, semi_sync_enforced, semi_sync_primary_enabled, semi_sync_primary_timeout, semi_sync_primary_wait_for_replica_count, semi_sync_replica_enabled, semi_sync_primary_status, semi_sync_primary_clients, semi_sync_replica_status, semi_sync_monitor_blocked, last_discovery_latency, is_disk_stalled, last_seen) VALUES - (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')) + (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')) ` a1 := `zone1-i710, i710, 3306, 710, , 5.6.7, 5.6, MySQL, false, false, STATEMENT, FULL, false, false, , 0, , 0, 0, 0, - false, false, 0, 0, false, false, false, , , , , , , , 0, mysql.000007, 10, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false, 0, false,` + false, false, 0, 0, false, false, false, , , , , , , , 0, mysql.000007, 10, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false, false, 0, false,` sql1, args1, err := mkInsertForInstances(instances[:1], false, true) require.NoError(t, err) @@ -87,16 +87,16 @@ func TestMkInsertThree(t *testing.T) { version, major_version, version_comment, binlog_server, read_only, binlog_format, binlog_row_image, log_bin, log_replica_updates, binary_log_file, binary_log_pos, source_host, source_port, replica_net_timeout, heartbeat_interval, replica_sql_running, replica_io_running, replication_sql_thread_state, replication_io_thread_state, has_replication_filters, supports_oracle_gtid, oracle_gtid, source_uuid, ancestry_uuid, executed_gtid_set, gtid_mode, gtid_purged, gtid_errant, - source_log_file, read_source_log_pos, relay_source_log_file, exec_source_log_pos, relay_log_file, relay_log_pos, last_sql_error, last_io_error, replication_lag_seconds, replica_lag_seconds, sql_delay, data_center, region, physical_environment, replication_depth, is_co_primary, has_replication_credentials, allow_tls, semi_sync_enforced, semi_sync_primary_enabled, semi_sync_primary_timeout, semi_sync_primary_wait_for_replica_count, semi_sync_replica_enabled, semi_sync_primary_status, semi_sync_primary_clients, semi_sync_replica_status, last_discovery_latency, is_disk_stalled, last_seen) + source_log_file, read_source_log_pos, relay_source_log_file, exec_source_log_pos, relay_log_file, relay_log_pos, last_sql_error, last_io_error, replication_lag_seconds, replica_lag_seconds, sql_delay, data_center, region, physical_environment, replication_depth, is_co_primary, has_replication_credentials, allow_tls, semi_sync_enforced, semi_sync_primary_enabled, semi_sync_primary_timeout, semi_sync_primary_wait_for_replica_count, semi_sync_replica_enabled, semi_sync_primary_status, semi_sync_primary_clients, semi_sync_replica_status, semi_sync_monitor_blocked, last_discovery_latency, is_disk_stalled, last_seen) VALUES - (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')), - (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')), - (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')) + (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')), + (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')), + (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')) ` a3 := ` - zone1-i710, i710, 3306, 710, , 5.6.7, 5.6, MySQL, false, false, STATEMENT, FULL, false, false, , 0, , 0, 0, 0, false, false, 0, 0, false, false, false, , , , , , , , 0, mysql.000007, 10, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false, 0, false, - zone1-i720, i720, 3306, 720, , 5.6.7, 5.6, MySQL, false, false, STATEMENT, FULL, false, false, , 0, , 0, 0, 0, false, false, 0, 0, false, false, false, , , , , , , , 0, mysql.000007, 20, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false, 0, false, - zone1-i730, i730, 3306, 730, , 5.6.7, 5.6, MySQL, false, false, STATEMENT, FULL, false, false, , 0, , 0, 0, 0, false, false, 0, 0, false, false, false, , , , , , , , 0, mysql.000007, 30, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false, 0, false, + zone1-i710, i710, 3306, 710, , 5.6.7, 5.6, MySQL, false, false, STATEMENT, FULL, false, false, , 0, , 0, 0, 0, false, false, 0, 0, false, false, false, , , , , , , , 0, mysql.000007, 10, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false ,false, 0, false, + zone1-i720, i720, 3306, 720, , 5.6.7, 5.6, MySQL, false, false, STATEMENT, FULL, false, false, , 0, , 0, 0, 0, false, false, 0, 0, false, false, false, , , , , , , , 0, mysql.000007, 20, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false, false, 0, false, + zone1-i730, i730, 3306, 730, , 5.6.7, 5.6, MySQL, false, false, STATEMENT, FULL, false, false, , 0, , 0, 0, 0, false, false, 0, 0, false, false, false, , , , , , , , 0, mysql.000007, 30, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false, false, 0, false, ` sql3, args3, err := mkInsertForInstances(instances[:3], true, true) diff --git a/go/vt/vtorc/logic/topology_recovery.go b/go/vt/vtorc/logic/topology_recovery.go index 41c0ca5d398..82894e8b369 100644 --- a/go/vt/vtorc/logic/topology_recovery.go +++ b/go/vt/vtorc/logic/topology_recovery.go @@ -293,7 +293,7 @@ func checkAndRecoverGenericProblem(ctx context.Context, analysisEntry *inst.Repl func getCheckAndRecoverFunctionCode(analysisCode inst.AnalysisCode, tabletAlias string) recoveryFunction { switch analysisCode { // primary - case inst.DeadPrimary, inst.DeadPrimaryAndSomeReplicas, inst.PrimaryDiskStalled: + case inst.DeadPrimary, inst.DeadPrimaryAndSomeReplicas, inst.PrimaryDiskStalled, inst.PrimarySemiSyncBlocked: // If ERS is disabled, we have no way of repairing the cluster. if !config.ERSEnabled() { log.Infof("VTOrc not configured to run ERS, skipping recovering %v", analysisCode) diff --git a/go/vt/vtorc/logic/topology_recovery_test.go b/go/vt/vtorc/logic/topology_recovery_test.go index 9df5fc989f0..68579dd05cb 100644 --- a/go/vt/vtorc/logic/topology_recovery_test.go +++ b/go/vt/vtorc/logic/topology_recovery_test.go @@ -49,6 +49,11 @@ func TestAnalysisEntriesHaveSameRecovery(t *testing.T) { prevAnalysisCode: inst.DeadPrimary, newAnalysisCode: inst.PrimaryDiskStalled, shouldBeEqual: true, + }, { + // PrimarySemiSyncBlocked and PrimaryDiskStalled have the same recovery + prevAnalysisCode: inst.PrimarySemiSyncBlocked, + newAnalysisCode: inst.PrimaryDiskStalled, + shouldBeEqual: true, }, { // DeadPrimary and PrimaryTabletDeleted are different recoveries. prevAnalysisCode: inst.DeadPrimary, @@ -232,6 +237,16 @@ func TestGetCheckAndRecoverFunctionCode(t *testing.T) { ersEnabled: false, analysisCode: inst.PrimaryDiskStalled, wantRecoveryFunction: noRecoveryFunc, + }, { + name: "PrimarySemiSyncBlocked with ERS enabled", + ersEnabled: true, + analysisCode: inst.PrimarySemiSyncBlocked, + wantRecoveryFunction: recoverDeadPrimaryFunc, + }, { + name: "PrimarySemiSyncBlocked with ERS disabled", + ersEnabled: false, + analysisCode: inst.PrimarySemiSyncBlocked, + wantRecoveryFunction: noRecoveryFunc, }, { name: "PrimaryTabletDeleted with ERS enabled", ersEnabled: true, diff --git a/go/vt/vtorc/test/recovery_analysis.go b/go/vt/vtorc/test/recovery_analysis.go index bb6e4132243..98493f629a0 100644 --- a/go/vt/vtorc/test/recovery_analysis.go +++ b/go/vt/vtorc/test/recovery_analysis.go @@ -65,6 +65,7 @@ type InfoForRecoveryAnalysis struct { CountValidBinlogServerReplicas uint SemiSyncPrimaryEnabled int SemiSyncPrimaryStatus int + SemiSyncMonitorBlocked int SemiSyncPrimaryWaitForReplicaCount uint SemiSyncPrimaryClients uint SemiSyncReplicaEnabled int @@ -142,6 +143,7 @@ func (info *InfoForRecoveryAnalysis) ConvertToRowMap() sqlutils.RowMap { rowMap["semi_sync_primary_clients"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncPrimaryClients), Valid: true} rowMap["semi_sync_primary_enabled"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncPrimaryEnabled), Valid: true} rowMap["semi_sync_primary_status"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncPrimaryStatus), Valid: true} + rowMap["semi_sync_monitor_blocked"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncMonitorBlocked), Valid: true} rowMap["semi_sync_primary_wait_for_replica_count"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncPrimaryWaitForReplicaCount), Valid: true} rowMap["semi_sync_replica_enabled"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncReplicaEnabled), Valid: true} res, _ := prototext.Marshal(info.TabletInfo) diff --git a/go/vt/vttablet/tabletmanager/rpc_replication.go b/go/vt/vttablet/tabletmanager/rpc_replication.go index 471b959e643..88337e2a692 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication.go @@ -187,6 +187,7 @@ func (tm *TabletManager) FullStatus(ctx context.Context) (*replicationdatapb.Ful SemiSyncPrimaryClients: semiSyncClients, SemiSyncPrimaryTimeout: semiSyncTimeout, SemiSyncWaitForReplicaCount: semiSyncNumReplicas, + SemiSyncMonitorBlocked: tm.QueryServiceControl.SemiSyncMonitorBlocked(), SuperReadOnly: superReadOnly, ReplicationConfiguration: replConfiguration, }, nil diff --git a/go/vt/vttablet/tabletserver/controller.go b/go/vt/vttablet/tabletserver/controller.go index 057eab3218a..ae67d19873b 100644 --- a/go/vt/vttablet/tabletserver/controller.go +++ b/go/vt/vttablet/tabletserver/controller.go @@ -128,6 +128,9 @@ type Controller interface { // WaitUntilSemiSyncBeingUnblocked waits until the primary is not blocked on semi-sync. WaitUntilSemiSyncBeingUnblocked(ctx context.Context) error + + // SemiSyncMonitorBlocked returns whether the semi-sync monitor has all its writes blocked. + SemiSyncMonitorBlocked() bool } // Ensure TabletServer satisfies Controller interface. diff --git a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go index 03187a2c5ca..aaebdcfa0b2 100644 --- a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go @@ -265,7 +265,9 @@ func (m *Monitor) incrementWriteCount() bool { // AllWritesBlocked returns if maxWritesPermitted number of writes // are already outstanding. func (m *Monitor) AllWritesBlocked() bool { - return m.inProgressWriteCount == maxWritesPermitted + m.mu.Lock() + defer m.mu.Unlock() + return m.isOpen && m.inProgressWriteCount == maxWritesPermitted } // decrementWriteCount decrements the write count. diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 5d78ab05905..50363db3876 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -783,6 +783,11 @@ func (tsv *TabletServer) WaitUntilSemiSyncBeingUnblocked(ctx context.Context) er return tsv.semiSyncMonitor.WaitUntilSemiSyncUnblocked(ctx) } +// SemiSyncMonitorBlocked returns whether the semi-sync monitor has all its writes blocked. +func (tsv *TabletServer) SemiSyncMonitorBlocked() bool { + return tsv.semiSyncMonitor.AllWritesBlocked() +} + // CreateTransaction creates the metadata for a 2PC transaction. func (tsv *TabletServer) CreateTransaction(ctx context.Context, target *querypb.Target, dtid string, participants []*querypb.Target) (err error) { return tsv.execRequest( diff --git a/go/vt/vttablet/tabletservermock/controller.go b/go/vt/vttablet/tabletservermock/controller.go index 20fadc11bc8..86c156557e1 100644 --- a/go/vt/vttablet/tabletservermock/controller.go +++ b/go/vt/vttablet/tabletservermock/controller.go @@ -291,6 +291,12 @@ func (tqsc *Controller) WaitUntilSemiSyncBeingUnblocked(context.Context) error { return nil } +// SemiSyncMonitorBlocked is part of the tabletserver.Controller interface +func (tqsc *Controller) SemiSyncMonitorBlocked() bool { + tqsc.MethodCalled["SemiSyncMonitorBlocked"] = true + return false +} + // EnterLameduck implements tabletserver.Controller. func (tqsc *Controller) EnterLameduck() { tqsc.mu.Lock() diff --git a/proto/replicationdata.proto b/proto/replicationdata.proto index eba4d323ee6..1cc90a91f1b 100644 --- a/proto/replicationdata.proto +++ b/proto/replicationdata.proto @@ -106,4 +106,5 @@ message FullStatus { bool super_read_only = 21; replicationdata.Configuration replication_configuration = 22; bool disk_stalled = 23; + bool semi_sync_monitor_blocked = 24; } diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts index 527adc01326..9cd09e771a1 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -48801,6 +48801,9 @@ export namespace replicationdata { /** FullStatus disk_stalled */ disk_stalled?: (boolean|null); + + /** FullStatus semi_sync_monitor_blocked */ + semi_sync_monitor_blocked?: (boolean|null); } /** Represents a FullStatus. */ @@ -48881,6 +48884,9 @@ export namespace replicationdata { /** FullStatus disk_stalled. */ public disk_stalled: boolean; + /** FullStatus semi_sync_monitor_blocked. */ + public semi_sync_monitor_blocked: boolean; + /** * Creates a new FullStatus instance using the specified properties. * @param [properties] Properties to set diff --git a/web/vtadmin/src/proto/vtadmin.js b/web/vtadmin/src/proto/vtadmin.js index 1209c59cbe9..9a73f5640f6 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -118503,6 +118503,7 @@ export const replicationdata = $root.replicationdata = (() => { * @property {boolean|null} [super_read_only] FullStatus super_read_only * @property {replicationdata.IConfiguration|null} [replication_configuration] FullStatus replication_configuration * @property {boolean|null} [disk_stalled] FullStatus disk_stalled + * @property {boolean|null} [semi_sync_monitor_blocked] FullStatus semi_sync_monitor_blocked */ /** @@ -118704,6 +118705,14 @@ export const replicationdata = $root.replicationdata = (() => { */ FullStatus.prototype.disk_stalled = false; + /** + * FullStatus semi_sync_monitor_blocked. + * @member {boolean} semi_sync_monitor_blocked + * @memberof replicationdata.FullStatus + * @instance + */ + FullStatus.prototype.semi_sync_monitor_blocked = false; + /** * Creates a new FullStatus instance using the specified properties. * @function create @@ -118774,6 +118783,8 @@ export const replicationdata = $root.replicationdata = (() => { $root.replicationdata.Configuration.encode(message.replication_configuration, writer.uint32(/* id 22, wireType 2 =*/178).fork()).ldelim(); if (message.disk_stalled != null && Object.hasOwnProperty.call(message, "disk_stalled")) writer.uint32(/* id 23, wireType 0 =*/184).bool(message.disk_stalled); + if (message.semi_sync_monitor_blocked != null && Object.hasOwnProperty.call(message, "semi_sync_monitor_blocked")) + writer.uint32(/* id 24, wireType 0 =*/192).bool(message.semi_sync_monitor_blocked); return writer; }; @@ -118900,6 +118911,10 @@ export const replicationdata = $root.replicationdata = (() => { message.disk_stalled = reader.bool(); break; } + case 24: { + message.semi_sync_monitor_blocked = reader.bool(); + break; + } default: reader.skipType(tag & 7); break; @@ -119010,6 +119025,9 @@ export const replicationdata = $root.replicationdata = (() => { if (message.disk_stalled != null && message.hasOwnProperty("disk_stalled")) if (typeof message.disk_stalled !== "boolean") return "disk_stalled: boolean expected"; + if (message.semi_sync_monitor_blocked != null && message.hasOwnProperty("semi_sync_monitor_blocked")) + if (typeof message.semi_sync_monitor_blocked !== "boolean") + return "semi_sync_monitor_blocked: boolean expected"; return null; }; @@ -119087,6 +119105,8 @@ export const replicationdata = $root.replicationdata = (() => { } if (object.disk_stalled != null) message.disk_stalled = Boolean(object.disk_stalled); + if (object.semi_sync_monitor_blocked != null) + message.semi_sync_monitor_blocked = Boolean(object.semi_sync_monitor_blocked); return message; }; @@ -119131,6 +119151,7 @@ export const replicationdata = $root.replicationdata = (() => { object.super_read_only = false; object.replication_configuration = null; object.disk_stalled = false; + object.semi_sync_monitor_blocked = false; } if (message.server_id != null && message.hasOwnProperty("server_id")) object.server_id = message.server_id; @@ -119181,6 +119202,8 @@ export const replicationdata = $root.replicationdata = (() => { object.replication_configuration = $root.replicationdata.Configuration.toObject(message.replication_configuration, options); if (message.disk_stalled != null && message.hasOwnProperty("disk_stalled")) object.disk_stalled = message.disk_stalled; + if (message.semi_sync_monitor_blocked != null && message.hasOwnProperty("semi_sync_monitor_blocked")) + object.semi_sync_monitor_blocked = message.semi_sync_monitor_blocked; return object; }; From d33b46043798fc4686c9d614bffc64792995d538 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Mon, 17 Feb 2025 14:52:11 +0530 Subject: [PATCH 09/27] feat: added test to verify VTOrc runs an ERS when semi-sync is blocked for too long and a add a few logs Signed-off-by: Manan Gupta --- .../endtoend/reparent/newfeaturetest/reparent_test.go | 8 +++++++- go/vt/vtorc/inst/analysis_dao.go | 10 +++++++--- go/vt/vtorc/inst/analysis_dao_test.go | 4 +++- go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go | 4 ++++ 4 files changed, 21 insertions(+), 5 deletions(-) diff --git a/go/test/endtoend/reparent/newfeaturetest/reparent_test.go b/go/test/endtoend/reparent/newfeaturetest/reparent_test.go index c9ef98b9e95..1db03f213a1 100644 --- a/go/test/endtoend/reparent/newfeaturetest/reparent_test.go +++ b/go/test/endtoend/reparent/newfeaturetest/reparent_test.go @@ -285,7 +285,13 @@ func TestSemiSyncBlockDueToDisruption(t *testing.T) { utils.ConfirmReplication(t, tablets[0], []*cluster.Vttablet{tablets[1], tablets[2], tablets[3]}) }() - time.Sleep(30 * time.Second) + // Starting VTOrc later now, because we don't want it to fix the heartbeat interval + // on the replica's before the disruption has been introduced. + err := clusterInstance.StartVTOrc(clusterInstance.Keyspaces[0].Name) + require.NoError(t, err) + // If the network disruption is too long lived, then we will end up running ERS from VTOrc. + networkDisruptionDuration := 43 * time.Second + time.Sleep(networkDisruptionDuration) // Restore the network runCommandWithSudo(t, "cp", "/etc/pf.conf.backup", "/etc/pf.conf") diff --git a/go/vt/vtorc/inst/analysis_dao.go b/go/vt/vtorc/inst/analysis_dao.go index 70b6d98e671..a4eee7c8bc5 100644 --- a/go/vt/vtorc/inst/analysis_dao.go +++ b/go/vt/vtorc/inst/analysis_dao.go @@ -462,6 +462,13 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna a.Analysis = PrimaryTabletDeleted a.Description = "Primary tablet has been deleted" ca.hasClusterwideAction = true + } else if a.IsPrimary && a.SemiSyncMonitorBlocked && a.CountSemiSyncReplicasEnabled >= a.SemiSyncPrimaryWaitForReplicaCount { + // The primary is reporting that semi-sync monitor is blocked on writes. + // There are enough replicas configured to send semi-sync ACKs such that the primary shouldn't be blocked. + // There is some network diruption in progress. We should run an ERS. + a.Analysis = PrimarySemiSyncBlocked + a.Description = "Writes seem to be blocked on semi-sync acks on the primary, even though sufficient replicas are configured to send ACKs" + ca.hasClusterwideAction = true } else if topo.IsReplicaType(a.TabletType) && !a.IsReadOnly { a.Analysis = ReplicaIsWritable a.Description = "Replica is writable" @@ -509,9 +516,6 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna a.Description = "Semi sync primary seems to be locked, more samplings needed to validate" } // - } else if a.IsPrimary && a.SemiSyncMonitorBlocked { - a.Analysis = PrimarySemiSyncBlocked - a.Description = "Writes seem to be blocked on semi-sync acks on the primary, even though sufficient replicas are configured to send ACKs" } else if a.IsPrimary && a.LastCheckValid && a.CountReplicas == 1 && a.CountValidReplicas == a.CountReplicas && a.CountValidReplicatingReplicas == 0 { a.Analysis = PrimarySingleReplicaNotReplicating a.Description = "Primary is reachable but its single replica is not replicating" diff --git a/go/vt/vtorc/inst/analysis_dao_test.go b/go/vt/vtorc/inst/analysis_dao_test.go index a954cf7f3de..76a86df43cc 100644 --- a/go/vt/vtorc/inst/analysis_dao_test.go +++ b/go/vt/vtorc/inst/analysis_dao_test.go @@ -140,7 +140,8 @@ func TestGetReplicationAnalysisDecision(t *testing.T) { SemiSyncPrimaryEnabled: 1, SemiSyncPrimaryStatus: 1, SemiSyncPrimaryWaitForReplicaCount: 2, - SemiSyncPrimaryClients: 2, + CountSemiSyncReplicasEnabled: 2, + SemiSyncPrimaryClients: 0, SemiSyncMonitorBlocked: 1, }}, keyspaceWanted: "ks", @@ -167,6 +168,7 @@ func TestGetReplicationAnalysisDecision(t *testing.T) { SemiSyncPrimaryEnabled: 1, SemiSyncPrimaryStatus: 1, SemiSyncPrimaryWaitForReplicaCount: 2, + CountSemiSyncReplicasEnabled: 1, SemiSyncPrimaryClients: 1, SemiSyncMonitorBlocked: 1, }}, diff --git a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go index aaebdcfa0b2..2ceb09b3757 100644 --- a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go @@ -186,15 +186,19 @@ func (m *Monitor) WaitUntilSemiSyncUnblocked(ctx context.Context) error { if !m.stillBlocked() { // If we find that the primary isn't blocked, we're good, // we don't need to wait for anything. + log.Infof("Primary not blocked on semi-sync ACKs") return nil } + log.Infof("Waiting for semi-sync to be unblocked") // The primary is blocked. We need to wait for it to be unblocked // or the context to expire. ch := m.addWaiter() select { case <-ch: + log.Infof("Finished waiting for semi-sync to be unblocked") return nil case <-ctx.Done(): + log.Infof("Error while waiting for semi-sync to be unblocked - %s", ctx.Err().Error()) return ctx.Err() } } From 609297855b38fcd601a8d2a67d60cf8a752c2c60 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Mon, 17 Feb 2025 15:32:50 +0530 Subject: [PATCH 10/27] feat: make the monitor interval configurable Signed-off-by: Manan Gupta --- go/flags/endtoend/vtcombo.txt | 1 + go/flags/endtoend/vttablet.txt | 1 + .../tabletserver/semisyncmonitor/monitor.go | 4 +- .../vttablet/tabletserver/tabletenv/config.go | 44 +++++++++++++++++++ .../tabletserver/tabletenv/config_test.go | 6 +++ 5 files changed, 53 insertions(+), 3 deletions(-) diff --git a/go/flags/endtoend/vtcombo.txt b/go/flags/endtoend/vtcombo.txt index 45ca4ee2a08..65767404046 100644 --- a/go/flags/endtoend/vtcombo.txt +++ b/go/flags/endtoend/vtcombo.txt @@ -322,6 +322,7 @@ Flags: --schema_change_signal Enable the schema tracker; requires queryserver-config-schema-change-signal to be enabled on the underlying vttablets for this to work (default true) --schema_dir string Schema base directory. Should contain one directory per keyspace, with a vschema.json file if necessary. --security_policy string the name of a registered security policy to use for controlling access to URLs - empty means allow all for anyone (built-in policies: deny-all, read-only) + --semi-sync-monitor-interval duration Interval between semi-sync monitor checks if the primary is blocked on semi-sync ACKs (default 10s) --service_map strings comma separated list of services to enable (or disable if prefixed with '-') Example: grpc-queryservice --serving_state_grace_period duration how long to pause after broadcasting health to vtgate, before enforcing a new serving state --shard_sync_retry_delay duration delay between retries of updates to keep the tablet and its shard record in sync (default 30s) diff --git a/go/flags/endtoend/vttablet.txt b/go/flags/endtoend/vttablet.txt index a463db305ba..7d9c60d1b05 100644 --- a/go/flags/endtoend/vttablet.txt +++ b/go/flags/endtoend/vttablet.txt @@ -322,6 +322,7 @@ Flags: --schema-change-reload-timeout duration query server schema change reload timeout, this is how long to wait for the signaled schema reload operation to complete before giving up (default 30s) --schema-version-max-age-seconds int max age of schema version records to kept in memory by the vreplication historian --security_policy string the name of a registered security policy to use for controlling access to URLs - empty means allow all for anyone (built-in policies: deny-all, read-only) + --semi-sync-monitor-interval duration Interval between semi-sync monitor checks if the primary is blocked on semi-sync ACKs (default 10s) --service_map strings comma separated list of services to enable (or disable if prefixed with '-') Example: grpc-queryservice --serving_state_grace_period duration how long to pause after broadcasting health to vtgate, before enforcing a new serving state --shard_sync_retry_delay duration delay between retries of updates to keep the tablet and its shard record in sync (default 30s) diff --git a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go index 2ceb09b3757..c4bcec58b81 100644 --- a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go @@ -75,11 +75,9 @@ type Monitor struct { // NewMonitor creates a new Monitor. func NewMonitor(env tabletenv.Env) *Monitor { - // TODO (@GuptaManan100): Parameterize the watch interval. - watchInterval := 10 * time.Second return &Monitor{ env: env, - ticks: timer.NewTimer(watchInterval), + ticks: timer.NewTimer(env.Config().SemiSyncMonitor.Interval), // We clear the data every day. We can make it configurable in the future, // but this seams fine for now. clearTicks: timer.NewTimer(clearTimerDuration), diff --git a/go/vt/vttablet/tabletserver/tabletenv/config.go b/go/vt/vttablet/tabletserver/tabletenv/config.go index ddab935d393..257361613c3 100644 --- a/go/vt/vttablet/tabletserver/tabletenv/config.go +++ b/go/vt/vttablet/tabletserver/tabletenv/config.go @@ -75,6 +75,7 @@ var ( heartbeatInterval time.Duration heartbeatOnDemandDuration time.Duration healthCheckInterval time.Duration + semiSyncMonitorInterval time.Duration degradedThreshold time.Duration unhealthyThreshold time.Duration transitionGracePeriod time.Duration @@ -203,6 +204,7 @@ func registerTabletEnvFlags(fs *pflag.FlagSet) { fs.DurationVar(°radedThreshold, "degraded_threshold", defaultConfig.Healthcheck.DegradedThreshold, "replication lag after which a replica is considered degraded") fs.DurationVar(&unhealthyThreshold, "unhealthy_threshold", defaultConfig.Healthcheck.UnhealthyThreshold, "replication lag after which a replica is considered unhealthy") fs.DurationVar(&transitionGracePeriod, "serving_state_grace_period", 0, "how long to pause after broadcasting health to vtgate, before enforcing a new serving state") + fs.DurationVar(&semiSyncMonitorInterval, "semi-sync-monitor-interval", defaultConfig.SemiSyncMonitor.Interval, "Interval between semi-sync monitor checks if the primary is blocked on semi-sync ACKs") fs.BoolVar(&enableReplicationReporter, "enable_replication_reporter", false, "Use polling to track replication lag.") fs.BoolVar(¤tConfig.EnableOnlineDDL, "queryserver_enable_online_ddl", true, "Enable online DDL.") @@ -278,6 +280,7 @@ func Init() { currentConfig.Healthcheck.DegradedThreshold = degradedThreshold currentConfig.Healthcheck.UnhealthyThreshold = unhealthyThreshold currentConfig.GracePeriods.Transition = transitionGracePeriod + currentConfig.SemiSyncMonitor.Interval = semiSyncMonitorInterval logFormat := streamlog.GetQueryLogConfig().Format switch logFormat { @@ -317,6 +320,8 @@ type TabletConfig struct { Healthcheck HealthcheckConfig `json:"healthcheck,omitempty"` GracePeriods GracePeriodsConfig `json:"gracePeriods,omitempty"` + SemiSyncMonitor SemiSyncMonitorConfig `json:"semiSyncMonitor,omitempty"` + ReplicationTracker ReplicationTrackerConfig `json:"replicationTracker,omitempty"` // Consolidator can be enable, disable, or notOnPrimary. Default is enable. @@ -613,6 +618,42 @@ type HotRowProtectionConfig struct { MaxConcurrency int `json:"maxConcurrency,omitempty"` } +// SemiSyncMonitorConfig contains the config for the semi-sync monitor. +type SemiSyncMonitorConfig struct { + Interval time.Duration +} + +func (cfg *SemiSyncMonitorConfig) MarshalJSON() ([]byte, error) { + var tmp struct { + IntervalSeconds string `json:"intervalSeconds,omitempty"` + } + + if d := cfg.Interval; d != 0 { + tmp.IntervalSeconds = d.String() + } + + return json.Marshal(&tmp) +} + +func (cfg *SemiSyncMonitorConfig) UnmarshalJSON(data []byte) (err error) { + var tmp struct { + Interval string `json:"intervalSeconds,omitempty"` + } + + if err = json.Unmarshal(data, &tmp); err != nil { + return err + } + + if tmp.Interval != "" { + cfg.Interval, err = time.ParseDuration(tmp.Interval) + if err != nil { + return err + } + } + + return nil +} + // HealthcheckConfig contains the config for healthcheck. type HealthcheckConfig struct { Interval time.Duration @@ -1008,6 +1049,9 @@ var defaultConfig = TabletConfig{ DegradedThreshold: 30 * time.Second, UnhealthyThreshold: 2 * time.Hour, }, + SemiSyncMonitor: SemiSyncMonitorConfig{ + Interval: 10 * time.Second, + }, ReplicationTracker: ReplicationTrackerConfig{ Mode: Disable, HeartbeatInterval: 250 * time.Millisecond, diff --git a/go/vt/vttablet/tabletserver/tabletenv/config_test.go b/go/vt/vttablet/tabletserver/tabletenv/config_test.go index 9ae653bafb9..628277b9e24 100644 --- a/go/vt/vttablet/tabletserver/tabletenv/config_test.go +++ b/go/vt/vttablet/tabletserver/tabletenv/config_test.go @@ -300,6 +300,12 @@ func TestFlags(t *testing.T) { want.Healthcheck.Interval = time.Second assert.Equal(t, want, currentConfig) + semiSyncMonitorInterval = time.Second + currentConfig.SemiSyncMonitor.Interval = 0 + Init() + want.SemiSyncMonitor.Interval = time.Second + assert.Equal(t, want, currentConfig) + degradedThreshold = 2 * time.Second currentConfig.Healthcheck.DegradedThreshold = 0 Init() From 728641953211d87c3a1ab07800fd96b36540064f Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Mon, 17 Feb 2025 15:42:40 +0530 Subject: [PATCH 11/27] feat: add a gauge for the monitor Signed-off-by: Manan Gupta --- .../tabletserver/semisyncmonitor/monitor.go | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go index c4bcec58b81..95072ec384e 100644 --- a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go @@ -23,6 +23,7 @@ import ( "time" "vitess.io/vitess/go/constants/sidecar" + "vitess.io/vitess/go/stats" "vitess.io/vitess/go/timer" "vitess.io/vitess/go/vt/dbconnpool" "vitess.io/vitess/go/vt/log" @@ -71,6 +72,8 @@ type Monitor struct { isBlocked bool // waiters stores the list of waiters that are waiting for the primary to be unblocked. waiters []chan any + // writesBlockedGauge is a gauge tracking the number of writes the monitor is blocked on. + writesBlockedGauge *stats.Gauge } // NewMonitor creates a new Monitor. @@ -80,10 +83,10 @@ func NewMonitor(env tabletenv.Env) *Monitor { ticks: timer.NewTimer(env.Config().SemiSyncMonitor.Interval), // We clear the data every day. We can make it configurable in the future, // but this seams fine for now. - clearTicks: timer.NewTimer(clearTimerDuration), - // TODO: Add a metric on the number of writes that are blocked. - appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), - waiters: make([]chan any, 0), + clearTicks: timer.NewTimer(clearTimerDuration), + writesBlockedGauge: env.Exporter().NewGauge("SemiSyncMonitorWritesBlocked", "Number of writes blocked in the semi-sync monitor"), + appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), + waiters: make([]chan any, 0), } } @@ -261,6 +264,7 @@ func (m *Monitor) incrementWriteCount() bool { return false } m.inProgressWriteCount = m.inProgressWriteCount + 1 + m.writesBlockedGauge.Set(int64(m.inProgressWriteCount)) return true } @@ -277,6 +281,7 @@ func (m *Monitor) decrementWriteCount() { m.mu.Lock() defer m.mu.Unlock() m.inProgressWriteCount = m.inProgressWriteCount - 1 + m.writesBlockedGauge.Set(int64(m.inProgressWriteCount)) } // write writes a heartbeat to unblock semi-sync being stuck. From 4725dd9da4244c8a023c0a48c420ee15092a4d35 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Mon, 17 Feb 2025 16:16:33 +0530 Subject: [PATCH 12/27] feat: move the semi-sync monitor from the tabletserver to tablet manager to only start it when required Signed-off-by: Manan Gupta --- go/cmd/vttablet/cli/cli.go | 2 + .../vttablet/tabletmanager/rpc_replication.go | 12 +++- .../semisyncmonitor/monitor.go | 29 +++++--- go/vt/vttablet/tabletmanager/tm_init.go | 2 + go/vt/vttablet/tabletserver/controller.go | 6 -- go/vt/vttablet/tabletserver/state_manager.go | 31 ++++----- .../tabletserver/state_manager_test.go | 66 +++++++++---------- go/vt/vttablet/tabletserver/tabletserver.go | 44 +++++-------- go/vt/vttablet/tabletservermock/controller.go | 12 ---- 9 files changed, 93 insertions(+), 111 deletions(-) rename go/vt/vttablet/{tabletserver => tabletmanager}/semisyncmonitor/monitor.go (92%) diff --git a/go/cmd/vttablet/cli/cli.go b/go/cmd/vttablet/cli/cli.go index e48a11c79dc..a953ed2b448 100644 --- a/go/cmd/vttablet/cli/cli.go +++ b/go/cmd/vttablet/cli/cli.go @@ -38,6 +38,7 @@ import ( "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtenv" "vitess.io/vitess/go/vt/vttablet/tabletmanager" + "vitess.io/vitess/go/vt/vttablet/tabletmanager/semisyncmonitor" "vitess.io/vitess/go/vt/vttablet/tabletmanager/vdiff" "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" "vitess.io/vitess/go/vt/vttablet/tabletserver" @@ -168,6 +169,7 @@ func run(cmd *cobra.Command, args []string) error { QueryServiceControl: qsc, UpdateStream: binlog.NewUpdateStream(ts, tablet.Keyspace, tabletAlias.Cell, qsc.SchemaEngine(), env.Parser()), VREngine: vreplication.NewEngine(env, config, ts, tabletAlias.Cell, mysqld, qsc.LagThrottler()), + SemiSyncMonitor: semisyncmonitor.NewMonitor(config, qsc.Exporter()), VDiffEngine: vdiff.NewEngine(ts, tablet, env.CollationEnv(), env.Parser()), } if err := tm.Start(tablet, config); err != nil { diff --git a/go/vt/vttablet/tabletmanager/rpc_replication.go b/go/vt/vttablet/tabletmanager/rpc_replication.go index 88337e2a692..5f6e154e2d6 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication.go @@ -187,7 +187,7 @@ func (tm *TabletManager) FullStatus(ctx context.Context) (*replicationdatapb.Ful SemiSyncPrimaryClients: semiSyncClients, SemiSyncPrimaryTimeout: semiSyncTimeout, SemiSyncWaitForReplicaCount: semiSyncNumReplicas, - SemiSyncMonitorBlocked: tm.QueryServiceControl.SemiSyncMonitorBlocked(), + SemiSyncMonitorBlocked: tm.SemiSyncMonitor.AllWritesBlocked(), SuperReadOnly: superReadOnly, ReplicationConfiguration: replConfiguration, }, nil @@ -588,7 +588,7 @@ func (tm *TabletManager) demotePrimary(ctx context.Context, revertPartialFailure // Now we know no writes are in-flight and no new writes can occur. // We just need to wait for no write being blocked on semi-sync ACKs. - err = tm.QueryServiceControl.WaitUntilSemiSyncBeingUnblocked(ctx) + err = tm.SemiSyncMonitor.WaitUntilSemiSyncUnblocked(ctx) if err != nil { return nil, err } @@ -1052,10 +1052,18 @@ func (tm *TabletManager) fixSemiSync(ctx context.Context, tabletType topodatapb. case SemiSyncActionNone: return nil case SemiSyncActionSet: + // We want to enable the semi-sync monitor only if the tablet is going to start + // expecting semi-sync ACKs. + if tabletType == topodatapb.TabletType_PRIMARY { + tm.SemiSyncMonitor.Open() + } else { + tm.SemiSyncMonitor.Close() + } // Always enable replica-side since it doesn't hurt to keep it on for a primary. // The primary-side needs to be off for a replica, or else it will get stuck. return tm.MysqlDaemon.SetSemiSyncEnabled(ctx, tabletType == topodatapb.TabletType_PRIMARY, true) case SemiSyncActionUnset: + tm.SemiSyncMonitor.Close() return tm.MysqlDaemon.SetSemiSyncEnabled(ctx, false, false) default: return vterrors.Errorf(vtrpc.Code_INTERNAL, "Unknown SemiSyncAction - %v", semiSync) diff --git a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go similarity index 92% rename from go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go rename to go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index 95072ec384e..4c8eb536aa6 100644 --- a/go/vt/vttablet/tabletserver/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -28,6 +28,7 @@ import ( "vitess.io/vitess/go/vt/dbconnpool" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/mysqlctl" + "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" ) @@ -47,8 +48,8 @@ const ( // blocking PRS. The monitor looks for this situation and manufactures a write // periodically to unblock the primary. type Monitor struct { - // env is used to get the connection parameters. - env tabletenv.Env + // config is used to get the connection parameters. + config *tabletenv.TabletConfig // ticks is the ticker on which we'll check // if the primary is blocked on semi-sync ACKs or not. ticks *timer.Timer @@ -77,15 +78,15 @@ type Monitor struct { } // NewMonitor creates a new Monitor. -func NewMonitor(env tabletenv.Env) *Monitor { +func NewMonitor(config *tabletenv.TabletConfig, exporter *servenv.Exporter) *Monitor { return &Monitor{ - env: env, - ticks: timer.NewTimer(env.Config().SemiSyncMonitor.Interval), + config: config, + ticks: timer.NewTimer(config.SemiSyncMonitor.Interval), // We clear the data every day. We can make it configurable in the future, // but this seams fine for now. clearTicks: timer.NewTimer(clearTimerDuration), - writesBlockedGauge: env.Exporter().NewGauge("SemiSyncMonitorWritesBlocked", "Number of writes blocked in the semi-sync monitor"), - appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", env.Exporter(), 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), + writesBlockedGauge: exporter.NewGauge("SemiSyncMonitorWritesBlocked", "Number of writes blocked in the semi-sync monitor"), + appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", exporter, 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), waiters: make([]chan any, 0), } } @@ -105,7 +106,7 @@ func (m *Monitor) Open() { // This function could be running from within a unit test scope, in which case we use // mock pools that are already open. This is why we test for the pool being open. if !m.appPool.IsOpen() { - m.appPool.Open(m.env.Config().DB.AppWithDB()) + m.appPool.Open(m.config.DB.AppWithDB()) } m.clearTicks.Start(m.clearAllData) m.ticks.Start(m.checkAndFixSemiSyncBlocked) @@ -179,9 +180,21 @@ func (m *Monitor) isSemiSyncBlocked(ctx context.Context) (bool, error) { return value != 0, err } +// isClosed returns if the monitor is currently closed or not. +func (m *Monitor) isClosed() bool { + m.mu.Lock() + defer m.mu.Lock() + return !m.isOpen +} + // WaitUntilSemiSyncUnblocked waits until the primary is not blocked // on semi-sync or until the context expires. func (m *Monitor) WaitUntilSemiSyncUnblocked(ctx context.Context) error { + // SemiSyncMonitor is closed, which means semi-sync is not enabled. + // We don't have anything to wait for. + if m.isClosed() { + return nil + } // run one iteration of checking if semi-sync is blocked or not. m.checkAndFixSemiSyncBlocked() if !m.stillBlocked() { diff --git a/go/vt/vttablet/tabletmanager/tm_init.go b/go/vt/vttablet/tabletmanager/tm_init.go index fbef04de357..4c7b706bc4b 100644 --- a/go/vt/vttablet/tabletmanager/tm_init.go +++ b/go/vt/vttablet/tabletmanager/tm_init.go @@ -73,6 +73,7 @@ import ( "vitess.io/vitess/go/vt/vtctl/reparentutil/policy" "vitess.io/vitess/go/vt/vtenv" "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vttablet/tabletmanager/semisyncmonitor" "vitess.io/vitess/go/vt/vttablet/tabletmanager/vdiff" "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" "vitess.io/vitess/go/vt/vttablet/tabletserver" @@ -162,6 +163,7 @@ type TabletManager struct { QueryServiceControl tabletserver.Controller UpdateStream binlog.UpdateStreamControl VREngine *vreplication.Engine + SemiSyncMonitor *semisyncmonitor.Monitor VDiffEngine *vdiff.Engine Env *vtenv.Environment diff --git a/go/vt/vttablet/tabletserver/controller.go b/go/vt/vttablet/tabletserver/controller.go index ae67d19873b..ab2875ae27b 100644 --- a/go/vt/vttablet/tabletserver/controller.go +++ b/go/vt/vttablet/tabletserver/controller.go @@ -125,12 +125,6 @@ type Controller interface { // IsDiskStalled returns if the disk is stalled. IsDiskStalled() bool - - // WaitUntilSemiSyncBeingUnblocked waits until the primary is not blocked on semi-sync. - WaitUntilSemiSyncBeingUnblocked(ctx context.Context) error - - // SemiSyncMonitorBlocked returns whether the semi-sync monitor has all its writes blocked. - SemiSyncMonitorBlocked() bool } // Ensure TabletServer satisfies Controller interface. diff --git a/go/vt/vttablet/tabletserver/state_manager.go b/go/vt/vttablet/tabletserver/state_manager.go index 1966c9e0375..0ccd0e42735 100644 --- a/go/vt/vttablet/tabletserver/state_manager.go +++ b/go/vt/vttablet/tabletserver/state_manager.go @@ -112,20 +112,19 @@ type stateManager struct { // Open must be done in forward order. // Close must be done in reverse order. // All Close functions must be called before Open. - hs *healthStreamer - se schemaEngine - rt replTracker - vstreamer subComponent - tracker subComponent - watcher subComponent - semiSyncMonitor subComponent - qe queryEngine - txThrottler txThrottler - te txEngine - messager subComponent - ddle onlineDDLExecutor - throttler lagThrottler - tableGC tableGarbageCollector + hs *healthStreamer + se schemaEngine + rt replTracker + vstreamer subComponent + tracker subComponent + watcher subComponent + qe queryEngine + txThrottler txThrottler + te txEngine + messager subComponent + ddle onlineDDLExecutor + throttler lagThrottler + tableGC tableGarbageCollector // hcticks starts on initialization and runs forever. hcticks *timer.Timer @@ -470,7 +469,6 @@ func (sm *stateManager) servePrimary() error { sm.throttler.Open() sm.tableGC.Open() sm.ddle.Open() - sm.semiSyncMonitor.Open() sm.setState(topodatapb.TabletType_PRIMARY, StateServing) return nil } @@ -487,7 +485,6 @@ func (sm *stateManager) unservePrimary() error { sm.se.MakePrimary(false) sm.hs.MakePrimary(false) sm.rt.MakePrimary() - sm.semiSyncMonitor.Open() sm.setState(topodatapb.TabletType_PRIMARY, StateNotServing) return nil } @@ -498,7 +495,6 @@ func (sm *stateManager) serveNonPrimary(wantTabletType topodatapb.TabletType) er cancel := sm.terminateAllQueries(nil) defer cancel() - sm.semiSyncMonitor.Close() sm.ddle.Close() sm.tableGC.Close() sm.messager.Close() @@ -521,7 +517,6 @@ func (sm *stateManager) serveNonPrimary(wantTabletType topodatapb.TabletType) er func (sm *stateManager) unserveNonPrimary(wantTabletType topodatapb.TabletType) error { sm.unserveCommon() - sm.semiSyncMonitor.Close() sm.se.MakeNonPrimary() sm.hs.MakeNonPrimary() diff --git a/go/vt/vttablet/tabletserver/state_manager_test.go b/go/vt/vttablet/tabletserver/state_manager_test.go index 1bc38117007..53b0db271a2 100644 --- a/go/vt/vttablet/tabletserver/state_manager_test.go +++ b/go/vt/vttablet/tabletserver/state_manager_test.go @@ -90,7 +90,6 @@ func TestStateManagerServePrimary(t *testing.T) { verifySubcomponent(t, 10, sm.throttler, testStateOpen) verifySubcomponent(t, 11, sm.tableGC, testStateOpen) verifySubcomponent(t, 12, sm.ddle, testStateOpen) - verifySubcomponent(t, 13, sm.semiSyncMonitor, testStateOpen) assert.False(t, sm.se.(*testSchemaEngine).nonPrimary) assert.True(t, sm.se.(*testSchemaEngine).ensureCalled) @@ -105,21 +104,20 @@ func TestStateManagerServeNonPrimary(t *testing.T) { err := sm.SetServingType(topodatapb.TabletType_REPLICA, testNow, StateServing, "") require.NoError(t, err) - verifySubcomponent(t, 1, sm.semiSyncMonitor, testStateClosed) - verifySubcomponent(t, 2, sm.ddle, testStateClosed) - verifySubcomponent(t, 3, sm.tableGC, testStateClosed) - verifySubcomponent(t, 4, sm.messager, testStateClosed) - verifySubcomponent(t, 5, sm.tracker, testStateClosed) + verifySubcomponent(t, 1, sm.ddle, testStateClosed) + verifySubcomponent(t, 2, sm.tableGC, testStateClosed) + verifySubcomponent(t, 3, sm.messager, testStateClosed) + verifySubcomponent(t, 4, sm.tracker, testStateClosed) assert.True(t, sm.se.(*testSchemaEngine).nonPrimary) - verifySubcomponent(t, 6, sm.se, testStateOpen) - verifySubcomponent(t, 7, sm.vstreamer, testStateOpen) - verifySubcomponent(t, 8, sm.qe, testStateOpen) - verifySubcomponent(t, 9, sm.txThrottler, testStateOpen) - verifySubcomponent(t, 10, sm.te, testStateNonPrimary) - verifySubcomponent(t, 11, sm.rt, testStateNonPrimary) - verifySubcomponent(t, 12, sm.watcher, testStateOpen) - verifySubcomponent(t, 13, sm.throttler, testStateOpen) + verifySubcomponent(t, 5, sm.se, testStateOpen) + verifySubcomponent(t, 6, sm.vstreamer, testStateOpen) + verifySubcomponent(t, 7, sm.qe, testStateOpen) + verifySubcomponent(t, 8, sm.txThrottler, testStateOpen) + verifySubcomponent(t, 9, sm.te, testStateNonPrimary) + verifySubcomponent(t, 10, sm.rt, testStateNonPrimary) + verifySubcomponent(t, 11, sm.watcher, testStateOpen) + verifySubcomponent(t, 12, sm.throttler, testStateOpen) assert.Equal(t, topodatapb.TabletType_REPLICA, sm.target.TabletType) assert.Equal(t, StateServing, sm.state) @@ -145,7 +143,6 @@ func TestStateManagerUnservePrimary(t *testing.T) { verifySubcomponent(t, 11, sm.txThrottler, testStateOpen) verifySubcomponent(t, 12, sm.rt, testStatePrimary) - verifySubcomponent(t, 13, sm.semiSyncMonitor, testStateOpen) assert.Equal(t, topodatapb.TabletType_PRIMARY, sm.target.TabletType) assert.Equal(t, StateNotServing, sm.state) @@ -198,14 +195,13 @@ func TestStateManagerUnserveNonPrimary(t *testing.T) { verifySubcomponent(t, 6, sm.tracker, testStateClosed) assert.True(t, sm.se.(*testSchemaEngine).nonPrimary) - verifySubcomponent(t, 7, sm.semiSyncMonitor, testStateClosed) - verifySubcomponent(t, 8, sm.se, testStateOpen) - verifySubcomponent(t, 9, sm.vstreamer, testStateOpen) - verifySubcomponent(t, 10, sm.qe, testStateOpen) - verifySubcomponent(t, 11, sm.txThrottler, testStateOpen) + verifySubcomponent(t, 7, sm.se, testStateOpen) + verifySubcomponent(t, 8, sm.vstreamer, testStateOpen) + verifySubcomponent(t, 9, sm.qe, testStateOpen) + verifySubcomponent(t, 10, sm.txThrottler, testStateOpen) - verifySubcomponent(t, 12, sm.rt, testStateNonPrimary) - verifySubcomponent(t, 13, sm.watcher, testStateOpen) + verifySubcomponent(t, 11, sm.rt, testStateNonPrimary) + verifySubcomponent(t, 12, sm.watcher, testStateOpen) assert.Equal(t, topodatapb.TabletType_RDONLY, sm.target.TabletType) assert.Equal(t, StateNotServing, sm.state) @@ -331,21 +327,20 @@ func TestStateManagerSetServingTypeNoChange(t *testing.T) { err = sm.SetServingType(topodatapb.TabletType_REPLICA, testNow, StateServing, "") require.NoError(t, err) - verifySubcomponent(t, 1, sm.semiSyncMonitor, testStateClosed) - verifySubcomponent(t, 2, sm.ddle, testStateClosed) - verifySubcomponent(t, 3, sm.tableGC, testStateClosed) - verifySubcomponent(t, 4, sm.messager, testStateClosed) - verifySubcomponent(t, 5, sm.tracker, testStateClosed) + verifySubcomponent(t, 1, sm.ddle, testStateClosed) + verifySubcomponent(t, 2, sm.tableGC, testStateClosed) + verifySubcomponent(t, 3, sm.messager, testStateClosed) + verifySubcomponent(t, 4, sm.tracker, testStateClosed) assert.True(t, sm.se.(*testSchemaEngine).nonPrimary) - verifySubcomponent(t, 6, sm.se, testStateOpen) - verifySubcomponent(t, 7, sm.vstreamer, testStateOpen) - verifySubcomponent(t, 8, sm.qe, testStateOpen) - verifySubcomponent(t, 9, sm.txThrottler, testStateOpen) - verifySubcomponent(t, 10, sm.te, testStateNonPrimary) - verifySubcomponent(t, 11, sm.rt, testStateNonPrimary) - verifySubcomponent(t, 12, sm.watcher, testStateOpen) - verifySubcomponent(t, 13, sm.throttler, testStateOpen) + verifySubcomponent(t, 5, sm.se, testStateOpen) + verifySubcomponent(t, 6, sm.vstreamer, testStateOpen) + verifySubcomponent(t, 7, sm.qe, testStateOpen) + verifySubcomponent(t, 8, sm.txThrottler, testStateOpen) + verifySubcomponent(t, 9, sm.te, testStateNonPrimary) + verifySubcomponent(t, 10, sm.rt, testStateNonPrimary) + verifySubcomponent(t, 11, sm.watcher, testStateOpen) + verifySubcomponent(t, 12, sm.throttler, testStateOpen) assert.Equal(t, topodatapb.TabletType_REPLICA, sm.target.TabletType) assert.Equal(t, StateServing, sm.state) @@ -825,7 +820,6 @@ func newTestStateManager() *stateManager { vstreamer: &testSubcomponent{}, tracker: &testSubcomponent{}, watcher: &testSubcomponent{}, - semiSyncMonitor: &testSubcomponent{}, qe: &testQueryEngine{}, txThrottler: &testTxThrottler{}, te: &testTxEngine{}, diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 50363db3876..e09e04a9679 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -66,7 +66,6 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletserver/repltracker" "vitess.io/vitess/go/vt/vttablet/tabletserver/rules" "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" - "vitess.io/vitess/go/vt/vttablet/tabletserver/semisyncmonitor" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" "vitess.io/vitess/go/vt/vttablet/tabletserver/throttle" "vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/base" @@ -105,22 +104,21 @@ type TabletServer struct { topoServer *topo.Server // These are sub-components of TabletServer. - statelessql *QueryList - statefulql *QueryList - olapql *QueryList - se *schema.Engine - rt *repltracker.ReplTracker - vstreamer *vstreamer.Engine - tracker *schema.Tracker - watcher *BinlogWatcher - semiSyncMonitor *semisyncmonitor.Monitor - qe *QueryEngine - txThrottler txthrottler.TxThrottler - te *TxEngine - messager *messager.Engine - hs *healthStreamer - lagThrottler *throttle.Throttler - tableGC *gc.TableGC + statelessql *QueryList + statefulql *QueryList + olapql *QueryList + se *schema.Engine + rt *repltracker.ReplTracker + vstreamer *vstreamer.Engine + tracker *schema.Tracker + watcher *BinlogWatcher + qe *QueryEngine + txThrottler txthrottler.TxThrottler + te *TxEngine + messager *messager.Engine + hs *healthStreamer + lagThrottler *throttle.Throttler + tableGC *gc.TableGC // sm manages state transitions. sm *stateManager @@ -183,7 +181,6 @@ func NewTabletServer(ctx context.Context, env *vtenv.Environment, name string, c tsv.vstreamer = vstreamer.NewEngine(tsv, srvTopoServer, tsv.se, tsv.lagThrottler, alias.Cell) tsv.tracker = schema.NewTracker(tsv, tsv.vstreamer, tsv.se) tsv.watcher = NewBinlogWatcher(tsv, tsv.vstreamer, tsv.config) - tsv.semiSyncMonitor = semisyncmonitor.NewMonitor(tsv) tsv.qe = NewQueryEngine(tsv, tsv.se) tsv.txThrottler = txthrottler.NewTxThrottler(tsv, topoServer) tsv.te = NewTxEngine(tsv, tsv.hs.sendUnresolvedTransactionSignal) @@ -202,7 +199,6 @@ func NewTabletServer(ctx context.Context, env *vtenv.Environment, name string, c vstreamer: tsv.vstreamer, tracker: tsv.tracker, watcher: tsv.watcher, - semiSyncMonitor: tsv.semiSyncMonitor, qe: tsv.qe, txThrottler: tsv.txThrottler, te: tsv.te, @@ -778,16 +774,6 @@ func (tsv *TabletServer) IsDiskStalled() bool { return tsv.sm.diskHealthMonitor.IsDiskStalled() } -// WaitUntilSemiSyncBeingUnblocked waits until semi-sync is unblocked or if context has expired. -func (tsv *TabletServer) WaitUntilSemiSyncBeingUnblocked(ctx context.Context) error { - return tsv.semiSyncMonitor.WaitUntilSemiSyncUnblocked(ctx) -} - -// SemiSyncMonitorBlocked returns whether the semi-sync monitor has all its writes blocked. -func (tsv *TabletServer) SemiSyncMonitorBlocked() bool { - return tsv.semiSyncMonitor.AllWritesBlocked() -} - // CreateTransaction creates the metadata for a 2PC transaction. func (tsv *TabletServer) CreateTransaction(ctx context.Context, target *querypb.Target, dtid string, participants []*querypb.Target) (err error) { return tsv.execRequest( diff --git a/go/vt/vttablet/tabletservermock/controller.go b/go/vt/vttablet/tabletservermock/controller.go index 86c156557e1..21b38755302 100644 --- a/go/vt/vttablet/tabletservermock/controller.go +++ b/go/vt/vttablet/tabletservermock/controller.go @@ -285,18 +285,6 @@ func (tqsc *Controller) IsDiskStalled() bool { return false } -// WaitUntilSemiSyncBeingUnblocked is part of the tabletserver.Controller interface -func (tqsc *Controller) WaitUntilSemiSyncBeingUnblocked(context.Context) error { - tqsc.MethodCalled["WaitUntilSemiSyncBeingUnblocked"] = true - return nil -} - -// SemiSyncMonitorBlocked is part of the tabletserver.Controller interface -func (tqsc *Controller) SemiSyncMonitorBlocked() bool { - tqsc.MethodCalled["SemiSyncMonitorBlocked"] = true - return false -} - // EnterLameduck implements tabletserver.Controller. func (tqsc *Controller) EnterLameduck() { tqsc.mu.Lock() From a8d2cbd9cb0f1a4a3840f05ffaa6142f17b1d057 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Mon, 17 Feb 2025 16:40:16 +0530 Subject: [PATCH 13/27] test: poll the new gauge in the test Signed-off-by: Manan Gupta --- .../reparent/newfeaturetest/reparent_test.go | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/go/test/endtoend/reparent/newfeaturetest/reparent_test.go b/go/test/endtoend/reparent/newfeaturetest/reparent_test.go index 1db03f213a1..81c66ebb05e 100644 --- a/go/test/endtoend/reparent/newfeaturetest/reparent_test.go +++ b/go/test/endtoend/reparent/newfeaturetest/reparent_test.go @@ -280,7 +280,7 @@ func TestSemiSyncBlockDueToDisruption(t *testing.T) { ch := make(chan any) go func() { defer func() { - ch <- true + close(ch) }() utils.ConfirmReplication(t, tablets[0], []*cluster.Vttablet{tablets[1], tablets[2], tablets[3]}) }() @@ -289,6 +289,19 @@ func TestSemiSyncBlockDueToDisruption(t *testing.T) { // on the replica's before the disruption has been introduced. err := clusterInstance.StartVTOrc(clusterInstance.Keyspaces[0].Name) require.NoError(t, err) + go func() { + for { + select { + case <-ch: + return + case <-time.After(1 * time.Second): + str, isPresent := tablets[0].VttabletProcess.GetVars()["SemiSyncMonitorWritesBlocked"] + if isPresent { + log.Errorf("SemiSyncMonitorWritesBlocked - %v", str) + } + } + } + }() // If the network disruption is too long lived, then we will end up running ERS from VTOrc. networkDisruptionDuration := 43 * time.Second time.Sleep(networkDisruptionDuration) From da4a8f877d07559a970b25af0ea12ebe82cdf3c7 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Mon, 17 Feb 2025 18:16:14 +0530 Subject: [PATCH 14/27] feat: fix bug in isClosed function where we locked twice :facepalm: Signed-off-by: Manan Gupta --- go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index 4c8eb536aa6..0761ee568f0 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -183,7 +183,7 @@ func (m *Monitor) isSemiSyncBlocked(ctx context.Context) (bool, error) { // isClosed returns if the monitor is currently closed or not. func (m *Monitor) isClosed() bool { m.mu.Lock() - defer m.mu.Lock() + defer m.mu.Unlock() return !m.isOpen } From 6cf02abd2f76da66a90c96ef6f6ea2a7e909368c Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Mon, 17 Feb 2025 20:58:34 +0530 Subject: [PATCH 15/27] test: add comprehensive testing for the semi-sync monitor Signed-off-by: Manan Gupta --- .../tabletmanager/semisyncmonitor/monitor.go | 8 +- .../semisyncmonitor/monitor_test.go | 714 ++++++++++++++++++ 2 files changed, 721 insertions(+), 1 deletion(-) create mode 100644 go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index 0761ee568f0..7bc23e56717 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -41,6 +41,12 @@ const ( clearTimerDuration = 24 * time.Hour ) +var ( + // waitBetweenWrites is the time to wait between consecutive writes. + // This is a variable instead of a constant only to be tweaked in tests. + waitBetweenWrites = 1 * time.Second +) + // Monitor is a monitor that checks if the primary tablet // is blocked on a semi-sync ack from the replica. // If the semi-sync ACK is lost in the network, @@ -262,7 +268,7 @@ func (m *Monitor) startWrites() { // exponentially backing off. We will not do more than maxWritesPermitted writes and once // all maxWritesPermitted writes are blocked, we'll wait for VTOrc to run an ERS. go m.write() - <-time.After(1 * time.Second) + <-time.After(waitBetweenWrites) } } diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go new file mode 100644 index 00000000000..0a1e98dd856 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go @@ -0,0 +1,714 @@ +/* +Copyright 2025 The Vitess Authors. + +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, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package semisyncmonitor + +import ( + "context" + "fmt" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/mysql/fakesqldb" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/dbconfigs" + "vitess.io/vitess/go/vt/servenv" + "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" +) + +var ( + exporter = servenv.NewExporter("TestSemiSyncMonitor", "") +) + +// createFakeDBAndMonitor created a fake DB and a monitor for testing. +func createFakeDBAndMonitor(t *testing.T) (*fakesqldb.DB, *Monitor) { + db := fakesqldb.New(t) + params := db.ConnParams() + cp := *params + dbc := dbconfigs.NewTestDBConfigs(cp, cp, "") + config := &tabletenv.TabletConfig{ + DB: dbc, + SemiSyncMonitor: tabletenv.SemiSyncMonitorConfig{ + Interval: 10 * time.Second, + }, + } + monitor := NewMonitor(config, exporter) + monitor.mu.Lock() + defer monitor.mu.Unlock() + monitor.isOpen = true + monitor.appPool.Open(config.DB.AppWithDB()) + return db, monitor + +} + +// TestMonitorIsSemiSyncBlocked tests the functionality of isSemiSyncBlocked. +func TestMonitorIsSemiSyncBlocked(t *testing.T) { + tests := []struct { + name string + res *sqltypes.Result + want bool + wantErr string + }{ + { + name: "no rows", + res: &sqltypes.Result{}, + want: false, + }, + { + name: "incorrect number of rows", + res: sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|1", "Rpl_semi_sync_master_wait_sessions|1"), + wantErr: "Row count exceeded 1", + }, + { + name: "incorrect number of fields", + res: sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value|a", "varchar|varchar|int"), "Rpl_semi_sync_source_wait_sessions|1|2"), + wantErr: "unexpected number of rows received", + }, + { + name: "Unblocked", + res: sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0"), + want: false, + }, + { + name: "Blocked", + res: sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|1"), + want: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + db.AddQuery(semiSyncWaitSessionsRead, tt.res) + got, err := m.isSemiSyncBlocked(context.Background()) + if tt.wantErr != "" { + require.EqualError(t, err, tt.wantErr) + return + } + require.NoError(t, err) + require.EqualValues(t, tt.want, got) + }) + } +} + +func TestMonitorBindSideCarDBName(t *testing.T) { + tests := []struct { + query string + expected string + }{ + { + query: semiSyncRecoverWrite, + expected: "INSERT INTO _vt.semisync_recover (ts) VALUES (NOW())", + }, + { + query: semiSyncRecoverClear, + expected: "TRUNCATE TABLE _vt.semisync_recover", + }, + } + for _, tt := range tests { + t.Run(tt.query, func(t *testing.T) { + m := &Monitor{} + require.EqualValues(t, tt.expected, m.bindSideCarDBName(tt.query)) + }) + } +} + +func TestMonitorClearAllData(t *testing.T) { + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + db.SetNeverFail(true) + m.clearAllData() + ql := db.QueryLog() + require.EqualValues(t, "truncate table _vt.semisync_recover", ql) +} + +// TestMonitorWaitMechanism tests that the wait mechanism works as intended. +// Setting the monitor to unblock state should unblock the waiters. +func TestMonitorWaitMechanism(t *testing.T) { + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + + // Add a waiter. + ch := m.addWaiter() + var waiterUnblocked atomic.Bool + go func() { + <-ch + waiterUnblocked.Store(true) + }() + + // Ensure that the waiter is currently blocked. + require.False(t, waiterUnblocked.Load()) + + // Verify that setting again to being blocked doesn't unblock the waiter. + m.setIsBlocked(true) + require.False(t, waiterUnblocked.Load()) + require.False(t, m.isClosed()) + require.True(t, m.stillBlocked()) + + // Verify that setting we are no longer blocked, unblocks the waiter. + m.setIsBlocked(false) + require.Eventually(t, func() bool { + return waiterUnblocked.Load() + }, time.Second, time.Millisecond*100) + require.False(t, m.stillBlocked()) + require.False(t, m.isClosed()) +} + +func TestMonitorIncrementWriteCount(t *testing.T) { + tests := []struct { + initVal int + finalVal int + want bool + }{ + { + initVal: maxWritesPermitted - 2, + finalVal: maxWritesPermitted - 1, + want: true, + }, { + initVal: maxWritesPermitted - 1, + finalVal: maxWritesPermitted, + want: true, + }, { + initVal: maxWritesPermitted, + finalVal: maxWritesPermitted, + want: false, + }, { + initVal: 0, + finalVal: 1, + want: true, + }, + } + for _, tt := range tests { + t.Run(fmt.Sprintf("%d", tt.initVal), func(t *testing.T) { + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + m.mu.Lock() + m.inProgressWriteCount = tt.initVal + m.mu.Unlock() + got := m.incrementWriteCount() + require.EqualValues(t, tt.want, got) + m.mu.Lock() + require.EqualValues(t, tt.finalVal, m.inProgressWriteCount) + require.EqualValues(t, tt.finalVal, m.writesBlockedGauge.Get()) + m.mu.Unlock() + }) + } +} + +func TestMonitorDecrementWriteCount(t *testing.T) { + tests := []struct { + initVal int + finalVal int + }{ + { + initVal: maxWritesPermitted - 1, + finalVal: maxWritesPermitted - 2, + }, { + initVal: maxWritesPermitted, + finalVal: maxWritesPermitted - 1, + }, { + initVal: 1, + finalVal: 0, + }, + } + for _, tt := range tests { + t.Run(fmt.Sprintf("%d", tt.initVal), func(t *testing.T) { + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + m.mu.Lock() + m.inProgressWriteCount = tt.initVal + m.mu.Unlock() + m.decrementWriteCount() + m.mu.Lock() + require.EqualValues(t, tt.finalVal, m.inProgressWriteCount) + require.EqualValues(t, tt.finalVal, m.writesBlockedGauge.Get()) + m.mu.Unlock() + }) + } +} + +func TestMonitorAllWritesBlocked(t *testing.T) { + tests := []struct { + initVal int + expected bool + }{ + { + initVal: maxWritesPermitted - 1, + expected: false, + }, { + initVal: maxWritesPermitted, + expected: true, + }, { + initVal: 1, + expected: false, + }, + } + for _, tt := range tests { + t.Run(fmt.Sprintf("%d", tt.initVal), func(t *testing.T) { + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + m.mu.Lock() + m.inProgressWriteCount = tt.initVal + m.mu.Unlock() + require.EqualValues(t, tt.expected, m.AllWritesBlocked()) + }) + } +} + +func TestMonitorWrite(t *testing.T) { + tests := []struct { + initVal int + queryLog string + }{ + { + initVal: maxWritesPermitted - 2, + queryLog: "insert into _vt.semisync_recover (ts) values (now())", + }, { + initVal: maxWritesPermitted - 1, + queryLog: "insert into _vt.semisync_recover (ts) values (now())", + }, { + initVal: maxWritesPermitted, + queryLog: "", + }, { + initVal: 0, + queryLog: "insert into _vt.semisync_recover (ts) values (now())", + }, + } + for _, tt := range tests { + t.Run(fmt.Sprintf("%d", tt.initVal), func(t *testing.T) { + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + db.SetNeverFail(true) + m.mu.Lock() + m.inProgressWriteCount = tt.initVal + m.writesBlockedGauge.Set(int64(tt.initVal)) + m.mu.Unlock() + m.write() + m.mu.Lock() + require.EqualValues(t, tt.initVal, m.inProgressWriteCount) + require.EqualValues(t, tt.initVal, m.writesBlockedGauge.Get()) + m.mu.Unlock() + require.EqualValues(t, tt.queryLog, db.QueryLog()) + }) + } +} + +// TestMonitorWriteBlocked tests the write function when the writes are blocked. +func TestMonitorWriteBlocked(t *testing.T) { + initialVal := waitBetweenWrites + waitBetweenWrites = 250 * time.Millisecond + defer func() { + waitBetweenWrites = initialVal + }() + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + + // Check the initial value of the inProgressWriteCount. + m.mu.Lock() + require.EqualValues(t, 0, m.inProgressWriteCount) + m.mu.Unlock() + + // Add a universal insert query pattern that would block until we make it unblock. + ch := make(chan int) + db.AddQueryPatternWithCallback("^INSERT INTO.*", sqltypes.MakeTestResult(nil), func(s string) { + <-ch + }) + + // Do a write, which we expect to block. + var writeFinished atomic.Bool + go func() { + m.write() + writeFinished.Store(true) + }() + // We should see the number of writes blocked to increase. + require.Eventually(t, func() bool { + m.mu.Lock() + defer m.mu.Unlock() + return m.inProgressWriteCount == 1 + }, 1*time.Second, 100*time.Millisecond) + + // Once the writers are unblocked, we expect to see a zero value again. + close(ch) + require.Eventually(t, func() bool { + m.mu.Lock() + defer m.mu.Unlock() + return m.inProgressWriteCount == 0 + }, 1*time.Second, 100*time.Millisecond) + + require.Eventually(t, func() bool { + return writeFinished.Load() + }, 1*time.Second, 100*time.Millisecond) +} + +// TestIsWriting checks the transitions for the isWriting field. +func TestIsWriting(t *testing.T) { + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + + // Check the initial value of the isWriting field. + m.mu.Lock() + require.False(t, m.isWriting) + m.mu.Unlock() + + // Clearing a false field does nothing. + m.clearIsWriting() + m.mu.Lock() + require.False(t, m.isWriting) + m.mu.Unlock() + + // Check and set should set the field. + set := m.checkAndSetIsWriting() + require.True(t, set) + m.mu.Lock() + require.True(t, m.isWriting) + m.mu.Unlock() + + // Checking and setting shouldn't do anything. + set = m.checkAndSetIsWriting() + require.False(t, set) + m.mu.Lock() + require.True(t, m.isWriting) + m.mu.Unlock() + + // Clearing should now make the field false. + m.clearIsWriting() + m.mu.Lock() + require.False(t, m.isWriting) + m.mu.Unlock() +} + +func TestStartWrites(t *testing.T) { + initialVal := waitBetweenWrites + waitBetweenWrites = 250 * time.Millisecond + defer func() { + waitBetweenWrites = initialVal + }() + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + + // Add a universal insert query pattern that would block until we make it unblock. + ch := make(chan int) + db.AddQueryPatternWithCallback("^INSERT INTO.*", sqltypes.MakeTestResult(nil), func(s string) { + <-ch + }) + + // If we aren't blocked, then start writes doesn't do anything. + m.startWrites() + require.EqualValues(t, "", db.QueryLog()) + + // Now we set the monitor to be blocked. + m.setIsBlocked(true) + + var writesFinished atomic.Bool + go func() { + m.startWrites() + writesFinished.Store(true) + }() + + // We should see the number of writes blocked to increase. + require.Eventually(t, func() bool { + m.mu.Lock() + defer m.mu.Unlock() + return m.inProgressWriteCount >= 1 + }, 1*time.Second, 100*time.Millisecond) + + // Once the writes have started, another call to startWrites shouldn't do anything + m.startWrites() + + // We should continue to see the number of writes blocked increase. + require.Eventually(t, func() bool { + m.mu.Lock() + defer m.mu.Unlock() + return m.inProgressWriteCount >= 2 + }, 1*time.Second, 100*time.Millisecond) + + // Check that the writes are still going. + require.False(t, writesFinished.Load()) + + // Make the monitor unblocked. This should stop the writes eventually. + m.setIsBlocked(false) + close(ch) + + require.Eventually(t, func() bool { + return writesFinished.Load() + }, 1*time.Second, 100*time.Millisecond) + + // Check that no writes are in progress anymore. + require.Eventually(t, func() bool { + m.mu.Lock() + defer m.mu.Unlock() + return m.inProgressWriteCount == 0 + }, 1*time.Second, 100*time.Millisecond) +} + +func TestCheckAndFixSemiSyncBlocked(t *testing.T) { + initialVal := waitBetweenWrites + waitBetweenWrites = 250 * time.Millisecond + defer func() { + waitBetweenWrites = initialVal + }() + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + + // Initially everything is unblocked. + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + // Add a universal insert query pattern that would block until we make it unblock. + ch := make(chan int) + db.AddQueryPatternWithCallback("^INSERT INTO.*", sqltypes.MakeTestResult(nil), func(s string) { + <-ch + }) + + // Check that the monitor thinks we are unblocked. + m.checkAndFixSemiSyncBlocked() + m.mu.Lock() + require.False(t, m.isBlocked) + m.mu.Unlock() + + // Now we set the monitor to be blocked. + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|2")) + m.checkAndFixSemiSyncBlocked() + + m.mu.Lock() + require.True(t, m.isBlocked) + m.mu.Unlock() + + // Checking again shouldn't make a difference. + m.checkAndFixSemiSyncBlocked() + m.mu.Lock() + require.True(t, m.isBlocked) + m.mu.Unlock() + + // Meanwhile writes should have started and should be getting blocked. + require.Eventually(t, func() bool { + m.mu.Lock() + defer m.mu.Unlock() + return m.inProgressWriteCount >= 2 + }, 1*time.Second, 100*time.Millisecond) + + // Now we set the monitor to be unblocked. + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + close(ch) + m.checkAndFixSemiSyncBlocked() + + // We expect the writes to clear out and also the monitor should think its unblocked. + m.mu.Lock() + require.False(t, m.isBlocked) + m.mu.Unlock() + require.Eventually(t, func() bool { + m.mu.Lock() + defer m.mu.Unlock() + return m.inProgressWriteCount == 0 && m.isWriting == false + }, 1*time.Second, 100*time.Millisecond) +} + +func TestWaitUntilSemiSyncUnblocked(t *testing.T) { + initialVal := waitBetweenWrites + waitBetweenWrites = 250 * time.Millisecond + defer func() { + waitBetweenWrites = initialVal + }() + db, m := createFakeDBAndMonitor(t) + defer db.Close() + defer m.Close() + + db.SetNeverFail(true) + // Initially everything is unblocked. + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + + // When everything is unblocked, then this should return without blocking. + err := m.WaitUntilSemiSyncUnblocked(context.Background()) + require.NoError(t, err) + + // Now we set the monitor to be blocked. + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|3")) + + // wg is used to keep track of all the go routines. + wg := sync.WaitGroup{} + // Start a cancellable context and use that to wait. + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + wg.Add(1) + var ctxErr error + var mu sync.Mutex + go func() { + defer wg.Done() + err := m.WaitUntilSemiSyncUnblocked(ctx) + mu.Lock() + ctxErr = err + mu.Unlock() + }() + + // Start another go routine, also waiting for semi-sync being unblocked, but not using the cancellable context. + wg.Add(1) + go func() { + defer wg.Done() + err := m.WaitUntilSemiSyncUnblocked(context.Background()) + require.NoError(t, err) + }() + + // Wait until the writes have started. + require.Eventually(t, func() bool { + m.mu.Lock() + defer m.mu.Unlock() + return m.isWriting + }, 1*time.Second, 100*time.Millisecond) + + // Now we cancel the context. This should fail the first wait. + cancel() + // Since we cancel the context before the semi-sync has been unblocked, we expect a context timeout error. + require.Eventually(t, func() bool { + mu.Lock() + defer mu.Unlock() + return ctxErr != nil + }, 1*time.Second, 100*time.Millisecond) + mu.Lock() + require.EqualError(t, ctxErr, "context canceled") + mu.Unlock() + + // Now we set the monitor to be unblocked. + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + err = m.WaitUntilSemiSyncUnblocked(context.Background()) + require.NoError(t, err) + // This should unblock the second wait. + wg.Wait() + // Eventually the writes should also stop. + require.Eventually(t, func() bool { + m.mu.Lock() + defer m.mu.Unlock() + return !m.isWriting + }, 1*time.Second, 100*time.Millisecond) + + // Also verify that if the monitor is closed, we don't wait. + m.Close() + err = m.WaitUntilSemiSyncUnblocked(context.Background()) + require.NoError(t, err) + require.True(t, m.isClosed()) +} + +// TestSemiSyncMonitor tests the semi-sync monitor as a black box. +// It only calls the exported methods to see they work as intended. +func TestSemiSyncMonitor(t *testing.T) { + initialVal := waitBetweenWrites + waitBetweenWrites = 250 * time.Millisecond + defer func() { + waitBetweenWrites = initialVal + }() + db := fakesqldb.New(t) + defer db.Close() + params := db.ConnParams() + cp := *params + dbc := dbconfigs.NewTestDBConfigs(cp, cp, "") + config := &tabletenv.TabletConfig{ + DB: dbc, + SemiSyncMonitor: tabletenv.SemiSyncMonitorConfig{ + Interval: 1 * time.Second, + }, + } + m := NewMonitor(config, exporter) + defer m.Close() + + db.SetNeverFail(true) + // Initially everything is unblocked. + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + + // Open the monitor. + m.Open() + + // Initially writes aren't blocked and the wait returns immediately. + require.False(t, m.AllWritesBlocked()) + err := m.WaitUntilSemiSyncUnblocked(context.Background()) + require.NoError(t, err) + + // Now we set the monitor to be blocked. + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|1")) + + // Start a go routine waiting for semi-sync being unblocked. + var waitFinished atomic.Bool + go func() { + err := m.WaitUntilSemiSyncUnblocked(context.Background()) + require.NoError(t, err) + waitFinished.Store(true) + }() + + // Even if we wait a second, the wait shouldn't be over. + time.Sleep(1 * time.Second) + require.False(t, waitFinished.Load()) + + // If we unblock the semi-sync, then the wait should finish. + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + require.Eventually(t, func() bool { + return waitFinished.Load() + }, 1*time.Second, 100*time.Millisecond) + require.False(t, m.AllWritesBlocked()) + + // Add a universal insert query pattern that would block until we make it unblock. + ch := make(chan int) + db.AddQueryPatternWithCallback("^INSERT INTO.*", sqltypes.MakeTestResult(nil), func(s string) { + <-ch + }) + // We block the semi-sync again. + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|1")) + + // Start another go routine, also waiting for semi-sync being unblocked. + waitFinished.Store(false) + go func() { + err := m.WaitUntilSemiSyncUnblocked(context.Background()) + require.NoError(t, err) + waitFinished.Store(true) + }() + + // Since the writes are now blocking, eventually all the writes should block. + require.Eventually(t, func() bool { + return m.AllWritesBlocked() + }, 10*time.Second, 100*time.Millisecond) + + // The wait should still not have ended. + require.False(t, waitFinished.Load()) + + // Now we unblock the writes and semi-sync. + close(ch) + db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + + // The wait should now finish. + require.Eventually(t, func() bool { + return waitFinished.Load() + }, 1*time.Second, 100*time.Millisecond) + require.False(t, m.AllWritesBlocked()) + + // Close the monitor. + m.Close() + // The test is technically over, but we wait for the writes to have stopped to prevent any data races. + require.Eventually(t, func() bool { + m.mu.Lock() + defer m.mu.Unlock() + return !m.isWriting + }, 1*time.Second, 100*time.Millisecond) +} From 942eaf0e010f38325a8e903db5592ae15634dcfc Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Tue, 18 Feb 2025 12:15:11 +0530 Subject: [PATCH 16/27] test: fix tests by doing nil checks and adding semi-sync monitor to fake tablet Signed-off-by: Manan Gupta --- go/cmd/vtctldclient/command/framework_test.go | 7 ++++++ go/vt/mysqlctl/fakemysqldaemon.go | 5 +++++ .../vttablet/tabletmanager/rpc_replication.go | 20 +++++++++++------ .../tabletmanager/rpc_replication_test.go | 5 ++++- .../tabletmanager/semisyncmonitor/monitor.go | 22 ++++++++++++++++++- go/vt/vttablet/tabletmanager/tm_init_test.go | 9 +++++++- go/vt/wrangler/fake_tablet_test.go | 7 ++++++ go/vt/wrangler/testlib/fake_tablet.go | 7 ++++++ 8 files changed, 72 insertions(+), 10 deletions(-) diff --git a/go/cmd/vtctldclient/command/framework_test.go b/go/cmd/vtctldclient/command/framework_test.go index 351356ea3a0..b4c06cdd014 100644 --- a/go/cmd/vtctldclient/command/framework_test.go +++ b/go/cmd/vtctldclient/command/framework_test.go @@ -30,12 +30,14 @@ import ( "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/dbconfigs" "vitess.io/vitess/go/vt/mysqlctl" + "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtenv" "vitess.io/vitess/go/vt/vttablet/grpctmserver" "vitess.io/vitess/go/vt/vttablet/tabletconntest" "vitess.io/vitess/go/vt/vttablet/tabletmanager" + "vitess.io/vitess/go/vt/vttablet/tabletmanager/semisyncmonitor" "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" "vitess.io/vitess/go/vt/vttablet/tabletservermock" "vitess.io/vitess/go/vt/vttablet/tmclient" @@ -158,6 +160,10 @@ func NewFakeTablet(t *testing.T, ts *topo.Server, cell string, uid uint32, table } } +var ( + exporter = servenv.NewExporter("TestVtctldClientCommand", "") +) + // StartActionLoop will start the action loop for a fake tablet, // using ft.FakeMysqlDaemon as the backing mysqld. func (ft *FakeTablet) StartActionLoop(t *testing.T, ts *topo.Server) { @@ -203,6 +209,7 @@ func (ft *FakeTablet) StartActionLoop(t *testing.T, ts *topo.Server) { DBConfigs: &dbconfigs.DBConfigs{}, QueryServiceControl: tabletservermock.NewController(), VREngine: vreplication.NewTestEngine(ts, ft.Tablet.Alias.Cell, ft.FakeMysqlDaemon, binlogplayer.NewFakeDBClient, binlogplayer.NewFakeDBClient, topoproto.TabletDbName(ft.Tablet), nil), + SemiSyncMonitor: semisyncmonitor.CreateTestSemiSyncMonitor(ft.FakeMysqlDaemon.DB(), exporter), Env: vtenv.NewTestEnv(), } if err := ft.TM.Start(ft.Tablet, nil); err != nil { diff --git a/go/vt/mysqlctl/fakemysqldaemon.go b/go/vt/mysqlctl/fakemysqldaemon.go index e6afe7917f1..a2b5e66bd49 100644 --- a/go/vt/mysqlctl/fakemysqldaemon.go +++ b/go/vt/mysqlctl/fakemysqldaemon.go @@ -217,6 +217,11 @@ func NewFakeMysqlDaemon(db *fakesqldb.DB) *FakeMysqlDaemon { return result } +// DB returns the fakesqldb.DB object. +func (fmd *FakeMysqlDaemon) DB() *fakesqldb.DB { + return fmd.db +} + // Start is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) Start(ctx context.Context, cnf *Mycnf, mysqldArgs ...string) error { if fmd.Running { diff --git a/go/vt/vttablet/tabletmanager/rpc_replication.go b/go/vt/vttablet/tabletmanager/rpc_replication.go index 5f6e154e2d6..9ed97490191 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication.go @@ -1052,18 +1052,24 @@ func (tm *TabletManager) fixSemiSync(ctx context.Context, tabletType topodatapb. case SemiSyncActionNone: return nil case SemiSyncActionSet: - // We want to enable the semi-sync monitor only if the tablet is going to start - // expecting semi-sync ACKs. - if tabletType == topodatapb.TabletType_PRIMARY { - tm.SemiSyncMonitor.Open() - } else { - tm.SemiSyncMonitor.Close() + if tm.SemiSyncMonitor != nil { + // We want to enable the semi-sync monitor only if the tablet is going to start + // expecting semi-sync ACKs. + if tabletType == topodatapb.TabletType_PRIMARY { + tm.SemiSyncMonitor.Open() + } else { + tm.SemiSyncMonitor.Close() + } } // Always enable replica-side since it doesn't hurt to keep it on for a primary. // The primary-side needs to be off for a replica, or else it will get stuck. return tm.MysqlDaemon.SetSemiSyncEnabled(ctx, tabletType == topodatapb.TabletType_PRIMARY, true) case SemiSyncActionUnset: - tm.SemiSyncMonitor.Close() + // The nil check is required for vtcombo, which doesn't run the semi-sync monitor + // but does try to turn off semi-sync. + if tm.SemiSyncMonitor != nil { + tm.SemiSyncMonitor.Close() + } return tm.MysqlDaemon.SetSemiSyncEnabled(ctx, false, false) default: return vterrors.Errorf(vtrpc.Code_INTERNAL, "Unknown SemiSyncAction - %v", semiSync) diff --git a/go/vt/vttablet/tabletmanager/rpc_replication_test.go b/go/vt/vttablet/tabletmanager/rpc_replication_test.go index b388235811b..33bbd8ec260 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication_test.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication_test.go @@ -26,6 +26,7 @@ import ( "golang.org/x/sync/semaphore" "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/vttablet/tabletmanager/semisyncmonitor" "vitess.io/vitess/go/vt/vttablet/tabletserver" ) @@ -77,15 +78,17 @@ func TestDemotePrimaryStalled(t *testing.T) { waitTime: 2 * time.Second, } // Create a tablet manager with a replica type tablet. + fakeDb := newTestMysqlDaemon(t, 1) tm := &TabletManager{ actionSema: semaphore.NewWeighted(1), - MysqlDaemon: newTestMysqlDaemon(t, 1), + MysqlDaemon: fakeDb, tmState: &tmState{ displayState: displayState{ tablet: newTestTablet(t, 100, "ks", "-", map[string]string{}), }, }, QueryServiceControl: qsc, + SemiSyncMonitor: semisyncmonitor.CreateTestSemiSyncMonitor(fakeDb.DB(), exporter), } // We make IsServing stall for over 2 seconds, which is longer than 10 * remote operation timeout. diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index 7bc23e56717..0be0527bbfc 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -23,8 +23,10 @@ import ( "time" "vitess.io/vitess/go/constants/sidecar" + "vitess.io/vitess/go/mysql/fakesqldb" "vitess.io/vitess/go/stats" "vitess.io/vitess/go/timer" + "vitess.io/vitess/go/vt/dbconfigs" "vitess.io/vitess/go/vt/dbconnpool" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/mysqlctl" @@ -97,11 +99,29 @@ func NewMonitor(config *tabletenv.TabletConfig, exporter *servenv.Exporter) *Mon } } +// CreateTestSemiSyncMonitor created a monitor for testing. +// It takes an optional fake db. +func CreateTestSemiSyncMonitor(db *fakesqldb.DB, exporter *servenv.Exporter) *Monitor { + var dbc *dbconfigs.DBConfigs + if db != nil { + params := db.ConnParams() + cp := *params + dbc = dbconfigs.NewTestDBConfigs(cp, cp, "") + } + return NewMonitor(&tabletenv.TabletConfig{ + DB: dbc, + SemiSyncMonitor: tabletenv.SemiSyncMonitorConfig{ + Interval: 10 * time.Second, + }, + }, exporter) +} + // Open starts the monitor. func (m *Monitor) Open() { m.mu.Lock() defer m.mu.Unlock() - if m.isOpen { + // The check for config being nil is only requried for tests. + if m.isOpen || m.config == nil || m.config.DB == nil { // If we are already open, then there is nothing to do return } diff --git a/go/vt/vttablet/tabletmanager/tm_init_test.go b/go/vt/vttablet/tabletmanager/tm_init_test.go index b8c9c54dcc2..a6c5d33c975 100644 --- a/go/vt/vttablet/tabletmanager/tm_init_test.go +++ b/go/vt/vttablet/tabletmanager/tm_init_test.go @@ -39,6 +39,7 @@ import ( "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/memorytopo" "vitess.io/vitess/go/vt/topotools" + "vitess.io/vitess/go/vt/vttablet/tabletmanager/semisyncmonitor" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" "vitess.io/vitess/go/vt/vttablet/tabletservermock" "vitess.io/vitess/go/vt/vttest" @@ -683,6 +684,10 @@ func newTestMysqlDaemon(t *testing.T, port int32) *mysqlctl.FakeMysqlDaemon { return mysqld } +var ( + exporter = servenv.NewExporter("TestTabletManager", "") +) + func newTestTM(t *testing.T, ts *topo.Server, uid int, keyspace, shard string, tags map[string]string) *TabletManager { // reset stats statsTabletTags.ResetAll() @@ -691,11 +696,13 @@ func newTestTM(t *testing.T, ts *topo.Server, uid int, keyspace, shard string, t t.Helper() ctx := context.Background() tablet := newTestTablet(t, uid, keyspace, shard, tags) + fakeDb := newTestMysqlDaemon(t, 1) tm := &TabletManager{ BatchCtx: ctx, TopoServer: ts, - MysqlDaemon: newTestMysqlDaemon(t, 1), + MysqlDaemon: fakeDb, DBConfigs: &dbconfigs.DBConfigs{}, + SemiSyncMonitor: semisyncmonitor.CreateTestSemiSyncMonitor(fakeDb.DB(), exporter), QueryServiceControl: tabletservermock.NewController(), } err := tm.Start(tablet, nil) diff --git a/go/vt/wrangler/fake_tablet_test.go b/go/vt/wrangler/fake_tablet_test.go index b70a64d644e..2eec782f301 100644 --- a/go/vt/wrangler/fake_tablet_test.go +++ b/go/vt/wrangler/fake_tablet_test.go @@ -33,6 +33,7 @@ import ( "vitess.io/vitess/go/vt/mysqlctl" querypb "vitess.io/vitess/go/vt/proto/query" topodatapb "vitess.io/vitess/go/vt/proto/topodata" + "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/vtenv" @@ -41,6 +42,7 @@ import ( "vitess.io/vitess/go/vt/vttablet/queryservice/fakes" "vitess.io/vitess/go/vt/vttablet/tabletconntest" "vitess.io/vitess/go/vt/vttablet/tabletmanager" + "vitess.io/vitess/go/vt/vttablet/tabletmanager/semisyncmonitor" vdiff2 "vitess.io/vitess/go/vt/vttablet/tabletmanager/vdiff" "vitess.io/vitess/go/vt/vttablet/tabletservermock" "vitess.io/vitess/go/vt/vttablet/tmclient" @@ -158,6 +160,10 @@ func newFakeTablet(t *testing.T, wr *Wrangler, cell string, uid uint32, tabletTy } } +var ( + exporter = servenv.NewExporter("TestWrangler", "") +) + // StartActionLoop will start the action loop for a fake tablet, // using ft.FakeMysqlDaemon as the backing mysqld. func (ft *fakeTablet) StartActionLoop(t *testing.T, wr *Wrangler) { @@ -199,6 +205,7 @@ func (ft *fakeTablet) StartActionLoop(t *testing.T, wr *Wrangler) { DBConfigs: &dbconfigs.DBConfigs{}, QueryServiceControl: tabletservermock.NewController(), VDiffEngine: vdiff2.NewEngine(wr.TopoServer(), ft.Tablet, collations.MySQL8(), sqlparser.NewTestParser()), + SemiSyncMonitor: semisyncmonitor.CreateTestSemiSyncMonitor(ft.FakeMysqlDaemon.DB(), exporter), Env: vtenv.NewTestEnv(), } if err := ft.TM.Start(ft.Tablet, nil); err != nil { diff --git a/go/vt/wrangler/testlib/fake_tablet.go b/go/vt/wrangler/testlib/fake_tablet.go index 9649d717d73..7881ebf6a2a 100644 --- a/go/vt/wrangler/testlib/fake_tablet.go +++ b/go/vt/wrangler/testlib/fake_tablet.go @@ -33,12 +33,14 @@ import ( "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/dbconfigs" "vitess.io/vitess/go/vt/mysqlctl" + "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtenv" "vitess.io/vitess/go/vt/vttablet/grpctmserver" "vitess.io/vitess/go/vt/vttablet/tabletconntest" "vitess.io/vitess/go/vt/vttablet/tabletmanager" + "vitess.io/vitess/go/vt/vttablet/tabletmanager/semisyncmonitor" "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" "vitess.io/vitess/go/vt/vttablet/tabletservermock" "vitess.io/vitess/go/vt/vttablet/tmclient" @@ -159,6 +161,10 @@ func NewFakeTablet(t *testing.T, wr *wrangler.Wrangler, cell string, uid uint32, } } +var ( + exporter = servenv.NewExporter("TestWranglerTestLib", "") +) + // StartActionLoop will start the action loop for a fake tablet, // using ft.FakeMysqlDaemon as the backing mysqld. func (ft *FakeTablet) StartActionLoop(t *testing.T, wr *wrangler.Wrangler) { @@ -202,6 +208,7 @@ func (ft *FakeTablet) StartActionLoop(t *testing.T, wr *wrangler.Wrangler) { DBConfigs: &dbconfigs.DBConfigs{}, QueryServiceControl: tabletservermock.NewController(), VREngine: vreplication.NewTestEngine(wr.TopoServer(), ft.Tablet.Alias.Cell, ft.FakeMysqlDaemon, binlogplayer.NewFakeDBClient, binlogplayer.NewFakeDBClient, topoproto.TabletDbName(ft.Tablet), nil), + SemiSyncMonitor: semisyncmonitor.CreateTestSemiSyncMonitor(ft.FakeMysqlDaemon.DB(), exporter), Env: vtenv.NewTestEnv(), } if err := ft.TM.Start(ft.Tablet, nil); err != nil { From 3a3db2a8c7f73b08908d2ad3f787d4a8b3b30994 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Tue, 18 Feb 2025 12:30:46 +0530 Subject: [PATCH 17/27] test: add demote primary test that makes sure we block when semi-sync is blocked Signed-off-by: Manan Gupta --- .../tabletmanager/rpc_replication_test.go | 44 +++++++++++++++++++ .../tabletmanager/semisyncmonitor/monitor.go | 2 +- 2 files changed, 45 insertions(+), 1 deletion(-) diff --git a/go/vt/vttablet/tabletmanager/rpc_replication_test.go b/go/vt/vttablet/tabletmanager/rpc_replication_test.go index 33bbd8ec260..9fd4beec667 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication_test.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication_test.go @@ -25,7 +25,10 @@ import ( "github.com/stretchr/testify/require" "golang.org/x/sync/semaphore" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/mysqlctl" "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/memorytopo" "vitess.io/vitess/go/vt/vttablet/tabletmanager/semisyncmonitor" "vitess.io/vitess/go/vt/vttablet/tabletserver" ) @@ -96,3 +99,44 @@ func TestDemotePrimaryStalled(t *testing.T) { tm.demotePrimary(context.Background(), false) require.True(t, qsc.primaryStalled.Load()) } + +// TestDemotePrimaryWaitingForSemiSyncUnblock tests that demote primary unblocks if the primary is blocked on semi-sync ACKs +// and doesn't issue the set super read-only query until all writes waiting on semi-sync ACKs have gone through. +func TestDemotePrimaryWaitingForSemiSyncUnblock(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + ts := memorytopo.NewServer(ctx, "cell1") + tm := newTestTM(t, ts, 1, "ks", "0", nil) + fakeMysqlDaemon := tm.MysqlDaemon.(*mysqlctl.FakeMysqlDaemon) + fakeDb := fakeMysqlDaemon.DB() + fakeDb.SetNeverFail(true) + + tm.SemiSyncMonitor.Open() + // Add a fake query that makes the semi-sync monitor believe that the tablet is blocked on semi-sync ACKs. + fakeDb.AddQuery("SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'", sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|1")) + + // Start the demote primary operation in a go routine. + var demotePrimaryFinished atomic.Bool + go func() { + _, err := tm.demotePrimary(context.Background(), false) + require.NoError(t, err) + demotePrimaryFinished.Store(true) + }() + + // Wait for the demote primary operation to be blocked on semi-sync. + time.Sleep(1 * time.Second) + // DemotePrimary shouldn't have finished yet. + require.False(t, demotePrimaryFinished.Load()) + // We shouldn't have seen the super-read only query either. + require.False(t, fakeMysqlDaemon.SuperReadOnly.Load()) + + // Now we unblock the semi-sync monitor. + fakeDb.AddQuery("SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'", sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + + // This should unblock the demote primary operation eventually. + require.Eventually(t, func() bool { + return demotePrimaryFinished.Load() + }, 5*time.Second, 100*time.Millisecond) + // We should have also seen the super-read only query. + require.True(t, fakeMysqlDaemon.SuperReadOnly.Load()) +} diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index 0be0527bbfc..f237608632f 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -111,7 +111,7 @@ func CreateTestSemiSyncMonitor(db *fakesqldb.DB, exporter *servenv.Exporter) *Mo return NewMonitor(&tabletenv.TabletConfig{ DB: dbc, SemiSyncMonitor: tabletenv.SemiSyncMonitorConfig{ - Interval: 10 * time.Second, + Interval: 1 * time.Second, }, }, exporter) } From eaa92462c7ee93fdca08d9d518315368d834e9d6 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Tue, 18 Feb 2025 12:38:14 +0530 Subject: [PATCH 18/27] summary: add summary changes to make users aware of the change Signed-off-by: Manan Gupta --- changelog/22.0/22.0.0/summary.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/changelog/22.0/22.0.0/summary.md b/changelog/22.0/22.0.0/summary.md index 1e7d713b60e..2acac12c694 100644 --- a/changelog/22.0/22.0.0/summary.md +++ b/changelog/22.0/22.0.0/summary.md @@ -18,6 +18,7 @@ - **[Update lite images to Debian Bookworm](#debian-bookworm)** - **[KeyRanges in `--clusters_to_watch` in VTOrc](#key-range-vtorc)** - **[Support for Filtering Query logs on Error](#query-logs)** + - **[Semi-sync monitor in vttablet](#semi-sync-monitor)** - **[Minor Changes](#minor-changes)** - **[VTTablet Flags](#flags-vttablet)** - **[VTTablet ACL enforcement and reloading](#reloading-vttablet-acl)** @@ -162,6 +163,14 @@ Users can continue to specify exact keyranges. The new feature is backward compa The `querylog-mode` setting can be configured to `error` to log only queries that result in errors. This option is supported in both VTGate and VTTablet. +### Semi-sync monitor in vttablet + +A new component has been added to the vttablet binary to monitor the semi-sync status of primary vttablets. We've observed cases where a brief network disruption can cause the primary to get stuck indefinitely waiting for semi-sync ACKs. In rare scenarios, this can block reparent operations and render the primary unresponsive. More information can be found in the issues https://github.com/vitessio/vitess/issues/17709 and https://github.com/vitessio/vitess/issues/17749. + +To address this, the new component continuously monitors the semi-sync status. If the primary becomes stuck on semi-sync ACKs, it generates writes to unblock it. If this fails, VTOrc is notified of the issue and initiates an emergency reparent operation. + +The monitoring interval can be adjusted using the `--semi-sync-monitor-interval` flag, which defaults to 10 seconds. + ## Minor Changes #### VTTablet Flags From a571e48b9acfe7193c5c18529bd937b5423e7359 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Tue, 18 Feb 2025 12:43:11 +0530 Subject: [PATCH 19/27] test: fix more tests by updating expectations Signed-off-by: Manan Gupta --- go/test/endtoend/vreplication/sidecardb_test.go | 2 +- go/vt/vttablet/tabletserver/tabletenv/config_test.go | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/go/test/endtoend/vreplication/sidecardb_test.go b/go/test/endtoend/vreplication/sidecardb_test.go index f908d66a2ec..ec3c13f5981 100644 --- a/go/test/endtoend/vreplication/sidecardb_test.go +++ b/go/test/endtoend/vreplication/sidecardb_test.go @@ -38,7 +38,7 @@ var ddls1, ddls2 []string func init() { sidecarDBTables = []string{"copy_state", "dt_participant", "dt_state", "heartbeat", "post_copy_action", - "redo_state", "redo_statement", "reparent_journal", "resharding_journal", "schema_migrations", "schema_version", + "redo_state", "redo_statement", "reparent_journal", "resharding_journal", "schema_migrations", "schema_version", "semisync_recover", "tables", "udfs", "vdiff", "vdiff_log", "vdiff_table", "views", "vreplication", "vreplication_log"} numSidecarDBTables = len(sidecarDBTables) ddls1 = []string{ diff --git a/go/vt/vttablet/tabletserver/tabletenv/config_test.go b/go/vt/vttablet/tabletserver/tabletenv/config_test.go index 628277b9e24..20f4541cdc8 100644 --- a/go/vt/vttablet/tabletserver/tabletenv/config_test.go +++ b/go/vt/vttablet/tabletserver/tabletenv/config_test.go @@ -97,6 +97,7 @@ rowStreamer: maxMySQLReplLagSecs: 400 schemaChangeReloadTimeout: 30s schemaReloadIntervalSeconds: 30m0s +semiSyncMonitor: {} txPool: {} ` assert.Equal(t, wantBytes, string(gotBytes)) @@ -164,6 +165,8 @@ rowStreamer: maxMySQLReplLagSecs: 43200 schemaChangeReloadTimeout: 30s schemaReloadIntervalSeconds: 30m0s +semiSyncMonitor: + intervalSeconds: 10s signalWhenSchemaChange: true streamBufferSize: 32768 txPool: From 05c3d2e3adc6cac95cd74425a7c3c606a0fa7401 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Thu, 20 Feb 2025 12:08:32 +0530 Subject: [PATCH 20/27] feat: make the test less flaky and improve flag documentation Signed-off-by: Manan Gupta --- go/flags/endtoend/vtcombo.txt | 2 +- go/flags/endtoend/vttablet.txt | 2 +- .../reparent/newfeaturetest/reparent_test.go | 9 ++++++++- .../tabletmanager/rpc_replication_test.go | 19 +++++++++++++++---- .../vttablet/tabletserver/tabletenv/config.go | 2 +- 5 files changed, 26 insertions(+), 8 deletions(-) diff --git a/go/flags/endtoend/vtcombo.txt b/go/flags/endtoend/vtcombo.txt index 65767404046..5436b6ef957 100644 --- a/go/flags/endtoend/vtcombo.txt +++ b/go/flags/endtoend/vtcombo.txt @@ -322,7 +322,7 @@ Flags: --schema_change_signal Enable the schema tracker; requires queryserver-config-schema-change-signal to be enabled on the underlying vttablets for this to work (default true) --schema_dir string Schema base directory. Should contain one directory per keyspace, with a vschema.json file if necessary. --security_policy string the name of a registered security policy to use for controlling access to URLs - empty means allow all for anyone (built-in policies: deny-all, read-only) - --semi-sync-monitor-interval duration Interval between semi-sync monitor checks if the primary is blocked on semi-sync ACKs (default 10s) + --semi-sync-monitor-interval duration How frequently the semi-sync monitor checks if the primary is blocked on semi-sync ACKs (default 10s) --service_map strings comma separated list of services to enable (or disable if prefixed with '-') Example: grpc-queryservice --serving_state_grace_period duration how long to pause after broadcasting health to vtgate, before enforcing a new serving state --shard_sync_retry_delay duration delay between retries of updates to keep the tablet and its shard record in sync (default 30s) diff --git a/go/flags/endtoend/vttablet.txt b/go/flags/endtoend/vttablet.txt index 7d9c60d1b05..f4c33d71b6b 100644 --- a/go/flags/endtoend/vttablet.txt +++ b/go/flags/endtoend/vttablet.txt @@ -322,7 +322,7 @@ Flags: --schema-change-reload-timeout duration query server schema change reload timeout, this is how long to wait for the signaled schema reload operation to complete before giving up (default 30s) --schema-version-max-age-seconds int max age of schema version records to kept in memory by the vreplication historian --security_policy string the name of a registered security policy to use for controlling access to URLs - empty means allow all for anyone (built-in policies: deny-all, read-only) - --semi-sync-monitor-interval duration Interval between semi-sync monitor checks if the primary is blocked on semi-sync ACKs (default 10s) + --semi-sync-monitor-interval duration How frequently the semi-sync monitor checks if the primary is blocked on semi-sync ACKs (default 10s) --service_map strings comma separated list of services to enable (or disable if prefixed with '-') Example: grpc-queryservice --serving_state_grace_period duration how long to pause after broadcasting health to vtgate, before enforcing a new serving state --shard_sync_retry_delay duration delay between retries of updates to keep the tablet and its shard record in sync (default 30s) diff --git a/go/test/endtoend/reparent/newfeaturetest/reparent_test.go b/go/test/endtoend/reparent/newfeaturetest/reparent_test.go index 81c66ebb05e..ca9cbcefcf5 100644 --- a/go/test/endtoend/reparent/newfeaturetest/reparent_test.go +++ b/go/test/endtoend/reparent/newfeaturetest/reparent_test.go @@ -19,7 +19,9 @@ package newfeaturetest import ( "context" "fmt" + "os" "os/exec" + "strings" "sync" "testing" "time" @@ -242,7 +244,12 @@ func TestBufferingWithMultipleDisruptions(t *testing.T) { // where a primary is stuck waiting for semi-sync ACKs due to a network issue, // even if no new writes from the user arrives. func TestSemiSyncBlockDueToDisruption(t *testing.T) { - t.Skip("Test not meant to be run on CI") + // This is always set to "true" on GitHub Actions runners: + // https://docs.github.com/en/actions/learn-github-actions/variables#default-environment-variables + ci, ok := os.LookupEnv("CI") + if ok && strings.ToLower(ci) == "true" { + t.Skip("Test not meant to be run on CI") + } clusterInstance := utils.SetupReparentCluster(t, policy.DurabilitySemiSync) defer utils.TeardownCluster(clusterInstance) tablets := clusterInstance.Keyspaces[0].Shards[0].Vttablets diff --git a/go/vt/vttablet/tabletmanager/rpc_replication_test.go b/go/vt/vttablet/tabletmanager/rpc_replication_test.go index 9fd4beec667..d02a210c3d5 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication_test.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication_test.go @@ -27,6 +27,7 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/mysqlctl" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/memorytopo" "vitess.io/vitess/go/vt/vttablet/tabletmanager/semisyncmonitor" @@ -103,10 +104,13 @@ func TestDemotePrimaryStalled(t *testing.T) { // TestDemotePrimaryWaitingForSemiSyncUnblock tests that demote primary unblocks if the primary is blocked on semi-sync ACKs // and doesn't issue the set super read-only query until all writes waiting on semi-sync ACKs have gone through. func TestDemotePrimaryWaitingForSemiSyncUnblock(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() ts := memorytopo.NewServer(ctx, "cell1") tm := newTestTM(t, ts, 1, "ks", "0", nil) + // Make the tablet a primary. + err := tm.ChangeType(ctx, topodatapb.TabletType_PRIMARY, false) + require.NoError(t, err) fakeMysqlDaemon := tm.MysqlDaemon.(*mysqlctl.FakeMysqlDaemon) fakeDb := fakeMysqlDaemon.DB() fakeDb.SetNeverFail(true) @@ -115,16 +119,23 @@ func TestDemotePrimaryWaitingForSemiSyncUnblock(t *testing.T) { // Add a fake query that makes the semi-sync monitor believe that the tablet is blocked on semi-sync ACKs. fakeDb.AddQuery("SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'", sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|1")) + // Verify that in the beginning the tablet is serving. + require.True(t, tm.QueryServiceControl.IsServing()) + // Start the demote primary operation in a go routine. var demotePrimaryFinished atomic.Bool go func() { - _, err := tm.demotePrimary(context.Background(), false) + _, err := tm.demotePrimary(ctx, false) require.NoError(t, err) demotePrimaryFinished.Store(true) }() - // Wait for the demote primary operation to be blocked on semi-sync. - time.Sleep(1 * time.Second) + // Wait for the demote primary operation to have changed the serving state. + // After that point, we can assume that the demote primary gets blocked on writes waiting for semi-sync ACKs. + require.Eventually(t, func() bool { + return !tm.QueryServiceControl.IsServing() + }, 5*time.Second, 100*time.Millisecond) + // DemotePrimary shouldn't have finished yet. require.False(t, demotePrimaryFinished.Load()) // We shouldn't have seen the super-read only query either. diff --git a/go/vt/vttablet/tabletserver/tabletenv/config.go b/go/vt/vttablet/tabletserver/tabletenv/config.go index 257361613c3..47a9acfc930 100644 --- a/go/vt/vttablet/tabletserver/tabletenv/config.go +++ b/go/vt/vttablet/tabletserver/tabletenv/config.go @@ -204,7 +204,7 @@ func registerTabletEnvFlags(fs *pflag.FlagSet) { fs.DurationVar(°radedThreshold, "degraded_threshold", defaultConfig.Healthcheck.DegradedThreshold, "replication lag after which a replica is considered degraded") fs.DurationVar(&unhealthyThreshold, "unhealthy_threshold", defaultConfig.Healthcheck.UnhealthyThreshold, "replication lag after which a replica is considered unhealthy") fs.DurationVar(&transitionGracePeriod, "serving_state_grace_period", 0, "how long to pause after broadcasting health to vtgate, before enforcing a new serving state") - fs.DurationVar(&semiSyncMonitorInterval, "semi-sync-monitor-interval", defaultConfig.SemiSyncMonitor.Interval, "Interval between semi-sync monitor checks if the primary is blocked on semi-sync ACKs") + fs.DurationVar(&semiSyncMonitorInterval, "semi-sync-monitor-interval", defaultConfig.SemiSyncMonitor.Interval, "How frequently the semi-sync monitor checks if the primary is blocked on semi-sync ACKs") fs.BoolVar(&enableReplicationReporter, "enable_replication_reporter", false, "Use polling to track replication lag.") fs.BoolVar(¤tConfig.EnableOnlineDDL, "queryserver_enable_online_ddl", true, "Enable online DDL.") From dddc6e56640459cb88fa71267439c2a3cec0562e Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Thu, 20 Feb 2025 12:37:59 +0530 Subject: [PATCH 21/27] feat: address review comments Signed-off-by: Manan Gupta --- .../tabletmanager/semisyncmonitor/monitor.go | 22 ++++++++++--------- .../semisyncmonitor/monitor_test.go | 6 +++-- 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index f237608632f..8bbc30afe01 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -80,7 +80,7 @@ type Monitor struct { // isBlocked stores if the primary is blocked on semi-sync ack. isBlocked bool // waiters stores the list of waiters that are waiting for the primary to be unblocked. - waiters []chan any + waiters []chan struct{} // writesBlockedGauge is a gauge tracking the number of writes the monitor is blocked on. writesBlockedGauge *stats.Gauge } @@ -95,7 +95,7 @@ func NewMonitor(config *tabletenv.TabletConfig, exporter *servenv.Exporter) *Mon clearTicks: timer.NewTimer(clearTimerDuration), writesBlockedGauge: exporter.NewGauge("SemiSyncMonitorWritesBlocked", "Number of writes blocked in the semi-sync monitor"), appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", exporter, 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), - waiters: make([]chan any, 0), + waiters: make([]chan struct{}, 0), } } @@ -288,7 +288,7 @@ func (m *Monitor) startWrites() { // exponentially backing off. We will not do more than maxWritesPermitted writes and once // all maxWritesPermitted writes are blocked, we'll wait for VTOrc to run an ERS. go m.write() - <-time.After(waitBetweenWrites) + time.Sleep(waitBetweenWrites) } } @@ -302,7 +302,7 @@ func (m *Monitor) incrementWriteCount() bool { if m.inProgressWriteCount == maxWritesPermitted { return false } - m.inProgressWriteCount = m.inProgressWriteCount + 1 + m.inProgressWriteCount++ m.writesBlockedGauge.Set(int64(m.inProgressWriteCount)) return true } @@ -319,7 +319,7 @@ func (m *Monitor) AllWritesBlocked() bool { func (m *Monitor) decrementWriteCount() { m.mu.Lock() defer m.mu.Unlock() - m.inProgressWriteCount = m.inProgressWriteCount - 1 + m.inProgressWriteCount-- m.writesBlockedGauge.Set(int64(m.inProgressWriteCount)) } @@ -331,9 +331,11 @@ func (m *Monitor) write() { } defer m.decrementWriteCount() // Get a connection from the pool - conn, err := m.appPool.Get(context.Background()) + ctx, cancel := context.WithTimeout(context.Background(), 15*time.Second) + defer cancel() + conn, err := m.appPool.Get(ctx) if err != nil { - log.Errorf("SemiSync Monitor: failed to write to semisync_recovery table: %v", err) + log.Errorf("SemiSync Monitor: failed to get a connection when writing to semisync_recovery table: %v", err) return } defer conn.Recycle() @@ -364,7 +366,7 @@ func (m *Monitor) clearAllData() { // Get a connection from the pool conn, err := m.appPool.Get(context.Background()) if err != nil { - log.Errorf("SemiSync Monitor: failed to clear semisync_recovery table: %v", err) + log.Errorf("SemiSync Monitor: failed get a connection to clear semisync_recovery table: %v", err) return } defer conn.Recycle() @@ -376,10 +378,10 @@ func (m *Monitor) clearAllData() { // addWaiter adds a waiter to the list of waiters // that will be unblocked when the primary is no longer blocked. -func (m *Monitor) addWaiter() chan any { +func (m *Monitor) addWaiter() chan struct{} { m.mu.Lock() defer m.mu.Unlock() - ch := make(chan any) + ch := make(chan struct{}) m.waiters = append(m.waiters, ch) return ch } diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go index 0a1e98dd856..578ad9b0d15 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go @@ -169,7 +169,7 @@ func TestMonitorWaitMechanism(t *testing.T) { m.setIsBlocked(false) require.Eventually(t, func() bool { return waiterUnblocked.Load() - }, time.Second, time.Millisecond*100) + }, 2*time.Second, time.Millisecond*100) require.False(t, m.stillBlocked()) require.False(t, m.isClosed()) } @@ -644,7 +644,9 @@ func TestSemiSyncMonitor(t *testing.T) { // Initially writes aren't blocked and the wait returns immediately. require.False(t, m.AllWritesBlocked()) - err := m.WaitUntilSemiSyncUnblocked(context.Background()) + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + defer cancel() + err := m.WaitUntilSemiSyncUnblocked(ctx) require.NoError(t, err) // Now we set the monitor to be blocked. From 9995a0d8f5bcb1ba829636f3d031179465de748a Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Thu, 20 Feb 2025 13:02:41 +0530 Subject: [PATCH 22/27] feat: change the read query to be more specific Signed-off-by: Manan Gupta --- go/vt/vttablet/tabletmanager/rpc_replication_test.go | 4 ++-- go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/rpc_replication_test.go b/go/vt/vttablet/tabletmanager/rpc_replication_test.go index d02a210c3d5..cdbdceefdc1 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication_test.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication_test.go @@ -117,7 +117,7 @@ func TestDemotePrimaryWaitingForSemiSyncUnblock(t *testing.T) { tm.SemiSyncMonitor.Open() // Add a fake query that makes the semi-sync monitor believe that the tablet is blocked on semi-sync ACKs. - fakeDb.AddQuery("SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'", sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|1")) + fakeDb.AddQuery("select variable_value from performance_schema.global_status where regexp_like(variable_name, 'Rpl_semi_sync_(source|master)_wait_sessions')", sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|1")) // Verify that in the beginning the tablet is serving. require.True(t, tm.QueryServiceControl.IsServing()) @@ -142,7 +142,7 @@ func TestDemotePrimaryWaitingForSemiSyncUnblock(t *testing.T) { require.False(t, fakeMysqlDaemon.SuperReadOnly.Load()) // Now we unblock the semi-sync monitor. - fakeDb.AddQuery("SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'", sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + fakeDb.AddQuery("select variable_value from performance_schema.global_status where regexp_like(variable_name, 'Rpl_semi_sync_(source|master)_wait_sessions')", sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) // This should unblock the demote primary operation eventually. require.Eventually(t, func() bool { diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index 8bbc30afe01..c9b4e65b2e2 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -36,7 +36,7 @@ import ( ) const ( - semiSyncWaitSessionsRead = "SHOW STATUS LIKE 'Rpl_semi_sync_%_wait_sessions'" + semiSyncWaitSessionsRead = "select variable_value from performance_schema.global_status where regexp_like(variable_name, 'Rpl_semi_sync_(source|master)_wait_sessions')" semiSyncRecoverWrite = "INSERT INTO %s.semisync_recover (ts) VALUES (NOW())" semiSyncRecoverClear = "TRUNCATE TABLE %s.semisync_recover" maxWritesPermitted = 15 From cdd609d3cd9f156f47e8666785306f4b40f41c65 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Thu, 20 Feb 2025 13:21:28 +0530 Subject: [PATCH 23/27] test: increase time for eventual checks Signed-off-by: Manan Gupta --- .../semisyncmonitor/monitor_test.go | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go index 578ad9b0d15..9d1039f002e 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go @@ -350,7 +350,7 @@ func TestMonitorWriteBlocked(t *testing.T) { m.mu.Lock() defer m.mu.Unlock() return m.inProgressWriteCount == 1 - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) // Once the writers are unblocked, we expect to see a zero value again. close(ch) @@ -358,11 +358,11 @@ func TestMonitorWriteBlocked(t *testing.T) { m.mu.Lock() defer m.mu.Unlock() return m.inProgressWriteCount == 0 - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) require.Eventually(t, func() bool { return writeFinished.Load() - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) } // TestIsWriting checks the transitions for the isWriting field. @@ -437,7 +437,7 @@ func TestStartWrites(t *testing.T) { m.mu.Lock() defer m.mu.Unlock() return m.inProgressWriteCount >= 1 - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) // Once the writes have started, another call to startWrites shouldn't do anything m.startWrites() @@ -447,7 +447,7 @@ func TestStartWrites(t *testing.T) { m.mu.Lock() defer m.mu.Unlock() return m.inProgressWriteCount >= 2 - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) // Check that the writes are still going. require.False(t, writesFinished.Load()) @@ -458,14 +458,14 @@ func TestStartWrites(t *testing.T) { require.Eventually(t, func() bool { return writesFinished.Load() - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) // Check that no writes are in progress anymore. require.Eventually(t, func() bool { m.mu.Lock() defer m.mu.Unlock() return m.inProgressWriteCount == 0 - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) } func TestCheckAndFixSemiSyncBlocked(t *testing.T) { @@ -511,7 +511,7 @@ func TestCheckAndFixSemiSyncBlocked(t *testing.T) { m.mu.Lock() defer m.mu.Unlock() return m.inProgressWriteCount >= 2 - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) // Now we set the monitor to be unblocked. db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) @@ -526,7 +526,7 @@ func TestCheckAndFixSemiSyncBlocked(t *testing.T) { m.mu.Lock() defer m.mu.Unlock() return m.inProgressWriteCount == 0 && m.isWriting == false - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) } func TestWaitUntilSemiSyncUnblocked(t *testing.T) { @@ -579,7 +579,7 @@ func TestWaitUntilSemiSyncUnblocked(t *testing.T) { m.mu.Lock() defer m.mu.Unlock() return m.isWriting - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) // Now we cancel the context. This should fail the first wait. cancel() @@ -588,7 +588,7 @@ func TestWaitUntilSemiSyncUnblocked(t *testing.T) { mu.Lock() defer mu.Unlock() return ctxErr != nil - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) mu.Lock() require.EqualError(t, ctxErr, "context canceled") mu.Unlock() @@ -604,7 +604,7 @@ func TestWaitUntilSemiSyncUnblocked(t *testing.T) { m.mu.Lock() defer m.mu.Unlock() return !m.isWriting - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) // Also verify that if the monitor is closed, we don't wait. m.Close() @@ -644,7 +644,7 @@ func TestSemiSyncMonitor(t *testing.T) { // Initially writes aren't blocked and the wait returns immediately. require.False(t, m.AllWritesBlocked()) - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) defer cancel() err := m.WaitUntilSemiSyncUnblocked(ctx) require.NoError(t, err) @@ -668,7 +668,7 @@ func TestSemiSyncMonitor(t *testing.T) { db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) require.Eventually(t, func() bool { return waitFinished.Load() - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) require.False(t, m.AllWritesBlocked()) // Add a universal insert query pattern that would block until we make it unblock. @@ -702,7 +702,7 @@ func TestSemiSyncMonitor(t *testing.T) { // The wait should now finish. require.Eventually(t, func() bool { return waitFinished.Load() - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) require.False(t, m.AllWritesBlocked()) // Close the monitor. @@ -712,5 +712,5 @@ func TestSemiSyncMonitor(t *testing.T) { m.mu.Lock() defer m.mu.Unlock() return !m.isWriting - }, 1*time.Second, 100*time.Millisecond) + }, 2*time.Second, 100*time.Millisecond) } From ffd5b5080399f3cb952fe2718c85c62dccd4b012 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Thu, 20 Feb 2025 14:16:15 +0530 Subject: [PATCH 24/27] feat: also mark semi-sync unblocked after a write succeeds Signed-off-by: Manan Gupta --- .../vttablet/tabletmanager/rpc_replication_test.go | 6 ++++++ .../tabletmanager/semisyncmonitor/monitor.go | 4 ++++ .../tabletmanager/semisyncmonitor/monitor_test.go | 14 +++++++++++++- 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/go/vt/vttablet/tabletmanager/rpc_replication_test.go b/go/vt/vttablet/tabletmanager/rpc_replication_test.go index cdbdceefdc1..eaf54e334f6 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication_test.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication_test.go @@ -116,6 +116,11 @@ func TestDemotePrimaryWaitingForSemiSyncUnblock(t *testing.T) { fakeDb.SetNeverFail(true) tm.SemiSyncMonitor.Open() + // Add a universal insert query pattern that would block until we make it unblock. + ch := make(chan int) + fakeDb.AddQueryPatternWithCallback("^INSERT INTO.*", sqltypes.MakeTestResult(nil), func(s string) { + <-ch + }) // Add a fake query that makes the semi-sync monitor believe that the tablet is blocked on semi-sync ACKs. fakeDb.AddQuery("select variable_value from performance_schema.global_status where regexp_like(variable_name, 'Rpl_semi_sync_(source|master)_wait_sessions')", sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|1")) @@ -143,6 +148,7 @@ func TestDemotePrimaryWaitingForSemiSyncUnblock(t *testing.T) { // Now we unblock the semi-sync monitor. fakeDb.AddQuery("select variable_value from performance_schema.global_status where regexp_like(variable_name, 'Rpl_semi_sync_(source|master)_wait_sessions')", sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + close(ch) // This should unblock the demote primary operation eventually. require.Eventually(t, func() bool { diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index c9b4e65b2e2..8a3eb4c1d25 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -342,6 +342,10 @@ func (m *Monitor) write() { _, err = conn.Conn.ExecuteFetch(m.bindSideCarDBName(semiSyncRecoverWrite), 0, false) if err != nil { log.Errorf("SemiSync Monitor: failed to write to semisync_recovery table: %v", err) + } else { + // One of the writes went through without an error. + // This means that we aren't blocked on semi-sync anymore. + m.setIsBlocked(false) } } diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go index 9d1039f002e..358d3e4f782 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go @@ -547,6 +547,11 @@ func TestWaitUntilSemiSyncUnblocked(t *testing.T) { err := m.WaitUntilSemiSyncUnblocked(context.Background()) require.NoError(t, err) + // Add a universal insert query pattern that would block until we make it unblock. + ch := make(chan int) + db.AddQueryPatternWithCallback("^INSERT INTO.*", sqltypes.MakeTestResult(nil), func(s string) { + <-ch + }) // Now we set the monitor to be blocked. db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|3")) @@ -595,6 +600,7 @@ func TestWaitUntilSemiSyncUnblocked(t *testing.T) { // Now we set the monitor to be unblocked. db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + close(ch) err = m.WaitUntilSemiSyncUnblocked(context.Background()) require.NoError(t, err) // This should unblock the second wait. @@ -649,6 +655,11 @@ func TestSemiSyncMonitor(t *testing.T) { err := m.WaitUntilSemiSyncUnblocked(ctx) require.NoError(t, err) + // Add a universal insert query pattern that would block until we make it unblock. + ch := make(chan int) + db.AddQueryPatternWithCallback("^INSERT INTO.*", sqltypes.MakeTestResult(nil), func(s string) { + <-ch + }) // Now we set the monitor to be blocked. db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|1")) @@ -666,13 +677,14 @@ func TestSemiSyncMonitor(t *testing.T) { // If we unblock the semi-sync, then the wait should finish. db.AddQuery(semiSyncWaitSessionsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "Rpl_semi_sync_source_wait_sessions|0")) + close(ch) require.Eventually(t, func() bool { return waitFinished.Load() }, 2*time.Second, 100*time.Millisecond) require.False(t, m.AllWritesBlocked()) // Add a universal insert query pattern that would block until we make it unblock. - ch := make(chan int) + ch = make(chan int) db.AddQueryPatternWithCallback("^INSERT INTO.*", sqltypes.MakeTestResult(nil), func(s string) { <-ch }) From 3bcbd3bd87ec79c2fa69ec9975a8d399cf0b403b Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Thu, 20 Feb 2025 14:52:18 +0530 Subject: [PATCH 25/27] feat: rename some fields Signed-off-by: Manan Gupta --- .../endtoend/vreplication/sidecardb_test.go | 2 +- .../vtorc/readtopologyinstance/main_test.go | 4 +-- .../replicationdata/replicationdata.pb.go | 31 +++++++++---------- .../replicationdata_vtproto.pb.go | 12 +++---- ...ync_recover.sql => semisync_heartbeat.sql} | 2 +- go/vt/vtorc/db/generate_base.go | 2 +- go/vt/vtorc/inst/analysis.go | 2 +- go/vt/vtorc/inst/analysis_dao.go | 8 ++--- go/vt/vtorc/inst/analysis_dao_test.go | 4 +-- go/vt/vtorc/inst/instance.go | 2 +- go/vt/vtorc/inst/instance_dao.go | 8 ++--- go/vt/vtorc/inst/instance_dao_test.go | 4 +-- go/vt/vtorc/test/recovery_analysis.go | 4 +-- .../vttablet/tabletmanager/rpc_replication.go | 2 +- .../tabletmanager/semisyncmonitor/monitor.go | 18 +++++------ .../semisyncmonitor/monitor_test.go | 16 +++++----- proto/replicationdata.proto | 2 +- web/vtadmin/src/proto/vtadmin.d.ts | 8 ++--- web/vtadmin/src/proto/vtadmin.js | 30 +++++++++--------- 19 files changed, 80 insertions(+), 81 deletions(-) rename go/vt/sidecardb/schema/misc/{semisync_recover.sql => semisync_heartbeat.sql} (93%) diff --git a/go/test/endtoend/vreplication/sidecardb_test.go b/go/test/endtoend/vreplication/sidecardb_test.go index ec3c13f5981..54c1a10130f 100644 --- a/go/test/endtoend/vreplication/sidecardb_test.go +++ b/go/test/endtoend/vreplication/sidecardb_test.go @@ -38,7 +38,7 @@ var ddls1, ddls2 []string func init() { sidecarDBTables = []string{"copy_state", "dt_participant", "dt_state", "heartbeat", "post_copy_action", - "redo_state", "redo_statement", "reparent_journal", "resharding_journal", "schema_migrations", "schema_version", "semisync_recover", + "redo_state", "redo_statement", "reparent_journal", "resharding_journal", "schema_migrations", "schema_version", "semisync_heartbeat", "tables", "udfs", "vdiff", "vdiff_log", "vdiff_table", "views", "vreplication", "vreplication_log"} numSidecarDBTables = len(sidecarDBTables) ddls1 = []string{ diff --git a/go/test/endtoend/vtorc/readtopologyinstance/main_test.go b/go/test/endtoend/vtorc/readtopologyinstance/main_test.go index a73afdc946d..531b069535a 100644 --- a/go/test/endtoend/vtorc/readtopologyinstance/main_test.go +++ b/go/test/endtoend/vtorc/readtopologyinstance/main_test.go @@ -91,7 +91,7 @@ func TestReadTopologyInstanceBufferable(t *testing.T) { assert.True(t, primaryInstance.SemiSyncReplicaEnabled) assert.True(t, primaryInstance.SemiSyncPrimaryStatus) assert.False(t, primaryInstance.SemiSyncReplicaStatus) - assert.False(t, primaryInstance.SemiSyncMonitorBlocked) + assert.False(t, primaryInstance.SemiSyncBlocked) assert.EqualValues(t, 2, primaryInstance.SemiSyncPrimaryClients) assert.EqualValues(t, 1, primaryInstance.SemiSyncPrimaryWaitForReplicaCount) assert.EqualValues(t, 1000000000000000000, primaryInstance.SemiSyncPrimaryTimeout) @@ -143,7 +143,7 @@ func TestReadTopologyInstanceBufferable(t *testing.T) { assert.False(t, replicaInstance.SemiSyncPrimaryEnabled) assert.True(t, replicaInstance.SemiSyncReplicaEnabled) assert.False(t, replicaInstance.SemiSyncPrimaryStatus) - assert.False(t, replicaInstance.SemiSyncMonitorBlocked) + assert.False(t, replicaInstance.SemiSyncBlocked) assert.True(t, replicaInstance.SemiSyncReplicaStatus) assert.EqualValues(t, 0, replicaInstance.SemiSyncPrimaryClients) assert.EqualValues(t, 1, replicaInstance.SemiSyncPrimaryWaitForReplicaCount) diff --git a/go/vt/proto/replicationdata/replicationdata.pb.go b/go/vt/proto/replicationdata/replicationdata.pb.go index e4cfdbd8ad6..2cd8424135a 100644 --- a/go/vt/proto/replicationdata/replicationdata.pb.go +++ b/go/vt/proto/replicationdata/replicationdata.pb.go @@ -504,7 +504,7 @@ type FullStatus struct { SuperReadOnly bool `protobuf:"varint,21,opt,name=super_read_only,json=superReadOnly,proto3" json:"super_read_only,omitempty"` ReplicationConfiguration *Configuration `protobuf:"bytes,22,opt,name=replication_configuration,json=replicationConfiguration,proto3" json:"replication_configuration,omitempty"` DiskStalled bool `protobuf:"varint,23,opt,name=disk_stalled,json=diskStalled,proto3" json:"disk_stalled,omitempty"` - SemiSyncMonitorBlocked bool `protobuf:"varint,24,opt,name=semi_sync_monitor_blocked,json=semiSyncMonitorBlocked,proto3" json:"semi_sync_monitor_blocked,omitempty"` + SemiSyncBlocked bool `protobuf:"varint,24,opt,name=semi_sync_blocked,json=semiSyncBlocked,proto3" json:"semi_sync_blocked,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -700,9 +700,9 @@ func (x *FullStatus) GetDiskStalled() bool { return false } -func (x *FullStatus) GetSemiSyncMonitorBlocked() bool { +func (x *FullStatus) GetSemiSyncBlocked() bool { if x != nil { - return x.SemiSyncMonitorBlocked + return x.SemiSyncBlocked } return false } @@ -794,7 +794,7 @@ var file_replicationdata_proto_rawDesc = string([]byte{ 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x69, 0x6c, 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, - 0x55, 0x75, 0x69, 0x64, 0x22, 0xa6, 0x09, 0x0a, 0x0a, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, + 0x55, 0x75, 0x69, 0x64, 0x22, 0x97, 0x09, 0x0a, 0x0a, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, @@ -865,18 +865,17 @@ var file_replicationdata_proto_rawDesc = string([]byte{ 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x73, 0x74, 0x61, 0x6c, 0x6c, 0x65, 0x64, 0x18, 0x17, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x64, 0x69, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x6c, 0x6c, - 0x65, 0x64, 0x12, 0x39, 0x0a, 0x19, 0x73, 0x65, 0x6d, 0x69, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, - 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x18, - 0x18, 0x20, 0x01, 0x28, 0x08, 0x52, 0x16, 0x73, 0x65, 0x6d, 0x69, 0x53, 0x79, 0x6e, 0x63, 0x4d, - 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x2a, 0x3b, 0x0a, - 0x13, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x12, 0x0a, 0x0e, 0x49, 0x4f, 0x41, 0x4e, 0x44, 0x53, 0x51, 0x4c, - 0x54, 0x48, 0x52, 0x45, 0x41, 0x44, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x49, 0x4f, 0x54, 0x48, - 0x52, 0x45, 0x41, 0x44, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0x01, 0x42, 0x2e, 0x5a, 0x2c, 0x76, 0x69, - 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, - 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x72, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, + 0x65, 0x64, 0x12, 0x2a, 0x0a, 0x11, 0x73, 0x65, 0x6d, 0x69, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x18, 0x18, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x73, + 0x65, 0x6d, 0x69, 0x53, 0x79, 0x6e, 0x63, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x2a, 0x3b, + 0x0a, 0x13, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x12, 0x0a, 0x0e, 0x49, 0x4f, 0x41, 0x4e, 0x44, 0x53, 0x51, + 0x4c, 0x54, 0x48, 0x52, 0x45, 0x41, 0x44, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x49, 0x4f, 0x54, + 0x48, 0x52, 0x45, 0x41, 0x44, 0x4f, 0x4e, 0x4c, 0x59, 0x10, 0x01, 0x42, 0x2e, 0x5a, 0x2c, 0x76, + 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, + 0x67, 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x72, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, }) var ( diff --git a/go/vt/proto/replicationdata/replicationdata_vtproto.pb.go b/go/vt/proto/replicationdata/replicationdata_vtproto.pb.go index b7ae5481a33..d4e2af4496e 100644 --- a/go/vt/proto/replicationdata/replicationdata_vtproto.pb.go +++ b/go/vt/proto/replicationdata/replicationdata_vtproto.pb.go @@ -143,7 +143,7 @@ func (m *FullStatus) CloneVT() *FullStatus { r.SuperReadOnly = m.SuperReadOnly r.ReplicationConfiguration = m.ReplicationConfiguration.CloneVT() r.DiskStalled = m.DiskStalled - r.SemiSyncMonitorBlocked = m.SemiSyncMonitorBlocked + r.SemiSyncBlocked = m.SemiSyncBlocked if len(m.unknownFields) > 0 { r.unknownFields = make([]byte, len(m.unknownFields)) copy(r.unknownFields, m.unknownFields) @@ -554,9 +554,9 @@ func (m *FullStatus) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if m.SemiSyncMonitorBlocked { + if m.SemiSyncBlocked { i-- - if m.SemiSyncMonitorBlocked { + if m.SemiSyncBlocked { dAtA[i] = 1 } else { dAtA[i] = 0 @@ -1004,7 +1004,7 @@ func (m *FullStatus) SizeVT() (n int) { if m.DiskStalled { n += 3 } - if m.SemiSyncMonitorBlocked { + if m.SemiSyncBlocked { n += 3 } n += len(m.unknownFields) @@ -2605,7 +2605,7 @@ func (m *FullStatus) UnmarshalVT(dAtA []byte) error { m.DiskStalled = bool(v != 0) case 24: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SemiSyncMonitorBlocked", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SemiSyncBlocked", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -2622,7 +2622,7 @@ func (m *FullStatus) UnmarshalVT(dAtA []byte) error { break } } - m.SemiSyncMonitorBlocked = bool(v != 0) + m.SemiSyncBlocked = bool(v != 0) default: iNdEx = preIndex skippy, err := protohelpers.Skip(dAtA[iNdEx:]) diff --git a/go/vt/sidecardb/schema/misc/semisync_recover.sql b/go/vt/sidecardb/schema/misc/semisync_heartbeat.sql similarity index 93% rename from go/vt/sidecardb/schema/misc/semisync_recover.sql rename to go/vt/sidecardb/schema/misc/semisync_heartbeat.sql index 7f6f0a55ade..01b8e5d21c2 100644 --- a/go/vt/sidecardb/schema/misc/semisync_recover.sql +++ b/go/vt/sidecardb/schema/misc/semisync_heartbeat.sql @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -CREATE TABLE IF NOT EXISTS semisync_recover +CREATE TABLE IF NOT EXISTS semisync_heartbeat ( ts BIGINT UNSIGNED NOT NULL, PRIMARY KEY (`ts`) diff --git a/go/vt/vtorc/db/generate_base.go b/go/vt/vtorc/db/generate_base.go index 03e89147c00..cdbc943690e 100644 --- a/go/vt/vtorc/db/generate_base.go +++ b/go/vt/vtorc/db/generate_base.go @@ -105,7 +105,7 @@ CREATE TABLE database_instance ( semi_sync_primary_status TINYint NOT NULL DEFAULT 0, semi_sync_replica_status TINYint NOT NULL DEFAULT 0, semi_sync_primary_clients int NOT NULL DEFAULT 0, - semi_sync_monitor_blocked tinyint NOT NULL DEFAULT 0, + semi_sync_blocked tinyint NOT NULL DEFAULT 0, is_disk_stalled TINYint NOT NULL DEFAULT 0, PRIMARY KEY (alias) )`, diff --git a/go/vt/vtorc/inst/analysis.go b/go/vt/vtorc/inst/analysis.go index 4c31389cd3c..3647134cd54 100644 --- a/go/vt/vtorc/inst/analysis.go +++ b/go/vt/vtorc/inst/analysis.go @@ -116,7 +116,7 @@ type ReplicationAnalysis struct { SemiSyncPrimaryWaitForReplicaCount uint SemiSyncPrimaryClients uint SemiSyncReplicaEnabled bool - SemiSyncMonitorBlocked bool + SemiSyncBlocked bool CountSemiSyncReplicasEnabled uint CountLoggingReplicas uint CountStatementBasedLoggingReplicas uint diff --git a/go/vt/vtorc/inst/analysis_dao.go b/go/vt/vtorc/inst/analysis_dao.go index a4eee7c8bc5..685d84b8a00 100644 --- a/go/vt/vtorc/inst/analysis_dao.go +++ b/go/vt/vtorc/inst/analysis_dao.go @@ -153,8 +153,8 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna primary_instance.semi_sync_primary_status ) AS semi_sync_primary_status, MIN( - primary_instance.semi_sync_monitor_blocked - ) AS semi_sync_monitor_blocked, + primary_instance.semi_sync_blocked + ) AS semi_sync_blocked, MIN( primary_instance.semi_sync_replica_enabled ) AS semi_sync_replica_enabled, @@ -336,7 +336,7 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna a.BinlogServerImmediateTopology = countValidBinlogServerReplicas == a.CountValidReplicas && a.CountValidReplicas > 0 a.SemiSyncPrimaryEnabled = m.GetBool("semi_sync_primary_enabled") a.SemiSyncPrimaryStatus = m.GetBool("semi_sync_primary_status") - a.SemiSyncMonitorBlocked = m.GetBool("semi_sync_monitor_blocked") + a.SemiSyncBlocked = m.GetBool("semi_sync_blocked") a.SemiSyncReplicaEnabled = m.GetBool("semi_sync_replica_enabled") a.CountSemiSyncReplicasEnabled = m.GetUint("count_semi_sync_replicas") // countValidSemiSyncReplicasEnabled := m.GetUint("count_valid_semi_sync_replicas") @@ -462,7 +462,7 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna a.Analysis = PrimaryTabletDeleted a.Description = "Primary tablet has been deleted" ca.hasClusterwideAction = true - } else if a.IsPrimary && a.SemiSyncMonitorBlocked && a.CountSemiSyncReplicasEnabled >= a.SemiSyncPrimaryWaitForReplicaCount { + } else if a.IsPrimary && a.SemiSyncBlocked && a.CountSemiSyncReplicasEnabled >= a.SemiSyncPrimaryWaitForReplicaCount { // The primary is reporting that semi-sync monitor is blocked on writes. // There are enough replicas configured to send semi-sync ACKs such that the primary shouldn't be blocked. // There is some network diruption in progress. We should run an ERS. diff --git a/go/vt/vtorc/inst/analysis_dao_test.go b/go/vt/vtorc/inst/analysis_dao_test.go index 76a86df43cc..a58414af3e8 100644 --- a/go/vt/vtorc/inst/analysis_dao_test.go +++ b/go/vt/vtorc/inst/analysis_dao_test.go @@ -142,7 +142,7 @@ func TestGetReplicationAnalysisDecision(t *testing.T) { SemiSyncPrimaryWaitForReplicaCount: 2, CountSemiSyncReplicasEnabled: 2, SemiSyncPrimaryClients: 0, - SemiSyncMonitorBlocked: 1, + SemiSyncBlocked: 1, }}, keyspaceWanted: "ks", shardWanted: "0", @@ -170,7 +170,7 @@ func TestGetReplicationAnalysisDecision(t *testing.T) { SemiSyncPrimaryWaitForReplicaCount: 2, CountSemiSyncReplicasEnabled: 1, SemiSyncPrimaryClients: 1, - SemiSyncMonitorBlocked: 1, + SemiSyncBlocked: 1, }}, keyspaceWanted: "ks", shardWanted: "0", diff --git a/go/vt/vtorc/inst/instance.go b/go/vt/vtorc/inst/instance.go index ba4a619d831..f8329e62288 100644 --- a/go/vt/vtorc/inst/instance.go +++ b/go/vt/vtorc/inst/instance.go @@ -85,7 +85,7 @@ type Instance struct { SemiSyncPrimaryStatus bool SemiSyncPrimaryClients uint SemiSyncReplicaStatus bool - SemiSyncMonitorBlocked bool + SemiSyncBlocked bool LastSeenTimestamp string IsLastCheckValid bool diff --git a/go/vt/vtorc/inst/instance_dao.go b/go/vt/vtorc/inst/instance_dao.go index e150a119d6d..6da7a7298e8 100644 --- a/go/vt/vtorc/inst/instance_dao.go +++ b/go/vt/vtorc/inst/instance_dao.go @@ -240,7 +240,7 @@ func ReadTopologyInstanceBufferable(tabletAlias string, latency *stopwatch.Named instance.SemiSyncPrimaryClients = uint(fs.SemiSyncPrimaryClients) instance.SemiSyncPrimaryStatus = fs.SemiSyncPrimaryStatus instance.SemiSyncReplicaStatus = fs.SemiSyncReplicaStatus - instance.SemiSyncMonitorBlocked = fs.SemiSyncMonitorBlocked + instance.SemiSyncBlocked = fs.SemiSyncBlocked if instance.IsOracleMySQL() || instance.IsPercona() { // Stuff only supported on Oracle / Percona MySQL @@ -580,7 +580,7 @@ func readInstanceRow(m sqlutils.RowMap) *Instance { instance.SemiSyncPrimaryStatus = m.GetBool("semi_sync_primary_status") instance.SemiSyncPrimaryClients = m.GetUint("semi_sync_primary_clients") instance.SemiSyncReplicaStatus = m.GetBool("semi_sync_replica_status") - instance.SemiSyncMonitorBlocked = m.GetBool("semi_sync_monitor_blocked") + instance.SemiSyncBlocked = m.GetBool("semi_sync_blocked") instance.ReplicationDepth = m.GetUint("replication_depth") instance.IsCoPrimary = m.GetBool("is_co_primary") instance.HasReplicationCredentials = m.GetBool("has_replication_credentials") @@ -881,7 +881,7 @@ func mkInsertForInstances(instances []*Instance, instanceWasActuallyFound bool, "semi_sync_primary_status", "semi_sync_primary_clients", "semi_sync_replica_status", - "semi_sync_monitor_blocked", + "semi_sync_blocked", "last_discovery_latency", "is_disk_stalled", } @@ -962,7 +962,7 @@ func mkInsertForInstances(instances []*Instance, instanceWasActuallyFound bool, args = append(args, instance.SemiSyncPrimaryStatus) args = append(args, instance.SemiSyncPrimaryClients) args = append(args, instance.SemiSyncReplicaStatus) - args = append(args, instance.SemiSyncMonitorBlocked) + args = append(args, instance.SemiSyncBlocked) args = append(args, instance.LastDiscoveryLatency.Nanoseconds()) args = append(args, instance.StalledDisk) } diff --git a/go/vt/vtorc/inst/instance_dao_test.go b/go/vt/vtorc/inst/instance_dao_test.go index 8276befe267..e518d563739 100644 --- a/go/vt/vtorc/inst/instance_dao_test.go +++ b/go/vt/vtorc/inst/instance_dao_test.go @@ -64,7 +64,7 @@ func TestMkInsertSingle(t *testing.T) { version, major_version, version_comment, binlog_server, read_only, binlog_format, binlog_row_image, log_bin, log_replica_updates, binary_log_file, binary_log_pos, source_host, source_port, replica_net_timeout, heartbeat_interval, replica_sql_running, replica_io_running, replication_sql_thread_state, replication_io_thread_state, has_replication_filters, supports_oracle_gtid, oracle_gtid, source_uuid, ancestry_uuid, executed_gtid_set, gtid_mode, gtid_purged, gtid_errant, - source_log_file, read_source_log_pos, relay_source_log_file, exec_source_log_pos, relay_log_file, relay_log_pos, last_sql_error, last_io_error, replication_lag_seconds, replica_lag_seconds, sql_delay, data_center, region, physical_environment, replication_depth, is_co_primary, has_replication_credentials, allow_tls, semi_sync_enforced, semi_sync_primary_enabled, semi_sync_primary_timeout, semi_sync_primary_wait_for_replica_count, semi_sync_replica_enabled, semi_sync_primary_status, semi_sync_primary_clients, semi_sync_replica_status, semi_sync_monitor_blocked, last_discovery_latency, is_disk_stalled, last_seen) + source_log_file, read_source_log_pos, relay_source_log_file, exec_source_log_pos, relay_log_file, relay_log_pos, last_sql_error, last_io_error, replication_lag_seconds, replica_lag_seconds, sql_delay, data_center, region, physical_environment, replication_depth, is_co_primary, has_replication_credentials, allow_tls, semi_sync_enforced, semi_sync_primary_enabled, semi_sync_primary_timeout, semi_sync_primary_wait_for_replica_count, semi_sync_replica_enabled, semi_sync_primary_status, semi_sync_primary_clients, semi_sync_replica_status, semi_sync_blocked, last_discovery_latency, is_disk_stalled, last_seen) VALUES (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')) ` @@ -87,7 +87,7 @@ func TestMkInsertThree(t *testing.T) { version, major_version, version_comment, binlog_server, read_only, binlog_format, binlog_row_image, log_bin, log_replica_updates, binary_log_file, binary_log_pos, source_host, source_port, replica_net_timeout, heartbeat_interval, replica_sql_running, replica_io_running, replication_sql_thread_state, replication_io_thread_state, has_replication_filters, supports_oracle_gtid, oracle_gtid, source_uuid, ancestry_uuid, executed_gtid_set, gtid_mode, gtid_purged, gtid_errant, - source_log_file, read_source_log_pos, relay_source_log_file, exec_source_log_pos, relay_log_file, relay_log_pos, last_sql_error, last_io_error, replication_lag_seconds, replica_lag_seconds, sql_delay, data_center, region, physical_environment, replication_depth, is_co_primary, has_replication_credentials, allow_tls, semi_sync_enforced, semi_sync_primary_enabled, semi_sync_primary_timeout, semi_sync_primary_wait_for_replica_count, semi_sync_replica_enabled, semi_sync_primary_status, semi_sync_primary_clients, semi_sync_replica_status, semi_sync_monitor_blocked, last_discovery_latency, is_disk_stalled, last_seen) + source_log_file, read_source_log_pos, relay_source_log_file, exec_source_log_pos, relay_log_file, relay_log_pos, last_sql_error, last_io_error, replication_lag_seconds, replica_lag_seconds, sql_delay, data_center, region, physical_environment, replication_depth, is_co_primary, has_replication_credentials, allow_tls, semi_sync_enforced, semi_sync_primary_enabled, semi_sync_primary_timeout, semi_sync_primary_wait_for_replica_count, semi_sync_replica_enabled, semi_sync_primary_status, semi_sync_primary_clients, semi_sync_replica_status, semi_sync_blocked, last_discovery_latency, is_disk_stalled, last_seen) VALUES (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')), (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')), diff --git a/go/vt/vtorc/test/recovery_analysis.go b/go/vt/vtorc/test/recovery_analysis.go index 98493f629a0..eaf0dac2258 100644 --- a/go/vt/vtorc/test/recovery_analysis.go +++ b/go/vt/vtorc/test/recovery_analysis.go @@ -65,7 +65,7 @@ type InfoForRecoveryAnalysis struct { CountValidBinlogServerReplicas uint SemiSyncPrimaryEnabled int SemiSyncPrimaryStatus int - SemiSyncMonitorBlocked int + SemiSyncBlocked int SemiSyncPrimaryWaitForReplicaCount uint SemiSyncPrimaryClients uint SemiSyncReplicaEnabled int @@ -143,7 +143,7 @@ func (info *InfoForRecoveryAnalysis) ConvertToRowMap() sqlutils.RowMap { rowMap["semi_sync_primary_clients"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncPrimaryClients), Valid: true} rowMap["semi_sync_primary_enabled"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncPrimaryEnabled), Valid: true} rowMap["semi_sync_primary_status"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncPrimaryStatus), Valid: true} - rowMap["semi_sync_monitor_blocked"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncMonitorBlocked), Valid: true} + rowMap["semi_sync_blocked"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncBlocked), Valid: true} rowMap["semi_sync_primary_wait_for_replica_count"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncPrimaryWaitForReplicaCount), Valid: true} rowMap["semi_sync_replica_enabled"] = sqlutils.CellData{String: fmt.Sprintf("%v", info.SemiSyncReplicaEnabled), Valid: true} res, _ := prototext.Marshal(info.TabletInfo) diff --git a/go/vt/vttablet/tabletmanager/rpc_replication.go b/go/vt/vttablet/tabletmanager/rpc_replication.go index 9ed97490191..787d0b0bdcb 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication.go @@ -187,7 +187,7 @@ func (tm *TabletManager) FullStatus(ctx context.Context) (*replicationdatapb.Ful SemiSyncPrimaryClients: semiSyncClients, SemiSyncPrimaryTimeout: semiSyncTimeout, SemiSyncWaitForReplicaCount: semiSyncNumReplicas, - SemiSyncMonitorBlocked: tm.SemiSyncMonitor.AllWritesBlocked(), + SemiSyncBlocked: tm.SemiSyncMonitor.AllWritesBlocked(), SuperReadOnly: superReadOnly, ReplicationConfiguration: replConfiguration, }, nil diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index 8a3eb4c1d25..3e8af58b228 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -37,8 +37,8 @@ import ( const ( semiSyncWaitSessionsRead = "select variable_value from performance_schema.global_status where regexp_like(variable_name, 'Rpl_semi_sync_(source|master)_wait_sessions')" - semiSyncRecoverWrite = "INSERT INTO %s.semisync_recover (ts) VALUES (NOW())" - semiSyncRecoverClear = "TRUNCATE TABLE %s.semisync_recover" + semiSyncHeartbeatWrite = "INSERT INTO %s.semisync_heartbeat (ts) VALUES (NOW())" + semiSyncHeartbeatClear = "TRUNCATE TABLE %s.semisync_heartbeat" maxWritesPermitted = 15 clearTimerDuration = 24 * time.Hour ) @@ -62,7 +62,7 @@ type Monitor struct { // if the primary is blocked on semi-sync ACKs or not. ticks *timer.Timer // clearTicks is the ticker to clear the data in - // the semisync_recover table. + // the semisync_heartbeat table. clearTicks *timer.Timer // mu protects the fields below. @@ -335,13 +335,13 @@ func (m *Monitor) write() { defer cancel() conn, err := m.appPool.Get(ctx) if err != nil { - log.Errorf("SemiSync Monitor: failed to get a connection when writing to semisync_recovery table: %v", err) + log.Errorf("SemiSync Monitor: failed to get a connection when writing to semisync_heartbeat table: %v", err) return } defer conn.Recycle() - _, err = conn.Conn.ExecuteFetch(m.bindSideCarDBName(semiSyncRecoverWrite), 0, false) + _, err = conn.Conn.ExecuteFetch(m.bindSideCarDBName(semiSyncHeartbeatWrite), 0, false) if err != nil { - log.Errorf("SemiSync Monitor: failed to write to semisync_recovery table: %v", err) + log.Errorf("SemiSync Monitor: failed to write to semisync_heartbeat table: %v", err) } else { // One of the writes went through without an error. // This means that we aren't blocked on semi-sync anymore. @@ -370,13 +370,13 @@ func (m *Monitor) clearAllData() { // Get a connection from the pool conn, err := m.appPool.Get(context.Background()) if err != nil { - log.Errorf("SemiSync Monitor: failed get a connection to clear semisync_recovery table: %v", err) + log.Errorf("SemiSync Monitor: failed get a connection to clear semisync_heartbeat table: %v", err) return } defer conn.Recycle() - _, err = conn.Conn.ExecuteFetch(m.bindSideCarDBName(semiSyncRecoverClear), 0, false) + _, err = conn.Conn.ExecuteFetch(m.bindSideCarDBName(semiSyncHeartbeatClear), 0, false) if err != nil { - log.Errorf("SemiSync Monitor: failed to clear semisync_recovery table: %v", err) + log.Errorf("SemiSync Monitor: failed to clear semisync_heartbeat table: %v", err) } } diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go index 358d3e4f782..97716569563 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor_test.go @@ -115,12 +115,12 @@ func TestMonitorBindSideCarDBName(t *testing.T) { expected string }{ { - query: semiSyncRecoverWrite, - expected: "INSERT INTO _vt.semisync_recover (ts) VALUES (NOW())", + query: semiSyncHeartbeatWrite, + expected: "INSERT INTO _vt.semisync_heartbeat (ts) VALUES (NOW())", }, { - query: semiSyncRecoverClear, - expected: "TRUNCATE TABLE _vt.semisync_recover", + query: semiSyncHeartbeatClear, + expected: "TRUNCATE TABLE _vt.semisync_heartbeat", }, } for _, tt := range tests { @@ -138,7 +138,7 @@ func TestMonitorClearAllData(t *testing.T) { db.SetNeverFail(true) m.clearAllData() ql := db.QueryLog() - require.EqualValues(t, "truncate table _vt.semisync_recover", ql) + require.EqualValues(t, "truncate table _vt.semisync_heartbeat", ql) } // TestMonitorWaitMechanism tests that the wait mechanism works as intended. @@ -285,16 +285,16 @@ func TestMonitorWrite(t *testing.T) { }{ { initVal: maxWritesPermitted - 2, - queryLog: "insert into _vt.semisync_recover (ts) values (now())", + queryLog: "insert into _vt.semisync_heartbeat (ts) values (now())", }, { initVal: maxWritesPermitted - 1, - queryLog: "insert into _vt.semisync_recover (ts) values (now())", + queryLog: "insert into _vt.semisync_heartbeat (ts) values (now())", }, { initVal: maxWritesPermitted, queryLog: "", }, { initVal: 0, - queryLog: "insert into _vt.semisync_recover (ts) values (now())", + queryLog: "insert into _vt.semisync_heartbeat (ts) values (now())", }, } for _, tt := range tests { diff --git a/proto/replicationdata.proto b/proto/replicationdata.proto index 1cc90a91f1b..0c9ad87e210 100644 --- a/proto/replicationdata.proto +++ b/proto/replicationdata.proto @@ -106,5 +106,5 @@ message FullStatus { bool super_read_only = 21; replicationdata.Configuration replication_configuration = 22; bool disk_stalled = 23; - bool semi_sync_monitor_blocked = 24; + bool semi_sync_blocked = 24; } diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts index 9cd09e771a1..55fb86d17e0 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -48802,8 +48802,8 @@ export namespace replicationdata { /** FullStatus disk_stalled */ disk_stalled?: (boolean|null); - /** FullStatus semi_sync_monitor_blocked */ - semi_sync_monitor_blocked?: (boolean|null); + /** FullStatus semi_sync_blocked */ + semi_sync_blocked?: (boolean|null); } /** Represents a FullStatus. */ @@ -48884,8 +48884,8 @@ export namespace replicationdata { /** FullStatus disk_stalled. */ public disk_stalled: boolean; - /** FullStatus semi_sync_monitor_blocked. */ - public semi_sync_monitor_blocked: boolean; + /** FullStatus semi_sync_blocked. */ + public semi_sync_blocked: boolean; /** * Creates a new FullStatus instance using the specified properties. diff --git a/web/vtadmin/src/proto/vtadmin.js b/web/vtadmin/src/proto/vtadmin.js index 9a73f5640f6..4cfc709fa9b 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -118503,7 +118503,7 @@ export const replicationdata = $root.replicationdata = (() => { * @property {boolean|null} [super_read_only] FullStatus super_read_only * @property {replicationdata.IConfiguration|null} [replication_configuration] FullStatus replication_configuration * @property {boolean|null} [disk_stalled] FullStatus disk_stalled - * @property {boolean|null} [semi_sync_monitor_blocked] FullStatus semi_sync_monitor_blocked + * @property {boolean|null} [semi_sync_blocked] FullStatus semi_sync_blocked */ /** @@ -118706,12 +118706,12 @@ export const replicationdata = $root.replicationdata = (() => { FullStatus.prototype.disk_stalled = false; /** - * FullStatus semi_sync_monitor_blocked. - * @member {boolean} semi_sync_monitor_blocked + * FullStatus semi_sync_blocked. + * @member {boolean} semi_sync_blocked * @memberof replicationdata.FullStatus * @instance */ - FullStatus.prototype.semi_sync_monitor_blocked = false; + FullStatus.prototype.semi_sync_blocked = false; /** * Creates a new FullStatus instance using the specified properties. @@ -118783,8 +118783,8 @@ export const replicationdata = $root.replicationdata = (() => { $root.replicationdata.Configuration.encode(message.replication_configuration, writer.uint32(/* id 22, wireType 2 =*/178).fork()).ldelim(); if (message.disk_stalled != null && Object.hasOwnProperty.call(message, "disk_stalled")) writer.uint32(/* id 23, wireType 0 =*/184).bool(message.disk_stalled); - if (message.semi_sync_monitor_blocked != null && Object.hasOwnProperty.call(message, "semi_sync_monitor_blocked")) - writer.uint32(/* id 24, wireType 0 =*/192).bool(message.semi_sync_monitor_blocked); + if (message.semi_sync_blocked != null && Object.hasOwnProperty.call(message, "semi_sync_blocked")) + writer.uint32(/* id 24, wireType 0 =*/192).bool(message.semi_sync_blocked); return writer; }; @@ -118912,7 +118912,7 @@ export const replicationdata = $root.replicationdata = (() => { break; } case 24: { - message.semi_sync_monitor_blocked = reader.bool(); + message.semi_sync_blocked = reader.bool(); break; } default: @@ -119025,9 +119025,9 @@ export const replicationdata = $root.replicationdata = (() => { if (message.disk_stalled != null && message.hasOwnProperty("disk_stalled")) if (typeof message.disk_stalled !== "boolean") return "disk_stalled: boolean expected"; - if (message.semi_sync_monitor_blocked != null && message.hasOwnProperty("semi_sync_monitor_blocked")) - if (typeof message.semi_sync_monitor_blocked !== "boolean") - return "semi_sync_monitor_blocked: boolean expected"; + if (message.semi_sync_blocked != null && message.hasOwnProperty("semi_sync_blocked")) + if (typeof message.semi_sync_blocked !== "boolean") + return "semi_sync_blocked: boolean expected"; return null; }; @@ -119105,8 +119105,8 @@ export const replicationdata = $root.replicationdata = (() => { } if (object.disk_stalled != null) message.disk_stalled = Boolean(object.disk_stalled); - if (object.semi_sync_monitor_blocked != null) - message.semi_sync_monitor_blocked = Boolean(object.semi_sync_monitor_blocked); + if (object.semi_sync_blocked != null) + message.semi_sync_blocked = Boolean(object.semi_sync_blocked); return message; }; @@ -119151,7 +119151,7 @@ export const replicationdata = $root.replicationdata = (() => { object.super_read_only = false; object.replication_configuration = null; object.disk_stalled = false; - object.semi_sync_monitor_blocked = false; + object.semi_sync_blocked = false; } if (message.server_id != null && message.hasOwnProperty("server_id")) object.server_id = message.server_id; @@ -119202,8 +119202,8 @@ export const replicationdata = $root.replicationdata = (() => { object.replication_configuration = $root.replicationdata.Configuration.toObject(message.replication_configuration, options); if (message.disk_stalled != null && message.hasOwnProperty("disk_stalled")) object.disk_stalled = message.disk_stalled; - if (message.semi_sync_monitor_blocked != null && message.hasOwnProperty("semi_sync_monitor_blocked")) - object.semi_sync_monitor_blocked = message.semi_sync_monitor_blocked; + if (message.semi_sync_blocked != null && message.hasOwnProperty("semi_sync_blocked")) + object.semi_sync_blocked = message.semi_sync_blocked; return object; }; From ee098c600e05b6be399bf888afb635b13ffcdad5 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Fri, 21 Feb 2025 15:12:10 +0530 Subject: [PATCH 26/27] feat: add a new counter for tracking number of errors Signed-off-by: Manan Gupta --- .../vttablet/tabletmanager/semisyncmonitor/monitor.go | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index 3e8af58b228..41b3124ca79 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -83,6 +83,9 @@ type Monitor struct { waiters []chan struct{} // writesBlockedGauge is a gauge tracking the number of writes the monitor is blocked on. writesBlockedGauge *stats.Gauge + // errorCount is the number of errors that the semi-sync monitor ran into. + // We ignore some of the errors, so the counter is a good way to track how many errors we have seen. + errorCount *stats.Counter } // NewMonitor creates a new Monitor. @@ -94,7 +97,8 @@ func NewMonitor(config *tabletenv.TabletConfig, exporter *servenv.Exporter) *Mon // but this seams fine for now. clearTicks: timer.NewTimer(clearTimerDuration), writesBlockedGauge: exporter.NewGauge("SemiSyncMonitorWritesBlocked", "Number of writes blocked in the semi-sync monitor"), - appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", exporter, 20, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), + errorCount: exporter.NewCounter("SemiSyncMonitorErrorCount", "Number of errors encountered by the semi-sync monitor"), + appPool: dbconnpool.NewConnectionPool("SemiSyncMonitorAppPool", exporter, maxWritesPermitted+5, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution), waiters: make([]chan struct{}, 0), } } @@ -162,6 +166,7 @@ func (m *Monitor) checkAndFixSemiSyncBlocked() { defer cancel() isBlocked, err := m.isSemiSyncBlocked(ctx) if err != nil { + m.errorCount.Add(1) // If we are unable to determine whether the primary is blocked or not, // then we can just abort the function and try again later. log.Errorf("SemiSync Monitor: failed to check if primary is blocked on semi-sync: %v", err) @@ -335,12 +340,14 @@ func (m *Monitor) write() { defer cancel() conn, err := m.appPool.Get(ctx) if err != nil { + m.errorCount.Add(1) log.Errorf("SemiSync Monitor: failed to get a connection when writing to semisync_heartbeat table: %v", err) return } defer conn.Recycle() _, err = conn.Conn.ExecuteFetch(m.bindSideCarDBName(semiSyncHeartbeatWrite), 0, false) if err != nil { + m.errorCount.Add(1) log.Errorf("SemiSync Monitor: failed to write to semisync_heartbeat table: %v", err) } else { // One of the writes went through without an error. @@ -370,12 +377,14 @@ func (m *Monitor) clearAllData() { // Get a connection from the pool conn, err := m.appPool.Get(context.Background()) if err != nil { + m.errorCount.Add(1) log.Errorf("SemiSync Monitor: failed get a connection to clear semisync_heartbeat table: %v", err) return } defer conn.Recycle() _, err = conn.Conn.ExecuteFetch(m.bindSideCarDBName(semiSyncHeartbeatClear), 0, false) if err != nil { + m.errorCount.Add(1) log.Errorf("SemiSync Monitor: failed to clear semisync_heartbeat table: %v", err) } } From 437340722dff14b75b68dbbf8733eb21d2e64999 Mon Sep 17 00:00:00 2001 From: Manan Gupta Date: Mon, 24 Feb 2025 11:42:19 +0530 Subject: [PATCH 27/27] feat: use remote operation timeout constant instead of direct 15 seconds value Signed-off-by: Manan Gupta --- go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go index 41b3124ca79..0ea37e29284 100644 --- a/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go +++ b/go/vt/vttablet/tabletmanager/semisyncmonitor/monitor.go @@ -32,6 +32,7 @@ import ( "vitess.io/vitess/go/vt/mysqlctl" "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" ) @@ -162,7 +163,7 @@ func (m *Monitor) Close() { // be called multiple times in parallel. func (m *Monitor) checkAndFixSemiSyncBlocked() { // Check if semi-sync is blocked or not - ctx, cancel := context.WithTimeout(context.Background(), 15*time.Second) + ctx, cancel := context.WithTimeout(context.Background(), topo.RemoteOperationTimeout) defer cancel() isBlocked, err := m.isSemiSyncBlocked(ctx) if err != nil { @@ -336,7 +337,7 @@ func (m *Monitor) write() { } defer m.decrementWriteCount() // Get a connection from the pool - ctx, cancel := context.WithTimeout(context.Background(), 15*time.Second) + ctx, cancel := context.WithTimeout(context.Background(), topo.RemoteOperationTimeout) defer cancel() conn, err := m.appPool.Get(ctx) if err != nil {