From ee90f1c88667714cc878474ced9675590a41e97a Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Sun, 22 Jun 2025 20:02:33 +0200 Subject: [PATCH 01/16] `vtorc`: use `*topoprotopb.TabletAlias` for tablet alias Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/discovery/queue.go | 49 ++++--- go/vt/vtorc/discovery/queue_test.go | 33 +++-- go/vt/vtorc/inst/analysis.go | 4 +- go/vt/vtorc/inst/analysis_dao.go | 29 ++-- go/vt/vtorc/inst/analysis_dao_test.go | 36 ++--- go/vt/vtorc/inst/audit_dao.go | 13 +- go/vt/vtorc/inst/audit_dao_test.go | 45 +++--- go/vt/vtorc/inst/instance.go | 2 +- go/vt/vtorc/inst/instance_dao.go | 129 ++++++++++-------- go/vt/vtorc/inst/instance_dao_test.go | 106 +++++++------- go/vt/vtorc/inst/instance_test.go | 8 +- go/vt/vtorc/inst/shard_dao.go | 26 ++-- go/vt/vtorc/inst/shard_dao_test.go | 10 +- go/vt/vtorc/inst/tablet_dao.go | 8 +- go/vt/vtorc/inst/tablet_dao_test.go | 14 +- .../logic/keyspace_shard_discovery_test.go | 22 +-- go/vt/vtorc/logic/tablet_discovery.go | 43 +++--- go/vt/vtorc/logic/tablet_discovery_test.go | 33 +++-- go/vt/vtorc/logic/topology_recovery.go | 54 ++++---- go/vt/vtorc/logic/topology_recovery_dao.go | 24 +++- .../vtorc/logic/topology_recovery_dao_test.go | 10 +- go/vt/vtorc/logic/topology_recovery_test.go | 21 ++- go/vt/vtorc/logic/vtorc.go | 51 ++++--- 23 files changed, 428 insertions(+), 342 deletions(-) diff --git a/go/vt/vtorc/discovery/queue.go b/go/vt/vtorc/discovery/queue.go index c56827b5899..57f2608d755 100644 --- a/go/vt/vtorc/discovery/queue.go +++ b/go/vt/vtorc/discovery/queue.go @@ -30,39 +30,41 @@ import ( "time" "vitess.io/vitess/go/vt/log" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtorc/config" ) // queueItem represents an item in the discovery.Queue. type queueItem struct { - Key string - PushedAt time.Time + TabletAlias *topodatapb.TabletAlias + PushedAt time.Time } // Queue is an ordered queue with deduplication. type Queue struct { mu sync.Mutex - enqueued map[string]struct{} + enqueued map[*topodatapb.TabletAlias]struct{} queue chan queueItem } // NewQueue creates a new queue. func NewQueue() *Queue { return &Queue{ - enqueued: make(map[string]struct{}), + enqueued: make(map[*topodatapb.TabletAlias]struct{}), queue: make(chan queueItem, config.DiscoveryQueueCapacity), } } -// setKeyCheckEnqueued returns true if a key is already enqueued, if -// not the key will be marked as enqueued and false is returned. -func (q *Queue) setKeyCheckEnqueued(key string) (alreadyEnqueued bool) { +// checkAndSetEnqueued returns true if a tablet alias is already enqueued, if +// not the tablet alias will be marked as enqueued and false is returned. +func (q *Queue) checkAndSetEnqueued(tabletAlias *topodatapb.TabletAlias) (alreadyEnqueued bool) { q.mu.Lock() defer q.mu.Unlock() - _, alreadyEnqueued = q.enqueued[key] + _, alreadyEnqueued = q.enqueued[tabletAlias] if !alreadyEnqueued { - q.enqueued[key] = struct{}{} + q.enqueued[tabletAlias] = struct{}{} } return alreadyEnqueued } @@ -75,36 +77,39 @@ func (q *Queue) QueueLen() int { return len(q.enqueued) } -// Push enqueues a key if it is not on a queue and is not being +// Push enqueues a tablet alias if it is not on a queue and is not being // processed; silently returns otherwise. -func (q *Queue) Push(key string) { - if q.setKeyCheckEnqueued(key) { +func (q *Queue) Push(tabletAlias *topodatapb.TabletAlias) { + if q.checkAndSetEnqueued(tabletAlias) || tabletAlias == nil { return } q.queue <- queueItem{ - Key: key, - PushedAt: time.Now(), + TabletAlias: tabletAlias, + PushedAt: time.Now(), } } -// Consume fetches a key to process; blocks if queue is empty. +// Consume fetches a tablet alias to process; blocks if queue is empty. // Release must be called once after Consume. -func (q *Queue) Consume() string { +func (q *Queue) Consume() *topodatapb.TabletAlias { item := <-q.queue timeOnQueue := time.Since(item.PushedAt) if timeOnQueue > config.GetInstancePollTime() { - log.Warningf("key %v spent %.4fs waiting on a discovery queue", item.Key, timeOnQueue.Seconds()) + log.Warningf("tablet %v spent %.4fs waiting on a discovery queue", + topoproto.TabletAliasString(item.TabletAlias), + timeOnQueue.Seconds(), + ) } - return item.Key + return item.TabletAlias } -// Release removes a key from a list of being processed keys -// which allows that key to be pushed into the queue again. -func (q *Queue) Release(key string) { +// Release removes a tablet alias from a list of being processed aliases +// which allows that tablet to be pushed into the queue again. +func (q *Queue) Release(tabletAlias *topodatapb.TabletAlias) { q.mu.Lock() defer q.mu.Unlock() - delete(q.enqueued, key) + delete(q.enqueued, tabletAlias) } diff --git a/go/vt/vtorc/discovery/queue_test.go b/go/vt/vtorc/discovery/queue_test.go index fa3e8c16c59..5ebb6bc3e25 100644 --- a/go/vt/vtorc/discovery/queue_test.go +++ b/go/vt/vtorc/discovery/queue_test.go @@ -17,44 +17,50 @@ limitations under the License. package discovery import ( - "strconv" "testing" "github.com/stretchr/testify/require" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) func TestQueue(t *testing.T) { q := NewQueue() require.Zero(t, q.QueueLen()) + tabletAlias := &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 1, + } + // Push - q.Push(t.Name()) + q.Push(tabletAlias) require.Equal(t, 1, q.QueueLen()) - _, found := q.enqueued[t.Name()] + _, found := q.enqueued[tabletAlias] require.True(t, found) // Push duplicate - q.Push(t.Name()) + q.Push(tabletAlias) require.Equal(t, 1, q.QueueLen()) // Consume - require.Equal(t, t.Name(), q.Consume()) + require.Equal(t, tabletAlias, q.Consume()) require.Equal(t, 1, q.QueueLen()) - _, found = q.enqueued[t.Name()] + _, found = q.enqueued[tabletAlias] require.True(t, found) // Release - q.Release(t.Name()) + q.Release(tabletAlias) require.Zero(t, q.QueueLen()) - _, found = q.enqueued[t.Name()] + _, found = q.enqueued[tabletAlias] require.False(t, found) } type testQueue interface { QueueLen() int - Push(string) - Consume() string - Release(string) + Push(*topodatapb.TabletAlias) + Consume() *topodatapb.TabletAlias + Release(*topodatapb.TabletAlias) } func BenchmarkQueues(b *testing.B) { @@ -69,7 +75,10 @@ func BenchmarkQueues(b *testing.B) { b.Run(test.name, func(b *testing.B) { for i := 0; i < b.N; i++ { for i := 0; i < 1000; i++ { - q.Push(b.Name() + strconv.Itoa(i)) + q.Push(&topodatapb.TabletAlias{ + Cell: "zone1", + Uid: uint32(i), + }) } q.QueueLen() for i := 0; i < 1000; i++ { diff --git a/go/vt/vtorc/inst/analysis.go b/go/vt/vtorc/inst/analysis.go index f6948ad89f4..dc7b47dc115 100644 --- a/go/vt/vtorc/inst/analysis.go +++ b/go/vt/vtorc/inst/analysis.go @@ -86,8 +86,8 @@ type ReplicationAnalysisHints struct { // ReplicationAnalysis notes analysis on replication chain status, per instance type ReplicationAnalysis struct { - AnalyzedInstanceAlias string - AnalyzedInstancePrimaryAlias string + AnalyzedInstanceAlias *topodatapb.TabletAlias + AnalyzedInstancePrimaryAlias *topodatapb.TabletAlias TabletType topodatapb.TabletType CurrentTabletType topodatapb.TabletType PrimaryTimeStamp time.Time diff --git a/go/vt/vtorc/inst/analysis_dao.go b/go/vt/vtorc/inst/analysis_dao.go index 0e480f74ee8..a26034b6d69 100644 --- a/go/vt/vtorc/inst/analysis_dao.go +++ b/go/vt/vtorc/inst/analysis_dao.go @@ -53,13 +53,13 @@ func initializeAnalysisDaoPostConfiguration() { type clusterAnalysis struct { hasClusterwideAction bool totalTablets int - primaryAlias string + primaryAlias *topodatapb.TabletAlias durability policy.Durabler } // GetReplicationAnalysis will check for replication problems (dead primary; unreachable primary; etc) func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAnalysisHints) ([]*ReplicationAnalysis, error) { - var result []*ReplicationAnalysis + result := make([]*ReplicationAnalysis, 0) appendAnalysis := func(analysis *ReplicationAnalysis) { if analysis.Analysis == NoProblem && len(analysis.StructureAnalysis) == 0 { return @@ -315,8 +315,8 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna a.ShardPrimaryTermTimestamp = m.GetString("shard_primary_term_timestamp") a.IsPrimary = m.GetBool("is_primary") - a.AnalyzedInstanceAlias = topoproto.TabletAliasString(tablet.Alias) - a.AnalyzedInstancePrimaryAlias = topoproto.TabletAliasString(primaryTablet.Alias) + a.AnalyzedInstanceAlias = tablet.Alias + a.AnalyzedInstancePrimaryAlias = primaryTablet.Alias a.AnalyzedInstanceBinlogCoordinates = BinlogCoordinates{ LogFile: m.GetString("binary_log_file"), LogPos: m.GetUint64("binary_log_pos"), @@ -458,12 +458,12 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna } else if topo.IsReplicaType(a.TabletType) && a.ErrantGTID != "" { a.Analysis = ErrantGTIDDetected a.Description = "Tablet has errant GTIDs" - } else if topo.IsReplicaType(a.TabletType) && ca.primaryAlias == "" && a.ShardPrimaryTermTimestamp == "" { + } else if topo.IsReplicaType(a.TabletType) && ca.primaryAlias == nil && a.ShardPrimaryTermTimestamp == "" { // ClusterHasNoPrimary should only be detected when the shard record doesn't have any primary term start time specified either. a.Analysis = ClusterHasNoPrimary a.Description = "Cluster has no primary" ca.hasClusterwideAction = true - } else if topo.IsReplicaType(a.TabletType) && ca.primaryAlias == "" && a.ShardPrimaryTermTimestamp != "" { + } else if topo.IsReplicaType(a.TabletType) && ca.primaryAlias == nil && a.ShardPrimaryTermTimestamp != "" { // If there are no primary tablets, but the shard primary start time isn't empty, then we know // the primary tablet was deleted. a.Analysis = PrimaryTabletDeleted @@ -488,7 +488,7 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna a.Analysis = ReplicaMisconfigured a.Description = "Replica has been misconfigured" // - } else if topo.IsReplicaType(a.TabletType) && !a.IsPrimary && ca.primaryAlias != "" && a.AnalyzedInstancePrimaryAlias != ca.primaryAlias { + } else if topo.IsReplicaType(a.TabletType) && !a.IsPrimary && ca.primaryAlias != nil && !topoproto.TabletAliasEqual(a.AnalyzedInstancePrimaryAlias, ca.primaryAlias) { a.Analysis = ConnectedToWrongPrimary a.Description = "Connected to wrong primary" // @@ -651,12 +651,13 @@ func postProcessAnalyses(result []*ReplicationAnalysis, clusters map[string]*clu // auditInstanceAnalysisInChangelog will write down an instance's analysis in the database_instance_analysis_changelog table. // To not repeat recurring analysis code, the database_instance_last_analysis table is used, so that only changes to // analysis codes are written. -func auditInstanceAnalysisInChangelog(tabletAlias string, analysisCode AnalysisCode) error { - if lastWrittenAnalysis, found := recentInstantAnalysis.Get(tabletAlias); found { +func auditInstanceAnalysisInChangelog(tabletAlias *topodatapb.TabletAlias, analysisCode AnalysisCode) error { + tabletAliasString := topoproto.TabletAliasString(tabletAlias) + if lastWrittenAnalysis, found := recentInstantAnalysis.Get(tabletAliasString); found { if lastWrittenAnalysis == analysisCode { // Surely nothing new. // And let's expand the timeout - recentInstantAnalysis.Set(tabletAlias, analysisCode, cache.DefaultExpiration) + recentInstantAnalysis.Set(tabletAliasString, analysisCode, cache.DefaultExpiration) return nil } } @@ -672,7 +673,7 @@ func auditInstanceAnalysisInChangelog(tabletAlias string, analysisCode AnalysisC alias = ? AND analysis != ? `, - string(analysisCode), tabletAlias, string(analysisCode), + string(analysisCode), tabletAliasString, string(analysisCode), ) if err != nil { log.Error(err) @@ -701,7 +702,7 @@ func auditInstanceAnalysisInChangelog(tabletAlias string, analysisCode AnalysisC DATETIME('now'), ? )`, - tabletAlias, string(analysisCode), + tabletAliasString, string(analysisCode), ) if err != nil { log.Error(err) @@ -714,7 +715,7 @@ func auditInstanceAnalysisInChangelog(tabletAlias string, analysisCode AnalysisC } firstInsertion = rows > 0 } - recentInstantAnalysis.Set(tabletAlias, analysisCode, cache.DefaultExpiration) + recentInstantAnalysis.Set(tabletAliasString, analysisCode, cache.DefaultExpiration) // If the analysis has changed or if it is the first insertion, we need to make sure we write this change to the database. if !lastAnalysisChanged && !firstInsertion { return nil @@ -730,7 +731,7 @@ func auditInstanceAnalysisInChangelog(tabletAlias string, analysisCode AnalysisC DATETIME('now'), ? )`, - tabletAlias, string(analysisCode), + tabletAliasString, string(analysisCode), ) if err == nil { analysisChangeWriteCounter.Add(1) diff --git a/go/vt/vtorc/inst/analysis_dao_test.go b/go/vt/vtorc/inst/analysis_dao_test.go index b24e15107d4..b78c3b71912 100644 --- a/go/vt/vtorc/inst/analysis_dao_test.go +++ b/go/vt/vtorc/inst/analysis_dao_test.go @@ -1106,24 +1106,24 @@ func TestAuditInstanceAnalysisInChangelog(t *testing.T) { }() updates := []struct { - tabletAlias string + tabletAlias *topodatapb.TabletAlias analysisCode AnalysisCode writeCounterExpectation int64 wantErr string }{ { // Store a new analysis for the zone1-100 tablet. - tabletAlias: "zone1-100", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, analysisCode: ReplicationStopped, writeCounterExpectation: 1, }, { // Write the same analysis, no new write should happen. - tabletAlias: "zone1-100", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, analysisCode: ReplicationStopped, writeCounterExpectation: 1, }, { // Change the analysis. This should trigger an update. - tabletAlias: "zone1-100", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, analysisCode: ReplicaSemiSyncMustBeSet, writeCounterExpectation: 2, }, @@ -1194,38 +1194,38 @@ func TestPostProcessAnalyses(t *testing.T) { analyses: []*ReplicationAnalysis{ { Analysis: InvalidPrimary, - AnalyzedInstanceAlias: "zone1-100", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, TabletType: topodatapb.TabletType_PRIMARY, ClusterDetails: ks0, }, { Analysis: NoProblem, LastCheckValid: true, - AnalyzedInstanceAlias: "zone1-202", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 202}, TabletType: topodatapb.TabletType_RDONLY, ClusterDetails: ks80, }, { Analysis: ConnectedToWrongPrimary, LastCheckValid: true, - AnalyzedInstanceAlias: "zone1-101", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 101}, TabletType: topodatapb.TabletType_REPLICA, ReplicationStopped: true, ClusterDetails: ks0, }, { Analysis: ReplicationStopped, LastCheckValid: true, - AnalyzedInstanceAlias: "zone1-102", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 102}, TabletType: topodatapb.TabletType_RDONLY, ReplicationStopped: true, ClusterDetails: ks0, }, { Analysis: InvalidReplica, - AnalyzedInstanceAlias: "zone1-108", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 108}, TabletType: topodatapb.TabletType_REPLICA, LastCheckValid: false, ClusterDetails: ks0, }, { Analysis: NoProblem, - AnalyzedInstanceAlias: "zone1-302", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 302}, LastCheckValid: true, TabletType: topodatapb.TabletType_REPLICA, ClusterDetails: ks80, @@ -1234,19 +1234,19 @@ func TestPostProcessAnalyses(t *testing.T) { want: []*ReplicationAnalysis{ { Analysis: DeadPrimary, - AnalyzedInstanceAlias: "zone1-100", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, TabletType: topodatapb.TabletType_PRIMARY, ClusterDetails: ks0, }, { Analysis: NoProblem, LastCheckValid: true, - AnalyzedInstanceAlias: "zone1-202", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 202}, TabletType: topodatapb.TabletType_RDONLY, ClusterDetails: ks80, }, { Analysis: NoProblem, LastCheckValid: true, - AnalyzedInstanceAlias: "zone1-302", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 302}, TabletType: topodatapb.TabletType_REPLICA, ClusterDetails: ks80, }, @@ -1257,32 +1257,32 @@ func TestPostProcessAnalyses(t *testing.T) { analyses: []*ReplicationAnalysis{ { Analysis: InvalidPrimary, - AnalyzedInstanceAlias: "zone1-100", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, TabletType: topodatapb.TabletType_PRIMARY, ClusterDetails: ks0, }, { Analysis: NoProblem, - AnalyzedInstanceAlias: "zone1-202", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 202}, LastCheckValid: true, TabletType: topodatapb.TabletType_RDONLY, ClusterDetails: ks80, }, { Analysis: NoProblem, LastCheckValid: true, - AnalyzedInstanceAlias: "zone1-101", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 101}, TabletType: topodatapb.TabletType_REPLICA, ClusterDetails: ks0, }, { Analysis: ReplicationStopped, LastCheckValid: true, - AnalyzedInstanceAlias: "zone1-102", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 102}, TabletType: topodatapb.TabletType_RDONLY, ReplicationStopped: true, ClusterDetails: ks0, }, { Analysis: NoProblem, LastCheckValid: true, - AnalyzedInstanceAlias: "zone1-302", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 302}, TabletType: topodatapb.TabletType_REPLICA, ClusterDetails: ks80, }, diff --git a/go/vt/vtorc/inst/audit_dao.go b/go/vt/vtorc/inst/audit_dao.go index 7ae60fba927..ed8463799a5 100644 --- a/go/vt/vtorc/inst/audit_dao.go +++ b/go/vt/vtorc/inst/audit_dao.go @@ -23,6 +23,8 @@ import ( "vitess.io/vitess/go/stats" "vitess.io/vitess/go/vt/log" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtorc/config" "vitess.io/vitess/go/vt/vtorc/db" ) @@ -30,12 +32,13 @@ import ( var auditOperationCounter = stats.NewCounter("AuditWrite", "Number of audit operations performed") // AuditOperation creates and writes a new audit entry by given params -func AuditOperation(auditType string, tabletAlias string, message string) error { +func AuditOperation(auditType string, tabletAlias *topodatapb.TabletAlias, message string) error { keyspace := "" shard := "" - if tabletAlias != "" { + if tabletAlias != nil { keyspace, shard, _ = GetKeyspaceShardName(tabletAlias) } + tabletAliasString := topoproto.TabletAliasString(tabletAlias) auditWrittenToFile := false if config.GetAuditFileLocation() != "" { @@ -48,7 +51,7 @@ func AuditOperation(auditType string, tabletAlias string, message string) error } defer f.Close() - text := fmt.Sprintf("%s\t%s\t%s\t[%s:%s]\t%s\t\n", time.Now().Format("2006-01-02 15:04:05"), auditType, tabletAlias, keyspace, shard, message) + text := fmt.Sprintf("%s\t%s\t%s\t[%s:%s]\t%s\t\n", time.Now().Format("2006-01-02 15:04:05"), auditType, tabletAliasString, keyspace, shard, message) if _, err = f.WriteString(text); err != nil { log.Error(err) } @@ -72,7 +75,7 @@ func AuditOperation(auditType string, tabletAlias string, message string) error ? )`, auditType, - tabletAlias, + tabletAliasString, keyspace, shard, message, @@ -82,7 +85,7 @@ func AuditOperation(auditType string, tabletAlias string, message string) error return err } } - logMessage := fmt.Sprintf("auditType:%s alias:%s keyspace:%s shard:%s message:%s", auditType, tabletAlias, keyspace, shard, message) + logMessage := fmt.Sprintf("auditType:%s alias:%s keyspace:%s shard:%s message:%s", auditType, tabletAliasString, keyspace, shard, message) if syslogMessage(logMessage) { auditWrittenToFile = true } diff --git a/go/vt/vtorc/inst/audit_dao_test.go b/go/vt/vtorc/inst/audit_dao_test.go index d22e9177dc3..011a14d6d89 100644 --- a/go/vt/vtorc/inst/audit_dao_test.go +++ b/go/vt/vtorc/inst/audit_dao_test.go @@ -73,7 +73,6 @@ func TestAuditOperation(t *testing.T) { err = SaveTablet(tab100) require.NoError(t, err) - tab100Alias := topoproto.TabletAliasString(tab100.Alias) auditType := "test-audit-operation" message := "test-message" @@ -83,26 +82,26 @@ func TestAuditOperation(t *testing.T) { config.SetAuditToBackend(true) // Auditing should succeed as expected - err = AuditOperation(auditType, tab100Alias, message) + err = AuditOperation(auditType, tab100.Alias, message) require.NoError(t, err) // Check that we can read the recent audits - audits, err := readRecentAudit(tab100Alias, 0) + audits, err := readRecentAudit(tab100.Alias, 0) require.NoError(t, err) require.Len(t, audits, 1) require.EqualValues(t, 1, audits[0].AuditID) require.EqualValues(t, auditType, audits[0].AuditType) require.EqualValues(t, message, audits[0].Message) - require.EqualValues(t, tab100Alias, audits[0].AuditTabletAlias) + testRequireTabletAliasEqual(t, tab100.Alias, audits[0].AuditTabletAlias) // Check the same for no-filtering - audits, err = readRecentAudit("", 0) + audits, err = readRecentAudit(nil, 0) require.NoError(t, err) require.Len(t, audits, 1) require.EqualValues(t, 1, audits[0].AuditID) require.EqualValues(t, auditType, audits[0].AuditType) require.EqualValues(t, message, audits[0].Message) - require.EqualValues(t, tab100Alias, audits[0].AuditTabletAlias) + testRequireTabletAliasEqual(t, tab100.Alias, audits[0].AuditTabletAlias) }) t.Run("audit to File", func(t *testing.T) { @@ -114,7 +113,7 @@ func TestAuditOperation(t *testing.T) { defer os.Remove(file.Name()) config.SetAuditFileLocation(file.Name()) - err = AuditOperation(auditType, tab100Alias, message) + err = AuditOperation(auditType, tab100.Alias, message) require.NoError(t, err) // Give a little time for the write to succeed since it happens in a separate go-routine @@ -132,42 +131,46 @@ type audit struct { AuditID int64 AuditTimestamp string AuditType string - AuditTabletAlias string + AuditTabletAlias *topodatapb.TabletAlias Message string } // readRecentAudit returns a list of audit entries order chronologically descending, using page number. -func readRecentAudit(tabletAlias string, page int) ([]audit, error) { +func readRecentAudit(tabletAlias *topodatapb.TabletAlias, page int) ([]audit, error) { res := []audit{} var args []any whereCondition := `` - if tabletAlias != "" { - whereCondition = `where alias=?` - args = append(args, tabletAlias) + if tabletAlias != nil { + whereCondition = `WHERE alias = ?` + args = append(args, topoproto.TabletAliasString(tabletAlias)) } query := fmt.Sprintf(` - select + SELECT audit_id, audit_timestamp, audit_type, alias, message - from + FROM audit %s - order by - audit_timestamp desc - limit ? - offset ? - `, whereCondition) + ORDER BY + audit_timestamp DESC + LIMIT ? + OFFSET ?`, + whereCondition, + ) args = append(args, config.AuditPageSize, page*config.AuditPageSize) - err := db.QueryVTOrc(query, args, func(m sqlutils.RowMap) error { + err := db.QueryVTOrc(query, args, func(m sqlutils.RowMap) (err error) { a := audit{} a.AuditID = m.GetInt64("audit_id") a.AuditTimestamp = m.GetString("audit_timestamp") a.AuditType = m.GetString("audit_type") - a.AuditTabletAlias = m.GetString("alias") a.Message = m.GetString("message") + a.AuditTabletAlias, err = topoproto.ParseTabletAlias(m.GetString("alias")) + if err != nil { + return err + } res = append(res, a) return nil diff --git a/go/vt/vtorc/inst/instance.go b/go/vt/vtorc/inst/instance.go index 6ed7ecfcc80..4718cc8bef6 100644 --- a/go/vt/vtorc/inst/instance.go +++ b/go/vt/vtorc/inst/instance.go @@ -30,7 +30,7 @@ import ( type Instance struct { Hostname string Port int - InstanceAlias string + InstanceAlias *topodatapb.TabletAlias // TabletType is the tablet type that the instance // publishes to the VtGates and believes itself to be. TabletType topodatapb.TabletType diff --git a/go/vt/vtorc/inst/instance_dao.go b/go/vt/vtorc/inst/instance_dao.go index eae38bc955f..91fa43f4aa5 100644 --- a/go/vt/vtorc/inst/instance_dao.go +++ b/go/vt/vtorc/inst/instance_dao.go @@ -80,6 +80,10 @@ func init() { func initializeInstanceDao() { config.WaitForConfigurationToBeLoaded() + InitializeForgetAliasesCache() +} + +func InitializeForgetAliasesCache() { forgetAliases = cache.New(config.GetInstancePollTime()*3, time.Second) cacheInitializationCompleted.Store(true) } @@ -136,19 +140,20 @@ func ExpireTableData(tableName string, timestampColumn string) error { // logReadTopologyInstanceError logs an error, if applicable, for a ReadTopologyInstance operation, // providing context and hint as for the source of the error. If there's no hint just provide the // original error. -func logReadTopologyInstanceError(tabletAlias string, hint string, err error) error { +func logReadTopologyInstanceError(tabletAlias *topodatapb.TabletAlias, hint string, err error) error { if err == nil { return nil } - if !util.ClearToLog("ReadTopologyInstance", tabletAlias) { + tabletAliasString := topoproto.TabletAliasString(tabletAlias) + if !util.ClearToLog("ReadTopologyInstance", tabletAliasString) { return err } var msg string if hint == "" { - msg = fmt.Sprintf("ReadTopologyInstance(%+v): %+v", tabletAlias, err) + msg = fmt.Sprintf("ReadTopologyInstance(%+v): %+v", tabletAliasString, err) } else { msg = fmt.Sprintf("ReadTopologyInstance(%+v) %+v: %+v", - tabletAlias, + tabletAliasString, strings.ReplaceAll(hint, "%", "%%"), // escape % err) } @@ -169,7 +174,7 @@ func RegisterStats() { // It writes the information retrieved into vtorc's backend. // - writes are optionally buffered. // - timing information can be collected for the stages performed. -func ReadTopologyInstanceBufferable(tabletAlias string, latency *stopwatch.NamedStopwatch) (inst *Instance, err error) { +func ReadTopologyInstanceBufferable(tabletAlias *topodatapb.TabletAlias, latency *stopwatch.NamedStopwatch) (inst *Instance, err error) { defer func() { if r := recover(); r != nil { err = logReadTopologyInstanceError(tabletAlias, "Unexpected, aborting", tb.Errorf("%+v", r)) @@ -186,7 +191,7 @@ func ReadTopologyInstanceBufferable(tabletAlias string, latency *stopwatch.Named partialSuccess := false errorChan := make(chan error, 32) - if tabletAlias == "" { + if tabletAlias == nil { return instance, fmt.Errorf("ReadTopologyInstance will not act on empty tablet alias") } @@ -338,7 +343,7 @@ func ReadTopologyInstanceBufferable(tabletAlias string, latency *stopwatch.Named // ------------------------------------------------------------------------- instance.DataCenter = tablet.Alias.Cell - instance.InstanceAlias = topoproto.TabletAliasString(tablet.Alias) + instance.InstanceAlias = tablet.Alias { latency.Start("backend") @@ -408,10 +413,10 @@ func detectErrantGTIDs(instance *Instance, tablet *topodatapb.Tablet) (err error if instance.primaryExecutedGtidSet == "" && instance.SourceHost == "" { var primaryInstance *Instance primaryAlias, _, _ := ReadShardPrimaryInformation(tablet.Keyspace, tablet.Shard) - if primaryAlias != "" { + if primaryAlias != nil { // Check if the current tablet is the primary. // If it is, then we don't need to run errant gtid detection on it. - if primaryAlias == instance.InstanceAlias { + if topoproto.TabletAliasEqual(primaryAlias, instance.InstanceAlias) { return nil } primaryInstance, _, _ = ReadInstance(primaryAlias) @@ -451,7 +456,7 @@ func detectErrantGTIDs(instance *Instance, tablet *topodatapb.Tablet) (err error if err == nil { var gtidCount int64 gtidCount, err = replication.GTIDCount(instance.GtidErrant) - currentErrantGTIDCount.Set(instance.InstanceAlias, gtidCount) + currentErrantGTIDCount.Set(topoproto.TabletAliasString(instance.InstanceAlias), gtidCount) } } } @@ -526,7 +531,7 @@ func ReadInstanceClusterAttributes(instance *Instance) (err error) { } // readInstanceRow reads a single instance row from the vtorc backend database. -func readInstanceRow(m sqlutils.RowMap) *Instance { +func readInstanceRow(m sqlutils.RowMap) (*Instance, error) { instance := NewInstance() instance.Hostname = m.GetString("hostname") @@ -594,9 +599,14 @@ func readInstanceRow(m sqlutils.RowMap) *Instance { instance.IsLastCheckValid = m.GetBool("is_last_check_valid") instance.SecondsSinceLastSeen = m.GetNullInt64("seconds_since_last_seen") instance.AllowTLS = m.GetBool("allow_tls") - instance.InstanceAlias = m.GetString("alias") instance.LastDiscoveryLatency = time.Duration(m.GetInt64("last_discovery_latency")) * time.Nanosecond + var err error + instance.InstanceAlias, err = topoproto.ParseTabletAlias(m.GetString("alias")) + if err != nil { + return instance, err + } + instance.applyFlavorName() // problems @@ -613,13 +623,13 @@ func readInstanceRow(m sqlutils.RowMap) *Instance { instance.Problems = append(instance.Problems, "errant_gtid") } - return instance + return instance, nil } // readInstancesByCondition is a generic function to read instances from the backend database func readInstancesByCondition(condition string, args []any, sort string) ([](*Instance), error) { readFunc := func() ([]*Instance, error) { - var instances []*Instance + instances := make([]*Instance, 0) if sort == "" { sort = `alias` @@ -642,7 +652,10 @@ func readInstancesByCondition(condition string, args []any, sort string) ([](*In ) err := db.QueryVTOrc(query, args, func(m sqlutils.RowMap) error { - instance := readInstanceRow(m) + instance, err := readInstanceRow(m) + if err != nil { + return err + } instances = append(instances, instance) return nil }) @@ -665,9 +678,10 @@ func readInstancesByCondition(condition string, args []any, sort string) ([](*In } // ReadInstance reads an instance from the vtorc backend database -func ReadInstance(tabletAlias string) (*Instance, bool, error) { +func ReadInstance(tabletAlias *topodatapb.TabletAlias) (*Instance, bool, error) { condition := `alias = ?` - instances, err := readInstancesByCondition(condition, sqlutils.Args(tabletAlias), "") + args := sqlutils.Args(topoproto.TabletAliasString(tabletAlias)) + instances, err := readInstancesByCondition(condition, args, "") // We know there will be at most one (alias is the PK). // And we expect to find one. readInstanceCounter.Add(1) @@ -681,7 +695,7 @@ func ReadInstance(tabletAlias string) (*Instance, bool, error) { } // ReadProblemInstances reads all instances with problems -func ReadProblemInstances(keyspace string, shard string) ([](*Instance), error) { +func ReadProblemInstances(keyspace, shard string) ([]*Instance, error) { condition := ` keyspace LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END) AND shard LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END) @@ -695,12 +709,15 @@ func ReadProblemInstances(keyspace string, shard string) ([](*Instance), error) OR (gtid_errant != '') )` - args := sqlutils.Args(keyspace, keyspace, shard, shard, config.GetInstancePollSeconds()*5, config.GetReasonableReplicationLagSeconds(), config.GetReasonableReplicationLagSeconds()) + args := sqlutils.Args(keyspace, keyspace, shard, shard, config.GetInstancePollSeconds()*5, + config.GetReasonableReplicationLagSeconds(), + config.GetReasonableReplicationLagSeconds(), + ) return readInstancesByCondition(condition, args, "") } // ReadInstancesWithErrantGTIds reads all instances with errant GTIDs -func ReadInstancesWithErrantGTIds(keyspace string, shard string) ([]*Instance, error) { +func ReadInstancesWithErrantGTIds(keyspace, shard string) ([]*Instance, error) { condition := ` keyspace LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END) AND shard LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END) @@ -711,7 +728,7 @@ func ReadInstancesWithErrantGTIds(keyspace string, shard string) ([]*Instance, e } // GetKeyspaceShardName gets the keyspace shard name for the given instance key -func GetKeyspaceShardName(tabletAlias string) (keyspace string, shard string, err error) { +func GetKeyspaceShardName(tabletAlias *topodatapb.TabletAlias) (keyspace, shard string, err error) { query := `SELECT keyspace, shard @@ -720,7 +737,8 @@ func GetKeyspaceShardName(tabletAlias string) (keyspace string, shard string, er WHERE alias = ? ` - err = db.QueryVTOrc(query, sqlutils.Args(tabletAlias), func(m sqlutils.RowMap) error { + args := sqlutils.Args(topoproto.TabletAliasString(tabletAlias)) + err = db.QueryVTOrc(query, args, func(m sqlutils.RowMap) error { keyspace = m.GetString("keyspace") shard = m.GetString("shard") return nil @@ -731,7 +749,7 @@ func GetKeyspaceShardName(tabletAlias string) (keyspace string, shard string, er return keyspace, shard, err } -// ReadOutdatedInstanceKeys reads and returns keys for all instances that are not up to date (i.e. +// ReadOutdatedInstanceAliases reads and returns tablet aliases for all instances that are not up to date (i.e. // pre-configured time has passed since they were last checked) or the ones whose tablet information was read // but not the mysql information. This could happen if the durability policy of the keyspace wasn't // available at the time it was discovered. This would lead to not having the record of the tablet in the @@ -740,8 +758,8 @@ func GetKeyspaceShardName(tabletAlias string) (keyspace string, shard string, er // resulted in an actual check! This can happen when TCP/IP connections are hung, in which case the "check" // never returns. In such case we multiply interval by a factor, so as not to open too many connections on // the instance. -func ReadOutdatedInstanceKeys() ([]string, error) { - var res []string +func ReadOutdatedInstanceAliases() ([]*topodatapb.TabletAlias, error) { + res := make([]*topodatapb.TabletAlias, 0) query := `SELECT alias FROM @@ -760,17 +778,19 @@ func ReadOutdatedInstanceKeys() ([]string, error) { vitess_tablet.alias = database_instance.alias ) WHERE - database_instance.alias IS NULL - ` + database_instance.alias IS NULL` args := sqlutils.Args(config.GetInstancePollSeconds(), 2*config.GetInstancePollSeconds()) err := db.QueryVTOrc(query, args, func(m sqlutils.RowMap) error { - tabletAlias := m.GetString("alias") + tabletAlias, err := topoproto.ParseTabletAlias(m.GetString("alias")) + if err != nil { + return err + } if !InstanceIsForgotten(tabletAlias) { // only if not in "forget" cache res = append(res, tabletAlias) } - // We don;t return an error because we want to keep filling the outdated instances list. + // We don't return an error because we want to keep filling the outdated instances list. return nil }) if err != nil { @@ -912,7 +932,7 @@ func mkInsertForInstances(instances []*Instance, instanceWasActuallyFound bool, for _, instance := range instances { // number of columns minus 2 as last_checked and last_attempted_check // updated with NOW() - args = append(args, instance.InstanceAlias) + args = append(args, topoproto.TabletAliasString(instance.InstanceAlias)) args = append(args, instance.Hostname) args = append(args, instance.Port) args = append(args, int(instance.TabletType)) @@ -1020,7 +1040,7 @@ func WriteInstance(instance *Instance, instanceWasActuallyFound bool, lastError // UpdateInstanceLastChecked updates the last_check timestamp in the vtorc backed database // for a given instance -func UpdateInstanceLastChecked(tabletAlias string, partialSuccess bool, stalledDisk bool) error { +func UpdateInstanceLastChecked(tabletAlias *topodatapb.TabletAlias, partialSuccess bool, stalledDisk bool) error { writeFunc := func() error { _, err := db.ExecVTOrc(`UPDATE database_instance SET @@ -1032,7 +1052,7 @@ func UpdateInstanceLastChecked(tabletAlias string, partialSuccess bool, stalledD `, partialSuccess, stalledDisk, - tabletAlias, + topoproto.TabletAliasString(tabletAlias), ) if err != nil { log.Error(err) @@ -1050,7 +1070,7 @@ func UpdateInstanceLastChecked(tabletAlias string, partialSuccess bool, stalledD // And so we make sure to note down *before* we even attempt to access the instance; and this raises a red flag when we // wish to access the instance again: if last_attempted_check is *newer* than last_checked, that's bad news and means // we have a "hanging" issue. -func UpdateInstanceLastAttemptedCheck(tabletAlias string) error { +func UpdateInstanceLastAttemptedCheck(tabletAlias *topodatapb.TabletAlias) error { writeFunc := func() error { _, err := db.ExecVTOrc(`UPDATE database_instance SET @@ -1058,7 +1078,7 @@ func UpdateInstanceLastAttemptedCheck(tabletAlias string) error { WHERE alias = ? `, - tabletAlias, + topoproto.TabletAliasString(tabletAlias), ) if err != nil { log.Error(err) @@ -1068,32 +1088,34 @@ func UpdateInstanceLastAttemptedCheck(tabletAlias string) error { return ExecDBWriteFunc(writeFunc) } -func InstanceIsForgotten(tabletAlias string) bool { - _, found := forgetAliases.Get(tabletAlias) +// InstanceIsForgotten returns true if an instance was forgotten. +func InstanceIsForgotten(tabletAlias *topodatapb.TabletAlias) bool { + tabletAliasString := topoproto.TabletAliasString(tabletAlias) + _, found := forgetAliases.Get(tabletAliasString) return found } // ForgetInstance removes an instance entry from the vtorc backed database. // It may be auto-rediscovered through topology or requested for discovery by multiple means. -func ForgetInstance(tabletAlias string) error { - if tabletAlias == "" { +func ForgetInstance(tabletAlias *topodatapb.TabletAlias) error { + if tabletAlias == nil { errMsg := "ForgetInstance(): empty tabletAlias" log.Errorf(errMsg) return errors.New(errMsg) } - forgetAliases.Set(tabletAlias, true, cache.DefaultExpiration) - log.Infof("Forgetting: %v", tabletAlias) + tabletAliasString := topoproto.TabletAliasString(tabletAlias) + forgetAliases.Set(tabletAliasString, true, cache.DefaultExpiration) + log.Infof("Forgetting: %v", tabletAliasString) // Remove this tablet from errant GTID count metric. - currentErrantGTIDCount.Reset(tabletAlias) + currentErrantGTIDCount.Reset(tabletAliasString) // Delete from the 'vitess_tablet' table. - _, err := db.ExecVTOrc(`DELETE - FROM vitess_tablet + _, err := db.ExecVTOrc(`DELETE FROM + vitess_tablet WHERE - alias = ? - `, - tabletAlias, + alias = ?`, + tabletAliasString, ) if err != nil { log.Error(err) @@ -1101,12 +1123,11 @@ func ForgetInstance(tabletAlias string) error { } // Also delete from the 'database_instance' table. - sqlResult, err := db.ExecVTOrc(`DELETE - FROM database_instance + sqlResult, err := db.ExecVTOrc(`DELETE FROM + database_instance WHERE - alias = ? - `, - tabletAlias, + alias = ?`, + tabletAliasString, ) if err != nil { log.Error(err) @@ -1119,7 +1140,7 @@ func ForgetInstance(tabletAlias string) error { return err } if rows == 0 { - errMsg := fmt.Sprintf("ForgetInstance(): tablet %+v not found", tabletAlias) + errMsg := fmt.Sprintf("ForgetInstance(): tablet %+v not found", tabletAliasString) log.Error(errMsg) return errors.New(errMsg) } @@ -1146,7 +1167,7 @@ func ForgetLongUnseenInstances() error { return err } if rows > 0 { - _ = AuditOperation("forget-unseen", "", fmt.Sprintf("Forgotten instances: %d", rows)) + _ = AuditOperation("forget-unseen", nil, fmt.Sprintf("Forgotten instances: %d", rows)) } return err } @@ -1213,7 +1234,7 @@ func GetDatabaseState() (string, error) { Rows []sqlutils.RowMap } - var dbState []tableState + dbState := make([]tableState, 0, len(db.TableNames)) for _, tableName := range db.TableNames { ts := tableState{ TableName: tableName, diff --git a/go/vt/vtorc/inst/instance_dao_test.go b/go/vt/vtorc/inst/instance_dao_test.go index c07f303b9d2..84fba38f057 100644 --- a/go/vt/vtorc/inst/instance_dao_test.go +++ b/go/vt/vtorc/inst/instance_dao_test.go @@ -38,9 +38,9 @@ func stripSpaces(s string) string { } func mkTestInstances() []*Instance { - i710 := Instance{InstanceAlias: "zone1-i710", Hostname: "i710", Port: 3306, TabletType: topodatapb.TabletType_PRIMARY, ServerID: 710, ExecBinlogCoordinates: BinlogCoordinates{LogFile: "mysql.000007", LogPos: 10}} - i720 := Instance{InstanceAlias: "zone1-i720", Hostname: "i720", Port: 3306, TabletType: topodatapb.TabletType_REPLICA, ServerID: 720, ExecBinlogCoordinates: BinlogCoordinates{LogFile: "mysql.000007", LogPos: 20}} - i730 := Instance{InstanceAlias: "zone1-i730", Hostname: "i730", Port: 3306, TabletType: topodatapb.TabletType_REPLICA, ServerID: 730, ExecBinlogCoordinates: BinlogCoordinates{LogFile: "mysql.000007", LogPos: 30}} + i710 := Instance{InstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 710}, Hostname: "i710", Port: 3306, TabletType: topodatapb.TabletType_PRIMARY, ServerID: 710, ExecBinlogCoordinates: BinlogCoordinates{LogFile: "mysql.000007", LogPos: 10}} + i720 := Instance{InstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 720}, Hostname: "i720", Port: 3306, TabletType: topodatapb.TabletType_REPLICA, ServerID: 720, ExecBinlogCoordinates: BinlogCoordinates{LogFile: "mysql.000007", LogPos: 20}} + i730 := Instance{InstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 730}, Hostname: "i730", Port: 3306, TabletType: topodatapb.TabletType_REPLICA, ServerID: 730, ExecBinlogCoordinates: BinlogCoordinates{LogFile: "mysql.000007", LogPos: 30}} instances := []*Instance{&i710, &i720, &i730} for _, instance := range instances { instance.Version = "5.6.7" @@ -69,7 +69,7 @@ func TestMkInsertSingle(t *testing.T) { VALUES (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')) ` - a1 := `zone1-i710, i710, 3306, 1, 710, , 5.6.7, 5.6, MySQL, false, false, STATEMENT, + a1 := `zone1-0000000710, i710, 3306, 1, 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,` @@ -95,9 +95,9 @@ func TestMkInsertThree(t *testing.T) { (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')) ` a3 := ` - zone1-i710, i710, 3306, 1, 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, 2, 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, 2, 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, + zone1-0000000710, i710, 3306, 1, 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-0000000720, i720, 3306, 2, 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-0000000730, i730, 3306, 2, 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) @@ -143,7 +143,7 @@ func TestGetKeyspaceShardName(t *testing.T) { err = SaveTablet(tab100) require.NoError(t, err) - keyspaceRead, shardRead, err := GetKeyspaceShardName(topoproto.TabletAliasString(tab100.Alias)) + keyspaceRead, shardRead, err := GetKeyspaceShardName(tab100.Alias) require.NoError(t, err) require.Equal(t, ks, keyspaceRead) require.Equal(t, shard, shardRead) @@ -153,16 +153,16 @@ func TestGetKeyspaceShardName(t *testing.T) { func TestReadInstance(t *testing.T) { tests := []struct { name string - tabletAliasToRead string + tabletAliasToRead *topodatapb.TabletAlias instanceFound bool }{ { name: "Read success", - tabletAliasToRead: "zone1-0000000100", + tabletAliasToRead: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, instanceFound: true, }, { name: "Unknown tablet", - tabletAliasToRead: "unknown-tablet", + tabletAliasToRead: &topodatapb.TabletAlias{Cell: "unknown", Uid: 123456789}, instanceFound: false, }, } @@ -178,7 +178,6 @@ func TestReadInstance(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got, found, err := ReadInstance(tt.tabletAliasToRead) require.NoError(t, err) require.Equal(t, tt.instanceFound, found) @@ -265,7 +264,7 @@ func TestReadProblemInstances(t *testing.T) { require.NoError(t, err) var tabletAliases []string for _, instance := range instances { - tabletAliases = append(tabletAliases, instance.InstanceAlias) + tabletAliases = append(tabletAliases, topoproto.TabletAliasString(instance.InstanceAlias)) } require.ElementsMatch(t, tabletAliases, tt.instancesRequired) }) @@ -349,7 +348,7 @@ func TestReadInstancesWithErrantGTIds(t *testing.T) { require.NoError(t, err) var tabletAliases []string for _, instance := range instances { - tabletAliases = append(tabletAliases, instance.InstanceAlias) + tabletAliases = append(tabletAliases, topoproto.TabletAliasString(instance.InstanceAlias)) } require.ElementsMatch(t, tabletAliases, tt.instancesRequired) }) @@ -370,7 +369,7 @@ func TestReadInstanceAllFields(t *testing.T) { wantInstance := &Instance{ Hostname: "localhost", Port: 6711, - InstanceAlias: "zone1-0000000100", + InstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, TabletType: topodatapb.TabletType_REPLICA, ServerID: 1094500338, ServerUUID: "729a5138-8680-11ed-acf8-d6b0ef9f4eaa", @@ -453,7 +452,7 @@ func TestReadInstanceAllFields(t *testing.T) { LastDiscoveryLatency: 0, } - instance, found, err := ReadInstance(`zone1-0000000100`) + instance, found, err := ReadInstance(&topodatapb.TabletAlias{Cell: "zone1", Uid: 100}) require.NoError(t, err) require.True(t, found) instance.SecondsSinceLastSeen = sql.NullInt64{} @@ -524,15 +523,15 @@ func TestReadInstancesByCondition(t *testing.T) { require.NoError(t, err) var tabletAliases []string for _, instance := range instances { - tabletAliases = append(tabletAliases, instance.InstanceAlias) + tabletAliases = append(tabletAliases, topoproto.TabletAliasString(instance.InstanceAlias)) } require.EqualValues(t, tt.instancesRequired, tabletAliases) }) } } -// TestReadOutdatedInstanceKeys is used to test the functionality of ReadOutdatedInstanceKeys and verify its failure modes and successes. -func TestReadOutdatedInstanceKeys(t *testing.T) { +// TestReadOutdatedInstanceAliases is used to test the functionality of ReadOutdatedInstanceAliases and verify its failure modes and successes. +func TestReadOutdatedInstanceAliases(t *testing.T) { // The test is intended to be used as follows. The initial data is stored into the database. Following this, some specific queries are run that each individual test specifies to get the desired state. tests := []struct { name string @@ -564,7 +563,7 @@ func TestReadOutdatedInstanceKeys(t *testing.T) { "update database_instance set last_checked = DATETIME('now', '-1 hour') where alias = 'zone1-0000000100'", `INSERT INTO vitess_tablet VALUES('zone1-0000000103','localhost',7706,'ks','0','zone1',2,'0001-01-01 00:00:00+00:00','');`, }, - instancesRequired: []string{"zone1-0000000103", "zone1-0000000100"}, + instancesRequired: []string{"zone1-0000000100", "zone1-0000000103"}, }, } @@ -593,7 +592,7 @@ func TestReadOutdatedInstanceKeys(t *testing.T) { require.NoError(t, err) } - tabletAliases, err := ReadOutdatedInstanceKeys() + tabletAliases, err := ReadOutdatedInstanceAliases() errInDataCollection := db.QueryVTOrcRowsMap(`select alias, last_checked, @@ -608,7 +607,11 @@ from database_instance`, func(rowMap sqlutils.RowMap) error { }) require.NoError(t, errInDataCollection) require.NoError(t, err) - require.ElementsMatch(t, tabletAliases, tt.instancesRequired) + var tabletAliasStrings []string + for _, tabletAlias := range tabletAliases { + tabletAliasStrings = append(tabletAliasStrings, topoproto.TabletAliasString(tabletAlias)) + } + require.EqualValues(t, tt.instancesRequired, tabletAliasStrings) }) } } @@ -617,32 +620,32 @@ from database_instance`, func(rowMap sqlutils.RowMap) error { func TestUpdateInstanceLastChecked(t *testing.T) { tests := []struct { name string - tabletAlias string + tabletAlias *topodatapb.TabletAlias partialSuccess bool stalledDisk bool conditionToCheck string }{ { name: "Verify updated last checked", - tabletAlias: "zone1-0000000100", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, partialSuccess: false, stalledDisk: false, conditionToCheck: "last_checked >= DATETIME('now', '-30 second') and last_check_partial_success = false and is_disk_stalled = false", }, { name: "Verify partial success", - tabletAlias: "zone1-0000000100", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, partialSuccess: true, stalledDisk: false, conditionToCheck: "last_checked >= datetime('now', '-30 second') and last_check_partial_success = true and is_disk_stalled = false", }, { name: "Verify stalled disk", - tabletAlias: "zone1-0000000100", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, partialSuccess: false, stalledDisk: true, conditionToCheck: "last_checked >= DATETIME('now', '-30 second') and last_check_partial_success = false and is_disk_stalled = true", }, { - name: "Verify no error on unknown tablet", - tabletAlias: "unknown tablet", + name: "Verify no error on nil tablet", + tabletAlias: nil, partialSuccess: true, stalledDisk: true, }, @@ -666,7 +669,7 @@ func TestUpdateInstanceLastChecked(t *testing.T) { // Verify the instance we just updated satisfies the condition specified. instances, err := readInstancesByCondition(tt.conditionToCheck, nil, "") require.NoError(t, err) - var tabletAliases []string + var tabletAliases []*topodatapb.TabletAlias for _, instance := range instances { tabletAliases = append(tabletAliases, instance.InstanceAlias) } @@ -680,16 +683,16 @@ func TestUpdateInstanceLastChecked(t *testing.T) { func TestUpdateInstanceLastAttemptedCheck(t *testing.T) { tests := []struct { name string - tabletAlias string + tabletAlias *topodatapb.TabletAlias conditionToCheck string }{ { name: "Verify updated last checked", - tabletAlias: "zone1-0000000100", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, conditionToCheck: "last_attempted_check >= DATETIME('now', '-30 second')", }, { - name: "Verify no error on unknown tablet", - tabletAlias: "unknown tablet", + name: "Verify no error on nil tablet", + tabletAlias: nil, }, } @@ -711,7 +714,7 @@ func TestUpdateInstanceLastAttemptedCheck(t *testing.T) { // Verify the instance we just updated satisfies the condition specified. instances, err := readInstancesByCondition(tt.conditionToCheck, nil, "") require.NoError(t, err) - var tabletAliases []string + var tabletAliases []*topodatapb.TabletAlias for _, instance := range instances { tabletAliases = append(tabletAliases, instance.InstanceAlias) } @@ -725,28 +728,31 @@ func TestUpdateInstanceLastAttemptedCheck(t *testing.T) { func TestForgetInstanceAndInstanceIsForgotten(t *testing.T) { tests := []struct { name string - tabletAlias string + tabletAlias *topodatapb.TabletAlias errExpected string instanceForgotten bool - tabletsExpected []string + tabletsExpected []*topodatapb.TabletAlias }{ { - name: "Unknown tablet", - tabletAlias: "unknown-tablet", - errExpected: "ForgetInstance(): tablet unknown-tablet not found", - instanceForgotten: true, - tabletsExpected: []string{"zone1-0000000100", "zone1-0000000101", "zone1-0000000112", "zone2-0000000200"}, - }, { name: "Empty tabletAlias", - tabletAlias: "", + tabletAlias: nil, errExpected: "ForgetInstance(): empty tabletAlias", instanceForgotten: false, - tabletsExpected: []string{"zone1-0000000100", "zone1-0000000101", "zone1-0000000112", "zone2-0000000200"}, + tabletsExpected: []*topodatapb.TabletAlias{ + {Cell: "zone1", Uid: 100}, + {Cell: "zone1", Uid: 101}, + {Cell: "zone1", Uid: 112}, + {Cell: "zone2", Uid: 200}, + }, }, { name: "Success", - tabletAlias: "zone1-0000000112", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 112}, instanceForgotten: true, - tabletsExpected: []string{"zone1-0000000100", "zone1-0000000101", "zone2-0000000200"}, + tabletsExpected: []*topodatapb.TabletAlias{ + {Cell: "zone1", Uid: 100}, + {Cell: "zone1", Uid: 101}, + {Cell: "zone2", Uid: 200}, + }, }, } @@ -779,7 +785,7 @@ func TestForgetInstanceAndInstanceIsForgotten(t *testing.T) { instances, err := readInstancesByCondition("1=1", nil, "") require.NoError(t, err) - var tabletAliases []string + var tabletAliases []*topodatapb.TabletAlias for _, instance := range instances { tabletAliases = append(tabletAliases, instance.InstanceAlias) } @@ -996,14 +1002,14 @@ func TestDetectErrantGTIDs(t *testing.T) { require.NoError(t, err) if tt.primaryInstance != nil { - tt.primaryInstance.InstanceAlias = topoproto.TabletAliasString(primaryTablet.Alias) + tt.primaryInstance.InstanceAlias = primaryTablet.Alias err = SaveTablet(primaryTablet) require.NoError(t, err) err = WriteInstance(tt.primaryInstance, true, nil) require.NoError(t, err) } - tt.instance.InstanceAlias = topoproto.TabletAliasString(tablet.Alias) + tt.instance.InstanceAlias = tablet.Alias err = detectErrantGTIDs(tt.instance, tablet) if tt.wantErr { require.Error(t, err) @@ -1035,7 +1041,7 @@ func TestPrimaryErrantGTIDs(t *testing.T) { instance := &Instance{ SourceHost: "", ExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10589,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34,316d193c-70e5-11e5-adb2-ecf4bb2262ff:1-341", - InstanceAlias: topoproto.TabletAliasString(tablet.Alias), + InstanceAlias: tablet.Alias, } // Save shard record for the primary tablet. diff --git a/go/vt/vtorc/inst/instance_test.go b/go/vt/vtorc/inst/instance_test.go index 9ca2f243999..cd3d226ef3e 100644 --- a/go/vt/vtorc/inst/instance_test.go +++ b/go/vt/vtorc/inst/instance_test.go @@ -21,6 +21,7 @@ import ( "github.com/stretchr/testify/require" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/vtorc/config" ) @@ -28,7 +29,10 @@ func init() { config.MarkConfigurationLoaded() } -var instance1 = Instance{InstanceAlias: "zone1-100"} +var instance1 = Instance{InstanceAlias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 100, +}} func TestReplicationThreads(t *testing.T) { { @@ -41,7 +45,7 @@ func TestReplicationThreads(t *testing.T) { require.True(t, instance1.ReplicationThreadsStopped()) } { - i := Instance{InstanceAlias: "zone1-100", ReplicationIOThreadState: ReplicationThreadStateNoThread, ReplicationSQLThreadState: ReplicationThreadStateNoThread} + i := Instance{InstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, ReplicationIOThreadState: ReplicationThreadStateNoThread, ReplicationSQLThreadState: ReplicationThreadStateNoThread} require.False(t, i.ReplicationThreadsExist()) } } diff --git a/go/vt/vtorc/inst/shard_dao.go b/go/vt/vtorc/inst/shard_dao.go index 7fe8df110f9..b9ffe82bd07 100644 --- a/go/vt/vtorc/inst/shard_dao.go +++ b/go/vt/vtorc/inst/shard_dao.go @@ -21,6 +21,7 @@ import ( "vitess.io/vitess/go/protoutil" "vitess.io/vitess/go/vt/external/golib/sqlutils" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtorc/db" @@ -42,7 +43,7 @@ func ReadShardNames(keyspaceName string) (shardNames []string, err error) { } // ReadShardPrimaryInformation reads the vitess shard record and gets the shard primary alias and timestamp. -func ReadShardPrimaryInformation(keyspaceName, shardName string) (primaryAlias string, primaryTimestamp string, err error) { +func ReadShardPrimaryInformation(keyspaceName, shardName string) (primaryAlias *topodatapb.TabletAlias, primaryTimestamp string, err error) { if err = topo.ValidateKeyspaceName(keyspaceName); err != nil { return } @@ -50,18 +51,23 @@ func ReadShardPrimaryInformation(keyspaceName, shardName string) (primaryAlias s return } - query := ` - select + query := `SELECT primary_alias, primary_timestamp - from + FROM vitess_shard - where keyspace=? and shard=? - ` + WHERE + keyspace = ? + AND shard = ?` args := sqlutils.Args(keyspaceName, shardName) shardFound := false - err = db.QueryVTOrc(query, args, func(row sqlutils.RowMap) error { + err = db.QueryVTOrc(query, args, func(row sqlutils.RowMap) (err error) { shardFound = true - primaryAlias = row.GetString("primary_alias") + if primaryAliasStr := row.GetString("primary_alias"); primaryAliasStr != "" { + primaryAlias, err = topoproto.ParseTabletAlias(primaryAliasStr) + if err != nil { + return err + } + } primaryTimestamp = row.GetString("primary_timestamp") return nil }) @@ -69,9 +75,9 @@ func ReadShardPrimaryInformation(keyspaceName, shardName string) (primaryAlias s return } if !shardFound { - return "", "", ErrShardNotFound + err = ErrShardNotFound } - return primaryAlias, primaryTimestamp, nil + return primaryAlias, primaryTimestamp, err } // SaveShard saves the shard record against the shard name. diff --git a/go/vt/vtorc/inst/shard_dao_test.go b/go/vt/vtorc/inst/shard_dao_test.go index 0077b3d64af..856b65d57e5 100644 --- a/go/vt/vtorc/inst/shard_dao_test.go +++ b/go/vt/vtorc/inst/shard_dao_test.go @@ -39,7 +39,7 @@ func TestSaveReadAndDeleteShard(t *testing.T) { keyspaceName string shardName string shard *topodatapb.Shard - primaryAliasWanted string + primaryAliasWanted *topodatapb.TabletAlias primaryTimestampWanted string err string }{ @@ -55,7 +55,7 @@ func TestSaveReadAndDeleteShard(t *testing.T) { PrimaryTermStartTime: protoutil.TimeToProto(timeToUse.Add(1 * time.Hour)), }, primaryTimestampWanted: "2023-07-24 06:00:05.000001 +0000 UTC", - primaryAliasWanted: "zone1-0000000301", + primaryAliasWanted: &topodatapb.TabletAlias{Cell: "zone1", Uid: 301}, }, { name: "Success with empty primary alias", keyspaceName: "ks1", @@ -64,7 +64,7 @@ func TestSaveReadAndDeleteShard(t *testing.T) { PrimaryTermStartTime: protoutil.TimeToProto(timeToUse), }, primaryTimestampWanted: "2023-07-24 05:00:05.000001 +0000 UTC", - primaryAliasWanted: "", + primaryAliasWanted: nil, }, { name: "Success with empty primary term start time", keyspaceName: "ks1", @@ -76,7 +76,7 @@ func TestSaveReadAndDeleteShard(t *testing.T) { }, }, primaryTimestampWanted: "", - primaryAliasWanted: "zone1-0000000301", + primaryAliasWanted: &topodatapb.TabletAlias{Cell: "zone1", Uid: 301}, }, { name: "No shard found", @@ -100,7 +100,7 @@ func TestSaveReadAndDeleteShard(t *testing.T) { return } require.NoError(t, err) - require.EqualValues(t, tt.primaryAliasWanted, shardPrimaryAlias) + testRequireTabletAliasEqual(t, tt.primaryAliasWanted, shardPrimaryAlias) require.EqualValues(t, tt.primaryTimestampWanted, primaryTimestamp) // ReadShardNames diff --git a/go/vt/vtorc/inst/tablet_dao.go b/go/vt/vtorc/inst/tablet_dao.go index bd8115de969..5bfbb0669a1 100644 --- a/go/vt/vtorc/inst/tablet_dao.go +++ b/go/vt/vtorc/inst/tablet_dao.go @@ -24,7 +24,6 @@ import ( "vitess.io/vitess/go/protoutil" "vitess.io/vitess/go/vt/external/golib/sqlutils" - replicationdatapb "vitess.io/vitess/go/vt/proto/replicationdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/topo" @@ -51,15 +50,14 @@ func fullStatus(tablet *topodatapb.Tablet) (*replicationdatapb.FullStatus, error } // ReadTablet reads the vitess tablet record. -func ReadTablet(tabletAlias string) (*topodatapb.Tablet, error) { +func ReadTablet(tabletAlias *topodatapb.TabletAlias) (*topodatapb.Tablet, error) { query := `SELECT info FROM vitess_tablet WHERE - alias = ? - ` - args := sqlutils.Args(tabletAlias) + alias = ?` + args := sqlutils.Args(topoproto.TabletAliasString(tabletAlias)) tablet := &topodatapb.Tablet{} opts := prototext.UnmarshalOptions{DiscardUnknown: true} err := db.QueryVTOrc(query, args, func(row sqlutils.RowMap) error { diff --git a/go/vt/vtorc/inst/tablet_dao_test.go b/go/vt/vtorc/inst/tablet_dao_test.go index d5de0eb268b..0a4898a5e70 100644 --- a/go/vt/vtorc/inst/tablet_dao_test.go +++ b/go/vt/vtorc/inst/tablet_dao_test.go @@ -12,6 +12,10 @@ import ( "vitess.io/vitess/go/vt/vtorc/db" ) +func testRequireTabletAliasEqual(t *testing.T, a, b *topodatapb.TabletAlias) { + require.Equal(t, topoproto.TabletAliasString(a), topoproto.TabletAliasString(b)) +} + func TestSaveAndReadTablet(t *testing.T) { // Clear the database after the test. The easiest way to do that is to run all the initialization commands again. defer func() { @@ -20,14 +24,14 @@ func TestSaveAndReadTablet(t *testing.T) { tests := []struct { name string - tabletAlias string + tabletAlias *topodatapb.TabletAlias tablet *topodatapb.Tablet tabletWanted *topodatapb.Tablet err string }{ { name: "Success with primary type", - tabletAlias: "zone1-0000000100", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, tablet: &topodatapb.Tablet{ Alias: &topodatapb.TabletAlias{ Cell: "zone1", @@ -47,7 +51,7 @@ func TestSaveAndReadTablet(t *testing.T) { tabletWanted: nil, }, { name: "Success with replica type", - tabletAlias: "zone1-0000000100", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, tablet: &topodatapb.Tablet{ Alias: &topodatapb.TabletAlias{ Cell: "zone1", @@ -63,7 +67,7 @@ func TestSaveAndReadTablet(t *testing.T) { tabletWanted: nil, }, { name: "No tablet found", - tabletAlias: "zone1-190734", + tabletAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 190734}, tablet: nil, tabletWanted: nil, err: ErrTabletAliasNil.Error(), @@ -87,7 +91,7 @@ func TestSaveAndReadTablet(t *testing.T) { } require.NoError(t, err) require.True(t, topotools.TabletEquality(tt.tabletWanted, readTable)) - require.Equal(t, tt.tabletAlias, topoproto.TabletAliasString(readTable.Alias)) + testRequireTabletAliasEqual(t, tt.tabletAlias, readTable.Alias) }) } } diff --git a/go/vt/vtorc/logic/keyspace_shard_discovery_test.go b/go/vt/vtorc/logic/keyspace_shard_discovery_test.go index b5b09aba95e..e9a7938ea5a 100644 --- a/go/vt/vtorc/logic/keyspace_shard_discovery_test.go +++ b/go/vt/vtorc/logic/keyspace_shard_discovery_test.go @@ -99,11 +99,11 @@ func TestRefreshAllKeyspaces(t *testing.T) { // Verify that we only have ks1 and ks3 in vtorc's db. verifyKeyspaceInfo(t, "ks1", keyspaceDurabilityNone, "") - verifyPrimaryAlias(t, "ks1", "-80", "zone_ks1-0000000100", "") + verifyPrimaryAlias(t, "ks1", "-80", &topodatapb.TabletAlias{Cell: "zone_ks1", Uid: 100}, "") verifyKeyspaceInfo(t, "ks2", nil, "keyspace not found") - verifyPrimaryAlias(t, "ks2", "80-", "", "shard not found") + verifyPrimaryAlias(t, "ks2", "80-", nil, "shard not found") verifyKeyspaceInfo(t, "ks3", keyspaceSnapshot, "") - verifyPrimaryAlias(t, "ks3", "80-", "zone_ks3-0000000101", "") + verifyPrimaryAlias(t, "ks3", "80-", &topodatapb.TabletAlias{Cell: "zone_ks3", Uid: 101}, "") verifyKeyspaceInfo(t, "ks4", nil, "keyspace not found") // Set clusters to watch to watch all keyspaces @@ -116,13 +116,13 @@ func TestRefreshAllKeyspaces(t *testing.T) { // Verify that all the keyspaces are correctly reloaded verifyKeyspaceInfo(t, "ks1", keyspaceDurabilitySemiSync, "") - verifyPrimaryAlias(t, "ks1", "-80", "zone_ks1-0000000100", "") + verifyPrimaryAlias(t, "ks1", "-80", &topodatapb.TabletAlias{Cell: "zone_ks1", Uid: 100}, "") verifyKeyspaceInfo(t, "ks2", keyspaceDurabilitySemiSync, "") - verifyPrimaryAlias(t, "ks2", "80-", "zone_ks2-0000000101", "") + verifyPrimaryAlias(t, "ks2", "80-", &topodatapb.TabletAlias{Cell: "zone_ks2", Uid: 101}, "") verifyKeyspaceInfo(t, "ks3", keyspaceSnapshot, "") - verifyPrimaryAlias(t, "ks3", "80-", "zone_ks3-0000000101", "") + verifyPrimaryAlias(t, "ks3", "80-", &topodatapb.TabletAlias{Cell: "zone_ks3", Uid: 101}, "") verifyKeyspaceInfo(t, "ks4", keyspaceDurabilityTest, "") - verifyPrimaryAlias(t, "ks4", "80-", "zone_ks4-0000000101", "") + verifyPrimaryAlias(t, "ks4", "80-", &topodatapb.TabletAlias{Cell: "zone_ks4", Uid: 101}, "") } func TestRefreshKeyspace(t *testing.T) { @@ -245,7 +245,7 @@ func TestRefreshShard(t *testing.T) { keyspaceName string shardName string shard *topodatapb.Shard - primaryAliasWanted string + primaryAliasWanted *topodatapb.TabletAlias err string }{ { @@ -258,14 +258,14 @@ func TestRefreshShard(t *testing.T) { Uid: 302, }, }, - primaryAliasWanted: "zone1-0000000302", + primaryAliasWanted: &topodatapb.TabletAlias{Cell: "zone1", Uid: 302}, err: "", }, { name: "Success with empty primaryAlias", keyspaceName: "ks1", shardName: "-80", shard: &topodatapb.Shard{}, - primaryAliasWanted: "", + primaryAliasWanted: nil, err: "", }, { name: "No shard found", @@ -302,7 +302,7 @@ func TestRefreshShard(t *testing.T) { } // verifyPrimaryAlias verifies the correct primary alias is stored in the database for the given keyspace shard. -func verifyPrimaryAlias(t *testing.T, keyspaceName, shardName string, primaryAliasWanted string, errString string) { +func verifyPrimaryAlias(t *testing.T, keyspaceName, shardName string, primaryAliasWanted *topodatapb.TabletAlias, errString string) { primaryAlias, _, err := inst.ReadShardPrimaryInformation(keyspaceName, shardName) if errString != "" { require.ErrorContains(t, err, errString) diff --git a/go/vt/vtorc/logic/tablet_discovery.go b/go/vt/vtorc/logic/tablet_discovery.go index eee788bd9aa..24477a5b746 100644 --- a/go/vt/vtorc/logic/tablet_discovery.go +++ b/go/vt/vtorc/logic/tablet_discovery.go @@ -20,7 +20,6 @@ import ( "context" "errors" "fmt" - "slices" "strings" "sync" "time" @@ -215,13 +214,13 @@ func getAllTablets(ctx context.Context, cells []string) (tabletsByCell map[strin // refreshAllTablets reloads the tablets from topo and discovers the ones which haven't been refreshed in a while func refreshAllTablets(ctx context.Context) error { - return refreshTabletsUsing(ctx, func(tabletAlias string) { + return refreshTabletsUsing(ctx, func(tabletAlias *topodatapb.TabletAlias) { DiscoverInstance(tabletAlias, false /* forceDiscovery */) }, false /* forceRefresh */) } // refreshTabletsUsing refreshes tablets using a provided loader. -func refreshTabletsUsing(ctx context.Context, loader func(tabletAlias string), forceRefresh bool) error { +func refreshTabletsUsing(ctx context.Context, loader func(*topodatapb.TabletAlias), forceRefresh bool) error { // Get all cells. cellsCtx, cellsCancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout) defer cellsCancel() @@ -256,7 +255,7 @@ func refreshTabletsUsing(ctx context.Context, loader func(tabletAlias string), f }() // Refresh the filtered tablets and forget stale tablets. - query := "select alias from vitess_tablet where cell = ?" + query := "SELECT alias FROM vitess_tablet WHERE cell = ?" args := sqlutils.Args(cell) refreshTablets(matchedTablets, query, args, loader, forceRefresh, nil) } @@ -267,10 +266,10 @@ func refreshTabletsUsing(ctx context.Context, loader func(tabletAlias string), f // forceRefreshAllTabletsInShard is used to refresh all the tablet's information (both MySQL information and topo records) // for a given shard. This function is meant to be called before or after a cluster-wide operation that we know will // change the replication information for the entire cluster drastically enough to warrant a full forceful refresh -func forceRefreshAllTabletsInShard(ctx context.Context, keyspace, shard string, tabletsToIgnore []string) { +func forceRefreshAllTabletsInShard(ctx context.Context, keyspace, shard string, tabletsToIgnore []*topodatapb.TabletAlias) { refreshCtx, refreshCancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout) defer refreshCancel() - refreshTabletsInKeyspaceShard(refreshCtx, keyspace, shard, func(tabletAlias string) { + refreshTabletsInKeyspaceShard(refreshCtx, keyspace, shard, func(tabletAlias *topodatapb.TabletAlias) { DiscoverInstance(tabletAlias, true) }, true, tabletsToIgnore) } @@ -279,32 +278,32 @@ func forceRefreshAllTabletsInShard(ctx context.Context, keyspace, shard string, // of the given keyspace-shard. func refreshTabletInfoOfShard(ctx context.Context, keyspace, shard string) { log.Infof("refresh of tablet records of shard - %v/%v", keyspace, shard) - refreshTabletsInKeyspaceShard(ctx, keyspace, shard, func(tabletAlias string) { + refreshTabletsInKeyspaceShard(ctx, keyspace, shard, func(*topodatapb.TabletAlias) { // No-op // We only want to refresh the tablet information for the given shard }, false, nil) } -func refreshTabletsInKeyspaceShard(ctx context.Context, keyspace, shard string, loader func(tabletAlias string), forceRefresh bool, tabletsToIgnore []string) { +func refreshTabletsInKeyspaceShard(ctx context.Context, keyspace, shard string, loader func(*topodatapb.TabletAlias), forceRefresh bool, tabletsToIgnore []*topodatapb.TabletAlias) { tablets, err := ts.GetTabletsByShard(ctx, keyspace, shard) if err != nil { log.Errorf("Error fetching tablets for keyspace/shard %v/%v: %v", keyspace, shard, err) return } - query := "select alias from vitess_tablet where keyspace = ? and shard = ?" + query := "SELECT alias FROM vitess_tablet WHERE keyspace = ? AND shard = ?" args := sqlutils.Args(keyspace, shard) refreshTablets(tablets, query, args, loader, forceRefresh, tabletsToIgnore) } -func refreshTablets(tablets []*topo.TabletInfo, query string, args []any, loader func(tabletAlias string), forceRefresh bool, tabletsToIgnore []string) { +func refreshTablets(tablets []*topo.TabletInfo, query string, args []any, loader func(*topodatapb.TabletAlias), forceRefresh bool, tabletsToIgnore []*topodatapb.TabletAlias) { // Discover new tablets. - latestInstances := make(map[string]bool) + latestInstances := make(map[string]bool, len(tablets)) var wg sync.WaitGroup for _, tabletInfo := range tablets { tablet := tabletInfo.Tablet tabletAliasString := topoproto.TabletAliasString(tablet.Alias) latestInstances[tabletAliasString] = true - old, err := inst.ReadTablet(tabletAliasString) + old, err := inst.ReadTablet(tablet.Alias) if err != nil && err != inst.ErrTabletAliasNil { log.Error(err) continue @@ -316,24 +315,30 @@ func refreshTablets(tablets []*topo.TabletInfo, query string, args []any, loader log.Error(err) continue } + wg.Add(1) go func() { defer wg.Done() - if slices.Contains(tabletsToIgnore, topoproto.TabletAliasString(tablet.Alias)) { - return + for _, tabletToIgnore := range tabletsToIgnore { + if topoproto.TabletAliasEqual(tabletToIgnore, tablet.Alias) { + return + } } - loader(tabletAliasString) + loader(tablet.Alias) }() log.Infof("Discovered: %v", tablet) } wg.Wait() // Forget tablets that were removed. - var toForget []string + toForget := make([]*topodatapb.TabletAlias, 0) err := db.QueryVTOrc(query, args, func(row sqlutils.RowMap) error { - tabletAlias := row.GetString("alias") - if !latestInstances[tabletAlias] { - toForget = append(toForget, tabletAlias) + tabletAliasString := row.GetString("alias") + if !latestInstances[tabletAliasString] { + tabletAlias, err := topoproto.ParseTabletAlias(tabletAliasString) + if err == nil { + toForget = append(toForget, tabletAlias) + } } return nil }) diff --git a/go/vt/vtorc/logic/tablet_discovery_test.go b/go/vt/vtorc/logic/tablet_discovery_test.go index c351fb41c0a..e19cd8e9a9d 100644 --- a/go/vt/vtorc/logic/tablet_discovery_test.go +++ b/go/vt/vtorc/logic/tablet_discovery_test.go @@ -332,6 +332,9 @@ func TestRefreshTabletsInKeyspaceShard(t *testing.T) { db.ClearVTOrcDatabase() }() + // Init forgetAliases cache in go/vt/vtorc/inst + inst.InitializeForgetAliasesCache() + // Create a memory topo-server and create the keyspace and shard records ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -364,7 +367,7 @@ func TestRefreshTabletsInKeyspaceShard(t *testing.T) { t.Run("call refreshTabletsInKeyspaceShard again - force refresh with ignore", func(t *testing.T) { // We expect 2 tablets to be refreshed since we requested force refresh, but we are ignoring one of them. - verifyRefreshTabletsInKeyspaceShard(t, true, 2, tablets, []string{topoproto.TabletAliasString(tab100.Alias)}) + verifyRefreshTabletsInKeyspaceShard(t, true, 2, tablets, []*topodatapb.TabletAlias{tab100.Alias}) }) t.Run("tablet shutdown removes mysql hostname and port. We shouldn't forget the tablet", func(t *testing.T) { @@ -419,7 +422,7 @@ func TestRefreshTabletsInKeyspaceShard(t *testing.T) { }) tab100.MysqlPort = 100 // We refresh once more to ensure we don't affect the next tests since we've made a change again. - refreshTabletsInKeyspaceShard(ctx, keyspace, shard, func(tabletAlias string) {}, false, nil) + refreshTabletsInKeyspaceShard(ctx, keyspace, shard, func(tabletAlias *topodatapb.TabletAlias) {}, false, nil) }() // Let's assume tab100 restarted on a different pod. This would change its tablet hostname and port _, err = ts.UpdateTabletFields(context.Background(), tab100.Alias, func(tablet *topodatapb.Tablet) error { @@ -441,7 +444,7 @@ func TestRefreshTabletsInKeyspaceShard(t *testing.T) { }) tab101.Type = topodatapb.TabletType_REPLICA // We refresh once more to ensure we don't affect the next tests since we've made a change again. - refreshTabletsInKeyspaceShard(ctx, keyspace, shard, func(tabletAlias string) {}, false, nil) + refreshTabletsInKeyspaceShard(ctx, keyspace, shard, func(tabletAlias *topodatapb.TabletAlias) {}, false, nil) }() // A replica tablet can be converted to drained type if it has an errant GTID. _, err = ts.UpdateTabletFields(context.Background(), tab101.Alias, func(tablet *topodatapb.Tablet) error { @@ -520,11 +523,11 @@ func TestShardPrimary(t *testing.T) { // verifyRefreshTabletsInKeyspaceShard calls refreshTabletsInKeyspaceShard with the forceRefresh parameter provided and verifies that // the number of instances refreshed matches the parameter and all the tablets match the ones provided -func verifyRefreshTabletsInKeyspaceShard(t *testing.T, forceRefresh bool, instanceRefreshRequired int, tablets []*topodatapb.Tablet, tabletsToIgnore []string) { +func verifyRefreshTabletsInKeyspaceShard(t *testing.T, forceRefresh bool, instanceRefreshRequired int, tablets []*topodatapb.Tablet, tabletsToIgnore []*topodatapb.TabletAlias) { var instancesRefreshed atomic.Int32 instancesRefreshed.Store(0) // call refreshTabletsInKeyspaceShard while counting all the instances that are refreshed - refreshTabletsInKeyspaceShard(context.Background(), keyspace, shard, func(string) { + refreshTabletsInKeyspaceShard(context.Background(), keyspace, shard, func(*topodatapb.TabletAlias) { instancesRefreshed.Add(1) }, forceRefresh, tabletsToIgnore) // Verify that all the tablets are present in the database @@ -540,13 +543,17 @@ func verifyRefreshTabletsInKeyspaceShard(t *testing.T, forceRefresh bool, instan // is the same as the one provided or reading it gives the same error as expected func verifyTabletInfo(t *testing.T, tabletWanted *topodatapb.Tablet, errString string) { t.Helper() - tabletAlias := topoproto.TabletAliasString(tabletWanted.Alias) - tablet, err := inst.ReadTablet(tabletAlias) + t.Logf("verifyTabletInfo tabletWanted.Alias: %v", topoproto.TabletAliasString(tabletWanted.Alias)) + tablet, err := inst.ReadTablet(tabletWanted.Alias) + t.Logf("verifyTabletInfo tablet: %v", tablet) if errString != "" { assert.EqualError(t, err, errString) } else { + t.Logf("verifyTabletInfo err: %v", err) + state, _ := inst.GetDatabaseState() + t.Logf("db state: %v", state) assert.NoError(t, err) - assert.EqualValues(t, tabletAlias, topoproto.TabletAliasString(tablet.Alias)) + assert.Equal(t, topoproto.TabletAliasString(tabletWanted.Alias), topoproto.TabletAliasString(tablet.Alias)) diff := cmp.Diff(tablet, tabletWanted, cmp.Comparer(proto.Equal)) assert.Empty(t, diff) } @@ -566,18 +573,18 @@ func verifyTabletCount(t *testing.T, countWanted int) { func TestGetLockAction(t *testing.T) { tests := []struct { - analysedInstance string + analysedInstance *topodatapb.TabletAlias code inst.AnalysisCode want string }{ { - analysedInstance: "zone1-100", + analysedInstance: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, code: inst.DeadPrimary, - want: "VTOrc Recovery for DeadPrimary on zone1-100", + want: "VTOrc Recovery for DeadPrimary on zone1-0000000100", }, { - analysedInstance: "zone1-200", + analysedInstance: &topodatapb.TabletAlias{Cell: "zone1", Uid: 200}, code: inst.ReplicationStopped, - want: "VTOrc Recovery for ReplicationStopped on zone1-200", + want: "VTOrc Recovery for ReplicationStopped on zone1-0000000200", }, } for _, tt := range tests { diff --git a/go/vt/vtorc/logic/topology_recovery.go b/go/vt/vtorc/logic/topology_recovery.go index 25b79e1f1bc..66af7b0504e 100644 --- a/go/vt/vtorc/logic/topology_recovery.go +++ b/go/vt/vtorc/logic/topology_recovery.go @@ -111,7 +111,7 @@ const ( type TopologyRecovery struct { ID int64 AnalysisEntry inst.ReplicationAnalysis - SuccessorAlias string + SuccessorAlias *topodatapb.TabletAlias IsSuccessful bool AllErrors []string RecoveryStartTimestamp string @@ -165,8 +165,8 @@ func initializeTopologyRecoveryPostConfiguration() { config.WaitForConfigurationToBeLoaded() } -func getLockAction(analysedInstance string, code inst.AnalysisCode) string { - return fmt.Sprintf("VTOrc Recovery for %v on %v", code, analysedInstance) +func getLockAction(tabletAlias *topodatapb.TabletAlias, code inst.AnalysisCode) string { + return fmt.Sprintf("VTOrc Recovery for %v on %v", code, topoproto.TabletAliasString(tabletAlias)) } // LockShard locks the keyspace-shard preventing others from performing conflicting actions. @@ -304,7 +304,7 @@ func runEmergencyReparentOp(ctx context.Context, analysisEntry *inst.Replication } if ev != nil && ev.NewPrimary != nil { - promotedReplica, _, _ = inst.ReadInstance(topoproto.TabletAliasString(ev.NewPrimary.Alias)) + promotedReplica, _, _ = inst.ReadInstance(ev.NewPrimary.Alias) } postErsCompletion(topologyRecovery, analysisEntry, recoveryName, promotedReplica) return true, topologyRecovery, err @@ -343,7 +343,7 @@ func checkAndRecoverGenericProblem(ctx context.Context, analysisEntry *inst.Repl } // getCheckAndRecoverFunctionCode gets the recovery function code to use for the given analysis. -func getCheckAndRecoverFunctionCode(analysisCode inst.AnalysisCode, tabletAlias string) recoveryFunction { +func getCheckAndRecoverFunctionCode(analysisCode inst.AnalysisCode, tabletAlias *topodatapb.TabletAlias) recoveryFunction { switch analysisCode { // primary case inst.DeadPrimary, inst.DeadPrimaryAndSomeReplicas, inst.PrimaryDiskStalled, inst.PrimarySemiSyncBlocked: @@ -515,21 +515,22 @@ func executeCheckAndRecoverFunction(analysisEntry *inst.ReplicationAnalysis) (er isActionableRecovery := hasActionableRecovery(checkAndRecoverFunctionCode) analysisEntry.IsActionableRecovery = isActionableRecovery + analyzedInstanceAliasString := topoproto.TabletAliasString(analysisEntry.AnalyzedInstanceAlias) if checkAndRecoverFunctionCode == noRecoveryFunc { logger.Warning("No recovery strategies for problem, aborting recovery") // Unhandled problem type if analysisEntry.Analysis != inst.NoProblem { - if util.ClearToLog("executeCheckAndRecoverFunction", analysisEntry.AnalyzedInstanceAlias) { + if util.ClearToLog("executeCheckAndRecoverFunction", analyzedInstanceAliasString) { logger.Warningf("executeCheckAndRecoverFunction: ignoring analysisEntry that has no action plan: tablet: %+v", - analysisEntry.AnalyzedInstanceAlias) + analyzedInstanceAliasString) } } return nil } // we have a recovery function; its execution still depends on filters if not disabled. - if isActionableRecovery || util.ClearToLog("executeCheckAndRecoverFunction: detection", analysisEntry.AnalyzedInstanceAlias) { - logger.Infof("executeCheckAndRecoverFunction: proceeding with %+v detection on %+v; isActionable?: %+v", analysisEntry.Analysis, analysisEntry.AnalyzedInstanceAlias, isActionableRecovery) + if isActionableRecovery || util.ClearToLog("executeCheckAndRecoverFunction: detection", analyzedInstanceAliasString) { + logger.Infof("executeCheckAndRecoverFunction: proceeding with %+v detection on %+v; isActionable?: %+v", analysisEntry.Analysis, analyzedInstanceAliasString, isActionableRecovery) } // At this point we have validated there's a failure scenario for which we have a recovery path. @@ -546,7 +547,7 @@ func executeCheckAndRecoverFunction(analysisEntry *inst.ReplicationAnalysis) (er logger.Errorf("Unable to determine if recovery is disabled globally, still attempting to recover: %v", err) } else if recoveryDisabledGlobally { logger.Infof("CheckAndRecover: Tablet: %+v: NOT Recovering host (disabled globally)", - analysisEntry.AnalyzedInstanceAlias) + analyzedInstanceAliasString) return err } @@ -580,7 +581,7 @@ func executeCheckAndRecoverFunction(analysisEntry *inst.ReplicationAnalysis) (er // changes, we should be checking that this failure is indeed needed to be fixed. We do this after locking the shard to be sure // that the data that we use now is up-to-date. if isActionableRecovery { - logger.Infof("executeCheckAndRecoverFunction: Proceeding with %v recovery on %v validation after acquiring shard lock.", analysisEntry.Analysis, analysisEntry.AnalyzedInstanceAlias) + logger.Infof("executeCheckAndRecoverFunction: Proceeding with %v recovery on %v validation after acquiring shard lock.", analysisEntry.Analysis, analyzedInstanceAliasString) // The first step we have to do is refresh the keyspace and shard information // This is required to know if the durability policies have changed or not // If they have, then recoveries like ReplicaSemiSyncMustNotBeSet, etc won't be valid anymore. @@ -595,7 +596,7 @@ func executeCheckAndRecoverFunction(analysisEntry *inst.ReplicationAnalysis) (er // of a shard because a new tablet could have been promoted, and we need to have this visibility before we // run a cluster operation of our own. if isClusterWideRecovery(checkAndRecoverFunctionCode) { - var tabletsToIgnore []string + tabletsToIgnore := make([]*topodatapb.TabletAlias, 0) if checkAndRecoverFunctionCode == recoverDeadPrimaryFunc { tabletsToIgnore = append(tabletsToIgnore, analysisEntry.AnalyzedInstanceAlias) } @@ -618,15 +619,14 @@ func executeCheckAndRecoverFunction(analysisEntry *inst.ReplicationAnalysis) (er primaryTablet, err := shardPrimary(analysisEntry.AnalyzedKeyspace, analysisEntry.AnalyzedShard) if err != nil { logger.Errorf("executeCheckAndRecoverFunction: Tablet: %+v: error while finding the shard primary: %v", - analysisEntry.AnalyzedInstanceAlias, err) + analyzedInstanceAliasString, err) return err } - primaryTabletAlias := topoproto.TabletAliasString(primaryTablet.Alias) // We can skip the refresh if we know the tablet we are looking at is the primary tablet. // This would be the case for PrimaryHasPrimary recovery. We don't need to refresh the same tablet twice. - if analysisEntry.AnalyzedInstanceAlias != primaryTabletAlias { + if !topoproto.TabletAliasEqual(analysisEntry.AnalyzedInstanceAlias, primaryTablet.Alias) { logger.Info("Discovering primary instance") - DiscoverInstance(primaryTabletAlias, true) + DiscoverInstance(primaryTablet.Alias, true) } } alreadyFixed, err := checkIfAlreadyFixed(analysisEntry) @@ -636,14 +636,14 @@ func executeCheckAndRecoverFunction(analysisEntry *inst.ReplicationAnalysis) (er return err } if alreadyFixed { - logger.Infof("Analysis: %v on tablet %v - No longer valid, some other agent must have fixed the problem.", analysisEntry.Analysis, analysisEntry.AnalyzedInstanceAlias) + logger.Infof("Analysis: %v on tablet %v - No longer valid, some other agent must have fixed the problem.", analysisEntry.Analysis, analyzedInstanceAliasString) return nil } } // Actually attempt recovery: - if isActionableRecovery || util.ClearToLog("executeCheckAndRecoverFunction: recovery", analysisEntry.AnalyzedInstanceAlias) { - logger.Infof("executeCheckAndRecoverFunction: proceeding with recovery on %+v; isRecoverable?: %+v", analysisEntry.AnalyzedInstanceAlias, isActionableRecovery) + if isActionableRecovery || util.ClearToLog("executeCheckAndRecoverFunction: recovery", analyzedInstanceAliasString) { + logger.Infof("executeCheckAndRecoverFunction: proceeding with recovery on %+v; isRecoverable?: %+v", analyzedInstanceAliasString, isActionableRecovery) } recoveryAttempted, topologyRecovery, err := getCheckAndRecoverFunction(checkAndRecoverFunctionCode)(ctx, analysisEntry, logger) if !recoveryAttempted { @@ -688,7 +688,7 @@ func executeCheckAndRecoverFunction(analysisEntry *inst.ReplicationAnalysis) (er // recheckPrimaryHealth check the health of the primary node. // It then checks whether, given the re-discovered primary health, the original recovery is still valid. // If not valid then it will abort the current analysis. -func recheckPrimaryHealth(analysisEntry *inst.ReplicationAnalysis, discoveryFunc func(string, bool)) error { +func recheckPrimaryHealth(analysisEntry *inst.ReplicationAnalysis, discoveryFunc func(*topodatapb.TabletAlias, bool)) error { originalAnalysisEntry := analysisEntry.Analysis primaryTabletAlias := analysisEntry.AnalyzedInstancePrimaryAlias @@ -707,7 +707,7 @@ func recheckPrimaryHealth(analysisEntry *inst.ReplicationAnalysis, discoveryFunc // This could mean that either the original analysis has changed or some other Vtorc instance has already performing the mitigation. // In either case, the original analysis is stale which can be safely aborted. if recoveryRequired { - log.Infof("recheckPrimaryHealth: Primary recovery is required, Tablet alias: %v", primaryTabletAlias) + log.Infof("recheckPrimaryHealth: Primary recovery is required, Tablet alias: %v", topoproto.TabletAliasString(primaryTabletAlias)) // original analysis is stale, abort. return fmt.Errorf("aborting %s, primary mitigation is required", originalAnalysisEntry) } @@ -725,7 +725,8 @@ func checkIfAlreadyFixed(analysisEntry *inst.ReplicationAnalysis) (bool, error) for _, entry := range analysisEntries { // If there is a analysis which has the same recovery required, then we should proceed with the recovery - if entry.AnalyzedInstanceAlias == analysisEntry.AnalyzedInstanceAlias && analysisEntriesHaveSameRecovery(analysisEntry, entry) { + tabletAliasesEqual := topoproto.TabletAliasEqual(entry.AnalyzedInstanceAlias, analysisEntry.AnalyzedInstanceAlias) + if tabletAliasesEqual && analysisEntriesHaveSameRecovery(analysisEntry, entry) { return false, nil } } @@ -751,7 +752,7 @@ func CheckAndRecover() { if e.Analysis != inst.NoProblem { names := [...]string{ string(e.Analysis), - e.AnalyzedInstanceAlias, + topoproto.TabletAliasString(e.AnalyzedInstanceAlias), e.AnalyzedKeyspace, e.AnalyzedShard, } @@ -783,10 +784,11 @@ func CheckAndRecover() { func postPrsCompletion(topologyRecovery *TopologyRecovery, analysisEntry *inst.ReplicationAnalysis, promotedReplica *inst.Instance) { if promotedReplica != nil { - message := fmt.Sprintf("promoted replica: %+v", promotedReplica.InstanceAlias) + tabletAliasString := topoproto.TabletAliasString(promotedReplica.InstanceAlias) + message := fmt.Sprintf("promoted replica: %+v", tabletAliasString) _ = AuditTopologyRecovery(topologyRecovery, message) _ = inst.AuditOperation(string(analysisEntry.Analysis), analysisEntry.AnalyzedInstanceAlias, message) - _ = AuditTopologyRecovery(topologyRecovery, fmt.Sprintf("%+v: successfully promoted %+v", analysisEntry.Analysis, promotedReplica.InstanceAlias)) + _ = AuditTopologyRecovery(topologyRecovery, fmt.Sprintf("%+v: successfully promoted %+v", analysisEntry.Analysis, tabletAliasString)) } } @@ -836,7 +838,7 @@ func electNewPrimary(ctx context.Context, analysisEntry *inst.ReplicationAnalysi ) if ev != nil && ev.NewPrimary != nil { - promotedReplica, _, _ = inst.ReadInstance(topoproto.TabletAliasString(ev.NewPrimary.Alias)) + promotedReplica, _, _ = inst.ReadInstance(ev.NewPrimary.Alias) } postPrsCompletion(topologyRecovery, analysisEntry, promotedReplica) return true, topologyRecovery, err diff --git a/go/vt/vtorc/logic/topology_recovery_dao.go b/go/vt/vtorc/logic/topology_recovery_dao.go index 137251c4fc8..15d9589e433 100644 --- a/go/vt/vtorc/logic/topology_recovery_dao.go +++ b/go/vt/vtorc/logic/topology_recovery_dao.go @@ -23,6 +23,7 @@ import ( "vitess.io/vitess/go/vt/external/golib/sqlutils" "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtorc/config" "vitess.io/vitess/go/vt/vtorc/db" "vitess.io/vitess/go/vt/vtorc/inst" @@ -44,7 +45,7 @@ func InsertRecoveryDetection(analysisEntry *inst.ReplicationAnalysis) error { ?, DATETIME('now') )`, - analysisEntry.AnalyzedInstanceAlias, + topoproto.TabletAliasString(analysisEntry.AnalyzedInstanceAlias), string(analysisEntry.Analysis), analysisEntry.ClusterDetails.Keyspace, analysisEntry.ClusterDetails.Shard, @@ -83,11 +84,11 @@ func writeTopologyRecovery(topologyRecovery *TopologyRecovery) (*TopologyRecover ? )`, sqlutils.NilIfZero(topologyRecovery.ID), - analysisEntry.AnalyzedInstanceAlias, + topoproto.TabletAliasString(analysisEntry.AnalyzedInstanceAlias), string(analysisEntry.Analysis), analysisEntry.ClusterDetails.Keyspace, analysisEntry.ClusterDetails.Shard, - analysisEntry.AnalyzedInstanceAlias, + topoproto.TabletAliasString(analysisEntry.AnalyzedInstanceAlias), analysisEntry.RecoveryId, ) if err != nil { @@ -145,7 +146,7 @@ func writeResolveRecovery(topologyRecovery *TopologyRecovery) error { recovery_id = ? `, topologyRecovery.IsSuccessful, - topologyRecovery.SuccessorAlias, + topoproto.TabletAliasString(topologyRecovery.SuccessorAlias), strings.Join(topologyRecovery.AllErrors, "\n"), topologyRecovery.ID, ) @@ -179,7 +180,7 @@ func readRecoveries(whereCondition string, limit string, args []any) ([]*Topolog whereCondition, limit, ) - err := db.QueryVTOrc(query, args, func(m sqlutils.RowMap) error { + err := db.QueryVTOrc(query, args, func(m sqlutils.RowMap) (err error) { topologyRecovery := *NewTopologyRecovery(inst.ReplicationAnalysis{}) topologyRecovery.ID = m.GetInt64("recovery_id") @@ -187,12 +188,21 @@ func readRecoveries(whereCondition string, limit string, args []any) ([]*Topolog topologyRecovery.RecoveryEndTimestamp = m.GetString("end_recovery") topologyRecovery.IsSuccessful = m.GetBool("is_successful") - topologyRecovery.AnalysisEntry.AnalyzedInstanceAlias = m.GetString("alias") + topologyRecovery.AnalysisEntry.AnalyzedInstanceAlias, err = topoproto.ParseTabletAlias(m.GetString("alias")) + if err != nil { + return err + } + topologyRecovery.AnalysisEntry.Analysis = inst.AnalysisCode(m.GetString("analysis")) topologyRecovery.AnalysisEntry.ClusterDetails.Keyspace = m.GetString("keyspace") topologyRecovery.AnalysisEntry.ClusterDetails.Shard = m.GetString("shard") - topologyRecovery.SuccessorAlias = m.GetString("successor_alias") + if successorAlias := m.GetString("successor_alias"); successorAlias != "" { + topologyRecovery.SuccessorAlias, err = topoproto.ParseTabletAlias(successorAlias) + if err != nil { + return err + } + } topologyRecovery.AllErrors = strings.Split(m.GetString("all_errors"), "\n") diff --git a/go/vt/vtorc/logic/topology_recovery_dao_test.go b/go/vt/vtorc/logic/topology_recovery_dao_test.go index 6a1d7c4c48f..2d1cb530bf6 100644 --- a/go/vt/vtorc/logic/topology_recovery_dao_test.go +++ b/go/vt/vtorc/logic/topology_recovery_dao_test.go @@ -23,6 +23,8 @@ import ( "github.com/stretchr/testify/require" "vitess.io/vitess/go/vt/external/golib/sqlutils" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtorc/config" "vitess.io/vitess/go/vt/vtorc/db" "vitess.io/vitess/go/vt/vtorc/inst" @@ -40,7 +42,7 @@ func TestTopologyRecovery(t *testing.T) { }() replicationAnalysis := inst.ReplicationAnalysis{ - AnalyzedInstanceAlias: "zone1-0000000101", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 101}, TabletType: tab101.Type, ClusterDetails: inst.ClusterInfo{ Keyspace: keyspace, @@ -143,7 +145,7 @@ func TestInsertRecoveryDetection(t *testing.T) { db.ClearVTOrcDatabase() }() ra := &inst.ReplicationAnalysis{ - AnalyzedInstanceAlias: "alias-1", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 1}, Analysis: inst.ClusterHasNoPrimary, ClusterDetails: inst.ClusterInfo{ Keyspace: keyspace, @@ -161,7 +163,9 @@ func TestInsertRecoveryDetection(t *testing.T) { }) require.NoError(t, err) require.Len(t, rows, 1) - require.EqualValues(t, ra.AnalyzedInstanceAlias, rows[0]["alias"].String) + tabletAlias, err := topoproto.ParseTabletAlias(rows[0]["alias"].String) + require.NoError(t, err) + require.EqualValues(t, ra.AnalyzedInstanceAlias, tabletAlias) require.EqualValues(t, ra.Analysis, rows[0]["analysis"].String) require.EqualValues(t, keyspace, rows[0]["keyspace"].String) require.EqualValues(t, shard, rows[0]["shard"].String) diff --git a/go/vt/vtorc/logic/topology_recovery_test.go b/go/vt/vtorc/logic/topology_recovery_test.go index 6f967f89051..f467b0af3fd 100644 --- a/go/vt/vtorc/logic/topology_recovery_test.go +++ b/go/vt/vtorc/logic/topology_recovery_test.go @@ -27,7 +27,6 @@ import ( "vitess.io/vitess/go/vt/external/golib/sqlutils" topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/topo/memorytopo" - "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtctl/reparentutil/policy" "vitess.io/vitess/go/vt/vtorc/config" "vitess.io/vitess/go/vt/vtorc/db" @@ -139,7 +138,7 @@ func TestElectNewPrimaryPanic(t *testing.T) { err = inst.SaveTablet(tablet) require.NoError(t, err) analysisEntry := &inst.ReplicationAnalysis{ - AnalyzedInstanceAlias: topoproto.TabletAliasString(tablet.Alias), + AnalyzedInstanceAlias: tablet.Alias, } ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -189,11 +188,11 @@ func TestRecoveryRegistration(t *testing.T) { err = inst.SaveTablet(replica) require.NoError(t, err) primaryAnalysisEntry := inst.ReplicationAnalysis{ - AnalyzedInstanceAlias: topoproto.TabletAliasString(primary.Alias), + AnalyzedInstanceAlias: primary.Alias, Analysis: inst.ReplicationStopped, } replicaAnalysisEntry := inst.ReplicationAnalysis{ - AnalyzedInstanceAlias: topoproto.TabletAliasString(replica.Alias), + AnalyzedInstanceAlias: replica.Alias, Analysis: inst.DeadPrimary, } ctx, cancel := context.WithCancel(context.Background()) @@ -309,7 +308,7 @@ func TestGetCheckAndRecoverFunctionCode(t *testing.T) { config.SetConvertTabletWithErrantGTIDs(tt.convertTabletWithErrantGTIDs) defer config.SetConvertTabletWithErrantGTIDs(convertErrantVal) - gotFunc := getCheckAndRecoverFunctionCode(tt.analysisCode, "") + gotFunc := getCheckAndRecoverFunctionCode(tt.analysisCode, nil) require.EqualValues(t, tt.wantRecoveryFunction, gotFunc) }) } @@ -325,7 +324,7 @@ func TestRecheckPrimaryHealth(t *testing.T) { name: "analysis change", info: []*test.InfoForRecoveryAnalysis{{ TabletInfo: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{Cell: "zon1", Uid: 100}, + Alias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, Hostname: "localhost", Keyspace: "ks", Shard: "0", @@ -345,7 +344,7 @@ func TestRecheckPrimaryHealth(t *testing.T) { name: "analysis did not change", info: []*test.InfoForRecoveryAnalysis{{ TabletInfo: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{Cell: "zon1", Uid: 101}, + Alias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 101}, Hostname: "localhost", Keyspace: "ks", Shard: "0", @@ -364,7 +363,7 @@ func TestRecheckPrimaryHealth(t *testing.T) { CurrentTabletType: int(topodatapb.TabletType_PRIMARY), }, { TabletInfo: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{Cell: "zon1", Uid: 100}, + Alias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, Hostname: "localhost", Keyspace: "ks", Shard: "0", @@ -374,7 +373,7 @@ func TestRecheckPrimaryHealth(t *testing.T) { }, DurabilityPolicy: policy.DurabilityNone, PrimaryTabletInfo: &topodatapb.Tablet{ - Alias: &topodatapb.TabletAlias{Cell: "zon1", Uid: 101}, + Alias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 101}, }, LastCheckValid: 1, ReadOnly: 1, @@ -401,11 +400,11 @@ func TestRecheckPrimaryHealth(t *testing.T) { db.Db = test.NewTestDB([][]sqlutils.RowMap{rowMaps}) err := recheckPrimaryHealth(&inst.ReplicationAnalysis{ - AnalyzedInstanceAlias: "zon1-0000000100", + AnalyzedInstanceAlias: &topodatapb.TabletAlias{Cell: "zone1", Uid: 100}, Analysis: inst.ReplicationStopped, AnalyzedKeyspace: "ks", AnalyzedShard: "0", - }, func(s string, b bool) { + }, func(*topodatapb.TabletAlias, bool) { // the implementation for DiscoverInstance is not required because we are mocking the db response. }) diff --git a/go/vt/vtorc/logic/vtorc.go b/go/vt/vtorc/logic/vtorc.go index 45568cd745c..df64b1fb5f3 100644 --- a/go/vt/vtorc/logic/vtorc.go +++ b/go/vt/vtorc/logic/vtorc.go @@ -28,7 +28,9 @@ import ( "vitess.io/vitess/go/stats" "vitess.io/vitess/go/vt/log" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/servenv" + "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtorc/collection" "vitess.io/vitess/go/vt/vtorc/config" "vitess.io/vitess/go/vt/vtorc/discovery" @@ -46,8 +48,8 @@ const ( // that were requested for discovery. It can be continuously updated // as discovery process progresses. var discoveryQueue *discovery.Queue -var snapshotDiscoveryKeys chan string -var snapshotDiscoveryKeysMutex sync.Mutex +var snapshotDiscoveryAliases chan *topodatapb.TabletAlias +var snapshotDiscoveryAliasesMutex sync.Mutex var hasReceivedSIGTERM int32 var ( @@ -68,7 +70,7 @@ var discoveryMetrics = collection.CreateOrReturnCollection(DiscoveryMetricsName) var recentDiscoveryOperationKeys *cache.Cache func init() { - snapshotDiscoveryKeys = make(chan string, 10) + snapshotDiscoveryAliases = make(chan *topodatapb.TabletAlias, 10) ometrics.OnMetricsTick(func() { discoveryQueueLengthGauge.Set(int64(discoveryQueue.QueueLen())) @@ -87,7 +89,7 @@ func closeVTOrc() { atomic.StoreInt32(&hasReceivedSIGTERM, 1) discoveryMetrics.StopAutoExpiration() // Poke other go routines to stop cleanly here ... - _ = inst.AuditOperation("shutdown", "", "Triggered via SIGTERM") + _ = inst.AuditOperation("shutdown", nil, "Triggered via SIGTERM") // wait for the locks to be released waitForLocksRelease() ts.Close() @@ -140,9 +142,10 @@ func handleDiscoveryRequests() { // DiscoverInstance will attempt to discover (poll) an instance (unless // it is already up-to-date) and will also ensure that its primary and // replicas (if any) are also checked. -func DiscoverInstance(tabletAlias string, forceDiscovery bool) { +func DiscoverInstance(tabletAlias *topodatapb.TabletAlias, forceDiscovery bool) { + tabletAliasString := topoproto.TabletAliasString(tabletAlias) if inst.InstanceIsForgotten(tabletAlias) { - log.Infof("discoverInstance: skipping discovery of %+v because it is set to be forgotten", tabletAlias) + log.Infof("discoverInstance: skipping discovery of %+v because it is set to be forgotten", tabletAliasString) return } @@ -159,21 +162,21 @@ func DiscoverInstance(tabletAlias string, forceDiscovery bool) { discoveryTime := latency.Elapsed("total") if discoveryTime > config.GetInstancePollTime() { instancePollSecondsExceededCounter.Add(1) - log.Warningf("discoverInstance exceeded InstancePollSeconds for %+v, took %.4fs", tabletAlias, discoveryTime.Seconds()) + log.Warningf("discoverInstance exceeded InstancePollSeconds for %+v, took %.4fs", tabletAliasString, discoveryTime.Seconds()) if metric != nil { metric.InstancePollSecondsDurationCount = 1 } } }() - if tabletAlias == "" { + if tabletAlias == nil { return } // Calculate the expiry period each time as InstancePollSeconds // _may_ change during the run of the process (via SIGHUP) and // it is not possible to change the cache's default expiry.. - if existsInCacheError := recentDiscoveryOperationKeys.Add(tabletAlias, true, config.GetInstancePollTime()); existsInCacheError != nil && !forceDiscovery { + if existsInCacheError := recentDiscoveryOperationKeys.Add(tabletAliasString, true, config.GetInstancePollTime()); existsInCacheError != nil && !forceDiscovery { // Just recently attempted return } @@ -209,16 +212,16 @@ func DiscoverInstance(tabletAlias string, forceDiscovery bool) { failedDiscoveriesCounter.Add(1) metric = &discovery.Metric{ Timestamp: time.Now(), - TabletAlias: tabletAlias, + TabletAlias: tabletAliasString, TotalLatency: totalLatency, BackendLatency: backendLatency, InstanceLatency: instanceLatency, Err: err, } _ = discoveryMetrics.Append(metric) - if util.ClearToLog("discoverInstance", tabletAlias) { + if util.ClearToLog("discoverInstance", tabletAliasString) { log.Warningf(" DiscoverInstance(%+v) instance is nil in %.3fs (Backend: %.3fs, Instance: %.3fs), error=%+v", - tabletAlias, + tabletAliasString, totalLatency.Seconds(), backendLatency.Seconds(), instanceLatency.Seconds(), @@ -229,7 +232,7 @@ func DiscoverInstance(tabletAlias string, forceDiscovery bool) { metric = &discovery.Metric{ Timestamp: time.Now(), - TabletAlias: tabletAlias, + TabletAlias: tabletAliasString, TotalLatency: totalLatency, BackendLatency: backendLatency, InstanceLatency: instanceLatency, @@ -240,7 +243,7 @@ func DiscoverInstance(tabletAlias string, forceDiscovery bool) { // onHealthTick handles the actions to take to discover/poll instances func onHealthTick() { - tabletAliases, err := inst.ReadOutdatedInstanceKeys() + tabletAliases, err := inst.ReadOutdatedInstanceAliases() if err != nil { log.Error(err) } @@ -248,21 +251,17 @@ func onHealthTick() { func() { // Normally onHealthTick() shouldn't run concurrently. It is kicked by a ticker. // However it _is_ invoked inside a goroutine. I like to be safe here. - snapshotDiscoveryKeysMutex.Lock() - defer snapshotDiscoveryKeysMutex.Unlock() + snapshotDiscoveryAliasesMutex.Lock() + defer snapshotDiscoveryAliasesMutex.Unlock() - countSnapshotKeys := len(snapshotDiscoveryKeys) - for i := 0; i < countSnapshotKeys; i++ { - tabletAliases = append(tabletAliases, <-snapshotDiscoveryKeys) + countSnapshotAliases := len(snapshotDiscoveryAliases) + for i := 0; i < countSnapshotAliases; i++ { + tabletAliases = append(tabletAliases, <-snapshotDiscoveryAliases) } }() - // avoid any logging unless there's something to be done - if len(tabletAliases) > 0 { - for _, tabletAlias := range tabletAliases { - if tabletAlias != "" { - discoveryQueue.Push(tabletAlias) - } - } + + for _, tabletAlias := range tabletAliases { + discoveryQueue.Push(tabletAlias) } } From 45041e026b0f9972a749eb377e913f1244683fbe Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Sun, 22 Jun 2025 20:12:20 +0200 Subject: [PATCH 02/16] rename func Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/inst/instance_dao.go | 4 ++-- go/vt/vtorc/inst/instance_dao_test.go | 6 +++--- go/vt/vtorc/logic/vtorc.go | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/go/vt/vtorc/inst/instance_dao.go b/go/vt/vtorc/inst/instance_dao.go index 91fa43f4aa5..6ff7db1197c 100644 --- a/go/vt/vtorc/inst/instance_dao.go +++ b/go/vt/vtorc/inst/instance_dao.go @@ -749,7 +749,7 @@ func GetKeyspaceShardName(tabletAlias *topodatapb.TabletAlias) (keyspace, shard return keyspace, shard, err } -// ReadOutdatedInstanceAliases reads and returns tablet aliases for all instances that are not up to date (i.e. +// ReadOutdatedInstances reads and returns tablet aliases for all instances that are not up to date (i.e. // pre-configured time has passed since they were last checked) or the ones whose tablet information was read // but not the mysql information. This could happen if the durability policy of the keyspace wasn't // available at the time it was discovered. This would lead to not having the record of the tablet in the @@ -758,7 +758,7 @@ func GetKeyspaceShardName(tabletAlias *topodatapb.TabletAlias) (keyspace, shard // resulted in an actual check! This can happen when TCP/IP connections are hung, in which case the "check" // never returns. In such case we multiply interval by a factor, so as not to open too many connections on // the instance. -func ReadOutdatedInstanceAliases() ([]*topodatapb.TabletAlias, error) { +func ReadOutdatedInstances() ([]*topodatapb.TabletAlias, error) { res := make([]*topodatapb.TabletAlias, 0) query := `SELECT alias diff --git a/go/vt/vtorc/inst/instance_dao_test.go b/go/vt/vtorc/inst/instance_dao_test.go index 84fba38f057..dbcf9809a83 100644 --- a/go/vt/vtorc/inst/instance_dao_test.go +++ b/go/vt/vtorc/inst/instance_dao_test.go @@ -530,8 +530,8 @@ func TestReadInstancesByCondition(t *testing.T) { } } -// TestReadOutdatedInstanceAliases is used to test the functionality of ReadOutdatedInstanceAliases and verify its failure modes and successes. -func TestReadOutdatedInstanceAliases(t *testing.T) { +// TestReadOutdatedInstances is used to test the functionality of ReadOutdatedInstances and verify its failure modes and successes. +func TestReadOutdatedInstances(t *testing.T) { // The test is intended to be used as follows. The initial data is stored into the database. Following this, some specific queries are run that each individual test specifies to get the desired state. tests := []struct { name string @@ -592,7 +592,7 @@ func TestReadOutdatedInstanceAliases(t *testing.T) { require.NoError(t, err) } - tabletAliases, err := ReadOutdatedInstanceAliases() + tabletAliases, err := ReadOutdatedInstances() errInDataCollection := db.QueryVTOrcRowsMap(`select alias, last_checked, diff --git a/go/vt/vtorc/logic/vtorc.go b/go/vt/vtorc/logic/vtorc.go index df64b1fb5f3..ce870104a8b 100644 --- a/go/vt/vtorc/logic/vtorc.go +++ b/go/vt/vtorc/logic/vtorc.go @@ -243,7 +243,7 @@ func DiscoverInstance(tabletAlias *topodatapb.TabletAlias, forceDiscovery bool) // onHealthTick handles the actions to take to discover/poll instances func onHealthTick() { - tabletAliases, err := inst.ReadOutdatedInstanceAliases() + tabletAliases, err := inst.ReadOutdatedInstances() if err != nil { log.Error(err) } From 032b6d33a49181fe72abcc0e8e9f94775231a861 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Sun, 22 Jun 2025 20:27:02 +0200 Subject: [PATCH 03/16] fix e2e test Signed-off-by: Tim Vaillancourt --- .../endtoend/vtorc/readtopologyinstance/main_test.go | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/go/test/endtoend/vtorc/readtopologyinstance/main_test.go b/go/test/endtoend/vtorc/readtopologyinstance/main_test.go index 581f37c4684..d5e383c1b0c 100644 --- a/go/test/endtoend/vtorc/readtopologyinstance/main_test.go +++ b/go/test/endtoend/vtorc/readtopologyinstance/main_test.go @@ -26,7 +26,7 @@ import ( "vitess.io/vitess/go/test/endtoend/vtorc/utils" topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/servenv" - + "vitess.io/vitess/go/vt/topo/topoproto" vtutils "vitess.io/vitess/go/vt/utils" "vitess.io/vitess/go/vt/vtorc/config" "vitess.io/vitess/go/vt/vtorc/inst" @@ -82,7 +82,13 @@ func TestReadTopologyInstanceBufferable(t *testing.T) { } } - primaryInstance, err := inst.ReadTopologyInstanceBufferable(primary.Alias, nil) + primaryAlias, err := topoproto.ParseTabletAlias(primary.Alias) + require.NoError(t, err) + + replicaAlias, err := topoproto.ParseTabletAlias(replica.Alias) + require.NoError(t, err) + + primaryInstance, err := inst.ReadTopologyInstanceBufferable(primaryAlias, nil) require.NoError(t, err) require.NotNil(t, primaryInstance) assert.Equal(t, utils.Hostname, primaryInstance.Hostname) @@ -134,7 +140,7 @@ func TestReadTopologyInstanceBufferable(t *testing.T) { err = logic.EnableRecovery() require.NoError(t, err) - replicaInstance, err := inst.ReadTopologyInstanceBufferable(replica.Alias, nil) + replicaInstance, err := inst.ReadTopologyInstanceBufferable(replicaAlias, nil) require.NoError(t, err) require.NotNil(t, replicaInstance) assert.Equal(t, utils.Hostname, replicaInstance.Hostname) From d490762e7a48ec059c932c7643127c639ce7ae05 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Mon, 23 Jun 2025 02:55:31 +0200 Subject: [PATCH 04/16] reorder Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/discovery/queue.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/go/vt/vtorc/discovery/queue.go b/go/vt/vtorc/discovery/queue.go index 57f2608d755..5a85074c9b0 100644 --- a/go/vt/vtorc/discovery/queue.go +++ b/go/vt/vtorc/discovery/queue.go @@ -80,7 +80,10 @@ func (q *Queue) QueueLen() int { // Push enqueues a tablet alias if it is not on a queue and is not being // processed; silently returns otherwise. func (q *Queue) Push(tabletAlias *topodatapb.TabletAlias) { - if q.checkAndSetEnqueued(tabletAlias) || tabletAlias == nil { + if tabletAlias == nil { + return + } + if q.checkAndSetEnqueued(tabletAlias) { return } q.queue <- queueItem{ From 96e14567bcbede2b2c8159fb4588c99d5c4d3d76 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Mon, 23 Jun 2025 03:07:13 +0200 Subject: [PATCH 05/16] more test cleanup Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/inst/analysis_dao.go | 22 ++++++++++++++++++---- go/vt/vtorc/inst/audit_dao.go | 3 +-- go/vt/vtorc/inst/instance_dao.go | 4 ++-- go/vt/vtorc/logic/tablet_discovery_test.go | 5 ----- 4 files changed, 21 insertions(+), 13 deletions(-) diff --git a/go/vt/vtorc/inst/analysis_dao.go b/go/vt/vtorc/inst/analysis_dao.go index a26034b6d69..68f3487f5b2 100644 --- a/go/vt/vtorc/inst/analysis_dao.go +++ b/go/vt/vtorc/inst/analysis_dao.go @@ -367,7 +367,17 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna if !a.LastCheckValid { analysisMessage := fmt.Sprintf("analysis: Alias: %+v, Keyspace: %+v, Shard: %+v, IsPrimary: %+v, LastCheckValid: %+v, LastCheckPartialSuccess: %+v, CountReplicas: %+v, CountValidReplicas: %+v, CountValidReplicatingReplicas: %+v, CountLaggingReplicas: %+v, CountDelayedReplicas: %+v", - a.AnalyzedInstanceAlias, a.ClusterDetails.Keyspace, a.ClusterDetails.Shard, a.IsPrimary, a.LastCheckValid, a.LastCheckPartialSuccess, a.CountReplicas, a.CountValidReplicas, a.CountValidReplicatingReplicas, a.CountLaggingReplicas, a.CountDelayedReplicas, + topoproto.TabletAliasString(a.AnalyzedInstanceAlias), + a.ClusterDetails.Keyspace, + a.ClusterDetails.Shard, + a.IsPrimary, + a.LastCheckValid, + a.LastCheckPartialSuccess, + a.CountReplicas, + a.CountValidReplicas, + a.CountValidReplicatingReplicas, + a.CountLaggingReplicas, + a.CountDelayedReplicas, ) if util.ClearToLog("analysis_dao", analysisMessage) { log.Infof(analysisMessage) @@ -673,7 +683,9 @@ func auditInstanceAnalysisInChangelog(tabletAlias *topodatapb.TabletAlias, analy alias = ? AND analysis != ? `, - string(analysisCode), tabletAliasString, string(analysisCode), + string(analysisCode), + tabletAliasString, + string(analysisCode), ) if err != nil { log.Error(err) @@ -702,7 +714,8 @@ func auditInstanceAnalysisInChangelog(tabletAlias *topodatapb.TabletAlias, analy DATETIME('now'), ? )`, - tabletAliasString, string(analysisCode), + tabletAliasString, + string(analysisCode), ) if err != nil { log.Error(err) @@ -731,7 +744,8 @@ func auditInstanceAnalysisInChangelog(tabletAlias *topodatapb.TabletAlias, analy DATETIME('now'), ? )`, - tabletAliasString, string(analysisCode), + tabletAliasString, + string(analysisCode), ) if err == nil { analysisChangeWriteCounter.Add(1) diff --git a/go/vt/vtorc/inst/audit_dao.go b/go/vt/vtorc/inst/audit_dao.go index ed8463799a5..0d75c87769d 100644 --- a/go/vt/vtorc/inst/audit_dao.go +++ b/go/vt/vtorc/inst/audit_dao.go @@ -33,8 +33,7 @@ var auditOperationCounter = stats.NewCounter("AuditWrite", "Number of audit oper // AuditOperation creates and writes a new audit entry by given params func AuditOperation(auditType string, tabletAlias *topodatapb.TabletAlias, message string) error { - keyspace := "" - shard := "" + var keyspace, shard string if tabletAlias != nil { keyspace, shard, _ = GetKeyspaceShardName(tabletAlias) } diff --git a/go/vt/vtorc/inst/instance_dao.go b/go/vt/vtorc/inst/instance_dao.go index 6ff7db1197c..0e9c801f17c 100644 --- a/go/vt/vtorc/inst/instance_dao.go +++ b/go/vt/vtorc/inst/instance_dao.go @@ -1212,8 +1212,8 @@ func ExpireStaleInstanceBinlogCoordinates() error { expireSeconds = config.StaleInstanceCoordinatesExpireSeconds } writeFunc := func() error { - _, err := db.ExecVTOrc(`DELETE - FROM database_instance_stale_binlog_coordinates + _, err := db.ExecVTOrc(`DELETE FROM + database_instance_stale_binlog_coordinates WHERE first_seen < DATETIME('now', PRINTF('-%d SECOND', ?)) `, diff --git a/go/vt/vtorc/logic/tablet_discovery_test.go b/go/vt/vtorc/logic/tablet_discovery_test.go index e19cd8e9a9d..d8df867ea50 100644 --- a/go/vt/vtorc/logic/tablet_discovery_test.go +++ b/go/vt/vtorc/logic/tablet_discovery_test.go @@ -543,15 +543,10 @@ func verifyRefreshTabletsInKeyspaceShard(t *testing.T, forceRefresh bool, instan // is the same as the one provided or reading it gives the same error as expected func verifyTabletInfo(t *testing.T, tabletWanted *topodatapb.Tablet, errString string) { t.Helper() - t.Logf("verifyTabletInfo tabletWanted.Alias: %v", topoproto.TabletAliasString(tabletWanted.Alias)) tablet, err := inst.ReadTablet(tabletWanted.Alias) - t.Logf("verifyTabletInfo tablet: %v", tablet) if errString != "" { assert.EqualError(t, err, errString) } else { - t.Logf("verifyTabletInfo err: %v", err) - state, _ := inst.GetDatabaseState() - t.Logf("db state: %v", state) assert.NoError(t, err) assert.Equal(t, topoproto.TabletAliasString(tabletWanted.Alias), topoproto.TabletAliasString(tablet.Alias)) diff := cmp.Diff(tablet, tabletWanted, cmp.Comparer(proto.Equal)) From 8fbfdd33b93e6b4a54263f6f7ce7296260068c20 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Sat, 28 Jun 2025 00:37:51 +0200 Subject: [PATCH 06/16] cleanup Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/inst/tablet_dao_test.go | 4 ++-- go/vt/vtorc/logic/tablet_discovery.go | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/go/vt/vtorc/inst/tablet_dao_test.go b/go/vt/vtorc/inst/tablet_dao_test.go index 0a4898a5e70..618dd4015e6 100644 --- a/go/vt/vtorc/inst/tablet_dao_test.go +++ b/go/vt/vtorc/inst/tablet_dao_test.go @@ -12,8 +12,8 @@ import ( "vitess.io/vitess/go/vt/vtorc/db" ) -func testRequireTabletAliasEqual(t *testing.T, a, b *topodatapb.TabletAlias) { - require.Equal(t, topoproto.TabletAliasString(a), topoproto.TabletAliasString(b)) +func testRequireTabletAliasEqual(t *testing.T, expected, got *topodatapb.TabletAlias) { + require.Equal(t, topoproto.TabletAliasString(expected), topoproto.TabletAliasString(got)) } func TestSaveAndReadTablet(t *testing.T) { diff --git a/go/vt/vtorc/logic/tablet_discovery.go b/go/vt/vtorc/logic/tablet_discovery.go index 24477a5b746..be24c90c543 100644 --- a/go/vt/vtorc/logic/tablet_discovery.go +++ b/go/vt/vtorc/logic/tablet_discovery.go @@ -335,8 +335,7 @@ func refreshTablets(tablets []*topo.TabletInfo, query string, args []any, loader err := db.QueryVTOrc(query, args, func(row sqlutils.RowMap) error { tabletAliasString := row.GetString("alias") if !latestInstances[tabletAliasString] { - tabletAlias, err := topoproto.ParseTabletAlias(tabletAliasString) - if err == nil { + if tabletAlias, err := topoproto.ParseTabletAlias(tabletAliasString); err == nil { toForget = append(toForget, tabletAlias) } } From f9328c33ed3f73aadc0e563dd4bf5ae5b77d3d81 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Tue, 1 Jul 2025 00:09:02 +0200 Subject: [PATCH 07/16] use `switch` statement Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/inst/analysis_dao.go | 82 ++++++++++++++------------------ 1 file changed, 36 insertions(+), 46 deletions(-) diff --git a/go/vt/vtorc/inst/analysis_dao.go b/go/vt/vtorc/inst/analysis_dao.go index e877ef53a15..0c33ce01985 100644 --- a/go/vt/vtorc/inst/analysis_dao.go +++ b/go/vt/vtorc/inst/analysis_dao.go @@ -412,117 +412,112 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna // We failed to load the durability policy, so we shouldn't run any analysis return nil } + isInvalid := m.GetBool("is_invalid") - if a.IsClusterPrimary && isInvalid { + + switch { + case a.IsClusterPrimary && isInvalid: a.Analysis = InvalidPrimary a.Description = "VTOrc hasn't been able to reach the primary even once since restart/shutdown" - } else if isInvalid { + case isInvalid: a.Analysis = InvalidReplica a.Description = "VTOrc hasn't been able to reach the replica even once since restart/shutdown" - } else if a.IsClusterPrimary && !a.LastCheckValid && a.IsDiskStalled { + case a.IsClusterPrimary && !a.LastCheckValid && a.IsDiskStalled: a.Analysis = PrimaryDiskStalled a.Description = "Primary has a stalled disk" ca.hasShardWideAction = true - } else if a.IsClusterPrimary && !a.LastCheckValid && a.CountReplicas == 0 { + case a.IsClusterPrimary && !a.LastCheckValid && a.CountReplicas == 0: a.Analysis = DeadPrimaryWithoutReplicas a.Description = "Primary cannot be reached by vtorc and has no replica" ca.hasShardWideAction = true // - } else if a.IsClusterPrimary && !a.LastCheckValid && a.CountValidReplicas == a.CountReplicas && a.CountValidReplicatingReplicas == 0 { + case a.IsClusterPrimary && !a.LastCheckValid && a.CountValidReplicas == a.CountReplicas && a.CountValidReplicatingReplicas == 0: a.Analysis = DeadPrimary a.Description = "Primary cannot be reached by vtorc and none of its replicas is replicating" ca.hasShardWideAction = true // - } else if a.IsClusterPrimary && !a.LastCheckValid && a.CountReplicas > 0 && a.CountValidReplicas == 0 && a.CountValidReplicatingReplicas == 0 { + case a.IsClusterPrimary && !a.LastCheckValid && a.CountReplicas > 0 && a.CountValidReplicas == 0 && a.CountValidReplicatingReplicas == 0: a.Analysis = DeadPrimaryAndReplicas a.Description = "Primary cannot be reached by vtorc and none of its replicas is replicating" ca.hasShardWideAction = true // - } else if a.IsClusterPrimary && !a.LastCheckValid && a.CountValidReplicas < a.CountReplicas && a.CountValidReplicas > 0 && a.CountValidReplicatingReplicas == 0 { + case a.IsClusterPrimary && !a.LastCheckValid && a.CountValidReplicas < a.CountReplicas && a.CountValidReplicas > 0 && a.CountValidReplicatingReplicas == 0: a.Analysis = DeadPrimaryAndSomeReplicas a.Description = "Primary cannot be reached by vtorc; some of its replicas are unreachable and none of its reachable replicas is replicating" ca.hasShardWideAction = true // - } else if a.IsClusterPrimary && !a.IsPrimary { + case a.IsClusterPrimary && !a.IsPrimary: a.Analysis = PrimaryHasPrimary a.Description = "Primary is replicating from somewhere else" ca.hasShardWideAction = true // - } else if a.IsClusterPrimary && a.IsReadOnly { + case a.IsClusterPrimary && a.IsReadOnly: a.Analysis = PrimaryIsReadOnly a.Description = "Primary is read-only" // - } else if a.IsClusterPrimary && policy.SemiSyncAckers(ca.durability, tablet) != 0 && !a.SemiSyncPrimaryEnabled { + case a.IsClusterPrimary && policy.SemiSyncAckers(ca.durability, tablet) != 0 && !a.SemiSyncPrimaryEnabled: a.Analysis = PrimarySemiSyncMustBeSet a.Description = "Primary semi-sync must be set" // - } else if a.IsClusterPrimary && policy.SemiSyncAckers(ca.durability, tablet) == 0 && a.SemiSyncPrimaryEnabled { + case a.IsClusterPrimary && policy.SemiSyncAckers(ca.durability, tablet) == 0 && a.SemiSyncPrimaryEnabled: a.Analysis = PrimarySemiSyncMustNotBeSet a.Description = "Primary semi-sync must not be set" // - } else if a.IsClusterPrimary && a.CurrentTabletType != topodatapb.TabletType_UNKNOWN && a.CurrentTabletType != topodatapb.TabletType_PRIMARY { + case a.IsClusterPrimary && a.CurrentTabletType != topodatapb.TabletType_UNKNOWN && a.CurrentTabletType != topodatapb.TabletType_PRIMARY: a.Analysis = PrimaryCurrentTypeMismatch a.Description = "Primary tablet's current type is not PRIMARY" - } else if topo.IsReplicaType(a.TabletType) && a.ErrantGTID != "" { + case topo.IsReplicaType(a.TabletType) && a.ErrantGTID != "": a.Analysis = ErrantGTIDDetected a.Description = "Tablet has errant GTIDs" - } else if topo.IsReplicaType(a.TabletType) && ca.primaryAlias == nil && a.ShardPrimaryTermTimestamp.IsZero() { + case topo.IsReplicaType(a.TabletType) && ca.primaryAlias == nil && a.ShardPrimaryTermTimestamp.IsZero(): // ClusterHasNoPrimary should only be detected when the shard record doesn't have any primary term start time specified either. a.Analysis = ClusterHasNoPrimary a.Description = "Cluster has no primary" ca.hasShardWideAction = true - } else if topo.IsReplicaType(a.TabletType) && ca.primaryAlias == nil && !a.ShardPrimaryTermTimestamp.IsZero() { + case topo.IsReplicaType(a.TabletType) && ca.primaryAlias == nil && !a.ShardPrimaryTermTimestamp.IsZero(): // If there are no primary tablets, but the shard primary start time isn't empty, then we know // the primary tablet was deleted. a.Analysis = PrimaryTabletDeleted a.Description = "Primary tablet has been deleted" ca.hasShardWideAction = true - } else if a.IsPrimary && a.SemiSyncBlocked && a.CountSemiSyncReplicasEnabled >= a.SemiSyncPrimaryWaitForReplicaCount { + case 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. 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.hasShardWideAction = true - } else if topo.IsReplicaType(a.TabletType) && !a.IsReadOnly { + case topo.IsReplicaType(a.TabletType) && !a.IsReadOnly: a.Analysis = ReplicaIsWritable a.Description = "Replica is writable" - // - } else if topo.IsReplicaType(a.TabletType) && a.IsPrimary { + case topo.IsReplicaType(a.TabletType) && a.IsPrimary: a.Analysis = NotConnectedToPrimary a.Description = "Not connected to the primary" - // - } else if topo.IsReplicaType(a.TabletType) && !a.IsPrimary && math.Round(a.HeartbeatInterval*2) != float64(a.ReplicaNetTimeout) { + case topo.IsReplicaType(a.TabletType) && !a.IsPrimary && math.Round(a.HeartbeatInterval*2) != float64(a.ReplicaNetTimeout): a.Analysis = ReplicaMisconfigured a.Description = "Replica has been misconfigured" - // - } else if topo.IsReplicaType(a.TabletType) && !a.IsPrimary && ca.primaryAlias != nil && !topoproto.TabletAliasEqual(a.AnalyzedInstancePrimaryAlias, ca.primaryAlias) { + case topo.IsReplicaType(a.TabletType) && !a.IsPrimary && ca.primaryAlias != nil && !topoproto.TabletAliasEqual(a.AnalyzedInstancePrimaryAlias, ca.primaryAlias): a.Analysis = ConnectedToWrongPrimary a.Description = "Connected to wrong primary" - // - } else if topo.IsReplicaType(a.TabletType) && !a.IsPrimary && a.ReplicationStopped { + case topo.IsReplicaType(a.TabletType) && !a.IsPrimary && a.ReplicationStopped: a.Analysis = ReplicationStopped a.Description = "Replication is stopped" - // - } else if topo.IsReplicaType(a.TabletType) && !a.IsPrimary && policy.IsReplicaSemiSync(ca.durability, primaryTablet, tablet) && !a.SemiSyncReplicaEnabled { + case topo.IsReplicaType(a.TabletType) && !a.IsPrimary && policy.IsReplicaSemiSync(ca.durability, primaryTablet, tablet) && !a.SemiSyncReplicaEnabled: a.Analysis = ReplicaSemiSyncMustBeSet a.Description = "Replica semi-sync must be set" - // - } else if topo.IsReplicaType(a.TabletType) && !a.IsPrimary && !policy.IsReplicaSemiSync(ca.durability, primaryTablet, tablet) && a.SemiSyncReplicaEnabled { + case topo.IsReplicaType(a.TabletType) && !a.IsPrimary && !policy.IsReplicaSemiSync(ca.durability, primaryTablet, tablet) && a.SemiSyncReplicaEnabled: a.Analysis = ReplicaSemiSyncMustNotBeSet a.Description = "Replica semi-sync must not be set" // // TODO(sougou): Events below here are either ignored or not possible. - } else if a.IsPrimary && !a.LastCheckValid && a.CountLaggingReplicas == a.CountReplicas && a.CountDelayedReplicas < a.CountReplicas && a.CountValidReplicatingReplicas > 0 { + case a.IsPrimary && !a.LastCheckValid && a.CountLaggingReplicas == a.CountReplicas && a.CountDelayedReplicas < a.CountReplicas && a.CountValidReplicatingReplicas > 0: a.Analysis = UnreachablePrimaryWithLaggingReplicas a.Description = "Primary cannot be reached by vtorc and all of its replicas are lagging" - // - } else if a.IsPrimary && !a.LastCheckValid && !a.LastCheckPartialSuccess && a.CountValidReplicas > 0 && a.CountValidReplicatingReplicas > 0 { + case a.IsPrimary && !a.LastCheckValid && !a.LastCheckPartialSuccess && a.CountValidReplicas > 0 && a.CountValidReplicatingReplicas > 0: // partial success is here to reduce noise a.Analysis = UnreachablePrimary a.Description = "Primary cannot be reached by vtorc but it has replicating replicas; possibly a network/host issue" - // - } else if a.IsPrimary && a.SemiSyncPrimaryEnabled && a.SemiSyncPrimaryStatus && a.SemiSyncPrimaryWaitForReplicaCount > 0 && a.SemiSyncPrimaryClients < a.SemiSyncPrimaryWaitForReplicaCount { + case a.IsPrimary && a.SemiSyncPrimaryEnabled && a.SemiSyncPrimaryStatus && a.SemiSyncPrimaryWaitForReplicaCount > 0 && a.SemiSyncPrimaryClients < a.SemiSyncPrimaryWaitForReplicaCount: if isStaleBinlogCoordinates { a.Analysis = LockedSemiSyncPrimary a.Description = "Semi sync primary is locked since it doesn't get enough replica acknowledgements" @@ -530,27 +525,22 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna a.Analysis = LockedSemiSyncPrimaryHypothesis a.Description = "Semi sync primary seems to be locked, more samplings needed to validate" } - // - } else if a.IsPrimary && a.LastCheckValid && a.CountReplicas == 1 && a.CountValidReplicas == a.CountReplicas && a.CountValidReplicatingReplicas == 0 { + case 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" - } else if a.IsPrimary && a.LastCheckValid && a.CountReplicas == 1 && a.CountValidReplicas == 0 { + case a.IsPrimary && a.LastCheckValid && a.CountReplicas == 1 && a.CountValidReplicas == 0: a.Analysis = PrimarySingleReplicaDead a.Description = "Primary is reachable but its single replica is dead" - // - } else if a.IsPrimary && a.LastCheckValid && a.CountReplicas > 1 && a.CountValidReplicas == a.CountReplicas && a.CountValidReplicatingReplicas == 0 { + case a.IsPrimary && a.LastCheckValid && a.CountReplicas > 1 && a.CountValidReplicas == a.CountReplicas && a.CountValidReplicatingReplicas == 0: a.Analysis = AllPrimaryReplicasNotReplicating a.Description = "Primary is reachable but none of its replicas is replicating" - // - } else if a.IsPrimary && a.LastCheckValid && a.CountReplicas > 1 && a.CountValidReplicas < a.CountReplicas && a.CountValidReplicas > 0 && a.CountValidReplicatingReplicas == 0 { + case a.IsPrimary && a.LastCheckValid && a.CountReplicas > 1 && a.CountValidReplicas < a.CountReplicas && a.CountValidReplicas > 0 && a.CountValidReplicatingReplicas == 0: a.Analysis = AllPrimaryReplicasNotReplicatingOrDead a.Description = "Primary is reachable but none of its replicas is replicating" - // + // case a.IsPrimary && a.CountReplicas == 0: + // a.Analysis = PrimaryWithoutReplicas + // a.Description = "Primary has no replicas" } - // else if a.IsPrimary && a.CountReplicas == 0 { - // a.Analysis = PrimaryWithoutReplicas - // a.Description = "Primary has no replicas" - // } { // Moving on to structure analysis From b1ac40cf6ab8e082b3b131764af668540b67d107 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Tue, 1 Jul 2025 00:10:20 +0200 Subject: [PATCH 08/16] use `switch` statement - cleanup Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/inst/analysis_dao.go | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/go/vt/vtorc/inst/analysis_dao.go b/go/vt/vtorc/inst/analysis_dao.go index 0c33ce01985..6d0ab08853c 100644 --- a/go/vt/vtorc/inst/analysis_dao.go +++ b/go/vt/vtorc/inst/analysis_dao.go @@ -430,39 +430,31 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna a.Analysis = DeadPrimaryWithoutReplicas a.Description = "Primary cannot be reached by vtorc and has no replica" ca.hasShardWideAction = true - // case a.IsClusterPrimary && !a.LastCheckValid && a.CountValidReplicas == a.CountReplicas && a.CountValidReplicatingReplicas == 0: a.Analysis = DeadPrimary a.Description = "Primary cannot be reached by vtorc and none of its replicas is replicating" ca.hasShardWideAction = true - // case a.IsClusterPrimary && !a.LastCheckValid && a.CountReplicas > 0 && a.CountValidReplicas == 0 && a.CountValidReplicatingReplicas == 0: a.Analysis = DeadPrimaryAndReplicas a.Description = "Primary cannot be reached by vtorc and none of its replicas is replicating" ca.hasShardWideAction = true - // case a.IsClusterPrimary && !a.LastCheckValid && a.CountValidReplicas < a.CountReplicas && a.CountValidReplicas > 0 && a.CountValidReplicatingReplicas == 0: a.Analysis = DeadPrimaryAndSomeReplicas a.Description = "Primary cannot be reached by vtorc; some of its replicas are unreachable and none of its reachable replicas is replicating" ca.hasShardWideAction = true - // case a.IsClusterPrimary && !a.IsPrimary: a.Analysis = PrimaryHasPrimary a.Description = "Primary is replicating from somewhere else" ca.hasShardWideAction = true - // case a.IsClusterPrimary && a.IsReadOnly: a.Analysis = PrimaryIsReadOnly a.Description = "Primary is read-only" - // case a.IsClusterPrimary && policy.SemiSyncAckers(ca.durability, tablet) != 0 && !a.SemiSyncPrimaryEnabled: a.Analysis = PrimarySemiSyncMustBeSet a.Description = "Primary semi-sync must be set" - // case a.IsClusterPrimary && policy.SemiSyncAckers(ca.durability, tablet) == 0 && a.SemiSyncPrimaryEnabled: a.Analysis = PrimarySemiSyncMustNotBeSet a.Description = "Primary semi-sync must not be set" - // case a.IsClusterPrimary && a.CurrentTabletType != topodatapb.TabletType_UNKNOWN && a.CurrentTabletType != topodatapb.TabletType_PRIMARY: a.Analysis = PrimaryCurrentTypeMismatch a.Description = "Primary tablet's current type is not PRIMARY" @@ -537,9 +529,9 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna case a.IsPrimary && a.LastCheckValid && a.CountReplicas > 1 && a.CountValidReplicas < a.CountReplicas && a.CountValidReplicas > 0 && a.CountValidReplicatingReplicas == 0: a.Analysis = AllPrimaryReplicasNotReplicatingOrDead a.Description = "Primary is reachable but none of its replicas is replicating" - // case a.IsPrimary && a.CountReplicas == 0: - // a.Analysis = PrimaryWithoutReplicas - // a.Description = "Primary has no replicas" + // case a.IsPrimary && a.CountReplicas == 0: + // a.Analysis = PrimaryWithoutReplicas + // a.Description = "Primary has no replicas" } { From 3bf9c4554993583ce2a15d7dbd490a00dbb9b828 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Tue, 1 Jul 2025 00:13:05 +0200 Subject: [PATCH 09/16] gofmt Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/inst/analysis_dao.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/go/vt/vtorc/inst/analysis_dao.go b/go/vt/vtorc/inst/analysis_dao.go index 6d0ab08853c..a9719ffc06a 100644 --- a/go/vt/vtorc/inst/analysis_dao.go +++ b/go/vt/vtorc/inst/analysis_dao.go @@ -529,9 +529,9 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna case a.IsPrimary && a.LastCheckValid && a.CountReplicas > 1 && a.CountValidReplicas < a.CountReplicas && a.CountValidReplicas > 0 && a.CountValidReplicatingReplicas == 0: a.Analysis = AllPrimaryReplicasNotReplicatingOrDead a.Description = "Primary is reachable but none of its replicas is replicating" - // case a.IsPrimary && a.CountReplicas == 0: - // a.Analysis = PrimaryWithoutReplicas - // a.Description = "Primary has no replicas" + // case a.IsPrimary && a.CountReplicas == 0: + // a.Analysis = PrimaryWithoutReplicas + // a.Description = "Primary has no replicas" } { From 2b6bce0b92706821b08799cd69d362c656683587 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Wed, 24 Sep 2025 00:09:43 +0200 Subject: [PATCH 10/16] return existing format Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/server/api.go | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/go/vt/vtorc/server/api.go b/go/vt/vtorc/server/api.go index 172760fd384..8d0e3c6a284 100644 --- a/go/vt/vtorc/server/api.go +++ b/go/vt/vtorc/server/api.go @@ -22,9 +22,12 @@ import ( "fmt" "net/http" + "github.com/go-viper/mapstructure/v2" + "vitess.io/vitess/go/acl" "vitess.io/vitess/go/viperutil/debug" "vitess.io/vitess/go/vt/servenv" + "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtorc/inst" "vitess.io/vitess/go/vt/vtorc/logic" "vitess.io/vitess/go/vt/vtorc/process" @@ -225,9 +228,23 @@ func detectionAnalysisAPIHandler(response http.ResponseWriter, request *http.Req return } + // return the tablet alias in topoproto.TabletAliasString(...) format. + // to achieve this, we make structs a map of string-to-interface and + // override the "AnalyzedInstanceAlias" field. + processedAnalysis := make([]map[string]interface{}, 0) + for _, analysis := range analysis { + out := make(map[string]interface{}) + if err := mapstructure.Decode(analysis, &out); err != nil { + http.Error(response, err.Error(), http.StatusInternalServerError) + return + } + out["AnalyzedInstanceAlias"] = topoproto.TabletAliasString(analysis.AnalyzedInstanceAlias) + processedAnalysis = append(processedAnalysis, out) + } + // TODO: We can also add filtering for a specific instance too based on the tablet alias. // Currently inst.DetectionAnalysis doesn't store the tablet alias, but once it does we can filter on that too - returnAsJSON(response, http.StatusOK, analysis) + returnAsJSON(response, http.StatusOK, processedAnalysis) } // healthAPIHandler is the handler for the healthAPI endpoint From df1686db64ff4865540309f1223606e1e8b818cb Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Wed, 24 Sep 2025 00:17:49 +0200 Subject: [PATCH 11/16] standardize on `github.com/go-viper/mapstructure/v2` Signed-off-by: Tim Vaillancourt --- go.mod | 4 +- go/viperutil/config.go | 2 +- go/vt/topo/txn.go | 101 +++++++++++++++++++++++++++++++++++++++++ 3 files changed, 104 insertions(+), 3 deletions(-) create mode 100644 go/vt/topo/txn.go diff --git a/go.mod b/go.mod index 0f5ce9a4757..addb4e901dd 100644 --- a/go.mod +++ b/go.mod @@ -100,7 +100,6 @@ require ( github.com/hashicorp/go-version v1.7.0 github.com/kr/pretty v0.3.1 github.com/kr/text v0.2.0 - github.com/mitchellh/mapstructure v1.5.1-0.20231216201459-8508981c8b6c github.com/nsf/jsondiff v0.0.0-20210926074059-1e845ec5d249 github.com/shirou/gopsutil/v4 v4.25.7 github.com/spf13/afero v1.14.0 @@ -133,6 +132,7 @@ require ( github.com/google/btree v1.1.3 // indirect github.com/google/pprof v0.0.0-20250820193118-f64d9cf942d6 // indirect github.com/json-iterator/go v1.1.12 // indirect + github.com/mitchellh/mapstructure v1.5.1-0.20231216201459-8508981c8b6c // indirect github.com/moby/sys/userns v0.1.0 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.3-0.20250322232337-35a7c28c31ee // indirect @@ -198,7 +198,7 @@ require ( github.com/go-logr/logr v1.4.3 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.3.0 // indirect - github.com/go-viper/mapstructure/v2 v2.4.0 // indirect + github.com/go-viper/mapstructure/v2 v2.4.0 github.com/gogo/protobuf v1.3.2 // indirect github.com/google/s2a-go v0.1.9 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.6 // indirect diff --git a/go/viperutil/config.go b/go/viperutil/config.go index b40d4a5989a..89cc5f78126 100644 --- a/go/viperutil/config.go +++ b/go/viperutil/config.go @@ -26,7 +26,7 @@ import ( "strings" "time" - "github.com/mitchellh/mapstructure" + "github.com/go-viper/mapstructure/v2" "github.com/spf13/pflag" "github.com/spf13/viper" diff --git a/go/vt/topo/txn.go b/go/vt/topo/txn.go new file mode 100644 index 00000000000..e7043682763 --- /dev/null +++ b/go/vt/topo/txn.go @@ -0,0 +1,101 @@ +/* +Copyright 2024 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 topo + +import ( + "context" +) + +// TxnOpResult represents the result of an operation within a transaction. +type TxnOpResult struct { + Contents []byte + DirEntries []DirEntry + Key string + Version Version +} + +// Txn defines the interface that must be implemented by topology +// plug-ins to be used with Vitess. +type Txn interface { + // + // Directory support + // + + // ListDir returns the entries in a directory. The returned + // list should be sorted by entry.Name. + // If there are no files under the provided path, returns ErrNoNode. + // dirPath is a path relative to the root directory of the cell. + // If full is set, we want all the fields in DirEntry to be filled in. + // If full is not set, only Name will be used. This is intended for + // implementations where getting more than the names is more expensive, + // as in most cases only the names are needed. + ListDir(dirPath string, full bool) Txn + + // + // File support + // if version == nil, then it’s an unconditional update / delete. + // + + // Create creates the initial version of a file. + // Returns ErrNodeExists if the file exists. + // filePath is a path relative to the root directory of the cell. + Create(filePath string, contents []byte) Txn + + // Update updates the file with the provided filename with the + // new content. + // If version is nil, it is an unconditional update + // (which is then the same as a Create is the file doesn't exist). + // filePath is a path relative to the root directory of the cell. + // It returns the new Version of the file after update. + // Returns ErrBadVersion if the provided version is not current. + Update(filePath string, contents []byte, version Version) Txn + + // Get returns the content and version of a file. + // filePath is a path relative to the root directory of the cell. + // Can return ErrNoNode if the file doesn't exist. + Get(filePath string) Txn + + // GetVersion returns the content of a file at the given version. + // filePath is a path relative to the root directory of the cell. + // Can return ErrNoNode if the file doesn't exist at the given + // version or ErrNoImplementation if the topo server does not + // support storing multiple versions and retrieving a specific one. + GetVersion(filePath string, version int64) Txn + + // List returns KV pairs, along with metadata like the version, for + // entries where the key contains the specified prefix. + // filePathPrefix is a path relative to the root directory of the cell. + // Can return ErrNoNode if there are no matches. + List(filePathPrefix string) Txn + + // Delete deletes the provided file. + // If version is nil, it is an unconditional delete. + // If the last entry of a directory is deleted, using ListDir + // on its parent directory should not return the directory. + // For instance, when deleting /keyspaces/aaa/Keyspace, and if + // there is no other file in /keyspaces/aaa, then aaa should not + // appear any more when listing /keyspaces. + // filePath is a path relative to the root directory of the cell. + // + // Delete will never be called on a directory. + // Returns ErrNodeExists if the file doesn't exist. + // Returns ErrBadVersion if the provided version is not current. + Delete(filePath string, version Version) Txn + + // Commit commits the transaction. + Commit(ctx context.Context) ([]TxnOpResult, error) +} From dc1b03ad065036c54d860ccef4049ff8a072d3f8 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Wed, 24 Sep 2025 00:29:55 +0200 Subject: [PATCH 12/16] use string for queue key Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/logic/discovery_queue.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/go/vt/vtorc/logic/discovery_queue.go b/go/vt/vtorc/logic/discovery_queue.go index 8b7428c5461..33b47397cc5 100644 --- a/go/vt/vtorc/logic/discovery_queue.go +++ b/go/vt/vtorc/logic/discovery_queue.go @@ -44,14 +44,14 @@ type queueItem struct { // DiscoveryQueue is an ordered queue with deduplication. type DiscoveryQueue struct { mu sync.Mutex - enqueued map[*topodatapb.TabletAlias]struct{} + enqueued map[string]struct{} queue chan queueItem } // NewDiscoveryQueue creates a new queue. func NewDiscoveryQueue() *DiscoveryQueue { return &DiscoveryQueue{ - enqueued: make(map[*topodatapb.TabletAlias]struct{}), + enqueued: make(map[string]struct{}), queue: make(chan queueItem, config.DiscoveryQueueCapacity), } } @@ -62,9 +62,10 @@ func (q *DiscoveryQueue) setKeyCheckEnqueued(tabletAlias *topodatapb.TabletAlias q.mu.Lock() defer q.mu.Unlock() - _, alreadyEnqueued = q.enqueued[tabletAlias] + tabletAliasString := topoproto.TabletAliasString(tabletAlias) + _, alreadyEnqueued = q.enqueued[tabletAliasString] if !alreadyEnqueued { - q.enqueued[tabletAlias] = struct{}{} + q.enqueued[tabletAliasString] = struct{}{} } return alreadyEnqueued } @@ -111,5 +112,5 @@ func (q *DiscoveryQueue) Release(tabletAlias *topodatapb.TabletAlias) { q.mu.Lock() defer q.mu.Unlock() - delete(q.enqueued, tabletAlias) + delete(q.enqueued, topoproto.TabletAliasString(tabletAlias)) } From 7b8a3cadd0b2098db58bf22b74f780ac84b7f777 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Tue, 30 Sep 2025 22:28:38 +0200 Subject: [PATCH 13/16] rm mistaken add Signed-off-by: Tim Vaillancourt --- go/vt/topo/txn.go | 101 ---------------------------------------------- 1 file changed, 101 deletions(-) delete mode 100644 go/vt/topo/txn.go diff --git a/go/vt/topo/txn.go b/go/vt/topo/txn.go deleted file mode 100644 index e7043682763..00000000000 --- a/go/vt/topo/txn.go +++ /dev/null @@ -1,101 +0,0 @@ -/* -Copyright 2024 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 topo - -import ( - "context" -) - -// TxnOpResult represents the result of an operation within a transaction. -type TxnOpResult struct { - Contents []byte - DirEntries []DirEntry - Key string - Version Version -} - -// Txn defines the interface that must be implemented by topology -// plug-ins to be used with Vitess. -type Txn interface { - // - // Directory support - // - - // ListDir returns the entries in a directory. The returned - // list should be sorted by entry.Name. - // If there are no files under the provided path, returns ErrNoNode. - // dirPath is a path relative to the root directory of the cell. - // If full is set, we want all the fields in DirEntry to be filled in. - // If full is not set, only Name will be used. This is intended for - // implementations where getting more than the names is more expensive, - // as in most cases only the names are needed. - ListDir(dirPath string, full bool) Txn - - // - // File support - // if version == nil, then it’s an unconditional update / delete. - // - - // Create creates the initial version of a file. - // Returns ErrNodeExists if the file exists. - // filePath is a path relative to the root directory of the cell. - Create(filePath string, contents []byte) Txn - - // Update updates the file with the provided filename with the - // new content. - // If version is nil, it is an unconditional update - // (which is then the same as a Create is the file doesn't exist). - // filePath is a path relative to the root directory of the cell. - // It returns the new Version of the file after update. - // Returns ErrBadVersion if the provided version is not current. - Update(filePath string, contents []byte, version Version) Txn - - // Get returns the content and version of a file. - // filePath is a path relative to the root directory of the cell. - // Can return ErrNoNode if the file doesn't exist. - Get(filePath string) Txn - - // GetVersion returns the content of a file at the given version. - // filePath is a path relative to the root directory of the cell. - // Can return ErrNoNode if the file doesn't exist at the given - // version or ErrNoImplementation if the topo server does not - // support storing multiple versions and retrieving a specific one. - GetVersion(filePath string, version int64) Txn - - // List returns KV pairs, along with metadata like the version, for - // entries where the key contains the specified prefix. - // filePathPrefix is a path relative to the root directory of the cell. - // Can return ErrNoNode if there are no matches. - List(filePathPrefix string) Txn - - // Delete deletes the provided file. - // If version is nil, it is an unconditional delete. - // If the last entry of a directory is deleted, using ListDir - // on its parent directory should not return the directory. - // For instance, when deleting /keyspaces/aaa/Keyspace, and if - // there is no other file in /keyspaces/aaa, then aaa should not - // appear any more when listing /keyspaces. - // filePath is a path relative to the root directory of the cell. - // - // Delete will never be called on a directory. - // Returns ErrNodeExists if the file doesn't exist. - // Returns ErrBadVersion if the provided version is not current. - Delete(filePath string, version Version) Txn - - // Commit commits the transaction. - Commit(ctx context.Context) ([]TxnOpResult, error) -} From 4aeb37ad5ba46b0421249c7ce2ae12926866383e Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Thu, 25 Dec 2025 01:39:40 +0100 Subject: [PATCH 14/16] fixes Signed-off-by: Tim Vaillancourt --- go/tools/codegen/random | 1 - go/tools/releases/README.md | 0 go/vt/vtorc/logic/vtorc.go | 4 ++-- 3 files changed, 2 insertions(+), 3 deletions(-) delete mode 100644 go/tools/codegen/random delete mode 100644 go/tools/releases/README.md diff --git a/go/tools/codegen/random b/go/tools/codegen/random deleted file mode 100644 index eeab28a038e..00000000000 --- a/go/tools/codegen/random +++ /dev/null @@ -1 +0,0 @@ -package random diff --git a/go/tools/releases/README.md b/go/tools/releases/README.md deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/go/vt/vtorc/logic/vtorc.go b/go/vt/vtorc/logic/vtorc.go index 6ec5cfe2a70..7fefdd80973 100644 --- a/go/vt/vtorc/logic/vtorc.go +++ b/go/vt/vtorc/logic/vtorc.go @@ -191,9 +191,9 @@ func DiscoverInstance(tabletAlias *topodatapb.TabletAlias, forceDiscovery bool) discoveryInstanceTimings.Add("Other", otherLatency) if err != nil { - log.Errorf("Failed to discover %s (force: %t), err: %v", tabletAlias, forceDiscovery, err) + log.Errorf("Failed to discover %s (force: %t), err: %v", tabletAliasString, forceDiscovery, err) } else { - log.Infof("Discovered %s (force: %t): %+v", tabletAlias, forceDiscovery, instance) + log.Infof("Discovered %s (force: %t): %+v", tabletAliasString, forceDiscovery, instance) } if instance == nil { From 2a2c5eb227c29718438caed443f562145c3d9e36 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Fri, 26 Dec 2025 00:07:20 +0100 Subject: [PATCH 15/16] fixes Signed-off-by: Tim Vaillancourt --- go/test/endtoend/vtorc/api/api_test.go | 16 +++--- go/test/endtoend/vtorc/general/vtorc_test.go | 8 +-- go/vt/vtorc/server/api.go | 52 +++++++++++--------- go/vt/vtorc/server/api_test.go | 3 -- 4 files changed, 41 insertions(+), 38 deletions(-) diff --git a/go/test/endtoend/vtorc/api/api_test.go b/go/test/endtoend/vtorc/api/api_test.go index fab5b80bdbf..25f82ca8bed 100644 --- a/go/test/endtoend/vtorc/api/api_test.go +++ b/go/test/endtoend/vtorc/api/api_test.go @@ -85,7 +85,7 @@ func TestAPIEndpoints(t *testing.T) { }) // Before we disable recoveries, let us wait until VTOrc has fixed all the issues (if any). - _, _ = utils.MakeAPICallRetry(t, vtorc, "/api/replication-analysis", func(_ int, response string) bool { + _, _ = utils.MakeAPICallRetry(t, vtorc, "/api/detection-analysis", func(i int, response string) bool { return response != "null" }) @@ -164,15 +164,15 @@ func TestAPIEndpoints(t *testing.T) { assert.Equal(t, "Global recoveries disabled\n", resp) }) - t.Run("Replication Analysis API", func(t *testing.T) { + t.Run("Detection Analysis API", func(t *testing.T) { // use vtctldclient to stop replication _, err := clusterInfo.ClusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("StopReplication", replica.Alias) require.NoError(t, err) // We know VTOrc won't fix this since we disabled global recoveries! // Wait until VTOrc picks up on this issue and verify - // that we see a not null result on the api/replication-analysis page - status, resp := utils.MakeAPICallRetry(t, vtorc, "/api/replication-analysis", func(_ int, response string) bool { + // that we see a not null result on the api/detection-analysis page + status, resp := utils.MakeAPICallRetry(t, vtorc, "/api/detection-analysis", func(_ int, response string) bool { return response == "null" }) assert.Equal(t, 200, status, resp) @@ -180,25 +180,25 @@ func TestAPIEndpoints(t *testing.T) { assert.Contains(t, resp, `"Analysis": "ReplicationStopped"`) // Verify that filtering also works in the API as intended - status, resp, err = utils.MakeAPICall(t, vtorc, "/api/replication-analysis?keyspace=ks&shard=0") + status, resp, err = utils.MakeAPICall(t, vtorc, "/api/detection-analysis?keyspace=ks&shard=0") require.NoError(t, err) assert.Equal(t, 200, status, resp) assert.Contains(t, resp, fmt.Sprintf(`"AnalyzedInstanceAlias": "%s"`, replica.Alias)) // Verify that filtering by keyspace also works in the API as intended - status, resp, err = utils.MakeAPICall(t, vtorc, "/api/replication-analysis?keyspace=ks") + status, resp, err = utils.MakeAPICall(t, vtorc, "/api/detection-analysis?keyspace=ks") require.NoError(t, err) assert.Equal(t, 200, status, resp) assert.Contains(t, resp, fmt.Sprintf(`"AnalyzedInstanceAlias": "%s"`, replica.Alias)) // Check that filtering using keyspace and shard works - status, resp, err = utils.MakeAPICall(t, vtorc, "/api/replication-analysis?keyspace=ks&shard=80-") + status, resp, err = utils.MakeAPICall(t, vtorc, "/api/detection-analysis?keyspace=ks&shard=80-") require.NoError(t, err) assert.Equal(t, 200, status, resp) assert.Equal(t, "null", resp) // Check that filtering using just the shard fails - status, resp, err = utils.MakeAPICall(t, vtorc, "/api/replication-analysis?shard=0") + status, resp, err = utils.MakeAPICall(t, vtorc, "/api/detection-analysis?shard=0") require.NoError(t, err) assert.Equal(t, 400, status, resp) assert.Equal(t, "Filtering by shard without keyspace isn't supported\n", resp) diff --git a/go/test/endtoend/vtorc/general/vtorc_test.go b/go/test/endtoend/vtorc/general/vtorc_test.go index 6158fe5161f..ae56c2b46a6 100644 --- a/go/test/endtoend/vtorc/general/vtorc_test.go +++ b/go/test/endtoend/vtorc/general/vtorc_test.go @@ -681,7 +681,7 @@ func TestFullStatusConnectionPooling(t *testing.T) { _ = curPrimary.VttabletProcess.Kill() // Wait until VTOrc notices some problems - status, resp := utils.MakeAPICallRetry(t, vtorc, "/api/replication-analysis", func(_ int, response string) bool { + status, resp := utils.MakeAPICallRetry(t, vtorc, "/api/detection-analysis", func(_ int, response string) bool { return response == "null" }) assert.Equal(t, 200, status) @@ -697,7 +697,7 @@ func TestFullStatusConnectionPooling(t *testing.T) { // See that VTOrc eventually reports no errors. // Wait until there are no problems and the api endpoint returns null - status, resp = utils.MakeAPICallRetry(t, vtorc, "/api/replication-analysis", func(_ int, response string) bool { + status, resp = utils.MakeAPICallRetry(t, vtorc, "/api/detection-analysis", func(_ int, response string) bool { return response != "null" }) assert.Equal(t, 200, status) @@ -708,7 +708,7 @@ func TestFullStatusConnectionPooling(t *testing.T) { _ = curPrimary.VttabletProcess.Kill() // Wait until VTOrc notices some problems - status, resp = utils.MakeAPICallRetry(t, vtorc, "/api/replication-analysis", func(_ int, response string) bool { + status, resp = utils.MakeAPICallRetry(t, vtorc, "/api/detection-analysis", func(_ int, response string) bool { return response == "null" }) assert.Equal(t, 200, status) @@ -724,7 +724,7 @@ func TestFullStatusConnectionPooling(t *testing.T) { // See that VTOrc eventually reports no errors. // Wait until there are no problems and the api endpoint returns null - status, resp = utils.MakeAPICallRetry(t, vtorc, "/api/replication-analysis", func(_ int, response string) bool { + status, resp = utils.MakeAPICallRetry(t, vtorc, "/api/detection-analysis", func(_ int, response string) bool { return response != "null" }) assert.Equal(t, 200, status) diff --git a/go/vt/vtorc/server/api.go b/go/vt/vtorc/server/api.go index 8d0e3c6a284..ec59743c4f5 100644 --- a/go/vt/vtorc/server/api.go +++ b/go/vt/vtorc/server/api.go @@ -22,8 +22,6 @@ import ( "fmt" "net/http" - "github.com/go-viper/mapstructure/v2" - "vitess.io/vitess/go/acl" "vitess.io/vitess/go/viperutil/debug" "vitess.io/vitess/go/vt/servenv" @@ -45,7 +43,6 @@ const ( disableGlobalRecoveriesAPI = "/api/disable-global-recoveries" enableGlobalRecoveriesAPI = "/api/enable-global-recoveries" detectionAnalysisAPI = "/api/detection-analysis" - replicationAnalysisAPI = "/api/replication-analysis" // TODO: remove in v24+ databaseStateAPI = "/api/database-state" configAPI = "/api/config" healthAPI = "/debug/health" @@ -62,7 +59,6 @@ var ( disableGlobalRecoveriesAPI, enableGlobalRecoveriesAPI, detectionAnalysisAPI, - replicationAnalysisAPI, databaseStateAPI, configAPI, healthAPI, @@ -88,7 +84,7 @@ func (v *vtorcAPI) ServeHTTP(response http.ResponseWriter, request *http.Request problemsAPIHandler(response, request) case errantGTIDsAPI: errantGTIDsAPIHandler(response, request) - case detectionAnalysisAPI, replicationAnalysisAPI: + case detectionAnalysisAPI: detectionAnalysisAPIHandler(response, request) case databaseStateAPI: databaseStateAPIHandler(response) @@ -108,7 +104,7 @@ func getACLPermissionLevelForAPI(apiEndpoint string) string { return acl.MONITORING case disableGlobalRecoveriesAPI, enableGlobalRecoveriesAPI: return acl.ADMIN - case detectionAnalysisAPI, replicationAnalysisAPI, configAPI: + case detectionAnalysisAPI, configAPI: return acl.MONITORING case healthAPI, databaseStateAPI: return acl.MONITORING @@ -213,6 +209,29 @@ func enableGlobalRecoveriesAPIHandler(response http.ResponseWriter) { writePlainTextResponse(response, "Global recoveries enabled", http.StatusOK) } +// LegacyDetectionAnalysisJSON is a wrapper to *inst.DetectionAnalysis that +// provides the AnalyzedInstanceAlias field in the old string-based format. +type LegacyDetectionAnalysisJSON struct { + *inst.DetectionAnalysis +} + +// NewLegacyDetectionAnalysisJSON wraps a *inst.DetectionAnalysis with *LegacyDetectionAnalysisJSON. +func NewLegacyDetectionAnalysisJSON(da *inst.DetectionAnalysis) *LegacyDetectionAnalysisJSON { + return &LegacyDetectionAnalysisJSON{da} +} + +// MarshalJSON converts a legacyDetectionAnalysisJSON to the legacy format JSON. +func (ldaj *LegacyDetectionAnalysisJSON) MarshalJSON() ([]byte, error) { + type Alias LegacyDetectionAnalysisJSON + return json.Marshal(&struct { + AnalyzedInstanceAlias string `json:"AnalyzedInstanceAlias"` + *Alias + }{ + AnalyzedInstanceAlias: topoproto.TabletAliasString(ldaj.AnalyzedInstanceAlias), + Alias: (*Alias)(ldaj), + }) +} + // detectionAnalysisAPIHandler is the handler for the detectionAnalysisAPI endpoint func detectionAnalysisAPIHandler(response http.ResponseWriter, request *http.Request) { // This api also supports filtering by shard and keyspace provided. @@ -227,24 +246,11 @@ func detectionAnalysisAPIHandler(response http.ResponseWriter, request *http.Req http.Error(response, err.Error(), http.StatusInternalServerError) return } - - // return the tablet alias in topoproto.TabletAliasString(...) format. - // to achieve this, we make structs a map of string-to-interface and - // override the "AnalyzedInstanceAlias" field. - processedAnalysis := make([]map[string]interface{}, 0) - for _, analysis := range analysis { - out := make(map[string]interface{}) - if err := mapstructure.Decode(analysis, &out); err != nil { - http.Error(response, err.Error(), http.StatusInternalServerError) - return - } - out["AnalyzedInstanceAlias"] = topoproto.TabletAliasString(analysis.AnalyzedInstanceAlias) - processedAnalysis = append(processedAnalysis, out) + processed := make([]*LegacyDetectionAnalysisJSON, 0) + for _, a := range analysis { + processed = append(processed, NewLegacyDetectionAnalysisJSON(a)) } - - // TODO: We can also add filtering for a specific instance too based on the tablet alias. - // Currently inst.DetectionAnalysis doesn't store the tablet alias, but once it does we can filter on that too - returnAsJSON(response, http.StatusOK, processedAnalysis) + returnAsJSON(response, http.StatusOK, processed) } // healthAPIHandler is the handler for the healthAPI endpoint diff --git a/go/vt/vtorc/server/api_test.go b/go/vt/vtorc/server/api_test.go index e46f4d86f8a..d04e7cb5621 100644 --- a/go/vt/vtorc/server/api_test.go +++ b/go/vt/vtorc/server/api_test.go @@ -28,9 +28,6 @@ func TestGetACLPermissionLevelForAPI(t *testing.T) { }, { apiEndpoint: detectionAnalysisAPI, want: acl.MONITORING, - }, { - apiEndpoint: replicationAnalysisAPI, - want: acl.MONITORING, }, { apiEndpoint: healthAPI, want: acl.MONITORING, From c3073b835035f2fc428ccb8ba3c04f66701b20d3 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Sun, 28 Dec 2025 01:19:40 +0100 Subject: [PATCH 16/16] add unmarshal Signed-off-by: Tim Vaillancourt --- go/vt/vtorc/server/api.go | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/go/vt/vtorc/server/api.go b/go/vt/vtorc/server/api.go index ec59743c4f5..92864923be5 100644 --- a/go/vt/vtorc/server/api.go +++ b/go/vt/vtorc/server/api.go @@ -220,7 +220,7 @@ func NewLegacyDetectionAnalysisJSON(da *inst.DetectionAnalysis) *LegacyDetection return &LegacyDetectionAnalysisJSON{da} } -// MarshalJSON converts a legacyDetectionAnalysisJSON to the legacy format JSON. +// MarshalJSON converts a *LegacyDetectionAnalysisJSON to the legacy format JSON. func (ldaj *LegacyDetectionAnalysisJSON) MarshalJSON() ([]byte, error) { type Alias LegacyDetectionAnalysisJSON return json.Marshal(&struct { @@ -232,6 +232,23 @@ func (ldaj *LegacyDetectionAnalysisJSON) MarshalJSON() ([]byte, error) { }) } +// UnmarshalJSON converts JSON to a *LegacyDetectionAnalysisJSON. +func (ldaj *LegacyDetectionAnalysisJSON) UnmarshalJSON(data []byte) (err error) { + type Alias LegacyDetectionAnalysisJSON + s := &struct { + AnalyzedInstanceAlias string `json:"AnalyzedInstanceAlias"` + *Alias + }{ + AnalyzedInstanceAlias: topoproto.TabletAliasString(ldaj.AnalyzedInstanceAlias), + Alias: (*Alias)(ldaj), + } + if err := json.Unmarshal(data, &s); err != nil { + return nil + } + ldaj.AnalyzedInstanceAlias, err = topoproto.ParseTabletAlias(s.AnalyzedInstanceAlias) + return err +} + // detectionAnalysisAPIHandler is the handler for the detectionAnalysisAPI endpoint func detectionAnalysisAPIHandler(response http.ResponseWriter, request *http.Request) { // This api also supports filtering by shard and keyspace provided.