diff --git a/go/cmd/vtcombo/main.go b/go/cmd/vtcombo/main.go index 4a1d446f3a5..62afaa640bd 100644 --- a/go/cmd/vtcombo/main.go +++ b/go/cmd/vtcombo/main.go @@ -55,7 +55,7 @@ var ( ts *topo.Server resilientServer *srvtopo.ResilientServer - healthCheck discovery.HealthCheck + healthCheck discovery.LegacyHealthCheck ) func init() { @@ -118,7 +118,7 @@ func main() { // vtgate configuration and init resilientServer = srvtopo.NewResilientServer(ts, "ResilientSrvTopoServer") - healthCheck = discovery.NewHealthCheck(1*time.Millisecond /*retryDelay*/, 1*time.Hour /*healthCheckTimeout*/) + healthCheck := discovery.NewLegacyHealthCheck(1*time.Millisecond /*retryDelay*/, 1*time.Hour /*healthCheckTimeout*/) tabletTypesToWait := []topodatapb.TabletType{ topodatapb.TabletType_MASTER, topodatapb.TabletType_REPLICA, @@ -128,7 +128,7 @@ func main() { vtgate.QueryLogHandler = "/debug/vtgate/querylog" vtgate.QueryLogzHandler = "/debug/vtgate/querylogz" vtgate.QueryzHandler = "/debug/vtgate/queryz" - vtg := vtgate.Init(context.Background(), healthCheck, resilientServer, tpb.Cells[0], 2 /*retryCount*/, tabletTypesToWait) + vtg := vtgate.LegacyInit(context.Background(), healthCheck, resilientServer, tpb.Cells[0], 2 /*retryCount*/, tabletTypesToWait) // vtctld configuration and init vtctld.InitVtctld(ts) diff --git a/go/cmd/vtcombo/status.go b/go/cmd/vtcombo/status.go index 4d9b3079a94..c7524cb17d4 100644 --- a/go/cmd/vtcombo/status.go +++ b/go/cmd/vtcombo/status.go @@ -43,7 +43,7 @@ func addStatusParts(vtg *vtgate.VTGate) { servenv.AddStatusPart("Gateway Status", vtgate.StatusTemplate, func() interface{} { return vtg.GetGatewayCacheStatus() }) - servenv.AddStatusPart("Health Check Cache", discovery.HealthCheckTemplate, func() interface{} { + servenv.AddStatusPart("Health Check Cache", discovery.LegacyHealthCheckTemplate, func() interface{} { return healthCheck.CacheStatus() }) } diff --git a/go/cmd/vtgate/status.go b/go/cmd/vtgate/status.go index f8315945bf8..b2548a24360 100644 --- a/go/cmd/vtgate/status.go +++ b/go/cmd/vtgate/status.go @@ -38,7 +38,13 @@ func addStatusParts(vtg *vtgate.VTGate) { servenv.AddStatusPart("Gateway Status", vtgate.StatusTemplate, func() interface{} { return vtg.GetGatewayCacheStatus() }) - servenv.AddStatusPart("Health Check Cache", discovery.HealthCheckTemplate, func() interface{} { - return healthCheck.CacheStatus() - }) + if *vtgate.GatewayImplementation == vtgate.GatewayImplementationDiscovery { + servenv.AddStatusPart("Health Check Cache", discovery.LegacyHealthCheckTemplate, func() interface{} { + return legacyHealthCheck.CacheStatus() + }) + } else { + servenv.AddStatusPart("Health Check Cache", discovery.HealthCheckTemplate, func() interface{} { + return vtg.Gateway().HealthCheck().CacheStatus() + }) + } } diff --git a/go/cmd/vtgate/vtgate.go b/go/cmd/vtgate/vtgate.go index 55779162e35..8c696f97887 100644 --- a/go/cmd/vtgate/vtgate.go +++ b/go/cmd/vtgate/vtgate.go @@ -37,15 +37,12 @@ import ( ) var ( - cell = flag.String("cell", "test_nj", "cell to use") - retryCount = flag.Int("retry-count", 2, "retry count") - healthCheckRetryDelay = flag.Duration("healthcheck_retry_delay", 2*time.Millisecond, "health check retry delay") - healthCheckTimeout = flag.Duration("healthcheck_timeout", time.Minute, "the health check timeout period") - tabletTypesToWait = flag.String("tablet_types_to_wait", "", "wait till connected for specified tablet types during Gateway initialization") + cell = flag.String("cell", "test_nj", "cell to use") + tabletTypesToWait = flag.String("tablet_types_to_wait", "", "wait till connected for specified tablet types during Gateway initialization") ) var resilientServer *srvtopo.ResilientServer -var healthCheck discovery.HealthCheck +var legacyHealthCheck discovery.LegacyHealthCheck func init() { rand.Seed(time.Now().UnixNano()) @@ -63,9 +60,6 @@ func main() { resilientServer = srvtopo.NewResilientServer(ts, "ResilientSrvTopoServer") - healthCheck = discovery.NewHealthCheck(*healthCheckRetryDelay, *healthCheckTimeout) - healthCheck.RegisterStats() - tabletTypes := make([]topodatapb.TabletType, 0, 1) if len(*tabletTypesToWait) != 0 { for _, ttStr := range strings.Split(*tabletTypesToWait, ",") { @@ -78,12 +72,28 @@ func main() { } } - vtg := vtgate.Init(context.Background(), healthCheck, resilientServer, *cell, *retryCount, tabletTypes) + var vtg *vtgate.VTGate + if *vtgate.GatewayImplementation == vtgate.GatewayImplementationDiscovery { + // default value + legacyHealthCheck = discovery.NewLegacyHealthCheck(*vtgate.HealthCheckRetryDelay, *vtgate.HealthCheckTimeout) + legacyHealthCheck.RegisterStats() + + vtg = vtgate.LegacyInit(context.Background(), legacyHealthCheck, resilientServer, *cell, *vtgate.RetryCount, tabletTypes) + } else { + // use new Init otherwise + vtg = vtgate.Init(context.Background(), resilientServer, *cell, tabletTypes) + } servenv.OnRun(func() { // Flags are parsed now. Parse the template using the actual flag value and overwrite the current template. discovery.ParseTabletURLTemplateFromFlag() addStatusParts(vtg) }) + servenv.OnClose(func() { + _ = vtg.Gateway().Close(context.Background()) + if legacyHealthCheck != nil { + _ = legacyHealthCheck.Close() + } + }) servenv.RunDefault() } diff --git a/go/test/endtoend/tabletgateway/buffer/buffer_test.go b/go/test/endtoend/tabletgateway/buffer/buffer_test.go new file mode 100644 index 00000000000..62d98ab641a --- /dev/null +++ b/go/test/endtoend/tabletgateway/buffer/buffer_test.go @@ -0,0 +1,394 @@ +/* +Copyright 2020 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. +*/ + +/* +Test the vtgate master buffer. + +During a master failover, vtgate should automatically buffer (stall) requests +for a configured time and retry them after the failover is over. + +The test reproduces such a scenario as follows: +- run two threads, the first thread continuously executes a critical read and the second executes a write (UPDATE) +- vtctl PlannedReparentShard runs a master failover +- both threads should not see any error during the failover +*/ + +package buffer + +import ( + "context" + "encoding/json" + "fmt" + "io/ioutil" + "math/rand" + "net/http" + "os" + "reflect" + "strconv" + "strings" + "sync" + "testing" + "time" + + "vitess.io/vitess/go/vt/log" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/test/endtoend/cluster" +) + +var ( + clusterInstance *cluster.LocalProcessCluster + vtParams mysql.ConnParams + keyspaceUnshardedName = "ks1" + cell = "zone1" + hostname = "localhost" + sqlSchema = ` + create table buffer( + id BIGINT NOT NULL, + msg VARCHAR(64) NOT NULL, + PRIMARY KEY (id) + ) Engine=InnoDB;` + wg = &sync.WaitGroup{} +) + +const ( + criticalReadRowID = 1 + updateRowID = 2 + demoteMasterQuery = "SET GLOBAL read_only = ON;FLUSH TABLES WITH READ LOCK;UNLOCK TABLES;" + disableSemiSyncMasterQuery = "SET GLOBAL rpl_semi_sync_master_enabled = 0" + enableSemiSyncMasterQuery = "SET GLOBAL rpl_semi_sync_master_enabled = 1" + promoteSlaveQuery = "STOP SLAVE;RESET SLAVE ALL;SET GLOBAL read_only = OFF;" +) + +//threadParams is set of params passed into read and write threads +type threadParams struct { + writable bool + quit bool + rpcs int // Number of queries successfully executed. + errors int // Number of failed queries. + waitForNotification chan bool // Channel used to notify the main thread that this thread executed + notifyLock sync.Mutex // notifyLock guards the two fields notifyAfterNSuccessfulRpcs/rpcsSoFar. + notifyAfterNSuccessfulRpcs int // If 0, notifications are disabled + rpcsSoFar int // Number of RPCs at the time a notification was requested + i int // + commitErrors int + executeFunction func(c *threadParams, conn *mysql.Conn) error // Implement the method for read/update. +} + +// Thread which constantly executes a query on vtgate. +func (c *threadParams) threadRun() { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &vtParams) + if err != nil { + log.Errorf("error connecting to mysql with params %v: %v", vtParams, err) + } + defer conn.Close() + for !c.quit { + err = c.executeFunction(c, conn) + if err != nil { + c.errors++ + log.Errorf("error executing function %v: %v", c.executeFunction, err) + } + c.rpcs++ + // If notifications are requested, check if we already executed the + // required number of successful RPCs. + // Use >= instead of == because we can miss the exact point due to + // slow thread scheduling. + c.notifyLock.Lock() + if c.notifyAfterNSuccessfulRpcs != 0 && c.rpcs >= (c.notifyAfterNSuccessfulRpcs+c.rpcsSoFar) { + c.waitForNotification <- true + c.notifyAfterNSuccessfulRpcs = 0 + } + c.notifyLock.Unlock() + // Wait 10ms seconds between two attempts. + time.Sleep(10 * time.Millisecond) + } + wg.Done() +} + +func (c *threadParams) setNotifyAfterNSuccessfulRpcs(n int) { + c.notifyLock.Lock() + c.notifyAfterNSuccessfulRpcs = n + c.rpcsSoFar = c.rpcs + c.notifyLock.Unlock() +} + +func (c *threadParams) stop() { + c.quit = true +} + +func readExecute(c *threadParams, conn *mysql.Conn) error { + _, err := conn.ExecuteFetch(fmt.Sprintf("SELECT * FROM buffer WHERE id = %d", criticalReadRowID), 1000, true) + return err +} + +func updateExecute(c *threadParams, conn *mysql.Conn) error { + attempt := c.i + // Value used in next UPDATE query. Increased after every query. + c.i++ + conn.ExecuteFetch("begin", 1000, true) + + result, err := conn.ExecuteFetch(fmt.Sprintf("UPDATE buffer SET msg='update %d' WHERE id = %d", attempt, updateRowID), 1000, true) + + // Sleep between [0, 1] seconds to prolong the time the transaction is in + // flight. This is more realistic because applications are going to keep + // their transactions open for longer as well. + time.Sleep(time.Duration(rand.Int31n(1000)) * time.Millisecond) + + if err == nil { + log.Infof("update attempt #%d affected %v rows", attempt, result.RowsAffected) + _, err = conn.ExecuteFetch("commit", 1000, true) + if err != nil { + _, errRollback := conn.ExecuteFetch("rollback", 1000, true) + if errRollback != nil { + log.Errorf("Error in rollback: %v", errRollback) + } + c.commitErrors++ + if c.commitErrors > 1 { + return err + } + log.Errorf("UPDATE %d failed during ROLLBACK. This is okay once because we do not support buffering it. err: %v", attempt, err) + } + } + if err != nil { + _, errRollback := conn.ExecuteFetch("rollback", 1000, true) + if errRollback != nil { + log.Errorf("Error in rollback: %v", errRollback) + } + c.commitErrors++ + if c.commitErrors > 1 { + return err + } + log.Errorf("UPDATE %d failed during COMMIT with err: %v.This is okay once because we do not support buffering it.", attempt, err) + } + return nil +} + +func createCluster() (*cluster.LocalProcessCluster, int) { + clusterInstance = cluster.NewCluster(cell, hostname) + + // Start topo server + if err := clusterInstance.StartTopo(); err != nil { + return nil, 1 + } + + // Start keyspace + keyspace := &cluster.Keyspace{ + Name: keyspaceUnshardedName, + SchemaSQL: sqlSchema, + } + clusterInstance.VtTabletExtraArgs = []string{"-health_check_interval", "1s"} + + if err := clusterInstance.StartUnshardedKeyspace(*keyspace, 1, false); err != nil { + return nil, 1 + } + + clusterInstance.VtGateExtraArgs = []string{ + "-enable_buffer", + // Long timeout in case failover is slow. + "-buffer_window", "10m", + "-buffer_max_failover_duration", "10m", + "-buffer_min_time_between_failovers", "20m", + "-gateway_implementation", "tabletgateway"} + + // Start vtgate + if err := clusterInstance.StartVtgate(); err != nil { + return nil, 1 + } + vtParams = mysql.ConnParams{ + Host: clusterInstance.Hostname, + Port: clusterInstance.VtgateMySQLPort, + } + rand.Seed(time.Now().UnixNano()) + return clusterInstance, 0 +} + +func exec(t *testing.T, conn *mysql.Conn, query string) *sqltypes.Result { + t.Helper() + qr, err := conn.ExecuteFetch(query, 1000, true) + require.Nil(t, err) + return qr +} + +func TestBufferInternalReparenting(t *testing.T) { + testBufferBase(t, false) +} + +func TestBufferExternalReparenting(t *testing.T) { + testBufferBase(t, true) +} + +func testBufferBase(t *testing.T, isExternalParent bool) { + defer cluster.PanicHandler(t) + clusterInstance, exitCode := createCluster() + if exitCode != 0 { + os.Exit(exitCode) + } + // Healthcheck interval on tablet is set to 1s, so sleep for 2s + time.Sleep(2 * time.Second) + ctx := context.Background() + conn, err := mysql.Connect(ctx, &vtParams) + require.Nil(t, err) + defer conn.Close() + + // Insert two rows for the later threads (critical read, update). + exec(t, conn, fmt.Sprintf("INSERT INTO buffer (id, msg) VALUES (%d, %s)", criticalReadRowID, "'critical read'")) + exec(t, conn, fmt.Sprintf("INSERT INTO buffer (id, msg) VALUES (%d, %s)", updateRowID, "'update'")) + + //Start both threads. + readThreadInstance := &threadParams{writable: false, quit: false, rpcs: 0, errors: 0, notifyAfterNSuccessfulRpcs: 0, rpcsSoFar: 0, executeFunction: readExecute, waitForNotification: make(chan bool)} + wg.Add(1) + go readThreadInstance.threadRun() + updateThreadInstance := &threadParams{writable: false, quit: false, rpcs: 0, errors: 0, notifyAfterNSuccessfulRpcs: 0, rpcsSoFar: 0, executeFunction: updateExecute, i: 1, commitErrors: 0, waitForNotification: make(chan bool)} + wg.Add(1) + go updateThreadInstance.threadRun() + + // Verify they got at least 2 RPCs through. + readThreadInstance.setNotifyAfterNSuccessfulRpcs(2) + updateThreadInstance.setNotifyAfterNSuccessfulRpcs(2) + + <-readThreadInstance.waitForNotification + <-updateThreadInstance.waitForNotification + + // Execute the failover. + readThreadInstance.setNotifyAfterNSuccessfulRpcs(10) + updateThreadInstance.setNotifyAfterNSuccessfulRpcs(10) + + if isExternalParent { + externalReparenting(ctx, t, clusterInstance) + } else { + //reparent call + clusterInstance.VtctlclientProcess.ExecuteCommand("PlannedReparentShard", "-keyspace_shard", + fmt.Sprintf("%s/%s", keyspaceUnshardedName, "0"), + "-new_master", clusterInstance.Keyspaces[0].Shards[0].Vttablets[1].Alias) + } + + <-readThreadInstance.waitForNotification + <-updateThreadInstance.waitForNotification + + // Stop threads + readThreadInstance.stop() + updateThreadInstance.stop() + + // Both threads must not see any error + assert.Equal(t, 0, readThreadInstance.errors) + assert.Equal(t, 0, updateThreadInstance.errors) + + //At least one thread should have been buffered. + //This may fail if a failover is too fast. Add retries then. + resp, err := http.Get(clusterInstance.VtgateProcess.VerifyURL) + require.Nil(t, err) + label := fmt.Sprintf("%s.%s", keyspaceUnshardedName, "0") + inFlightMax := 0 + masterPromotedCount := 0 + durationMs := 0 + bufferingStops := 0 + if resp.StatusCode == 200 { + resultMap := make(map[string]interface{}) + respByte, _ := ioutil.ReadAll(resp.Body) + err := json.Unmarshal(respByte, &resultMap) + if err != nil { + panic(err) + } + inFlightMax = getVarFromVtgate(t, label, "BufferLastRequestsInFlightMax", resultMap) + masterPromotedCount = getVarFromVtgate(t, label, "HealthcheckMasterPromoted", resultMap) + durationMs = getVarFromVtgate(t, label, "BufferFailoverDurationSumMs", resultMap) + bufferingStops = getVarFromVtgate(t, "NewMasterSeen", "BufferStops", resultMap) + } + if inFlightMax == 0 { + // Missed buffering is okay when we observed the failover during the + // COMMIT (which cannot trigger the buffering). + assert.Greater(t, updateThreadInstance.commitErrors, 0, "No buffering took place and the update thread saw no error during COMMIT. But one of it must happen.") + } else { + assert.Greater(t, inFlightMax, 0) + } + + // There was a failover and the HealthCheck module must have seen it. + if masterPromotedCount > 0 { + assert.Greater(t, masterPromotedCount, 0) + } + + if durationMs > 0 { + // Number of buffering stops must be equal to the number of seen failovers. + assert.Equal(t, masterPromotedCount, bufferingStops) + } + wg.Wait() + clusterInstance.Teardown() +} + +func getVarFromVtgate(t *testing.T, label string, param string, resultMap map[string]interface{}) int { + paramVal := 0 + var err error + object := reflect.ValueOf(resultMap[param]) + if object.Kind() == reflect.Map { + for _, key := range object.MapKeys() { + if strings.Contains(key.String(), label) { + v := object.MapIndex(key) + s := fmt.Sprintf("%v", v.Interface()) + paramVal, err = strconv.Atoi(s) + require.Nil(t, err) + } + } + } + return paramVal +} + +func externalReparenting(ctx context.Context, t *testing.T, clusterInstance *cluster.LocalProcessCluster) { + start := time.Now() + + // Demote master Query + master := clusterInstance.Keyspaces[0].Shards[0].Vttablets[0] + replica := clusterInstance.Keyspaces[0].Shards[0].Vttablets[1] + oldMaster := master + newMaster := replica + master.VttabletProcess.QueryTablet(demoteMasterQuery, keyspaceUnshardedName, true) + if master.VttabletProcess.EnableSemiSync { + master.VttabletProcess.QueryTablet(disableSemiSyncMasterQuery, keyspaceUnshardedName, true) + } + + // Wait for replica to catch up to master. + cluster.WaitForReplicationPos(t, master, replica, "localhost", 60.0) + + duration := time.Since(start) + minUnavailabilityInS := 1.0 + if duration.Seconds() < minUnavailabilityInS { + w := minUnavailabilityInS - duration.Seconds() + log.Infof("Waiting for %.1f seconds because the failover was too fast (took only %.3f seconds)", w, duration.Seconds()) + time.Sleep(time.Duration(w) * time.Second) + } + + // Promote replica to new master. + replica.VttabletProcess.QueryTablet(promoteSlaveQuery, keyspaceUnshardedName, true) + + if replica.VttabletProcess.EnableSemiSync { + replica.VttabletProcess.QueryTablet(enableSemiSyncMasterQuery, keyspaceUnshardedName, true) + } + + // Configure old master to replicate from new master. + + _, gtID := cluster.GetMasterPosition(t, *newMaster, hostname) + + // Use 'localhost' as hostname because Travis CI worker hostnames + // are too long for MySQL replication. + changeMasterCommands := fmt.Sprintf("RESET SLAVE;SET GLOBAL gtid_slave_pos = '%s';CHANGE MASTER TO MASTER_HOST='%s', MASTER_PORT=%d ,MASTER_USER='vt_repl', MASTER_USE_GTID = slave_pos;START SLAVE;", gtID, "localhost", newMaster.MySQLPort) + oldMaster.VttabletProcess.QueryTablet(changeMasterCommands, keyspaceUnshardedName, true) + + // Notify the new vttablet master about the reparent. + clusterInstance.VtctlclientProcess.ExecuteCommand("TabletExternallyReparented", newMaster.Alias) +} diff --git a/go/test/endtoend/tabletgateway/cellalias/cell_alias_test.go b/go/test/endtoend/tabletgateway/cellalias/cell_alias_test.go new file mode 100644 index 00000000000..3ee4809ff8f --- /dev/null +++ b/go/test/endtoend/tabletgateway/cellalias/cell_alias_test.go @@ -0,0 +1,370 @@ +/* +Copyright 2020 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. + +This test cell aliases feature + +We start with no aliases and assert that vtgates can't route to replicas/rondly tablets. +Then we add an alias, and these tablets should be routable +*/ + +package binlog + +import ( + "encoding/json" + "flag" + "fmt" + "os" + "os/exec" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/test/endtoend/cluster" + "vitess.io/vitess/go/test/endtoend/sharding" + "vitess.io/vitess/go/vt/proto/topodata" +) + +var ( + localCluster *cluster.LocalProcessCluster + cell1 = "zone1" + cell2 = "zone2" + hostname = "localhost" + keyspaceName = "ks" + tableName = "test_table" + sqlSchema = ` + create table %s( + id bigint(20) unsigned auto_increment, + msg varchar(64), + primary key (id), + index by_msg (msg) + ) Engine=InnoDB +` + commonTabletArg = []string{ + "-vreplication_healthcheck_topology_refresh", "1s", + "-vreplication_healthcheck_retry_delay", "1s", + "-vreplication_retry_delay", "1s", + "-degraded_threshold", "5s", + "-lock_tables_timeout", "5s", + "-watch_replication_stream", + "-enable_replication_reporter", + "-serving_state_grace_period", "1s", + "-binlog_player_protocol", "grpc", + "-enable-autocommit", + } + vSchema = ` + { + "sharded": true, + "vindexes": { + "hash_index": { + "type": "hash" + } + }, + "tables": { + "%s": { + "column_vindexes": [ + { + "column": "id", + "name": "hash_index" + } + ] + } + } + } +` + shard1Master *cluster.Vttablet + shard1Replica *cluster.Vttablet + shard1Rdonly *cluster.Vttablet + shard2Master *cluster.Vttablet + shard2Replica *cluster.Vttablet + shard2Rdonly *cluster.Vttablet +) + +func TestMain(m *testing.M) { + defer cluster.PanicHandler(nil) + flag.Parse() + + exitcode, err := func() (int, error) { + localCluster = cluster.NewCluster(cell1, hostname) + defer localCluster.Teardown() + localCluster.Keyspaces = append(localCluster.Keyspaces, cluster.Keyspace{ + Name: keyspaceName, + }) + + // Start topo server + if err := localCluster.StartTopo(); err != nil { + return 1, err + } + + // Adding another cell in the same cluster + err := localCluster.TopoProcess.ManageTopoDir("mkdir", "/vitess/"+cell2) + if err != nil { + return 1, err + } + err = localCluster.VtctlProcess.AddCellInfo(cell2) + if err != nil { + return 1, err + } + + shard1Master = localCluster.GetVttabletInstance("master", 0, "") + shard1Replica = localCluster.GetVttabletInstance("replica", 0, cell2) + shard1Rdonly = localCluster.GetVttabletInstance("rdonly", 0, cell2) + + shard2Master = localCluster.GetVttabletInstance("master", 0, "") + shard2Replica = localCluster.GetVttabletInstance("replica", 0, cell2) + shard2Rdonly = localCluster.GetVttabletInstance("rdonly", 0, cell2) + + var mysqlProcs []*exec.Cmd + for _, tablet := range []*cluster.Vttablet{shard1Master, shard1Replica, shard1Rdonly, shard2Master, shard2Replica, shard2Rdonly} { + tablet.MysqlctlProcess = *cluster.MysqlCtlProcessInstance(tablet.TabletUID, tablet.MySQLPort, localCluster.TmpDirectory) + tablet.VttabletProcess = cluster.VttabletProcessInstance(tablet.HTTPPort, + tablet.GrpcPort, + tablet.TabletUID, + tablet.Cell, + "", + keyspaceName, + localCluster.VtctldProcess.Port, + tablet.Type, + localCluster.TopoPort, + hostname, + localCluster.TmpDirectory, + commonTabletArg, + true, + ) + tablet.VttabletProcess.SupportsBackup = true + proc, err := tablet.MysqlctlProcess.StartProcess() + if err != nil { + return 1, err + } + mysqlProcs = append(mysqlProcs, proc) + } + for _, proc := range mysqlProcs { + if err := proc.Wait(); err != nil { + return 1, err + } + } + + if err := localCluster.VtctlProcess.CreateKeyspace(keyspaceName); err != nil { + return 1, err + } + + shard1 := cluster.Shard{ + Name: "-80", + Vttablets: []*cluster.Vttablet{shard1Master, shard1Replica, shard1Rdonly}, + } + for idx := range shard1.Vttablets { + shard1.Vttablets[idx].VttabletProcess.Shard = shard1.Name + } + localCluster.Keyspaces[0].Shards = append(localCluster.Keyspaces[0].Shards, shard1) + + shard2 := cluster.Shard{ + Name: "80-", + Vttablets: []*cluster.Vttablet{shard2Master, shard2Replica, shard2Rdonly}, + } + for idx := range shard2.Vttablets { + shard2.Vttablets[idx].VttabletProcess.Shard = shard2.Name + } + localCluster.Keyspaces[0].Shards = append(localCluster.Keyspaces[0].Shards, shard2) + + for _, tablet := range shard1.Vttablets { + if err := localCluster.VtctlclientProcess.InitTablet(tablet, tablet.Cell, keyspaceName, hostname, shard1.Name); err != nil { + return 1, err + } + if err := tablet.VttabletProcess.CreateDB(keyspaceName); err != nil { + return 1, err + } + if err := tablet.VttabletProcess.Setup(); err != nil { + return 1, err + } + } + if err := localCluster.VtctlclientProcess.InitShardMaster(keyspaceName, shard1.Name, shard1Master.Cell, shard1Master.TabletUID); err != nil { + return 1, err + } + + // run a health check on source replica so it responds to discovery + // (for binlog players) and on the source rdonlys (for workers) + for _, tablet := range []string{shard1Replica.Alias, shard1Rdonly.Alias} { + if err := localCluster.VtctlclientProcess.ExecuteCommand("RunHealthCheck", tablet); err != nil { + return 1, err + } + } + + for _, tablet := range shard2.Vttablets { + if err := localCluster.VtctlclientProcess.InitTablet(tablet, tablet.Cell, keyspaceName, hostname, shard2.Name); err != nil { + return 1, err + } + if err := tablet.VttabletProcess.CreateDB(keyspaceName); err != nil { + return 1, err + } + if err := tablet.VttabletProcess.Setup(); err != nil { + return 1, err + } + } + + if err := localCluster.VtctlclientProcess.InitShardMaster(keyspaceName, shard2.Name, shard2Master.Cell, shard2Master.TabletUID); err != nil { + return 1, err + } + + if err := localCluster.VtctlclientProcess.ApplySchema(keyspaceName, fmt.Sprintf(sqlSchema, tableName)); err != nil { + return 1, err + } + if err := localCluster.VtctlclientProcess.ApplyVSchema(keyspaceName, fmt.Sprintf(vSchema, tableName)); err != nil { + return 1, err + } + + _ = localCluster.VtctlclientProcess.ExecuteCommand("RebuildKeyspaceGraph", keyspaceName) + + return m.Run(), nil + }() + if err != nil { + fmt.Printf("%v\n", err) + os.Exit(1) + } else { + os.Exit(exitcode) + } +} + +func TestAlias(t *testing.T) { + defer cluster.PanicHandler(t) + + insertInitialValues(t) + defer deleteInitialValues(t) + + err := localCluster.VtctlclientProcess.ExecuteCommand("RebuildKeyspaceGraph", keyspaceName) + require.Nil(t, err) + shard1 := localCluster.Keyspaces[0].Shards[0] + shard2 := localCluster.Keyspaces[0].Shards[1] + allCells := fmt.Sprintf("%s,%s", cell1, cell2) + + expectedPartitions := map[topodata.TabletType][]string{} + expectedPartitions[topodata.TabletType_MASTER] = []string{shard1.Name, shard2.Name} + expectedPartitions[topodata.TabletType_REPLICA] = []string{shard1.Name, shard2.Name} + expectedPartitions[topodata.TabletType_RDONLY] = []string{shard1.Name, shard2.Name} + sharding.CheckSrvKeyspace(t, cell1, keyspaceName, "", 0, expectedPartitions, *localCluster) + sharding.CheckSrvKeyspace(t, cell2, keyspaceName, "", 0, expectedPartitions, *localCluster) + + // Adds alias so vtgate can route to replica/rdonly tablets that are not in the same cell, but same alias + err = localCluster.VtctlclientProcess.ExecuteCommand("AddCellsAlias", + "-cells", allCells, + "region_east_coast") + require.Nil(t, err) + err = localCluster.VtctlclientProcess.ExecuteCommand("UpdateCellsAlias", + "-cells", allCells, + "region_east_coast") + require.Nil(t, err) + + vtgateInstance := localCluster.GetVtgateInstance() + vtgateInstance.CellsToWatch = allCells + vtgateInstance.TabletTypesToWait = "MASTER,REPLICA" + vtgateInstance.GatewayImplementation = "tabletgateway" + err = vtgateInstance.Setup() + require.Nil(t, err) + + // Cluster teardown will not teardown vtgate because we are not + // actually setting this on localCluster.VtgateInstance + defer vtgateInstance.TearDown() + + waitTillAllTabletsAreHealthyInVtgate(t, *vtgateInstance, shard1.Name, shard2.Name) + + testQueriesOnTabletType(t, "master", vtgateInstance.GrpcPort, false) + testQueriesOnTabletType(t, "replica", vtgateInstance.GrpcPort, false) + testQueriesOnTabletType(t, "rdonly", vtgateInstance.GrpcPort, false) + + // now, delete the alias, so that if we run above assertions again, it will fail for replica,rdonly target type + err = localCluster.VtctlclientProcess.ExecuteCommand("DeleteCellsAlias", + "region_east_coast") + require.Nil(t, err) + + // restarts the vtgate process + vtgateInstance.TabletTypesToWait = "MASTER" + err = vtgateInstance.TearDown() + require.Nil(t, err) + err = vtgateInstance.Setup() + require.Nil(t, err) + + // since replica and rdonly tablets of all shards in cell2, the last 2 assertion is expected to fail + testQueriesOnTabletType(t, "master", vtgateInstance.GrpcPort, false) + testQueriesOnTabletType(t, "replica", vtgateInstance.GrpcPort, true) + testQueriesOnTabletType(t, "rdonly", vtgateInstance.GrpcPort, true) + +} + +func waitTillAllTabletsAreHealthyInVtgate(t *testing.T, vtgateInstance cluster.VtgateProcess, shards ...string) { + for _, shard := range shards { + err := vtgateInstance.WaitForStatusOfTabletInShard(fmt.Sprintf("%s.%s.master", keyspaceName, shard), 1) + require.Nil(t, err) + err = vtgateInstance.WaitForStatusOfTabletInShard(fmt.Sprintf("%s.%s.replica", keyspaceName, shard), 1) + require.Nil(t, err) + err = vtgateInstance.WaitForStatusOfTabletInShard(fmt.Sprintf("%s.%s.rdonly", keyspaceName, shard), 1) + require.Nil(t, err) + } +} + +func testQueriesOnTabletType(t *testing.T, tabletType string, vtgateGrpcPort int, shouldFail bool) { + output, err := localCluster.VtctlProcess.ExecuteCommandWithOutput("VtGateExecute", "-json", + "-server", fmt.Sprintf("%s:%d", localCluster.Hostname, vtgateGrpcPort), + "-target", "@"+tabletType, + fmt.Sprintf(`select * from %s`, tableName)) + if shouldFail { + require.Error(t, err) + return + } + require.Nil(t, err) + var result sqltypes.Result + + err = json.Unmarshal([]byte(output), &result) + require.Nil(t, err) + assert.Equal(t, len(result.Rows), 3) +} + +func insertInitialValues(t *testing.T) { + sharding.ExecuteOnTablet(t, + fmt.Sprintf(sharding.InsertTabletTemplateKsID, tableName, 1, "msg1", 1), + *shard1Master, + keyspaceName, + false) + + sharding.ExecuteOnTablet(t, + fmt.Sprintf(sharding.InsertTabletTemplateKsID, tableName, 2, "msg2", 2), + *shard1Master, + keyspaceName, + false) + + sharding.ExecuteOnTablet(t, + fmt.Sprintf(sharding.InsertTabletTemplateKsID, tableName, 4, "msg4", 4), + *shard2Master, + keyspaceName, + false) +} + +func deleteInitialValues(t *testing.T) { + sharding.ExecuteOnTablet(t, + fmt.Sprintf("delete from %s where id = %v", tableName, 1), + *shard1Master, + keyspaceName, + false) + + sharding.ExecuteOnTablet(t, + fmt.Sprintf("delete from %s where id = %v", tableName, 2), + *shard1Master, + keyspaceName, + false) + + sharding.ExecuteOnTablet(t, + fmt.Sprintf("delete from %s where id = %v", tableName, 4), + *shard2Master, + keyspaceName, + false) +} diff --git a/go/test/endtoend/tabletgateway/healthcheck/main_test.go b/go/test/endtoend/tabletgateway/healthcheck/main_test.go new file mode 100644 index 00000000000..e2054fc2554 --- /dev/null +++ b/go/test/endtoend/tabletgateway/healthcheck/main_test.go @@ -0,0 +1,103 @@ +/* +Copyright 2020 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 healthcheck + +import ( + "flag" + "os" + "testing" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/test/endtoend/cluster" +) + +var ( + clusterInstance *cluster.LocalProcessCluster + vtParams mysql.ConnParams + keyspaceName = "commerce" + cell = "zone1" + sqlSchema = `create table product( + sku varbinary(128), + description varbinary(128), + price bigint, + primary key(sku) + ) ENGINE=InnoDB; + create table customer( + id bigint not null auto_increment, + email varchar(128), + primary key(id) + ) ENGINE=InnoDB; + create table corder( + order_id bigint not null auto_increment, + customer_id bigint, + sku varbinary(128), + price bigint, + primary key(order_id) + ) ENGINE=InnoDB;` + + vSchema = `{ + "tables": { + "product": {}, + "customer": {}, + "corder": {} + } + }` +) + +func TestMain(m *testing.M) { + defer cluster.PanicHandler(nil) + flag.Parse() + + exitCode := func() int { + clusterInstance = cluster.NewCluster(cell, "localhost") + clusterInstance.VtTabletExtraArgs = []string{"-health_check_interval", "1s"} + defer clusterInstance.Teardown() + + // Start topo server + err := clusterInstance.StartTopo() + if err != nil { + return 1 + } + + // Start keyspace + keyspace := &cluster.Keyspace{ + Name: keyspaceName, + SchemaSQL: sqlSchema, + VSchema: vSchema, + } + err = clusterInstance.StartUnshardedKeyspace(*keyspace, 1, true) + if err != nil { + return 1 + } + + // Start vtgate + vtgateInstance := clusterInstance.GetVtgateInstance() + vtgateInstance.GatewayImplementation = "tabletgateway" + err = vtgateInstance.Setup() + if err != nil { + return 1 + } + // ensure it is torn down during cluster TearDown + clusterInstance.VtgateProcess = *vtgateInstance + vtParams = mysql.ConnParams{ + Host: clusterInstance.Hostname, + Port: clusterInstance.VtgateMySQLPort, + } + return m.Run() + }() + os.Exit(exitCode) +} diff --git a/go/test/endtoend/tabletgateway/healthcheck/vtgate_test.go b/go/test/endtoend/tabletgateway/healthcheck/vtgate_test.go new file mode 100644 index 00000000000..26925afd96d --- /dev/null +++ b/go/test/endtoend/tabletgateway/healthcheck/vtgate_test.go @@ -0,0 +1,107 @@ +/* +Copyright 2020 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. + +This tests select/insert using the unshared keyspace added in main_test +*/ +package healthcheck + +import ( + "context" + "encoding/json" + "fmt" + "io/ioutil" + "net/http" + "reflect" + "strings" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/test/endtoend/cluster" +) + +func TestVtgateProcess(t *testing.T) { + defer cluster.PanicHandler(t) + // Healthcheck interval on tablet is set to 1s, so sleep for 2s + time.Sleep(2 * time.Second) + verifyVtgateVariables(t, clusterInstance.VtgateProcess.VerifyURL) + ctx := context.Background() + conn, err := mysql.Connect(ctx, &vtParams) + require.Nil(t, err) + defer conn.Close() + + exec(t, conn, "insert into customer(id, email) values(1,'email1')") + + qr := exec(t, conn, "select id, email from customer") + assert.Equal(t, fmt.Sprintf("%v", qr.Rows), `[[INT64(1) VARCHAR("email1")]]`, "select returned wrong result") + qr = exec(t, conn, "show vitess_tablets") + assert.Equal(t, len(qr.Rows), 3, "wrong number of results from show") +} + +func verifyVtgateVariables(t *testing.T, url string) { + resp, _ := http.Get(url) + require.True(t, resp != nil && resp.StatusCode == 200, "Vtgate api url response not found") + resultMap := make(map[string]interface{}) + respByte, _ := ioutil.ReadAll(resp.Body) + err := json.Unmarshal(respByte, &resultMap) + require.Nil(t, err) + assert.True(t, resultMap["VtgateVSchemaCounts"] != nil, "Vschema count should be present in variables") + vschemaCountMap := getMapFromJSON(resultMap, "VtgateVSchemaCounts") + _, present := vschemaCountMap["Reload"] + assert.True(t, present, "Reload count should be present in vschemacount") + object := reflect.ValueOf(vschemaCountMap["Reload"]) + assert.True(t, object.NumField() > 0, "Reload count should be greater than 0") + _, present = vschemaCountMap["WatchError"] + assert.False(t, present, "There should not be any WatchError in VschemaCount") + _, present = vschemaCountMap["Parsing"] + assert.False(t, present, "There should not be any Parsing in VschemaCount") + + assert.True(t, resultMap["HealthcheckConnections"] != nil, "HealthcheckConnections count should be present in variables") + healthCheckConnection := getMapFromJSON(resultMap, "HealthcheckConnections") + assert.True(t, len(healthCheckConnection) > 0, "Atleast one healthy tablet needs to be present") + assert.True(t, isMasterTabletPresent(healthCheckConnection), "Atleast one master tablet needs to be present") +} + +func getMapFromJSON(JSON map[string]interface{}, key string) map[string]interface{} { + result := make(map[string]interface{}) + object := reflect.ValueOf(JSON[key]) + if object.Kind() == reflect.Map { + for _, key := range object.MapKeys() { + value := object.MapIndex(key) + result[key.String()] = value + } + } + return result +} + +func isMasterTabletPresent(tablets map[string]interface{}) bool { + for key := range tablets { + if strings.Contains(key, "master") { + return true + } + } + return false +} + +func exec(t *testing.T, conn *mysql.Conn, query string) *sqltypes.Result { + t.Helper() + qr, err := conn.ExecuteFetch(query, 1000, true) + require.Nil(t, err) + return qr +} diff --git a/go/vt/discovery/fake_healthcheck.go b/go/vt/discovery/fake_healthcheck.go index ed3fe2a5996..460aa5a934d 100644 --- a/go/vt/discovery/fake_healthcheck.go +++ b/go/vt/discovery/fake_healthcheck.go @@ -19,9 +19,6 @@ package discovery import ( "sort" "sync" - "time" - - "vitess.io/vitess/go/vt/logutil" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" @@ -33,7 +30,7 @@ import ( ) // This file contains the definitions for a FakeHealthCheck class to -// simulate a HealthCheck module. Note it is not in a sub-package because +// simulate a LegacyHealthCheck module. Note it is not in a sub-package because // otherwise it couldn't be used in this package's tests because of // circular dependencies. @@ -44,43 +41,35 @@ func NewFakeHealthCheck() *FakeHealthCheck { } } -// FakeHealthCheck implements discovery.HealthCheck. +// FakeHealthCheck implements discovery.LegacyHealthCheck. type FakeHealthCheck struct { - listener HealthCheckStatsListener - // mu protects the items map mu sync.RWMutex items map[string]*fhcItem } type fhcItem struct { - ts *TabletStats + ts *TabletHealth conn queryservice.QueryService } // -// discovery.HealthCheck interface methods +// discovery.LegacyHealthCheck interface methods // // RegisterStats is not implemented. func (fhc *FakeHealthCheck) RegisterStats() { } -// SetListener is not implemented. -func (fhc *FakeHealthCheck) SetListener(listener HealthCheckStatsListener, sendDownEvents bool) { - fhc.listener = listener -} - // WaitForInitialStatsUpdates is not implemented. func (fhc *FakeHealthCheck) WaitForInitialStatsUpdates() { } // AddTablet adds the tablet and calls the listener. -func (fhc *FakeHealthCheck) AddTablet(tablet *topodatapb.Tablet, name string) { +func (fhc *FakeHealthCheck) AddTablet(tablet *topodatapb.Tablet) { key := TabletToMapKey(tablet) item := &fhcItem{ - ts: &TabletStats{ - Key: key, + ts: &TabletHealth{ Tablet: tablet, Target: &querypb.Target{ Keyspace: tablet.Keyspace, @@ -88,8 +77,6 @@ func (fhc *FakeHealthCheck) AddTablet(tablet *topodatapb.Tablet, name string) { TabletType: tablet.Type, }, Serving: true, - Up: true, - Name: name, Stats: &querypb.RealtimeStats{}, }, } @@ -97,10 +84,6 @@ func (fhc *FakeHealthCheck) AddTablet(tablet *topodatapb.Tablet, name string) { fhc.mu.Lock() defer fhc.mu.Unlock() fhc.items[key] = item - - if fhc.listener != nil { - fhc.listener.StatsUpdate(item.ts) - } } // RemoveTablet removes the tablet. @@ -123,9 +106,9 @@ func (fhc *FakeHealthCheck) RemoveTablet(tablet *topodatapb.Tablet) { } // ReplaceTablet removes the old tablet and adds the new. -func (fhc *FakeHealthCheck) ReplaceTablet(old, new *topodatapb.Tablet, name string) { +func (fhc *FakeHealthCheck) ReplaceTablet(old, new *topodatapb.Tablet) { fhc.RemoveTablet(old) - fhc.AddTablet(new, name) + fhc.AddTablet(new) } // GetConnection returns the TabletConn of the given tablet. @@ -182,8 +165,6 @@ func (fhc *FakeHealthCheck) AddFakeTablet(cell, host string, port int32, keyspac t.Shard = shard t.Type = tabletType t.PortMap["vt"] = port - // reparentTS only has precision to seconds - t.MasterTermStartTime = logutil.TimeToProto(time.Unix(reparentTS, 0)) key := TabletToMapKey(t) fhc.mu.Lock() @@ -191,10 +172,8 @@ func (fhc *FakeHealthCheck) AddFakeTablet(cell, host string, port int32, keyspac item := fhc.items[key] if item == nil { item = &fhcItem{ - ts: &TabletStats{ - Key: key, + ts: &TabletHealth{ Tablet: t, - Up: true, }, } fhc.items[key] = item @@ -205,15 +184,12 @@ func (fhc *FakeHealthCheck) AddFakeTablet(cell, host string, port int32, keyspac TabletType: tabletType, } item.ts.Serving = serving - item.ts.TabletExternallyReparentedTimestamp = reparentTS + item.ts.MasterTermStartTime = reparentTS item.ts.Stats = &querypb.RealtimeStats{} item.ts.LastError = err conn := connFactory(t) item.conn = conn - if fhc.listener != nil { - fhc.listener.StatsUpdate(item.ts) - } return conn } diff --git a/go/vt/discovery/fake_legacy_healthcheck.go b/go/vt/discovery/fake_legacy_healthcheck.go new file mode 100644 index 00000000000..3abca835fd2 --- /dev/null +++ b/go/vt/discovery/fake_legacy_healthcheck.go @@ -0,0 +1,238 @@ +/* +Copyright 2019 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 discovery + +import ( + "sort" + "sync" + "time" + + "vitess.io/vitess/go/vt/logutil" + + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/vttablet/queryservice" + "vitess.io/vitess/go/vt/vttablet/sandboxconn" + + querypb "vitess.io/vitess/go/vt/proto/query" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +// This file contains the definitions for a FakeLegacyHealthCheck class to +// simulate a LegacyHealthCheck module. Note it is not in a sub-package because +// otherwise it couldn't be used in this package's tests because of +// circular dependencies. + +// NewFakeLegacyHealthCheck returns the fake healthcheck object. +func NewFakeLegacyHealthCheck() *FakeLegacyHealthCheck { + return &FakeLegacyHealthCheck{ + items: make(map[string]*flhcItem), + } +} + +// FakeLegacyHealthCheck implements discovery.LegacyHealthCheck. +type FakeLegacyHealthCheck struct { + listener LegacyHealthCheckStatsListener + + // mu protects the items map + mu sync.RWMutex + items map[string]*flhcItem +} + +type flhcItem struct { + ts *LegacyTabletStats + conn queryservice.QueryService +} + +// +// discovery.LegacyHealthCheck interface methods +// + +// RegisterStats is not implemented. +func (fhc *FakeLegacyHealthCheck) RegisterStats() { +} + +// SetListener is not implemented. +func (fhc *FakeLegacyHealthCheck) SetListener(listener LegacyHealthCheckStatsListener, sendDownEvents bool) { + fhc.listener = listener +} + +// WaitForInitialStatsUpdates is not implemented. +func (fhc *FakeLegacyHealthCheck) WaitForInitialStatsUpdates() { +} + +// AddTablet adds the tablet and calls the listener. +func (fhc *FakeLegacyHealthCheck) AddTablet(tablet *topodatapb.Tablet, name string) { + key := TabletToMapKey(tablet) + item := &flhcItem{ + ts: &LegacyTabletStats{ + Key: key, + Tablet: tablet, + Target: &querypb.Target{ + Keyspace: tablet.Keyspace, + Shard: tablet.Shard, + TabletType: tablet.Type, + }, + Serving: true, + Up: true, + Name: name, + Stats: &querypb.RealtimeStats{}, + }, + } + + fhc.mu.Lock() + defer fhc.mu.Unlock() + fhc.items[key] = item + + if fhc.listener != nil { + fhc.listener.StatsUpdate(item.ts) + } +} + +// RemoveTablet removes the tablet. +func (fhc *FakeLegacyHealthCheck) RemoveTablet(tablet *topodatapb.Tablet) { + fhc.mu.Lock() + defer fhc.mu.Unlock() + key := TabletToMapKey(tablet) + item, ok := fhc.items[key] + if !ok { + return + } + // Make sure the key still corresponds to the tablet we want to delete. + // If it doesn't match, we should do nothing. The tablet we were asked to + // delete is already gone, and some other tablet is using the key + // (host:port) that the original tablet used to use, which is fine. + if !topoproto.TabletAliasEqual(tablet.Alias, item.ts.Tablet.Alias) { + return + } + delete(fhc.items, key) +} + +// ReplaceTablet removes the old tablet and adds the new. +func (fhc *FakeLegacyHealthCheck) ReplaceTablet(old, new *topodatapb.Tablet, name string) { + fhc.RemoveTablet(old) + fhc.AddTablet(new, name) +} + +// GetConnection returns the TabletConn of the given tablet. +func (fhc *FakeLegacyHealthCheck) GetConnection(key string) queryservice.QueryService { + fhc.mu.RLock() + defer fhc.mu.RUnlock() + if item := fhc.items[key]; item != nil { + return item.conn + } + return nil +} + +// CacheStatus returns the status for each tablet +func (fhc *FakeLegacyHealthCheck) CacheStatus() LegacyTabletsCacheStatusList { + fhc.mu.Lock() + defer fhc.mu.Unlock() + + stats := make(LegacyTabletsCacheStatusList, 0, len(fhc.items)) + for _, item := range fhc.items { + stats = append(stats, &LegacyTabletsCacheStatus{ + Cell: "FakeCell", + Target: item.ts.Target, + TabletsStats: LegacyTabletStatsList{item.ts}, + }) + } + sort.Sort(stats) + return stats +} + +// Close is not implemented. +func (fhc *FakeLegacyHealthCheck) Close() error { + return nil +} + +// +// Management methods +// + +// Reset cleans up the internal state. +func (fhc *FakeLegacyHealthCheck) Reset() { + fhc.mu.Lock() + defer fhc.mu.Unlock() + + fhc.items = make(map[string]*flhcItem) +} + +// AddFakeTablet inserts a fake entry into FakeLegacyHealthCheck. +// The Tablet can be talked to using the provided connection. +// The Listener is called, as if AddTablet had been called. +// For flexibility the connection is created via a connFactory callback +func (fhc *FakeLegacyHealthCheck) AddFakeTablet(cell, host string, port int32, keyspace, shard string, tabletType topodatapb.TabletType, serving bool, reparentTS int64, err error, connFactory func(*topodatapb.Tablet) queryservice.QueryService) queryservice.QueryService { + t := topo.NewTablet(0, cell, host) + t.Keyspace = keyspace + t.Shard = shard + t.Type = tabletType + t.PortMap["vt"] = port + // reparentTS only has precision to seconds + t.MasterTermStartTime = logutil.TimeToProto(time.Unix(reparentTS, 0)) + key := TabletToMapKey(t) + + fhc.mu.Lock() + defer fhc.mu.Unlock() + item := fhc.items[key] + if item == nil { + item = &flhcItem{ + ts: &LegacyTabletStats{ + Key: key, + Tablet: t, + Up: true, + }, + } + fhc.items[key] = item + } + item.ts.Target = &querypb.Target{ + Keyspace: keyspace, + Shard: shard, + TabletType: tabletType, + } + item.ts.Serving = serving + item.ts.TabletExternallyReparentedTimestamp = reparentTS + item.ts.Stats = &querypb.RealtimeStats{} + item.ts.LastError = err + conn := connFactory(t) + item.conn = conn + + if fhc.listener != nil { + fhc.listener.StatsUpdate(item.ts) + } + return conn +} + +// AddTestTablet adds a fake tablet for tests using the SandboxConn and returns +// the fake connection +func (fhc *FakeLegacyHealthCheck) AddTestTablet(cell, host string, port int32, keyspace, shard string, tabletType topodatapb.TabletType, serving bool, reparentTS int64, err error) *sandboxconn.SandboxConn { + conn := fhc.AddFakeTablet(cell, host, port, keyspace, shard, tabletType, serving, reparentTS, err, func(tablet *topodatapb.Tablet) queryservice.QueryService { + return sandboxconn.NewSandboxConn(tablet) + }) + return conn.(*sandboxconn.SandboxConn) +} + +// GetAllTablets returns all the tablets we have. +func (fhc *FakeLegacyHealthCheck) GetAllTablets() map[string]*topodatapb.Tablet { + res := make(map[string]*topodatapb.Tablet) + fhc.mu.RLock() + defer fhc.mu.RUnlock() + for key, t := range fhc.items { + res[key] = t.ts.Tablet + } + return res +} diff --git a/go/vt/discovery/healthcheck.go b/go/vt/discovery/healthcheck.go index 9a39e54a0dc..5b6fceec399 100644 --- a/go/vt/discovery/healthcheck.go +++ b/go/vt/discovery/healthcheck.go @@ -1,5 +1,5 @@ /* -Copyright 2019 The Vitess Authors. +Copyright 2020 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. @@ -19,18 +19,13 @@ limitations under the License. // // Use the HealthCheck object to query for tablets and their health. // -// For an example how to use the HealthCheck object, see worker/topo_utils.go. +// For an example how to use the HealthCheck object, see vtgate/tabletgateway.go // // Tablets have to be manually added to the HealthCheck using AddTablet(). // Alternatively, use a Watcher implementation which will constantly watch // a source (e.g. the topology) and add and remove tablets as they are // added or removed from the source. -// For a Watcher example have a look at NewShardReplicationWatcher(). -// -// Each HealthCheck has a HealthCheckStatsListener that will receive -// notification of when tablets go up and down. -// TabletStatsCache is one implementation, that caches the known tablets -// and the healthy ones per keyspace/shard/tabletType. +// For a Watcher example have a look at NewCellTabletsWatcher(). // // Internally, the HealthCheck module is connected to each tablet and has a // streaming RPC (StreamHealth) open to receive periodic health infos. @@ -38,6 +33,7 @@ package discovery import ( "bytes" + "context" "encoding/json" "flag" "fmt" @@ -49,21 +45,15 @@ import ( "sync" "time" - "github.com/golang/protobuf/proto" - "golang.org/x/net/context" - "vitess.io/vitess/go/netutil" + "vitess.io/vitess/go/flagutil" + + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/stats" - "vitess.io/vitess/go/sync2" - "vitess.io/vitess/go/vt/grpcclient" "vitess.io/vitess/go/vt/log" - "vitess.io/vitess/go/vt/servenv" + "vitess.io/vitess/go/vt/proto/query" + "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/topo/topoproto" - "vitess.io/vitess/go/vt/topotools" - "vitess.io/vitess/go/vt/vttablet/queryservice" - "vitess.io/vitess/go/vt/vttablet/tabletconn" - - querypb "vitess.io/vitess/go/vt/proto/query" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) var ( @@ -74,6 +64,24 @@ var ( // TabletURLTemplateString is a flag to generate URLs for the tablets that vtgate discovers. TabletURLTemplateString = flag.String("tablet_url_template", "http://{{.GetTabletHostPort}}", "format string describing debug tablet url formatting. See the Go code for getTabletDebugURL() how to customize this.") tabletURLTemplate *template.Template + + //TODO(deepthi): change these vars back to unexported when discoveryGateway is removed + + // CellsToWatch is the list of cells the healthcheck operates over. If it is empty, only the local cell is watched + CellsToWatch = flag.String("cells_to_watch", "", "comma-separated list of cells for watching tablets") + // AllowedTabletTypes is the list of allowed tablet types. e.g. {MASTER, REPLICA} + AllowedTabletTypes []topodata.TabletType + // TabletFilters are the keyspace|shard or keyrange filters to apply to the full set of tablets + TabletFilters flagutil.StringListValue + // KeyspacesToWatch - if provided this specifies which keyspaces should be + // visible to the healthcheck. By default the healthcheck will watch all keyspaces. + KeyspacesToWatch flagutil.StringListValue + // RefreshInterval is the interval at which healthcheck refreshes its list of tablets from topo + RefreshInterval = flag.Duration("tablet_refresh_interval", 1*time.Minute, "tablet refresh interval") + // RefreshKnownTablets tells us whether to process all tablets or only new tablets + RefreshKnownTablets = flag.Bool("tablet_refresh_known_tablets", true, "tablet refresh reloads the tablet address/port map from topo in case it changes") + // TopoReadConcurrency tells us how many topo reads are allowed in parallel + TopoReadConcurrency = flag.Int("topo_read_concurrency", 32, "concurrent topo reads") ) // See the documentation for NewHealthCheck below for an explanation of these parameters. @@ -81,12 +89,11 @@ const ( DefaultHealthCheckRetryDelay = 5 * time.Second DefaultHealthCheckTimeout = 1 * time.Minute - // DefaultTopoReadConcurrency can be used as default value for the topoReadConcurrency parameter of a TopologyWatcher. + // DefaultTopoReadConcurrency is used as the default value for the TopoReadConcurrency parameter of a TopologyWatcher. DefaultTopoReadConcurrency int = 5 - // DefaultTopologyWatcherRefreshInterval can be used as the default value for + // DefaultTopologyWatcherRefreshInterval is used as the default value for // the refresh interval of a topology watcher. DefaultTopologyWatcherRefreshInterval = 1 * time.Minute - // HealthCheckTemplate is the HTML code to display a TabletsCacheStatusList HealthCheckTemplate = ` + + + + + + + + + + + + {{range $i, $ts := .}} + + + + + + + + {{end}} +
HealthCheck Tablet Cache
CellKeyspaceShardTabletTypetabletStats
{{github_com_vitessio_vitess_vtctld_srv_cell $ts.Cell}}{{github_com_vitessio_vitess_vtctld_srv_keyspace $ts.Cell $ts.Target.Keyspace}}{{$ts.Target.Shard}}{{$ts.Target.TabletType}}{{$ts.StatusAsHTML}}
+` +) + +func init() { + // Flags are not parsed at this point and the default value of the flag (just the hostname) will be used. + ParseTabletURLTemplateFromFlag() +} + +// LegacyHealthCheckStatsListener is the listener to receive health check stats update. +type LegacyHealthCheckStatsListener interface { + // StatsUpdate is called when: + // - a new tablet is known to the LegacyHealthCheck, and its first + // streaming healthcheck is returned. (then ts.Up is true). + // - a tablet is removed from the list of tablets we watch + // (then ts.Up is false). + // - a tablet dynamically changes its type. When registering the + // listener, if sendDownEvents is true, two events are generated + // (ts.Up false on the old type, ts.Up true on the new type). + // If it is false, only one event is sent (ts.Up true on the new + // type). + StatsUpdate(*LegacyTabletStats) +} + +// LegacyTabletStats is returned when getting the set of tablets. +type LegacyTabletStats struct { + // Key uniquely identifies that serving tablet. It is computed + // from the Tablet's record Hostname and PortMap. If a tablet + // is restarted on different ports, its Key will be different. + // Key is computed using the TabletToMapKey method below. + // key can be used in GetConnection(). + Key string + // Tablet is the tablet object that was sent to LegacyHealthCheck.AddTablet. + Tablet *topodatapb.Tablet + // Name is an optional tag (e.g. alternative address) for the + // tablet. It is supposed to represent the tablet as a task, + // not as a process. For instance, it can be a + // cell+keyspace+shard+tabletType+taskIndex value. + Name string + // Target is the current target as returned by the streaming + // StreamHealth RPC. + Target *querypb.Target + // Up describes whether the tablet is added or removed. + Up bool + // Serving describes if the tablet can be serving traffic. + Serving bool + // TabletExternallyReparentedTimestamp is the last timestamp + // that this tablet was either elected the master, or received + // a TabletExternallyReparented event. It is set to 0 if the + // tablet doesn't think it's a master. + TabletExternallyReparentedTimestamp int64 + // Stats is the current health status, as received by the + // StreamHealth RPC (replication lag, ...). + Stats *querypb.RealtimeStats + // LastError is the error we last saw when trying to get the + // tablet's healthcheck. + LastError error +} + +// String is defined because we want to print a []*LegacyTabletStats array nicely. +func (e *LegacyTabletStats) String() string { + return fmt.Sprint(*e) +} + +// DeepEqual compares two LegacyTabletStats. Since we include protos, we +// need to use proto.Equal on these. +func (e *LegacyTabletStats) DeepEqual(f *LegacyTabletStats) bool { + return e.Key == f.Key && + proto.Equal(e.Tablet, f.Tablet) && + e.Name == f.Name && + proto.Equal(e.Target, f.Target) && + e.Up == f.Up && + e.Serving == f.Serving && + e.TabletExternallyReparentedTimestamp == f.TabletExternallyReparentedTimestamp && + proto.Equal(e.Stats, f.Stats) && + ((e.LastError == nil && f.LastError == nil) || + (e.LastError != nil && f.LastError != nil && e.LastError.Error() == f.LastError.Error())) +} + +// Copy produces a copy of LegacyTabletStats. +func (e *LegacyTabletStats) Copy() *LegacyTabletStats { + ts := *e + return &ts +} + +// GetTabletHostPort formats a tablet host port address. +func (e LegacyTabletStats) GetTabletHostPort() string { + vtPort := e.Tablet.PortMap["vt"] + return netutil.JoinHostPort(e.Tablet.Hostname, vtPort) +} + +// GetHostNameLevel returns the specified hostname level. If the level does not exist it will pick the closest level. +// This seems unused but can be utilized by certain url formatting templates. See getTabletDebugURL for more details. +func (e LegacyTabletStats) GetHostNameLevel(level int) string { + chunkedHostname := strings.Split(e.Tablet.Hostname, ".") + + if level < 0 { + return chunkedHostname[0] + } else if level >= len(chunkedHostname) { + return chunkedHostname[len(chunkedHostname)-1] + } else { + return chunkedHostname[level] + } +} + +// NamedStatusURL returns the URL for the case where a tablet server is named. +func (e LegacyTabletStats) NamedStatusURL() string { + return "/" + topoproto.TabletAliasString(e.Tablet.Alias) + servenv.StatusURLPath() +} + +// getTabletDebugURL formats a debug url to the tablet. +// It uses a format string that can be passed into the app to format +// the debug URL to accommodate different network setups. It applies +// the html/template string defined to a LegacyTabletStats object. The +// format string can refer to members and functions of LegacyTabletStats +// like a regular html/template string. +// +// For instance given a tablet with hostname:port of host.dc.domain:22 +// could be configured as follows: +// http://{{.GetTabletHostPort}} -> http://host.dc.domain:22 +// https://{{.Tablet.Hostname}} -> https://host.dc.domain +// https://{{.GetHostNameLevel 0}}.bastion.corp -> https://host.bastion.corp +// {{.NamedStatusURL}} -> test-0000000001/debug/status +func (e LegacyTabletStats) getTabletDebugURL() string { + var buffer bytes.Buffer + tabletURLTemplate.Execute(&buffer, e) + return buffer.String() +} + +// TrivialStatsUpdate returns true iff the old and new LegacyTabletStats +// haven't changed enough to warrant re-calling FilterLegacyStatsByReplicationLag. +func (e *LegacyTabletStats) TrivialStatsUpdate(n *LegacyTabletStats) bool { + // Skip replag filter when replag remains in the low rep lag range, + // which should be the case majority of the time. + lowRepLag := lowReplicationLag.Seconds() + oldRepLag := float64(e.Stats.SecondsBehindMaster) + newRepLag := float64(n.Stats.SecondsBehindMaster) + if oldRepLag <= lowRepLag && newRepLag <= lowRepLag { + return true + } + + // Skip replag filter when replag remains in the high rep lag range, + // and did not change beyond +/- 10%. + // when there is a high rep lag, it takes a long time for it to reduce, + // so it is not necessary to re-calculate every time. + // In that case, we won't save the new record, so we still + // remember the original replication lag. + if oldRepLag > lowRepLag && newRepLag > lowRepLag && newRepLag < oldRepLag*1.1 && newRepLag > oldRepLag*0.9 { + return true + } + + return false +} + +// LegacyTabletRecorder is the part of the LegacyHealthCheck interface that can +// add or remove tablets. We define it as a sub-interface here so we +// can add filters on tablets if needed. +type LegacyTabletRecorder interface { + // AddTablet adds the tablet. + // Name is an alternate name, like an address. + AddTablet(tablet *topodatapb.Tablet, name string) + + // RemoveTablet removes the tablet. + RemoveTablet(tablet *topodatapb.Tablet) + + // ReplaceTablet does an AddTablet and RemoveTablet in one call, effectively replacing the old tablet with the new. + ReplaceTablet(old, new *topodatapb.Tablet, name string) +} + +// LegacyHealthCheck defines the interface of health checking module. +// The goal of this object is to maintain a StreamHealth RPC +// to a lot of tablets. Tablets are added / removed by calling the +// AddTablet / RemoveTablet methods (other discovery module objects +// can for instance watch the topology and call these). +// +// Updates to the health of all registered tablet can be watched by +// registering a listener. To get the underlying "TabletConn" object +// which is used for each tablet, use the "GetConnection()" method +// below and pass in the Key string which is also sent to the +// listener in each update (as it is part of LegacyTabletStats). +type LegacyHealthCheck interface { + // LegacyTabletRecorder interface adds AddTablet and RemoveTablet methods. + // AddTablet adds the tablet, and starts health check on it. + // RemoveTablet removes the tablet, and stops its StreamHealth RPC. + LegacyTabletRecorder + + // RegisterStats registers the connection counts and checksum stats. + // It can only be called on one Healthcheck object per process. + RegisterStats() + // SetListener sets the listener for healthcheck + // updates. sendDownEvents is used when a tablet changes type + // (from replica to master for instance). If the listener + // wants two events (Up=false on old type, Up=True on new + // type), sendDownEvents should be set. Otherwise, the + // healthcheck will only send one event (Up=true on new type). + // + // Note that the default implementation requires to set the + // listener before any tablets are added to the healthcheck. + SetListener(listener LegacyHealthCheckStatsListener, sendDownEvents bool) + // WaitForInitialStatsUpdates waits until all tablets added via + // AddTablet() call were propagated to the listener via correspondingdiscovert + // StatsUpdate() calls. Note that code path from AddTablet() to + // corresponding StatsUpdate() is asynchronous but not cancelable, thus + // this function is also non-cancelable and can't return error. Also + // note that all AddTablet() calls should happen before calling this + // method. WaitForInitialStatsUpdates won't wait for StatsUpdate() calls + // corresponding to AddTablet() calls made during its execution. + WaitForInitialStatsUpdates() + // GetConnection returns the TabletConn of the given tablet. + GetConnection(key string) queryservice.QueryService + // CacheStatus returns a displayable version of the cache. + CacheStatus() LegacyTabletsCacheStatusList + // Close stops the healthcheck. + Close() error +} + +// LegacyHealthCheckImpl performs health checking and notifies downstream components about any changes. +// It contains a map of legacyTabletHealth objects, each of which stores the health information for +// a tablet. A checkConn goroutine is spawned for each legacyTabletHealth, which is responsible for +// keeping that legacyTabletHealth up-to-date. This is done through callbacks to updateHealth. +// If checkConn terminates for any reason, it updates legacyTabletHealth.Up as false. If a legacyTabletHealth +// gets removed from the map, its cancelFunc gets called, which ensures that the associated +// checkConn goroutine eventually terminates. +type LegacyHealthCheckImpl struct { + // Immutable fields set at construction time. + listener LegacyHealthCheckStatsListener + sendDownEvents bool + retryDelay time.Duration + healthCheckTimeout time.Duration + // connsWG keeps track of all launched Go routines that monitor tablet connections. + connsWG sync.WaitGroup + + // mu protects all the following fields. + mu sync.Mutex + + // addrToHealth maps from address to legacyTabletHealth. + addrToHealth map[string]*legacyTabletHealth + + // Wait group that's used to wait until all initial StatsUpdate() calls are made after the AddTablet() calls. + initialUpdatesWG sync.WaitGroup +} + +// legacyHealthCheckConn is a structure that lives within the scope of +// the checkConn goroutine to maintain its internal state. Therefore, +// it does not require synchronization. Changes that are relevant to +// healthcheck are transmitted through calls to LegacyHealthCheckImpl.updateHealth. +// TODO(sougou): move this and associated functions to a separate file. +type legacyHealthCheckConn struct { + ctx context.Context + + conn queryservice.QueryService + tabletStats LegacyTabletStats + loggedServingState bool + lastResponseTimestamp time.Time // timestamp of the last healthcheck response +} + +// legacyTabletHealth maintains the health status of a tablet. A map of this +// structure is maintained in LegacyHealthCheckImpl. +type legacyTabletHealth struct { + // cancelFunc must be called before discarding legacyTabletHealth. + // This will ensure that the associated checkConn goroutine will terminate. + cancelFunc context.CancelFunc + // conn is the connection associated with the tablet. + conn queryservice.QueryService + // latestTabletStats stores the latest health stats of the tablet. + latestTabletStats LegacyTabletStats +} + +// NewLegacyDefaultHealthCheck creates a new LegacyHealthCheck object with a default configuration. +func NewLegacyDefaultHealthCheck() LegacyHealthCheck { + return NewLegacyHealthCheck(DefaultHealthCheckRetryDelay, DefaultHealthCheckTimeout) +} + +// NewLegacyHealthCheck creates a new LegacyHealthCheck object. +// Parameters: +// retryDelay. +// The duration to wait before retrying to connect (e.g. after a failed connection +// attempt). +// healthCheckTimeout. +// The duration for which we consider a health check response to be 'fresh'. If we don't get +// a health check response from a tablet for more than this duration, we consider the tablet +// not healthy. +func NewLegacyHealthCheck(retryDelay, healthCheckTimeout time.Duration) LegacyHealthCheck { + hc := &LegacyHealthCheckImpl{ + addrToHealth: make(map[string]*legacyTabletHealth), + retryDelay: retryDelay, + healthCheckTimeout: healthCheckTimeout, + } + + healthcheckOnce.Do(func() { + http.Handle("/debug/gateway", hc) + }) + + return hc +} + +// RegisterStats registers the connection counts stats +func (hc *LegacyHealthCheckImpl) RegisterStats() { + stats.NewGaugesFuncWithMultiLabels( + "HealthcheckConnections", + "the number of healthcheck connections registered", + []string{"Keyspace", "ShardName", "TabletType"}, + hc.servingConnStats) + + stats.NewGaugeFunc( + "HealthcheckChecksum", + "crc32 checksum of the current healthcheck state", + hc.stateChecksum) +} + +// ServeHTTP is part of the http.Handler interface. It renders the current state of the discovery gateway tablet cache into json. +func (hc *LegacyHealthCheckImpl) ServeHTTP(w http.ResponseWriter, _ *http.Request) { + w.Header().Set("Content-Type", "application/json; charset=utf-8") + status := hc.cacheStatusMap() + b, err := json.MarshalIndent(status, "", " ") + if err != nil { + w.Write([]byte(err.Error())) + return + } + + buf := bytes.NewBuffer(nil) + json.HTMLEscape(buf, b) + w.Write(buf.Bytes()) +} + +// servingConnStats returns the number of serving tablets per keyspace/shard/tablet type. +func (hc *LegacyHealthCheckImpl) servingConnStats() map[string]int64 { + res := make(map[string]int64) + hc.mu.Lock() + defer hc.mu.Unlock() + for _, th := range hc.addrToHealth { + if !th.latestTabletStats.Up || !th.latestTabletStats.Serving || th.latestTabletStats.LastError != nil { + continue + } + key := fmt.Sprintf("%s.%s.%s", th.latestTabletStats.Target.Keyspace, th.latestTabletStats.Target.Shard, topoproto.TabletTypeLString(th.latestTabletStats.Target.TabletType)) + res[key]++ + } + return res +} + +// stateChecksum returns a crc32 checksum of the healthcheck state +func (hc *LegacyHealthCheckImpl) stateChecksum() int64 { + // CacheStatus is sorted so this should be stable across vtgates + cacheStatus := hc.CacheStatus() + var buf bytes.Buffer + for _, st := range cacheStatus { + fmt.Fprintf(&buf, + "%v%v%v%v\n", + st.Cell, + st.Target.Keyspace, + st.Target.Shard, + st.Target.TabletType.String(), + ) + sort.Sort(st.TabletsStats) + for _, ts := range st.TabletsStats { + fmt.Fprintf(&buf, "%v%v%v\n", ts.Up, ts.Serving, ts.TabletExternallyReparentedTimestamp) + } + } + + return int64(crc32.ChecksumIEEE(buf.Bytes())) +} + +// updateHealth updates the legacyTabletHealth record and transmits the tablet stats +// to the listener. +func (hc *LegacyHealthCheckImpl) updateHealth(ts *LegacyTabletStats, conn queryservice.QueryService) { + // Unconditionally send the received update at the end. + defer func() { + if hc.listener != nil { + hc.listener.StatsUpdate(ts) + } + }() + + hc.mu.Lock() + th, ok := hc.addrToHealth[ts.Key] + if !ok { + // This can happen on delete because the entry is removed first, + // or if LegacyHealthCheckImpl has been closed. + hc.mu.Unlock() + return + } + oldts := th.latestTabletStats + th.latestTabletStats = *ts + th.conn = conn + hc.mu.Unlock() + + // In the case where a tablet changes type (but not for the + // initial message), we want to log it, and maybe advertise it too. + if oldts.Target.TabletType != topodatapb.TabletType_UNKNOWN && oldts.Target.TabletType != ts.Target.TabletType { + // Log and maybe notify + log.Infof("HealthCheckUpdate(Type Change): %v, tablet: %s, target %+v => %+v, reparent time: %v", + oldts.Name, topotools.TabletIdent(oldts.Tablet), topotools.TargetIdent(oldts.Target), topotools.TargetIdent(ts.Target), ts.TabletExternallyReparentedTimestamp) + if hc.listener != nil && hc.sendDownEvents { + oldts.Up = false + hc.listener.StatsUpdate(&oldts) + } + + // Track how often a tablet gets promoted to master. It is used for + // comparing against the variables in go/vtgate/buffer/variables.go. + if oldts.Target.TabletType != topodatapb.TabletType_MASTER && ts.Target.TabletType == topodatapb.TabletType_MASTER { + hcMasterPromotedCounters.Add([]string{ts.Target.Keyspace, ts.Target.Shard}, 1) + } + } +} + +// finalizeConn closes the health checking connection and sends the final +// notification about the tablet to downstream. To be called only on exit from +// checkConn(). +func (hc *LegacyHealthCheckImpl) finalizeConn(hcc *legacyHealthCheckConn) { + hcc.tabletStats.Up = false + hcc.setServingState(false, "finalizeConn closing connection") + // Note: checkConn() exits only when hcc.ctx.Done() is closed. Thus it's + // safe to simply get Err() value here and assign to LastError. + hcc.tabletStats.LastError = hcc.ctx.Err() + hc.updateHealth(hcc.tabletStats.Copy(), nil) + if hcc.conn != nil { + // Don't use hcc.ctx because it's already closed. + // Use a separate context, and add a timeout to prevent unbounded waits. + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + hcc.conn.Close(ctx) + hcc.conn = nil + } +} + +// checkConn performs health checking on the given tablet. +func (hc *LegacyHealthCheckImpl) checkConn(hcc *legacyHealthCheckConn, name string) { + defer hc.connsWG.Done() + defer hc.finalizeConn(hcc) + + // Initial notification for downstream about the tablet existence. + hc.updateHealth(hcc.tabletStats.Copy(), hcc.conn) + hc.initialUpdatesWG.Done() + + retryDelay := hc.retryDelay + for { + streamCtx, streamCancel := context.WithCancel(hcc.ctx) + + // Setup a watcher that restarts the timer every time an update is received. + // If a timeout occurs for a serving tablet, we make it non-serving and send + // a status update. The stream is also terminated so it can be retried. + // servingStatus feeds into the serving var, which keeps track of the serving + // status transmitted by the tablet. + servingStatus := make(chan bool, 1) + // timedout is accessed atomically because there could be a race + // between the goroutine that sets it and the check for its value + // later. + timedout := sync2.NewAtomicBool(false) + go func() { + for { + select { + case <-servingStatus: + continue + case <-time.After(hc.healthCheckTimeout): + timedout.Set(true) + streamCancel() + return + case <-streamCtx.Done(): + // If the stream is done, stop watching. + return + } + } + }() + + // Read stream health responses. + hcc.stream(streamCtx, hc, func(shr *querypb.StreamHealthResponse) error { + // We received a message. Reset the back-off. + retryDelay = hc.retryDelay + // Don't block on send to avoid deadlocks. + select { + case servingStatus <- shr.Serving: + default: + } + return hcc.processResponse(hc, shr) + }) + + // streamCancel to make sure the watcher goroutine terminates. + streamCancel() + + // If there was a timeout send an error. We do this after stream has returned. + // This will ensure that this update prevails over any previous message that + // stream could have sent. + if timedout.Get() { + hcc.tabletStats.LastError = fmt.Errorf("healthcheck timed out (latest %v)", hcc.lastResponseTimestamp) + hcc.setServingState(false, hcc.tabletStats.LastError.Error()) + hc.updateHealth(hcc.tabletStats.Copy(), hcc.conn) + hcErrorCounters.Add([]string{hcc.tabletStats.Target.Keyspace, hcc.tabletStats.Target.Shard, topoproto.TabletTypeLString(hcc.tabletStats.Target.TabletType)}, 1) + } + + // Streaming RPC failed e.g. because vttablet was restarted or took too long. + // Sleep until the next retry is up or the context is done/canceled. + select { + case <-hcc.ctx.Done(): + return + case <-time.After(retryDelay): + // Exponentially back-off to prevent tight-loop. + retryDelay *= 2 + // Limit the retry delay backoff to the health check timeout + if retryDelay > hc.healthCheckTimeout { + retryDelay = hc.healthCheckTimeout + } + } + } +} + +// setServingState sets the tablet state to the given value. +// +// If the state changes, it logs the change so that failures +// from the health check connection are logged the first time, +// but don't continue to log if the connection stays down. +// +// hcc.mu must be locked before calling this function +func (hcc *legacyHealthCheckConn) setServingState(serving bool, reason string) { + if !hcc.loggedServingState || (serving != hcc.tabletStats.Serving) { + // Emit the log from a separate goroutine to avoid holding + // the hcc lock while logging is happening + go log.Infof("HealthCheckUpdate(Serving State): %v, tablet: %v serving => %v for %v/%v (%v) reason: %s", + hcc.tabletStats.Name, + topotools.TabletIdent(hcc.tabletStats.Tablet), + serving, + hcc.tabletStats.Tablet.GetKeyspace(), + hcc.tabletStats.Tablet.GetShard(), + hcc.tabletStats.Target.GetTabletType(), + reason, + ) + hcc.loggedServingState = true + } + + hcc.tabletStats.Serving = serving +} + +// stream streams healthcheck responses to callback. +func (hcc *legacyHealthCheckConn) stream(ctx context.Context, hc *LegacyHealthCheckImpl, callback func(*querypb.StreamHealthResponse) error) { + if hcc.conn == nil { + conn, err := tabletconn.GetDialer()(hcc.tabletStats.Tablet, grpcclient.FailFast(true)) + if err != nil { + hcc.tabletStats.LastError = err + return + } + hcc.conn = conn + hcc.tabletStats.LastError = nil + } + + if err := hcc.conn.StreamHealth(ctx, callback); err != nil { + log.Warningf("tablet %v healthcheck stream error: %v", hcc.tabletStats.Tablet.Alias, err) + hcc.setServingState(false, err.Error()) + hcc.tabletStats.LastError = err + // Send nil because we intend to close the connection. + hc.updateHealth(hcc.tabletStats.Copy(), nil) + hcc.conn.Close(ctx) + hcc.conn = nil + } +} + +// processResponse reads one health check response, and notifies LegacyHealthCheckStatsListener. +func (hcc *legacyHealthCheckConn) processResponse(hc *LegacyHealthCheckImpl, shr *querypb.StreamHealthResponse) error { + select { + case <-hcc.ctx.Done(): + return hcc.ctx.Err() + default: + } + + // Check for invalid data, better than panicking. + if shr.Target == nil || shr.RealtimeStats == nil { + return fmt.Errorf("health stats is not valid: %v", shr) + } + + // an app-level error from tablet, force serving state. + var healthErr error + serving := shr.Serving + if shr.RealtimeStats.HealthError != "" { + healthErr = fmt.Errorf("vttablet error: %v", shr.RealtimeStats.HealthError) + serving = false + } + + // hcc.LegacyTabletStats.Tablet.Alias.Uid may be 0 because the youtube internal mechanism uses a different + // code path to initialize this value. If so, we should skip this check. + if shr.TabletAlias != nil && hcc.tabletStats.Tablet.Alias.Uid != 0 && !proto.Equal(shr.TabletAlias, hcc.tabletStats.Tablet.Alias) { + return fmt.Errorf("health stats mismatch, tablet %+v alias does not match response alias %v", hcc.tabletStats.Tablet, shr.TabletAlias) + } + + // In this case where a new tablet is initialized or a tablet type changes, we want to + // initialize the counter so the rate can be calculated correctly. + if hcc.tabletStats.Target.TabletType != shr.Target.TabletType { + hcErrorCounters.Add([]string{shr.Target.Keyspace, shr.Target.Shard, topoproto.TabletTypeLString(shr.Target.TabletType)}, 0) + } + + // Update our record, and notify downstream for tabletType and + // realtimeStats change. + hcc.lastResponseTimestamp = time.Now() + hcc.tabletStats.Target = shr.Target + hcc.tabletStats.TabletExternallyReparentedTimestamp = shr.TabletExternallyReparentedTimestamp + hcc.tabletStats.Stats = shr.RealtimeStats + hcc.tabletStats.LastError = healthErr + reason := "healthCheck update" + if healthErr != nil { + reason = "healthCheck update error: " + healthErr.Error() + } + hcc.setServingState(serving, reason) + hc.updateHealth(hcc.tabletStats.Copy(), hcc.conn) + return nil +} + +func (hc *LegacyHealthCheckImpl) deleteConn(tablet *topodatapb.Tablet) { + hc.mu.Lock() + defer hc.mu.Unlock() + + key := TabletToMapKey(tablet) + th, ok := hc.addrToHealth[key] + if !ok { + return + } + // Make sure the key still corresponds to the tablet we want to delete. + // If it doesn't match, we should do nothing. The tablet we were asked to + // delete is already gone, and some other tablet is using the key + // (host:port) that the original tablet used to use, which is fine. + if !topoproto.TabletAliasEqual(tablet.Alias, th.latestTabletStats.Tablet.Alias) { + return + } + hc.deleteConnLocked(key, th) +} + +func (hc *LegacyHealthCheckImpl) deleteConnLocked(key string, th *legacyTabletHealth) { + th.latestTabletStats.Up = false + th.cancelFunc() + delete(hc.addrToHealth, key) +} + +// SetListener sets the listener for healthcheck updates. +// It must be called after NewLegacyHealthCheck and before any tablets are added +// (either through AddTablet or through a Watcher). +func (hc *LegacyHealthCheckImpl) SetListener(listener LegacyHealthCheckStatsListener, sendDownEvents bool) { + if hc.listener != nil { + panic("must not call SetListener twice") + } + + hc.mu.Lock() + defer hc.mu.Unlock() + if len(hc.addrToHealth) > 0 { + panic("must not call SetListener after tablets were added") + } + + hc.listener = listener + hc.sendDownEvents = sendDownEvents +} + +// AddTablet adds the tablet, and starts health check. +// It does not block on making connection. +// name is an optional tag for the tablet, e.g. an alternative address. +func (hc *LegacyHealthCheckImpl) AddTablet(tablet *topodatapb.Tablet, name string) { + ctx, cancelFunc := context.WithCancel(context.Background()) + key := TabletToMapKey(tablet) + hcc := &legacyHealthCheckConn{ + ctx: ctx, + tabletStats: LegacyTabletStats{ + Key: key, + Tablet: tablet, + Name: name, + Target: &querypb.Target{}, + Up: true, + }, + } + hc.mu.Lock() + if hc.addrToHealth == nil { + // already closed. + hc.mu.Unlock() + return + } + if th, ok := hc.addrToHealth[key]; ok { + // Something already exists at this key. + // If it's the same tablet, something is wrong. + if topoproto.TabletAliasEqual(th.latestTabletStats.Tablet.Alias, tablet.Alias) { + hc.mu.Unlock() + log.Warningf("refusing to add duplicate tablet %v for %v: %+v", name, tablet.Alias.Cell, tablet) + return + } + // If it's a different tablet, then we trust this new tablet that claims + // it has taken over the host:port that the old tablet used to be on. + // Remove the old tablet to clear the way. + hc.deleteConnLocked(key, th) + } + hc.addrToHealth[key] = &legacyTabletHealth{ + cancelFunc: cancelFunc, + latestTabletStats: hcc.tabletStats, + } + hc.initialUpdatesWG.Add(1) + hc.connsWG.Add(1) + hc.mu.Unlock() + + go hc.checkConn(hcc, name) +} + +// RemoveTablet removes the tablet, and stops the health check. +// It does not block. +func (hc *LegacyHealthCheckImpl) RemoveTablet(tablet *topodatapb.Tablet) { + hc.deleteConn(tablet) +} + +// ReplaceTablet removes the old tablet and adds the new tablet. +func (hc *LegacyHealthCheckImpl) ReplaceTablet(old, new *topodatapb.Tablet, name string) { + hc.deleteConn(old) + hc.AddTablet(new, name) +} + +// WaitForInitialStatsUpdates waits until all tablets added via AddTablet() call +// were propagated to downstream via corresponding StatsUpdate() calls. +func (hc *LegacyHealthCheckImpl) WaitForInitialStatsUpdates() { + hc.initialUpdatesWG.Wait() +} + +// GetConnection returns the TabletConn of the given tablet. +func (hc *LegacyHealthCheckImpl) GetConnection(key string) queryservice.QueryService { + hc.mu.Lock() + defer hc.mu.Unlock() + + th := hc.addrToHealth[key] + if th == nil { + return nil + } + return th.conn +} + +// LegacyTabletsCacheStatus is the current tablets for a cell/target. +type LegacyTabletsCacheStatus struct { + Cell string + Target *querypb.Target + TabletsStats LegacyTabletStatsList +} + +// LegacyTabletStatsList is used for sorting. +type LegacyTabletStatsList []*LegacyTabletStats + +// Len is part of sort.Interface. +func (tsl LegacyTabletStatsList) Len() int { + return len(tsl) +} + +// Less is part of sort.Interface +func (tsl LegacyTabletStatsList) Less(i, j int) bool { + name1 := tsl[i].Name + if name1 == "" { + name1 = tsl[i].Key + } + name2 := tsl[j].Name + if name2 == "" { + name2 = tsl[j].Key + } + return name1 < name2 +} + +// Swap is part of sort.Interface +func (tsl LegacyTabletStatsList) Swap(i, j int) { + tsl[i], tsl[j] = tsl[j], tsl[i] +} + +// StatusAsHTML returns an HTML version of the status. +func (tcs *LegacyTabletsCacheStatus) StatusAsHTML() template.HTML { + tLinks := make([]string, 0, 1) + if tcs.TabletsStats != nil { + sort.Sort(tcs.TabletsStats) + } + for _, ts := range tcs.TabletsStats { + color := "green" + extra := "" + if ts.LastError != nil { + color = "red" + extra = fmt.Sprintf(" (%v)", ts.LastError) + } else if !ts.Serving { + color = "red" + extra = " (Not Serving)" + } else if !ts.Up { + color = "red" + extra = " (Down)" + } else if ts.Target.TabletType == topodatapb.TabletType_MASTER { + extra = fmt.Sprintf(" (MasterTS: %v)", ts.TabletExternallyReparentedTimestamp) + } else { + extra = fmt.Sprintf(" (RepLag: %v)", ts.Stats.SecondsBehindMaster) + } + name := ts.Name + if name == "" { + name = ts.GetTabletHostPort() + } + tLinks = append(tLinks, fmt.Sprintf(`%v%v`, ts.getTabletDebugURL(), color, name, extra)) + } + return template.HTML(strings.Join(tLinks, "
")) +} + +// LegacyTabletsCacheStatusList is used for sorting. +type LegacyTabletsCacheStatusList []*LegacyTabletsCacheStatus + +// Len is part of sort.Interface. +func (tcsl LegacyTabletsCacheStatusList) Len() int { + return len(tcsl) +} + +// Less is part of sort.Interface +func (tcsl LegacyTabletsCacheStatusList) Less(i, j int) bool { + return tcsl[i].Cell+"."+tcsl[i].Target.Keyspace+"."+tcsl[i].Target.Shard+"."+string(tcsl[i].Target.TabletType) < + tcsl[j].Cell+"."+tcsl[j].Target.Keyspace+"."+tcsl[j].Target.Shard+"."+string(tcsl[j].Target.TabletType) +} + +// Swap is part of sort.Interface +func (tcsl LegacyTabletsCacheStatusList) Swap(i, j int) { + tcsl[i], tcsl[j] = tcsl[j], tcsl[i] +} + +// CacheStatus returns a displayable version of the cache. +func (hc *LegacyHealthCheckImpl) CacheStatus() LegacyTabletsCacheStatusList { + tcsMap := hc.cacheStatusMap() + tcsl := make(LegacyTabletsCacheStatusList, 0, len(tcsMap)) + for _, tcs := range tcsMap { + tcsl = append(tcsl, tcs) + } + sort.Sort(tcsl) + return tcsl +} + +func (hc *LegacyHealthCheckImpl) cacheStatusMap() map[string]*LegacyTabletsCacheStatus { + tcsMap := make(map[string]*LegacyTabletsCacheStatus) + hc.mu.Lock() + defer hc.mu.Unlock() + for _, th := range hc.addrToHealth { + key := fmt.Sprintf("%v.%v.%v.%v", th.latestTabletStats.Tablet.Alias.Cell, th.latestTabletStats.Target.Keyspace, th.latestTabletStats.Target.Shard, th.latestTabletStats.Target.TabletType.String()) + var tcs *LegacyTabletsCacheStatus + var ok bool + if tcs, ok = tcsMap[key]; !ok { + tcs = &LegacyTabletsCacheStatus{ + Cell: th.latestTabletStats.Tablet.Alias.Cell, + Target: th.latestTabletStats.Target, + } + tcsMap[key] = tcs + } + tabletStats := th.latestTabletStats + tcs.TabletsStats = append(tcs.TabletsStats, &tabletStats) + } + return tcsMap +} + +// Close stops the healthcheck. +// After Close() returned, it's guaranteed that the listener isn't +// currently executing and won't be called again. +func (hc *LegacyHealthCheckImpl) Close() error { + hc.mu.Lock() + for _, th := range hc.addrToHealth { + th.cancelFunc() + } + hc.addrToHealth = nil + // Release the lock early or a pending checkHealthCheckTimeout + // cannot get a read lock on it. + hc.mu.Unlock() + + // Wait for the checkHealthCheckTimeout Go routine and each Go + // routine per tablet. + hc.connsWG.Wait() + + return nil +} + +// TabletToMapKey creates a key to the map from tablet's host and ports. +// It should only be used in discovery and related module. +func TabletToMapKey(tablet *topodatapb.Tablet) string { + parts := make([]string, 0, 1) + for name, port := range tablet.PortMap { + parts = append(parts, netutil.JoinHostPort(name, port)) + } + sort.Strings(parts) + parts = append([]string{tablet.Hostname}, parts...) + return strings.Join(parts, ",") +} diff --git a/go/vt/discovery/healthcheck_flaky_test.go b/go/vt/discovery/legacy_healthcheck_flaky_test.go similarity index 84% rename from go/vt/discovery/healthcheck_flaky_test.go rename to go/vt/discovery/legacy_healthcheck_flaky_test.go index 58fa12ca0c3..8d28ad60abd 100644 --- a/go/vt/discovery/healthcheck_flaky_test.go +++ b/go/vt/discovery/legacy_healthcheck_flaky_test.go @@ -21,10 +21,8 @@ import ( "flag" "fmt" "html/template" - "io" "reflect" "strings" - "sync" "testing" "time" @@ -32,7 +30,6 @@ import ( "vitess.io/vitess/go/vt/grpcclient" "vitess.io/vitess/go/vt/status" "vitess.io/vitess/go/vt/topo" - "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vttablet/queryservice" "vitess.io/vitess/go/vt/vttablet/queryservice/fakes" "vitess.io/vitess/go/vt/vttablet/tabletconn" @@ -56,21 +53,22 @@ func testChecksum(t *testing.T, want, got int64) { } } -func TestHealthCheck(t *testing.T) { +func TestLegacyHealthCheck(t *testing.T) { + hcErrorCounters.ResetAll() tablet := topo.NewTablet(0, "cell", "a") tablet.PortMap["vt"] = 1 input := make(chan *querypb.StreamHealthResponse) createFakeConn(tablet, input) t.Logf(`createFakeConn({Host: "a", PortMap: {"vt": 1}}, c)`) l := newListener() - hc := NewHealthCheck(1*time.Millisecond, time.Hour).(*HealthCheckImpl) + hc := NewLegacyHealthCheck(1*time.Millisecond, time.Hour).(*LegacyHealthCheckImpl) hc.SetListener(l, true) testChecksum(t, 0, hc.stateChecksum()) hc.AddTablet(tablet, "") - t.Logf(`hc = HealthCheck(); hc.AddTablet({Host: "a", PortMap: {"vt": 1}}, "")`) + t.Logf(`hc = LegacyHealthCheck(); hc.AddTablet({Host: "a", PortMap: {"vt": 1}}, "")`) // Immediately after AddTablet() there will be the first notification. - want := &TabletStats{ + want := &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{}, @@ -90,7 +88,7 @@ func TestHealthCheck(t *testing.T) { TabletExternallyReparentedTimestamp: 10, RealtimeStats: &querypb.RealtimeStats{SecondsBehindMaster: 1, CpuUsage: 0.2}, } - want = &TabletStats{ + want = &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_MASTER}, @@ -112,10 +110,10 @@ func TestHealthCheck(t *testing.T) { } tcsl := hc.CacheStatus() - tcslWant := TabletsCacheStatusList{{ + tcslWant := LegacyTabletsCacheStatusList{{ Cell: "cell", Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_MASTER}, - TabletsStats: TabletStatsList{{ + TabletsStats: LegacyTabletStatsList{{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_MASTER}, @@ -139,7 +137,7 @@ func TestHealthCheck(t *testing.T) { } input <- shr t.Logf(`input <- {{Keyspace: "k", Shard: "s", TabletType: REPLICA}, Serving: true, TabletExternallyReparentedTimestamp: 0, {SecondsBehindMaster: 1, CpuUsage: 0.5}}`) - want = &TabletStats{ + want = &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_MASTER}, @@ -152,7 +150,7 @@ func TestHealthCheck(t *testing.T) { if !reflect.DeepEqual(res, want) { t.Errorf(`<-l.output: %+v; want %+v`, res, want) } - want = &TabletStats{ + want = &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -178,7 +176,7 @@ func TestHealthCheck(t *testing.T) { TabletExternallyReparentedTimestamp: 0, RealtimeStats: &querypb.RealtimeStats{SecondsBehindMaster: 1, CpuUsage: 0.3}, } - want = &TabletStats{ + want = &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -202,7 +200,7 @@ func TestHealthCheck(t *testing.T) { TabletExternallyReparentedTimestamp: 0, RealtimeStats: &querypb.RealtimeStats{HealthError: "some error", SecondsBehindMaster: 1, CpuUsage: 0.3}, } - want = &TabletStats{ + want = &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -223,7 +221,7 @@ func TestHealthCheck(t *testing.T) { // remove tablet hc.deleteConn(tablet) t.Logf(`hc.RemoveTablet({Host: "a", PortMap: {"vt": 1}})`) - want = &TabletStats{ + want = &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -243,7 +241,7 @@ func TestHealthCheck(t *testing.T) { hc.Close() } -func TestHealthCheckStreamError(t *testing.T) { +func TestLegacyHealthCheckStreamError(t *testing.T) { tablet := topo.NewTablet(0, "cell", "a") tablet.PortMap["vt"] = 1 input := make(chan *querypb.StreamHealthResponse) @@ -251,13 +249,13 @@ func TestHealthCheckStreamError(t *testing.T) { fc.errCh = make(chan error) t.Logf(`createFakeConn({Host: "a", PortMap: {"vt": 1}}, c)`) l := newListener() - hc := NewHealthCheck(1*time.Millisecond, time.Hour).(*HealthCheckImpl) + hc := NewLegacyHealthCheck(1*time.Millisecond, time.Hour).(*LegacyHealthCheckImpl) hc.SetListener(l, true) hc.AddTablet(tablet, "") - t.Logf(`hc = HealthCheck(); hc.AddTablet({Host: "a", PortMap: {"vt": 1}}, "")`) + t.Logf(`hc = LegacyHealthCheck(); hc.AddTablet({Host: "a", PortMap: {"vt": 1}}, "")`) // Immediately after AddTablet() there will be the first notification. - want := &TabletStats{ + want := &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{}, @@ -276,7 +274,7 @@ func TestHealthCheckStreamError(t *testing.T) { TabletExternallyReparentedTimestamp: 0, RealtimeStats: &querypb.RealtimeStats{SecondsBehindMaster: 1, CpuUsage: 0.2}, } - want = &TabletStats{ + want = &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -294,7 +292,7 @@ func TestHealthCheckStreamError(t *testing.T) { // Stream error fc.errCh <- fmt.Errorf("some stream error") - want = &TabletStats{ + want = &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -313,7 +311,7 @@ func TestHealthCheckStreamError(t *testing.T) { hc.Close() } -func TestHealthCheckVerifiesTabletAlias(t *testing.T) { +func TestLegacyHealthCheckVerifiesTabletAlias(t *testing.T) { t.Logf("starting") tablet := topo.NewTablet(1, "cell", "a") tablet.PortMap["vt"] = 1 @@ -323,13 +321,13 @@ func TestHealthCheckVerifiesTabletAlias(t *testing.T) { t.Logf(`createFakeConn({Host: "a", PortMap: {"vt": 1}}, c)`) l := newListener() - hc := NewHealthCheck(1*time.Millisecond, time.Hour).(*HealthCheckImpl) + hc := NewLegacyHealthCheck(1*time.Millisecond, time.Hour).(*LegacyHealthCheckImpl) hc.SetListener(l, false) hc.AddTablet(tablet, "") - t.Logf(`hc = HealthCheck(); hc.AddTablet({Host: "a", PortMap: {"vt": 1}}, "")`) + t.Logf(`hc = LegacyHealthCheck(); hc.AddTablet({Host: "a", PortMap: {"vt": 1}}, "")`) // Immediately after AddTablet() there will be the first notification. - want := &TabletStats{ + want := &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{}, @@ -378,9 +376,9 @@ func TestHealthCheckVerifiesTabletAlias(t *testing.T) { hc.Close() } -// TestHealthCheckCloseWaitsForGoRoutines tests that Close() waits for all Go +// TestLegacyHealthCheckCloseWaitsForGoRoutines tests that Close() waits for all Go // routines to finish and the listener won't be called anymore. -func TestHealthCheckCloseWaitsForGoRoutines(t *testing.T) { +func TestLegacyHealthCheckCloseWaitsForGoRoutines(t *testing.T) { tablet := topo.NewTablet(0, "cell", "a") tablet.PortMap["vt"] = 1 input := make(chan *querypb.StreamHealthResponse, 1) @@ -389,13 +387,13 @@ func TestHealthCheckCloseWaitsForGoRoutines(t *testing.T) { t.Logf(`createFakeConn({Host: "a", PortMap: {"vt": 1}}, c)`) l := newListener() - hc := NewHealthCheck(1*time.Millisecond, time.Hour).(*HealthCheckImpl) + hc := NewLegacyHealthCheck(1*time.Millisecond, time.Hour).(*LegacyHealthCheckImpl) hc.SetListener(l, false) hc.AddTablet(tablet, "") - t.Logf(`hc = HealthCheck(); hc.AddTablet({Host: "a", PortMap: {"vt": 1}}, "")`) + t.Logf(`hc = LegacyHealthCheck(); hc.AddTablet({Host: "a", PortMap: {"vt": 1}}, "")`) // Immediately after AddTablet() there will be the first notification. - want := &TabletStats{ + want := &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{}, @@ -414,7 +412,7 @@ func TestHealthCheckCloseWaitsForGoRoutines(t *testing.T) { TabletExternallyReparentedTimestamp: 10, RealtimeStats: &querypb.RealtimeStats{SecondsBehindMaster: 1, CpuUsage: 0.2}, } - want = &TabletStats{ + want = &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_MASTER}, @@ -452,7 +450,7 @@ func TestHealthCheckCloseWaitsForGoRoutines(t *testing.T) { select { case res = <-l.output: if res.TabletExternallyReparentedTimestamp == 10 && res.LastError == context.Canceled { - // HealthCheck repeats the previous stats if there is an error. + // LegacyHealthCheck repeats the previous stats if there is an error. // This is expected. break } @@ -476,7 +474,7 @@ func TestHealthCheckCloseWaitsForGoRoutines(t *testing.T) { } } -func TestHealthCheckTimeout(t *testing.T) { +func TestLegacyHealthCheckTimeout(t *testing.T) { timeout := 500 * time.Millisecond tablet := topo.NewTablet(0, "cell", "a") tablet.PortMap["vt"] = 1 @@ -484,13 +482,13 @@ func TestHealthCheckTimeout(t *testing.T) { fc := createFakeConn(tablet, input) t.Logf(`createFakeConn({Host: "a", PortMap: {"vt": 1}}, c)`) l := newListener() - hc := NewHealthCheck(1*time.Millisecond, timeout).(*HealthCheckImpl) + hc := NewLegacyHealthCheck(1*time.Millisecond, timeout).(*LegacyHealthCheckImpl) hc.SetListener(l, false) hc.AddTablet(tablet, "") - t.Logf(`hc = HealthCheck(); hc.AddTablet({Host: "a", PortMap: {"vt": 1}}, "")`) + t.Logf(`hc = LegacyHealthCheck(); hc.AddTablet({Host: "a", PortMap: {"vt": 1}}, "")`) // Immediately after AddTablet() there will be the first notification. - want := &TabletStats{ + want := &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{}, @@ -509,7 +507,7 @@ func TestHealthCheckTimeout(t *testing.T) { TabletExternallyReparentedTimestamp: 10, RealtimeStats: &querypb.RealtimeStats{SecondsBehindMaster: 1, CpuUsage: 0.2}, } - want = &TabletStats{ + want = &LegacyTabletStats{ Key: "a,vt:1", Tablet: tablet, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_MASTER}, @@ -579,9 +577,9 @@ func TestHealthCheckTimeout(t *testing.T) { hc.Close() } -func TestTemplate(t *testing.T) { +func TestLegacyTemplate(t *testing.T) { tablet := topo.NewTablet(0, "cell", "a") - ts := []*TabletStats{ + ts := []*LegacyTabletStats{ { Key: "a", Tablet: tablet, @@ -592,28 +590,28 @@ func TestTemplate(t *testing.T) { TabletExternallyReparentedTimestamp: 0, }, } - tcs := &TabletsCacheStatus{ + tcs := &LegacyTabletsCacheStatus{ Cell: "cell", Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, TabletsStats: ts, } templ := template.New("").Funcs(status.StatusFuncs) - templ, err := templ.Parse(HealthCheckTemplate) + templ, err := templ.Parse(LegacyHealthCheckTemplate) if err != nil { t.Fatalf("error parsing template: %v", err) } wr := &bytes.Buffer{} - if err := templ.Execute(wr, []*TabletsCacheStatus{tcs}); err != nil { + if err := templ.Execute(wr, []*LegacyTabletsCacheStatus{tcs}); err != nil { t.Fatalf("error executing template: %v", err) } } -func TestDebugURLFormatting(t *testing.T) { +func TestLegacyDebugURLFormatting(t *testing.T) { flag.Set("tablet_url_template", "https://{{.GetHostNameLevel 0}}.bastion.{{.Tablet.Alias.Cell}}.corp") ParseTabletURLTemplateFromFlag() tablet := topo.NewTablet(0, "cell", "host.dc.domain") - ts := []*TabletStats{ + ts := []*LegacyTabletStats{ { Key: "a", Tablet: tablet, @@ -624,18 +622,18 @@ func TestDebugURLFormatting(t *testing.T) { TabletExternallyReparentedTimestamp: 0, }, } - tcs := &TabletsCacheStatus{ + tcs := &LegacyTabletsCacheStatus{ Cell: "cell", Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, TabletsStats: ts, } templ := template.New("").Funcs(status.StatusFuncs) - templ, err := templ.Parse(HealthCheckTemplate) + templ, err := templ.Parse(LegacyHealthCheckTemplate) if err != nil { t.Fatalf("error parsing template: %v", err) } wr := &bytes.Buffer{} - if err := templ.Execute(wr, []*TabletsCacheStatus{tcs}); err != nil { + if err := templ.Execute(wr, []*LegacyTabletsCacheStatus{tcs}); err != nil { t.Fatalf("error executing template: %v", err) } expectedURL := `"https://host.bastion.cell.corp"` @@ -645,45 +643,17 @@ func TestDebugURLFormatting(t *testing.T) { } type listener struct { - output chan *TabletStats + output chan *LegacyTabletStats } func newListener() *listener { - return &listener{output: make(chan *TabletStats, 2)} + return &listener{output: make(chan *LegacyTabletStats, 2)} } -func (l *listener) StatsUpdate(ts *TabletStats) { +func (l *listener) StatsUpdate(ts *LegacyTabletStats) { l.output <- ts } -type fakeConn struct { - queryservice.QueryService - tablet *topodatapb.Tablet - // If fixedResult is set, the channels are not used. - fixedResult *querypb.StreamHealthResponse - // hcChan should be an unbuffered channel which holds the tablet's next health response. - hcChan chan *querypb.StreamHealthResponse - // errCh is either an unbuffered channel which holds the stream error to return, or nil. - errCh chan error - // cbErrCh is a channel which receives errors returned from the supplied callback. - cbErrCh chan error - - mu sync.Mutex - canceled bool -} - -func createFakeConn(tablet *topodatapb.Tablet, c chan *querypb.StreamHealthResponse) *fakeConn { - key := TabletToMapKey(tablet) - conn := &fakeConn{ - QueryService: fakes.ErrorQueryService, - tablet: tablet, - hcChan: c, - cbErrCh: make(chan error, 1), - } - connMap[key] = conn - return conn -} - func createFixedHealthConn(tablet *topodatapb.Tablet, fixedResult *querypb.StreamHealthResponse) *fakeConn { key := TabletToMapKey(tablet) conn := &fakeConn{ @@ -702,58 +672,3 @@ func discoveryDialer(tablet *topodatapb.Tablet, failFast grpcclient.FailFast) (q } return nil, fmt.Errorf("tablet %v not found", key) } - -// StreamHealth implements queryservice.QueryService. -func (fc *fakeConn) StreamHealth(ctx context.Context, callback func(shr *querypb.StreamHealthResponse) error) error { - if fc.fixedResult != nil { - return callback(fc.fixedResult) - } - for { - select { - case shr := <-fc.hcChan: - if err := callback(shr); err != nil { - if err == io.EOF { - return nil - } - select { - case fc.cbErrCh <- err: - case <-ctx.Done(): - } - return err - } - case err := <-fc.errCh: - return err - case <-ctx.Done(): - fc.mu.Lock() - fc.canceled = true - fc.mu.Unlock() - return nil - } - } -} - -func (fc *fakeConn) isCanceled() bool { - fc.mu.Lock() - defer fc.mu.Unlock() - return fc.canceled -} - -func (fc *fakeConn) resetCanceledFlag() { - fc.mu.Lock() - defer fc.mu.Unlock() - fc.canceled = false -} - -func checkErrorCounter(keyspace, shard string, tabletType topodatapb.TabletType, want int64) error { - statsKey := []string{keyspace, shard, topoproto.TabletTypeLString(tabletType)} - name := strings.Join(statsKey, ".") - got, ok := hcErrorCounters.Counts()[name] - if !ok { - return fmt.Errorf("hcErrorCounters not correctly initialized") - } - if got != want { - return fmt.Errorf("wrong value for hcErrorCounters got = %v, want = %v", got, want) - } - - return nil -} diff --git a/go/vt/discovery/legacy_replicationlag.go b/go/vt/discovery/legacy_replicationlag.go new file mode 100644 index 00000000000..7b21c78ddc3 --- /dev/null +++ b/go/vt/discovery/legacy_replicationlag.go @@ -0,0 +1,201 @@ +/* +Copyright 2019 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 discovery + +import ( + "fmt" + "sort" +) + +// LegacyIsReplicationLagHigh verifies that the given LegacyTabletStats refers to a tablet with high +// replication lag, i.e. higher than the configured discovery_low_replication_lag flag. +func LegacyIsReplicationLagHigh(tabletStats *LegacyTabletStats) bool { + return float64(tabletStats.Stats.SecondsBehindMaster) > lowReplicationLag.Seconds() +} + +// LegacyIsReplicationLagVeryHigh verifies that the given LegacyTabletStats refers to a tablet with very high +// replication lag, i.e. higher than the configured discovery_high_replication_lag_minimum_serving flag. +func LegacyIsReplicationLagVeryHigh(tabletStats *LegacyTabletStats) bool { + return float64(tabletStats.Stats.SecondsBehindMaster) > highReplicationLagMinServing.Seconds() +} + +// FilterLegacyStatsByReplicationLag filters the list of LegacyTabletStats by LegacyTabletStats.Stats.SecondsBehindMaster. +// Note that LegacyTabletStats that is non-serving or has error is ignored. +// +// The simplified logic: +// - Return tablets that have lag <= lowReplicationLag. +// - Make sure we return at least minNumTablets tablets, if there are enough one with lag <= highReplicationLagMinServing. +// For example, with the default of 30s / 2h / 2, this means: +// - lags of (5s, 10s, 15s, 120s) return the first three +// - lags of (30m, 35m, 40m, 45m) return the first two +// - lags of (2h, 3h, 4h, 5h) return the first one +// +// The legacy algorithm (default for now): +// - Return the list if there is 0 or 1 tablet. +// - Return the list if all tablets have <=30s lag. +// - Filter by replication lag: for each tablet, if the mean value without it is more than 0.7 of the mean value across all tablets, it is valid. +// - Make sure we return at least minNumTablets tablets (if there are enough one with only low replication lag). +// - If one tablet is removed, run above steps again in case there are two tablets with high replication lag. (It should cover most cases.) +// For example, lags of (5s, 10s, 15s, 120s) return the first three; +// lags of (30m, 35m, 40m, 45m) return all. +// +// One thing to know about this code: vttablet also has a couple flags that impact the logic here: +// * unhealthy_threshold: if replication lag is higher than this, a tablet will be reported as unhealthy. +// The default for this is 2h, same as the discovery_high_replication_lag_minimum_serving here. +// * degraded_threshold: this is only used by vttablet for display. It should match +// discovery_low_replication_lag here, so the vttablet status display matches what vtgate will do of it. +func FilterLegacyStatsByReplicationLag(tabletStatsList []*LegacyTabletStats) []*LegacyTabletStats { + if !*legacyReplicationLagAlgorithm { + return filterLegacyStatsByLag(tabletStatsList) + } + + res := filterLegacyStatsByLagWithLegacyAlgorithm(tabletStatsList) + // run the filter again if exactly one tablet is removed, + // and we have spare tablets. + if len(res) > *minNumTablets && len(res) == len(tabletStatsList)-1 { + res = filterLegacyStatsByLagWithLegacyAlgorithm(res) + } + return res +} + +func filterLegacyStatsByLag(tabletStatsList []*LegacyTabletStats) []*LegacyTabletStats { + list := make([]legacyTabletLagSnapshot, 0, len(tabletStatsList)) + // filter non-serving tablets and those with very high replication lag + for _, ts := range tabletStatsList { + if !ts.Serving || ts.LastError != nil || ts.Stats == nil || LegacyIsReplicationLagVeryHigh(ts) { + continue + } + // Pull the current replication lag for a stable sort later. + list = append(list, legacyTabletLagSnapshot{ + ts: ts, + replag: ts.Stats.SecondsBehindMaster}) + } + + // Sort by replication lag. + sort.Sort(byLegacyReplag(list)) + + // Pick those with low replication lag, but at least minNumTablets tablets regardless. + res := make([]*LegacyTabletStats, 0, len(list)) + for i := 0; i < len(list); i++ { + if !LegacyIsReplicationLagHigh(list[i].ts) || i < *minNumTablets { + res = append(res, list[i].ts) + } + } + return res +} + +func filterLegacyStatsByLagWithLegacyAlgorithm(tabletStatsList []*LegacyTabletStats) []*LegacyTabletStats { + list := make([]*LegacyTabletStats, 0, len(tabletStatsList)) + // filter non-serving tablets + for _, ts := range tabletStatsList { + if !ts.Serving || ts.LastError != nil || ts.Stats == nil { + continue + } + list = append(list, ts) + } + if len(list) <= 1 { + return list + } + // if all have low replication lag (<=30s), return all tablets. + allLowLag := true + for _, ts := range list { + if LegacyIsReplicationLagHigh(ts) { + allLowLag = false + break + } + } + if allLowLag { + return list + } + // filter those affecting "mean" lag significantly + // calculate mean for all tablets + res := make([]*LegacyTabletStats, 0, len(list)) + m, _ := legacyMean(list, -1) + for i, ts := range list { + // calculate mean by excluding ith tablet + mi, _ := legacyMean(list, i) + if float64(mi) > float64(m)*0.7 { + res = append(res, ts) + } + } + if len(res) >= *minNumTablets { + return res + } + // return at least minNumTablets tablets to avoid over loading, + // if there is enough tablets with replication lag < highReplicationLagMinServing. + // Pull the current replication lag for a stable sort. + snapshots := make([]legacyTabletLagSnapshot, 0, len(list)) + for _, ts := range list { + if !LegacyIsReplicationLagVeryHigh(ts) { + snapshots = append(snapshots, legacyTabletLagSnapshot{ + ts: ts, + replag: ts.Stats.SecondsBehindMaster}) + } + } + if len(snapshots) == 0 { + // We get here if all tablets are over the high + // replication lag threshold, and their lag is + // different enough that the 70% mean computation up + // there didn't find them all in a group. For + // instance, if *minNumTablets = 2, and we have two + // tablets with lag of 3h and 30h. In that case, we + // just use them all. + for _, ts := range list { + snapshots = append(snapshots, legacyTabletLagSnapshot{ + ts: ts, + replag: ts.Stats.SecondsBehindMaster}) + } + } + + // Sort by replication lag. + sort.Sort(byLegacyReplag(snapshots)) + + // Pick the first minNumTablets tablets. + res = make([]*LegacyTabletStats, 0, *minNumTablets) + for i := 0; i < min(*minNumTablets, len(snapshots)); i++ { + res = append(res, snapshots[i].ts) + } + return res +} + +type legacyTabletLagSnapshot struct { + ts *LegacyTabletStats + replag uint32 +} +type byLegacyReplag []legacyTabletLagSnapshot + +func (a byLegacyReplag) Len() int { return len(a) } +func (a byLegacyReplag) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a byLegacyReplag) Less(i, j int) bool { return a[i].replag < a[j].replag } + +// mean calculates the mean value over the given list, +// while excluding the item with the specified index. +func legacyMean(tabletStatsList []*LegacyTabletStats, idxExclude int) (uint64, error) { + var sum uint64 + var count uint64 + for i, ts := range tabletStatsList { + if i == idxExclude { + continue + } + sum = sum + uint64(ts.Stats.SecondsBehindMaster) + count++ + } + if count == 0 { + return 0, fmt.Errorf("empty list") + } + return sum / count, nil +} diff --git a/go/vt/discovery/legacy_replicationlag_test.go b/go/vt/discovery/legacy_replicationlag_test.go new file mode 100644 index 00000000000..e6026ea40a0 --- /dev/null +++ b/go/vt/discovery/legacy_replicationlag_test.go @@ -0,0 +1,370 @@ +/* +Copyright 2019 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 discovery + +import ( + "fmt" + "testing" + + querypb "vitess.io/vitess/go/vt/proto/query" + "vitess.io/vitess/go/vt/topo" +) + +// testSetLegacyReplicationLagAlgorithm is a test helper function, if this is used by a production code path, something is wrong. +func testSetLegacyReplicationLagAlgorithm(newLegacy bool) { + *legacyReplicationLagAlgorithm = newLegacy +} + +func TestFilterLegacyStatsByReplicationLagUnhealthy(t *testing.T) { + // 1 healthy serving tablet, 1 not healhty + ts1 := &LegacyTabletStats{ + Tablet: topo.NewTablet(1, "cell", "host1"), + Serving: true, + Stats: &querypb.RealtimeStats{}, + } + ts2 := &LegacyTabletStats{ + Tablet: topo.NewTablet(2, "cell", "host2"), + Serving: false, + Stats: &querypb.RealtimeStats{}, + } + got := FilterLegacyStatsByReplicationLag([]*LegacyTabletStats{ts1, ts2}) + if len(got) != 1 { + t.Errorf("len(FilterLegacyStatsByReplicationLag([{Tablet: {Uid: 1}, Serving: true}, {Tablet: {Uid: 2}, Serving: false}])) = %v, want 1", len(got)) + } + if len(got) > 0 && !got[0].DeepEqual(ts1) { + t.Errorf("FilterLegacyStatsByReplicationLag([{Tablet: {Uid: 1}, Serving: true}, {Tablet: {Uid: 2}, Serving: false}]) = %+v, want %+v", got[0], ts1) + } +} + +func TestFilterLegacyStatsByReplicationLag(t *testing.T) { + // Use simplified logic + testSetLegacyReplicationLagAlgorithm(false) + + cases := []struct { + description string + input []uint32 + output []uint32 + }{ + { + "0 tablet", + []uint32{}, + []uint32{}, + }, + { + "lags of (1s) - return all items with low lag.", + []uint32{1}, + []uint32{1}, + }, + { + "lags of (1s, 1s, 1s, 30s) - return all items with low lag.", + []uint32{1, 1, 1, 30}, + []uint32{1, 1, 1, 30}, + }, + { + "lags of (1s, 1s, 1s, 40m, 40m, 40m) - return all items with low lag.", + []uint32{1, 1, 1, 40 * 60, 40 * 60, 40 * 60}, + []uint32{1, 1, 1}, + }, + { + "lags of (1s, 40m, 40m, 40m) - return at least 2 items if they don't have very high lag.", + []uint32{1, 40 * 60, 40 * 60, 40 * 60}, + []uint32{1, 40 * 60}, + }, + { + "lags of (30m, 35m, 40m, 45m) - return at least 2 items if they don't have very high lag.", + []uint32{30 * 60, 35 * 60, 40 * 60, 45 * 60}, + []uint32{30 * 60, 35 * 60}, + }, + { + "lags of (2h, 3h, 4h, 5h) - return <2 items if the others have very high lag.", + []uint32{2 * 60 * 60, 3 * 60 * 60, 4 * 60 * 60, 5 * 60 * 60}, + []uint32{2 * 60 * 60}, + }, + { + "lags of (3h, 30h) - return nothing if all have very high lag.", + []uint32{3 * 60 * 60, 30 * 60 * 60}, + []uint32{}, + }, + } + + for _, tc := range cases { + lts := make([]*LegacyTabletStats, len(tc.input)) + for i, lag := range tc.input { + lts[i] = &LegacyTabletStats{ + Tablet: topo.NewTablet(uint32(i+1), "cell", fmt.Sprintf("host-%vs-behind", lag)), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: lag}, + } + } + got := FilterLegacyStatsByReplicationLag(lts) + if len(got) != len(tc.output) { + t.Errorf("FilterLegacyStatsByReplicationLag(%v) failed: got output:\n%v\nExpected: %v", tc.description, got, tc.output) + continue + } + for i, elag := range tc.output { + if got[i].Stats.SecondsBehindMaster != elag { + t.Errorf("FilterLegacyStatsByReplicationLag(%v) failed: got output:\n%v\nExpected value index %v to be %v", tc.description, got, i, elag) + } + } + } + + // Reset to the default + testSetLegacyReplicationLagAlgorithm(true) +} + +func TestFilterLegacyStatysByReplicationLagWithLegacyAlgorithm(t *testing.T) { + // Use legacy algorithm by default for now + + cases := []struct { + description string + input []uint32 + output []uint32 + }{ + { + "0 tablet", + []uint32{}, + []uint32{}, + }, + { + "1 serving tablet", + []uint32{1}, + []uint32{1}, + }, + { + "lags of (1s, 1s, 1s, 30s)", + []uint32{1, 1, 1, 30}, + []uint32{1, 1, 1, 30}, + }, + { + "lags of (30m, 35m, 40m, 45m)", + []uint32{30 * 60, 35 * 60, 40 * 60, 45 * 60}, + []uint32{30 * 60, 35 * 60, 40 * 60, 45 * 60}, + }, + { + "lags of (1s, 1s, 1m, 40m, 40m) - not run filter the second time as first run removed two items.", + []uint32{1, 1, 60, 40 * 60, 40 * 60}, + []uint32{1, 1, 60}, + }, + { + "lags of (1s, 1s, 10m, 40m) - run filter twice to remove two items", + []uint32{1, 1, 10 * 60, 40 * 60}, + []uint32{1, 1}, + }, + { + "lags of (1m, 100m) - return at least 2 items to avoid overloading if the 2nd one is not delayed too much.", + []uint32{1 * 60, 100 * 60}, + []uint32{1 * 60, 100 * 60}, + }, + { + "lags of (1m, 3h) - return 1 if the 2nd one is delayed too much.", + []uint32{1 * 60, 3 * 60 * 60}, + []uint32{1 * 60}, + }, + { + "lags of (3h) - return 1 as they're all delayed too much.", + []uint32{3 * 60 * 60}, + []uint32{3 * 60 * 60}, + }, + { + "lags of (3h, 4h) - return 2 as they're all delayed too much, but still in a good group.", + []uint32{3 * 60 * 60, 4 * 60 * 60}, + []uint32{3 * 60 * 60, 4 * 60 * 60}, + }, + { + "lags of (3h, 3h, 4h) - return 3 as they're all delayed too much, but still in a good group.", + []uint32{3 * 60 * 60, 3 * 60 * 60, 4 * 60 * 60}, + []uint32{3 * 60 * 60, 3 * 60 * 60, 4 * 60 * 60}, + }, + { + "lags of (3h, 15h, 18h) - return 3 as they're all delayed too much, but still in a good group." + + "(different test case than above to show how absurb the good group logic is)", + []uint32{3 * 60 * 60, 15 * 60 * 60, 18 * 60 * 60}, + []uint32{3 * 60 * 60, 15 * 60 * 60, 18 * 60 * 60}, + }, + { + "lags of (3h, 12h, 18h) - return 2 as they're all delayed too much, but 18h is now considered an outlier." + + "(different test case than above to show how absurb the good group logic is)", + []uint32{3 * 60 * 60, 12 * 60 * 60, 18 * 60 * 60}, + []uint32{3 * 60 * 60, 12 * 60 * 60}, + }, + { + "lags of (3h, 30h) - return 2 as they're all delayed too much." + + "(different test case that before, as both tablet stats are" + + "widely different, not within 70% of eachother)", + []uint32{3 * 60 * 60, 30 * 60 * 60}, + []uint32{3 * 60 * 60, 30 * 60 * 60}, + }, + } + + for _, tc := range cases { + lts := make([]*LegacyTabletStats, len(tc.input)) + for i, lag := range tc.input { + lts[i] = &LegacyTabletStats{ + Tablet: topo.NewTablet(uint32(i+1), "cell", fmt.Sprintf("host-%vs-behind", lag)), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: lag}, + } + } + got := FilterLegacyStatsByReplicationLag(lts) + if len(got) != len(tc.output) { + t.Errorf("FilterLegacyStatsByReplicationLag(%v) failed: got output:\n%v\nExpected: %v", tc.description, got, tc.output) + continue + } + for i, elag := range tc.output { + if got[i].Stats.SecondsBehindMaster != elag { + t.Errorf("FilterLegacyStatsByReplicationLag(%v) failed: got output:\n%v\nExpected value index %v to be %v", tc.description, got, i, elag) + } + } + } +} + +func TestFilterLegacyStatsByReplicationLagThreeTabletMin(t *testing.T) { + // Use at least 3 tablets if possible + testSetMinNumTablets(3) + // lags of (1s, 1s, 10m, 11m) - returns at least32 items where the slightly delayed ones that are returned are the 10m and 11m ones. + ts1 := &LegacyTabletStats{ + Tablet: topo.NewTablet(1, "cell", "host1"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1}, + } + ts2 := &LegacyTabletStats{ + Tablet: topo.NewTablet(2, "cell", "host2"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1}, + } + ts3 := &LegacyTabletStats{ + Tablet: topo.NewTablet(3, "cell", "host3"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 10 * 60}, + } + ts4 := &LegacyTabletStats{ + Tablet: topo.NewTablet(4, "cell", "host4"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 11 * 60}, + } + got := FilterLegacyStatsByReplicationLag([]*LegacyTabletStats{ts1, ts2, ts3, ts4}) + if len(got) != 3 || !got[0].DeepEqual(ts1) || !got[1].DeepEqual(ts2) || !got[2].DeepEqual(ts3) { + t.Errorf("FilterLegacyStatsByReplicationLag([1s, 1s, 10m, 11m]) = %+v, want [1s, 1s, 10m]", got) + } + // lags of (11m, 10m, 1s, 1s) - reordered tablets returns the same 3 items where the slightly delayed one that is returned is the 10m and 11m ones. + ts1 = &LegacyTabletStats{ + Tablet: topo.NewTablet(1, "cell", "host1"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 11 * 60}, + } + ts2 = &LegacyTabletStats{ + Tablet: topo.NewTablet(2, "cell", "host2"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 10 * 60}, + } + ts3 = &LegacyTabletStats{ + Tablet: topo.NewTablet(3, "cell", "host3"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1}, + } + ts4 = &LegacyTabletStats{ + Tablet: topo.NewTablet(4, "cell", "host4"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1}, + } + got = FilterLegacyStatsByReplicationLag([]*LegacyTabletStats{ts1, ts2, ts3, ts4}) + if len(got) != 3 || !got[0].DeepEqual(ts3) || !got[1].DeepEqual(ts4) || !got[2].DeepEqual(ts2) { + t.Errorf("FilterLegacyStatsByReplicationLag([1s, 1s, 10m, 11m]) = %+v, want [1s, 1s, 10m]", got) + } + // Reset to the default + testSetMinNumTablets(2) +} + +func TestFilterByReplicationLagOneTabletMin(t *testing.T) { + // Use at least 1 tablets if possible + testSetMinNumTablets(1) + // lags of (1s, 100m) - return only healthy tablet if that is all that is available. + ts1 := &LegacyTabletStats{ + Tablet: topo.NewTablet(1, "cell", "host1"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1}, + } + ts2 := &LegacyTabletStats{ + Tablet: topo.NewTablet(2, "cell", "host2"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 100 * 60}, + } + got := FilterLegacyStatsByReplicationLag([]*LegacyTabletStats{ts1, ts2}) + if len(got) != 1 || !got[0].DeepEqual(ts1) { + t.Errorf("FilterLegacyStatsByReplicationLag([1s, 100m]) = %+v, want [1s]", got) + } + // lags of (1m, 100m) - return only healthy tablet if that is all that is healthy enough. + ts1 = &LegacyTabletStats{ + Tablet: topo.NewTablet(1, "cell", "host1"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1 * 60}, + } + ts2 = &LegacyTabletStats{ + Tablet: topo.NewTablet(2, "cell", "host2"), + Serving: true, + Stats: &querypb.RealtimeStats{SecondsBehindMaster: 100 * 60}, + } + got = FilterLegacyStatsByReplicationLag([]*LegacyTabletStats{ts1, ts2}) + if len(got) != 1 || !got[0].DeepEqual(ts1) { + t.Errorf("FilterLegacyStatsByReplicationLag([1m, 100m]) = %+v, want [1m]", got) + } + // Reset to the default + testSetMinNumTablets(2) +} + +func TestTrivialLegacyStatsUpdate(t *testing.T) { + // Note the healthy threshold is set to 30s. + cases := []struct { + o uint32 + n uint32 + expected bool + }{ + // both are under 30s + {o: 0, n: 1, expected: true}, + {o: 15, n: 20, expected: true}, + + // one is under 30s, the other isn't + {o: 2, n: 40, expected: false}, + {o: 40, n: 10, expected: false}, + + // both are over 30s, but close enough + {o: 100, n: 100, expected: true}, + {o: 100, n: 105, expected: true}, + {o: 105, n: 100, expected: true}, + + // both are over 30s, but too far + {o: 100, n: 120, expected: false}, + {o: 120, n: 100, expected: false}, + } + + for _, c := range cases { + o := &LegacyTabletStats{ + Stats: &querypb.RealtimeStats{ + SecondsBehindMaster: c.o, + }, + } + n := &LegacyTabletStats{ + Stats: &querypb.RealtimeStats{ + SecondsBehindMaster: c.n, + }, + } + got := o.TrivialStatsUpdate(n) + if got != c.expected { + t.Errorf("TrivialStatsUpdate(%v, %v) = %v, expected %v", c.o, c.n, got, c.expected) + } + } +} diff --git a/go/vt/discovery/tablet_stats_cache.go b/go/vt/discovery/legacy_tablet_stats_cache.go similarity index 69% rename from go/vt/discovery/tablet_stats_cache.go rename to go/vt/discovery/legacy_tablet_stats_cache.go index 14771e7aec9..196f3c00b46 100644 --- a/go/vt/discovery/tablet_stats_cache.go +++ b/go/vt/discovery/legacy_tablet_stats_cache.go @@ -23,15 +23,14 @@ import ( "vitess.io/vitess/go/vt/log" querypb "vitess.io/vitess/go/vt/proto/query" topodatapb "vitess.io/vitess/go/vt/proto/topodata" - "vitess.io/vitess/go/vt/srvtopo" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" ) -// TabletStatsCache is a HealthCheckStatsListener that keeps both the -// current list of available TabletStats, and a serving list: +// LegacyTabletStatsCache is a LegacyHealthCheckStatsListener that keeps both the +// current list of available LegacyTabletStats, and a serving list: // - for master tablets, only the current master is kept. -// - for non-master tablets, we filter the list using FilterByReplicationLag. +// - for non-master tablets, we filter the list using FilterLegacyStatsByReplicationLag. // It keeps entries for all tablets in the cell(s) it's configured to serve for, // and for the master independently of which cell it's in. // Note the healthy tablet computation is done when we receive a tablet @@ -39,7 +38,7 @@ import ( // Also note the cache may not have the last entry received by the tablet. // For instance, if a tablet was healthy, and is still healthy, we do not // keep its new update. -type TabletStatsCache struct { +type LegacyTabletStatsCache struct { // cell is the cell we are keeping all tablets for. // Note we keep track of all master tablets in all cells. cell string @@ -49,26 +48,24 @@ type TabletStatsCache struct { // entries in the entries map. mu sync.RWMutex // entries maps from keyspace/shard/tabletType to our cache. - entries map[string]map[string]map[topodatapb.TabletType]*tabletStatsCacheEntry - // tsm is a helper to broadcast aggregate stats. - tsm srvtopo.TargetStatsMultiplexer + entries map[string]map[string]map[topodatapb.TabletType]*legacyTabletStatsCacheEntry // cellAliases is a cache of cell aliases cellAliases map[string]string } -// tabletStatsCacheEntry is the per keyspace/shard/tabletType -// entry of the in-memory map for TabletStatsCache. -type tabletStatsCacheEntry struct { +// legacyTabletStatsCacheEntry is the per keyspace/shard/tabletType +// entry of the in-memory map for LegacyTabletStatsCache. +type legacyTabletStatsCacheEntry struct { // mu protects the rest of this structure. mu sync.RWMutex // all has the valid tablets, indexed by TabletToMapKey(ts.Tablet), - // as it is the index used by HealthCheck. - all map[string]*TabletStats + // as it is the index used by LegacyHealthCheck. + all map[string]*LegacyTabletStats // healthy only has the healthy ones. - healthy []*TabletStats + healthy []*LegacyTabletStats } -func (e *tabletStatsCacheEntry) updateHealthyMapForMaster(ts *TabletStats) { +func (e *legacyTabletStatsCacheEntry) updateHealthyMapForMaster(ts *LegacyTabletStats) { if ts.Up { // We have an Up master. if len(e.healthy) == 0 { @@ -103,32 +100,31 @@ func (e *tabletStatsCacheEntry) updateHealthyMapForMaster(ts *TabletStats) { } } -// NewTabletStatsCache creates a TabletStatsCache, and registers -// it as HealthCheckStatsListener of the provided healthcheck. +// NewLegacyTabletStatsCache creates a LegacyTabletStatsCache, and registers +// it as LegacyHealthCheckStatsListener of the provided healthcheck. // Note we do the registration in this code to guarantee we call // SetListener with sendDownEvents=true, as we need these events // to maintain the integrity of our cache. -func NewTabletStatsCache(hc HealthCheck, ts *topo.Server, cell string) *TabletStatsCache { - return newTabletStatsCache(hc, ts, cell, true /* setListener */) +func NewLegacyTabletStatsCache(hc LegacyHealthCheck, ts *topo.Server, cell string) *LegacyTabletStatsCache { + return newLegacyTabletStatsCache(hc, ts, cell, true /* setListener */) } -// NewTabletStatsCacheDoNotSetListener is identical to NewTabletStatsCache +// NewTabletStatsCacheDoNotSetListener is identical to NewLegacyTabletStatsCache // but does not automatically set the returned object as listener for "hc". -// Instead, it's up to the caller to ensure that TabletStatsCache.StatsUpdate() +// Instead, it's up to the caller to ensure that LegacyTabletStatsCache.StatsUpdate() // gets called properly. This is useful for chaining multiple listeners. // When the caller sets its own listener on "hc", they must make sure that they // set the parameter "sendDownEvents" to "true" or this cache won't properly // remove tablets whose tablet type changes. -func NewTabletStatsCacheDoNotSetListener(ts *topo.Server, cell string) *TabletStatsCache { - return newTabletStatsCache(nil, ts, cell, false /* setListener */) +func NewTabletStatsCacheDoNotSetListener(ts *topo.Server, cell string) *LegacyTabletStatsCache { + return newLegacyTabletStatsCache(nil, ts, cell, false /* setListener */) } -func newTabletStatsCache(hc HealthCheck, ts *topo.Server, cell string, setListener bool) *TabletStatsCache { - tc := &TabletStatsCache{ +func newLegacyTabletStatsCache(hc LegacyHealthCheck, ts *topo.Server, cell string, setListener bool) *LegacyTabletStatsCache { + tc := &LegacyTabletStatsCache{ cell: cell, ts: ts, - entries: make(map[string]map[string]map[topodatapb.TabletType]*tabletStatsCacheEntry), - tsm: srvtopo.NewTargetStatsMultiplexer(), + entries: make(map[string]map[string]map[topodatapb.TabletType]*legacyTabletStatsCacheEntry), cellAliases: make(map[string]string), } @@ -140,9 +136,9 @@ func newTabletStatsCache(hc HealthCheck, ts *topo.Server, cell string, setListen return tc } -// getEntry returns an existing tabletStatsCacheEntry in the cache, or nil +// getEntry returns an existing legacyTabletStatsCacheEntry in the cache, or nil // if the entry does not exist. It only takes a Read lock on mu. -func (tc *TabletStatsCache) getEntry(keyspace, shard string, tabletType topodatapb.TabletType) *tabletStatsCacheEntry { +func (tc *LegacyTabletStatsCache) getEntry(keyspace, shard string, tabletType topodatapb.TabletType) *legacyTabletStatsCacheEntry { tc.mu.RLock() defer tc.mu.RUnlock() @@ -156,9 +152,9 @@ func (tc *TabletStatsCache) getEntry(keyspace, shard string, tabletType topodata return nil } -// getOrCreateEntry returns an existing tabletStatsCacheEntry from the cache, +// getOrCreateEntry returns an existing legacyTabletStatsCacheEntry from the cache, // or creates it if it doesn't exist. -func (tc *TabletStatsCache) getOrCreateEntry(target *querypb.Target) *tabletStatsCacheEntry { +func (tc *LegacyTabletStatsCache) getOrCreateEntry(target *querypb.Target) *legacyTabletStatsCacheEntry { // Fast path (most common path too): Read-lock, return the entry. if e := tc.getEntry(target.Keyspace, target.Shard, target.TabletType); e != nil { return e @@ -170,25 +166,25 @@ func (tc *TabletStatsCache) getOrCreateEntry(target *querypb.Target) *tabletStat s, ok := tc.entries[target.Keyspace] if !ok { - s = make(map[string]map[topodatapb.TabletType]*tabletStatsCacheEntry) + s = make(map[string]map[topodatapb.TabletType]*legacyTabletStatsCacheEntry) tc.entries[target.Keyspace] = s } t, ok := s[target.Shard] if !ok { - t = make(map[topodatapb.TabletType]*tabletStatsCacheEntry) + t = make(map[topodatapb.TabletType]*legacyTabletStatsCacheEntry) s[target.Shard] = t } e, ok := t[target.TabletType] if !ok { - e = &tabletStatsCacheEntry{ - all: make(map[string]*TabletStats), + e = &legacyTabletStatsCacheEntry{ + all: make(map[string]*LegacyTabletStats), } t[target.TabletType] = e } return e } -func (tc *TabletStatsCache) getAliasByCell(cell string) string { +func (tc *LegacyTabletStatsCache) getAliasByCell(cell string) string { tc.mu.Lock() defer tc.mu.Unlock() @@ -202,8 +198,8 @@ func (tc *TabletStatsCache) getAliasByCell(cell string) string { return alias } -// StatsUpdate is part of the HealthCheckStatsListener interface. -func (tc *TabletStatsCache) StatsUpdate(ts *TabletStats) { +// StatsUpdate is part of the LegacyHealthCheckStatsListener interface. +func (tc *LegacyTabletStatsCache) StatsUpdate(ts *LegacyTabletStats) { if ts.Target.TabletType != topodatapb.TabletType_MASTER && ts.Tablet.Alias.Cell != tc.cell && tc.getAliasByCell(ts.Tablet.Alias.Cell) != tc.getAliasByCell(tc.cell) { @@ -221,7 +217,8 @@ func (tc *TabletStatsCache) StatsUpdate(ts *TabletStats) { if ts.Up { // We have an existing entry, and a new entry. // Remember if they are both good (most common case). - trivialNonMasterUpdate = existing.LastError == nil && existing.Serving && ts.LastError == nil && ts.Serving && ts.Target.TabletType != topodatapb.TabletType_MASTER && TrivialStatsUpdate(existing, ts) + trivialNonMasterUpdate = existing.LastError == nil && existing.Serving && ts.LastError == nil && + ts.Serving && ts.Target.TabletType != topodatapb.TabletType_MASTER && existing.TrivialStatsUpdate(ts) // We already have the entry, update the // values if necessary. (will update both @@ -245,7 +242,7 @@ func (tc *TabletStatsCache) StatsUpdate(ts *TabletStats) { } // Update our healthy list. - var allArray []*TabletStats + var allArray []*LegacyTabletStats if ts.Target.TabletType == topodatapb.TabletType_MASTER { // The healthy list is different for TabletType_MASTER: we // only keep the most recent one. @@ -259,17 +256,17 @@ func (tc *TabletStatsCache) StatsUpdate(ts *TabletStats) { } // Now we need to do some work. Recompute our healthy list. - allArray = make([]*TabletStats, 0, len(e.all)) + allArray = make([]*LegacyTabletStats, 0, len(e.all)) for _, s := range e.all { allArray = append(allArray, s) } - e.healthy = FilterByReplicationLag(allArray) + e.healthy = FilterLegacyStatsByReplicationLag(allArray) } } // GetTabletStats returns the full list of available targets. // The returned array is owned by the caller. -func (tc *TabletStatsCache) GetTabletStats(keyspace, shard string, tabletType topodatapb.TabletType) []TabletStats { +func (tc *LegacyTabletStatsCache) GetTabletStats(keyspace, shard string, tabletType topodatapb.TabletType) []LegacyTabletStats { e := tc.getEntry(keyspace, shard, tabletType) if e == nil { return nil @@ -277,7 +274,7 @@ func (tc *TabletStatsCache) GetTabletStats(keyspace, shard string, tabletType to e.mu.RLock() defer e.mu.RUnlock() - result := make([]TabletStats, 0, len(e.all)) + result := make([]LegacyTabletStats, 0, len(e.all)) for _, s := range e.all { result = append(result, *s) } @@ -288,7 +285,7 @@ func (tc *TabletStatsCache) GetTabletStats(keyspace, shard string, tabletType to // The returned array is owned by the caller. // For TabletType_MASTER, this will only return at most one entry, // the most recent tablet of type master. -func (tc *TabletStatsCache) GetHealthyTabletStats(keyspace, shard string, tabletType topodatapb.TabletType) []TabletStats { +func (tc *LegacyTabletStatsCache) GetHealthyTabletStats(keyspace, shard string, tabletType topodatapb.TabletType) []LegacyTabletStats { e := tc.getEntry(keyspace, shard, tabletType) if e == nil { return nil @@ -296,7 +293,7 @@ func (tc *TabletStatsCache) GetHealthyTabletStats(keyspace, shard string, tablet e.mu.RLock() defer e.mu.RUnlock() - result := make([]TabletStats, len(e.healthy)) + result := make([]LegacyTabletStats, len(e.healthy)) for i, ts := range e.healthy { result[i] = *ts } @@ -304,12 +301,12 @@ func (tc *TabletStatsCache) GetHealthyTabletStats(keyspace, shard string, tablet } // ResetForTesting is for use in tests only. -func (tc *TabletStatsCache) ResetForTesting() { +func (tc *LegacyTabletStatsCache) ResetForTesting() { tc.mu.Lock() defer tc.mu.Unlock() - tc.entries = make(map[string]map[string]map[topodatapb.TabletType]*tabletStatsCacheEntry) + tc.entries = make(map[string]map[string]map[topodatapb.TabletType]*legacyTabletStatsCacheEntry) } // Compile-time interface check. -var _ HealthCheckStatsListener = (*TabletStatsCache)(nil) +var _ LegacyHealthCheckStatsListener = (*LegacyTabletStatsCache)(nil) diff --git a/go/vt/discovery/tablet_stats_cache_test.go b/go/vt/discovery/legacy_tablet_stats_cache_test.go similarity index 93% rename from go/vt/discovery/tablet_stats_cache_test.go rename to go/vt/discovery/legacy_tablet_stats_cache_test.go index 9897d503db7..4a50109cafe 100644 --- a/go/vt/discovery/tablet_stats_cache_test.go +++ b/go/vt/discovery/legacy_tablet_stats_cache_test.go @@ -27,8 +27,8 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) -// TestTabletStatsCache tests the functionality of the TabletStatsCache class. -func TestTabletStatsCache(t *testing.T) { +// TestTabletStatsCache tests the functionality of the LegacyTabletStatsCache class. +func TestLegacyTabletStatsCache(t *testing.T) { ts := memorytopo.NewServer("cell", "cell1", "cell2") cellsAlias := &topodatapb.CellsAlias{ @@ -47,13 +47,13 @@ func TestTabletStatsCache(t *testing.T) { defer ts.DeleteCellsAlias(context.Background(), "region2") - // We want to unit test TabletStatsCache without a full-blown - // HealthCheck object, so we can't call NewTabletStatsCache. + // We want to unit test LegacyTabletStatsCache without a full-blown + // LegacyHealthCheck object, so we can't call NewLegacyTabletStatsCache. // So we just construct this object here. - tsc := &TabletStatsCache{ + tsc := &LegacyTabletStatsCache{ cell: "cell", ts: ts, - entries: make(map[string]map[string]map[topodatapb.TabletType]*tabletStatsCacheEntry), + entries: make(map[string]map[string]map[topodatapb.TabletType]*legacyTabletStatsCacheEntry), cellAliases: make(map[string]string), } @@ -65,7 +65,7 @@ func TestTabletStatsCache(t *testing.T) { // add a tablet tablet1 := topo.NewTablet(10, "cell", "host1") - ts1 := &TabletStats{ + ts1 := &LegacyTabletStats{ Key: "t1", Tablet: tablet1, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -86,7 +86,7 @@ func TestTabletStatsCache(t *testing.T) { } // update stats with a change that won't change health array - stillHealthyTs1 := &TabletStats{ + stillHealthyTs1 := &LegacyTabletStats{ Key: "t1", Tablet: tablet1, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -107,7 +107,7 @@ func TestTabletStatsCache(t *testing.T) { } // update stats with a change that will change arrays - notHealthyTs1 := &TabletStats{ + notHealthyTs1 := &LegacyTabletStats{ Key: "t1", Tablet: tablet1, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -129,7 +129,7 @@ func TestTabletStatsCache(t *testing.T) { // add a second tablet tablet2 := topo.NewTablet(11, "cell", "host2") - ts2 := &TabletStats{ + ts2 := &LegacyTabletStats{ Key: "t2", Tablet: tablet2, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -232,7 +232,7 @@ func TestTabletStatsCache(t *testing.T) { // add a third tablet as slave in diff cell, same region tablet3 := topo.NewTablet(12, "cell1", "host3") - ts3 := &TabletStats{ + ts3 := &LegacyTabletStats{ Key: "t3", Tablet: tablet3, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -253,7 +253,7 @@ func TestTabletStatsCache(t *testing.T) { // add a 4th slave tablet in a diff cell, diff region tablet4 := topo.NewTablet(13, "cell2", "host4") - ts4 := &TabletStats{ + ts4 := &LegacyTabletStats{ Key: "t4", Tablet: tablet4, Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, diff --git a/go/vt/discovery/tablet_stats_cache_wait.go b/go/vt/discovery/legacy_tablet_stats_cache_wait.go similarity index 82% rename from go/vt/discovery/tablet_stats_cache_wait.go rename to go/vt/discovery/legacy_tablet_stats_cache_wait.go index 1b1123a7439..d984ff7d1ce 100644 --- a/go/vt/discovery/tablet_stats_cache_wait.go +++ b/go/vt/discovery/legacy_tablet_stats_cache_wait.go @@ -33,7 +33,7 @@ var ( // WaitForTablets waits for at least one tablet in the given // keyspace / shard / tablet type before returning. The tablets do not // have to be healthy. It will return ctx.Err() if the context is canceled. -func (tc *TabletStatsCache) WaitForTablets(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType) error { +func (tc *LegacyTabletStatsCache) WaitForTablets(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType) error { targets := []*querypb.Target{ { Keyspace: keyspace, @@ -48,12 +48,12 @@ func (tc *TabletStatsCache) WaitForTablets(ctx context.Context, keyspace, shard // each given target before returning. // It will return ctx.Err() if the context is canceled. // It will return an error if it can't read the necessary topology records. -func (tc *TabletStatsCache) WaitForAllServingTablets(ctx context.Context, targets []*querypb.Target) error { +func (tc *LegacyTabletStatsCache) WaitForAllServingTablets(ctx context.Context, targets []*querypb.Target) error { return tc.waitForTablets(ctx, targets, true) } // waitForTablets is the internal method that polls for tablets. -func (tc *TabletStatsCache) waitForTablets(ctx context.Context, targets []*querypb.Target, requireServing bool) error { +func (tc *LegacyTabletStatsCache) waitForTablets(ctx context.Context, targets []*querypb.Target, requireServing bool) error { for { // We nil targets as we find them. allPresent := true @@ -62,7 +62,7 @@ func (tc *TabletStatsCache) waitForTablets(ctx context.Context, targets []*query continue } - var stats []TabletStats + var stats []LegacyTabletStats if requireServing { stats = tc.GetHealthyTabletStats(target.Keyspace, target.Shard, target.TabletType) } else { @@ -92,7 +92,7 @@ func (tc *TabletStatsCache) waitForTablets(ctx context.Context, targets []*query } // WaitByFilter waits for at least one tablet based on the filter function. -func (tc *TabletStatsCache) WaitByFilter(ctx context.Context, keyspace, shard string, tabletTypes []topodatapb.TabletType, filter func([]TabletStats) []TabletStats) error { +func (tc *LegacyTabletStatsCache) WaitByFilter(ctx context.Context, keyspace, shard string, tabletTypes []topodatapb.TabletType, filter func([]LegacyTabletStats) []LegacyTabletStats) error { for { for _, tt := range tabletTypes { stats := tc.GetTabletStats(keyspace, shard, tt) diff --git a/go/vt/discovery/tablet_stats_cache_wait_test.go b/go/vt/discovery/legacy_tablet_stats_cache_wait_test.go similarity index 95% rename from go/vt/discovery/tablet_stats_cache_wait_test.go rename to go/vt/discovery/legacy_tablet_stats_cache_wait_test.go index 0cf309931c1..1eb7c4df470 100644 --- a/go/vt/discovery/tablet_stats_cache_wait_test.go +++ b/go/vt/discovery/legacy_tablet_stats_cache_wait_test.go @@ -38,8 +38,8 @@ func TestWaitForTablets(t *testing.T) { input := make(chan *querypb.StreamHealthResponse) createFakeConn(tablet, input) - hc := NewHealthCheck(1*time.Millisecond, 1*time.Hour) - tsc := NewTabletStatsCache(hc, nil, "cell") + hc := NewLegacyHealthCheck(1*time.Millisecond, 1*time.Hour) + tsc := NewLegacyTabletStatsCache(hc, nil, "cell") hc.AddTablet(tablet, "") // this should time out diff --git a/go/vt/discovery/legacy_topology_watcher.go b/go/vt/discovery/legacy_topology_watcher.go new file mode 100644 index 00000000000..02370a511a8 --- /dev/null +++ b/go/vt/discovery/legacy_topology_watcher.go @@ -0,0 +1,456 @@ +/* +Copyright 2019 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 discovery + +import ( + "bytes" + "fmt" + "hash/crc32" + "sort" + "strings" + "sync" + "time" + + "golang.org/x/net/context" + "vitess.io/vitess/go/trace" + + "vitess.io/vitess/go/vt/key" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/topoproto" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +// tabletInfo is used internally by the TopologyWatcher class +type legacyTabletInfo struct { + alias string + key string + tablet *topodatapb.Tablet +} + +// NewLegacyCellTabletsWatcher returns a LegacyTopologyWatcher that monitors all +// the tablets in a cell, and starts refreshing. +func NewLegacyCellTabletsWatcher(ctx context.Context, topoServer *topo.Server, tr LegacyTabletRecorder, cell string, refreshInterval time.Duration, refreshKnownTablets bool, topoReadConcurrency int) *LegacyTopologyWatcher { + return NewLegacyTopologyWatcher(ctx, topoServer, tr, cell, refreshInterval, refreshKnownTablets, topoReadConcurrency, func(tw *LegacyTopologyWatcher) ([]*topodatapb.TabletAlias, error) { + return tw.topoServer.GetTabletsByCell(ctx, tw.cell) + }) +} + +// NewLegacyShardReplicationWatcher returns a LegacyTopologyWatcher that +// monitors the tablets in a cell/keyspace/shard, and starts refreshing. +func NewLegacyShardReplicationWatcher(ctx context.Context, topoServer *topo.Server, tr LegacyTabletRecorder, cell, keyspace, shard string, refreshInterval time.Duration, topoReadConcurrency int) *LegacyTopologyWatcher { + return NewLegacyTopologyWatcher(ctx, topoServer, tr, cell, refreshInterval, true /* RefreshKnownTablets */, topoReadConcurrency, func(tw *LegacyTopologyWatcher) ([]*topodatapb.TabletAlias, error) { + sri, err := tw.topoServer.GetShardReplication(ctx, tw.cell, keyspace, shard) + switch { + case err == nil: + // we handle this case after this switch block + case topo.IsErrType(err, topo.NoNode): + // this is not an error + return nil, nil + default: + return nil, err + } + + result := make([]*topodatapb.TabletAlias, len(sri.Nodes)) + for i, node := range sri.Nodes { + result[i] = node.TabletAlias + } + return result, nil + }) +} + +// LegacyTopologyWatcher polls tablet from a configurable set of tablets +// periodically. When tablets are added / removed, it calls +// the LegacyTabletRecorder AddTablet / RemoveTablet interface appropriately. +type LegacyTopologyWatcher struct { + // set at construction time + topoServer *topo.Server + tr LegacyTabletRecorder + cell string + refreshInterval time.Duration + refreshKnownTablets bool + getTablets func(tw *LegacyTopologyWatcher) ([]*topodatapb.TabletAlias, error) + sem chan int + ctx context.Context + cancelFunc context.CancelFunc + // wg keeps track of all launched Go routines. + wg sync.WaitGroup + + // mu protects all variables below + mu sync.Mutex + // tablets contains a map of alias -> tabletInfo for all known tablets + tablets map[string]*legacyTabletInfo + // topoChecksum stores a crc32 of the tablets map and is exported as a metric + topoChecksum uint32 + // lastRefresh records the timestamp of the last topo refresh + lastRefresh time.Time + // firstLoadDone is true when first load of the topology data is done. + firstLoadDone bool + // firstLoadChan is closed when the initial loading of topology data is done. + firstLoadChan chan struct{} +} + +// NewLegacyTopologyWatcher returns a LegacyTopologyWatcher that monitors all +// the tablets in a cell, and starts refreshing. +func NewLegacyTopologyWatcher(ctx context.Context, topoServer *topo.Server, tr LegacyTabletRecorder, cell string, refreshInterval time.Duration, refreshKnownTablets bool, topoReadConcurrency int, getTablets func(tw *LegacyTopologyWatcher) ([]*topodatapb.TabletAlias, error)) *LegacyTopologyWatcher { + tw := &LegacyTopologyWatcher{ + topoServer: topoServer, + tr: tr, + cell: cell, + refreshInterval: refreshInterval, + refreshKnownTablets: refreshKnownTablets, + getTablets: getTablets, + sem: make(chan int, topoReadConcurrency), + tablets: make(map[string]*legacyTabletInfo), + } + tw.firstLoadChan = make(chan struct{}) + + // We want the span from the context, but not the cancelation that comes with it + spanContext := trace.CopySpan(context.Background(), ctx) + tw.ctx, tw.cancelFunc = context.WithCancel(spanContext) + tw.wg.Add(1) + go tw.watch() + return tw +} + +// watch polls all tablets and notifies LegacyTabletRecorder by adding/removing tablets. +func (tw *LegacyTopologyWatcher) watch() { + defer tw.wg.Done() + ticker := time.NewTicker(tw.refreshInterval) + defer ticker.Stop() + for { + tw.loadTablets() + select { + case <-tw.ctx.Done(): + return + case <-ticker.C: + } + } +} + +// loadTablets reads all tablets from topology, and updates LegacyTabletRecorder. +func (tw *LegacyTopologyWatcher) loadTablets() { + var wg sync.WaitGroup + newTablets := make(map[string]*legacyTabletInfo) + replacedTablets := make(map[string]*legacyTabletInfo) + + tabletAliases, err := tw.getTablets(tw) + topologyWatcherOperations.Add(topologyWatcherOpListTablets, 1) + if err != nil { + topologyWatcherErrors.Add(topologyWatcherOpListTablets, 1) + select { + case <-tw.ctx.Done(): + return + default: + } + log.Errorf("cannot get tablets for cell: %v: %v", tw.cell, err) + return + } + + // Accumulate a list of all known alias strings to use later + // when sorting + tabletAliasStrs := make([]string, 0, len(tabletAliases)) + + tw.mu.Lock() + for _, tAlias := range tabletAliases { + aliasStr := topoproto.TabletAliasString(tAlias) + tabletAliasStrs = append(tabletAliasStrs, aliasStr) + + if !tw.refreshKnownTablets { + if val, ok := tw.tablets[aliasStr]; ok { + newTablets[aliasStr] = val + continue + } + } + + wg.Add(1) + go func(alias *topodatapb.TabletAlias) { + defer wg.Done() + tw.sem <- 1 // Wait for active queue to drain. + tablet, err := tw.topoServer.GetTablet(tw.ctx, alias) + topologyWatcherOperations.Add(topologyWatcherOpGetTablet, 1) + <-tw.sem // Done; enable next request to run + if err != nil { + topologyWatcherErrors.Add(topologyWatcherOpGetTablet, 1) + select { + case <-tw.ctx.Done(): + return + default: + } + log.Errorf("cannot get tablet for alias %v: %v", alias, err) + return + } + tw.mu.Lock() + aliasStr := topoproto.TabletAliasString(alias) + newTablets[aliasStr] = &legacyTabletInfo{ + alias: aliasStr, + key: TabletToMapKey(tablet.Tablet), + tablet: tablet.Tablet, + } + tw.mu.Unlock() + }(tAlias) + } + + tw.mu.Unlock() + wg.Wait() + tw.mu.Lock() + + for alias, newVal := range newTablets { + if val, ok := tw.tablets[alias]; !ok { + // Check if there's a tablet with the same address key but a + // different alias. If so, replace it and keep track of the + // replaced alias to make sure it isn't removed later. + found := false + for _, otherVal := range tw.tablets { + if newVal.key == otherVal.key { + found = true + tw.tr.ReplaceTablet(otherVal.tablet, newVal.tablet, alias) + topologyWatcherOperations.Add(topologyWatcherOpReplaceTablet, 1) + replacedTablets[otherVal.alias] = newVal + } + } + if !found { + tw.tr.AddTablet(newVal.tablet, alias) + topologyWatcherOperations.Add(topologyWatcherOpAddTablet, 1) + } + + } else if val.key != newVal.key { + // Handle the case where the same tablet alias is now reporting + // a different address key. + replacedTablets[alias] = newVal + tw.tr.ReplaceTablet(val.tablet, newVal.tablet, alias) + topologyWatcherOperations.Add(topologyWatcherOpReplaceTablet, 1) + } + } + + for _, val := range tw.tablets { + if _, ok := newTablets[val.alias]; !ok { + if _, ok2 := replacedTablets[val.alias]; !ok2 { + tw.tr.RemoveTablet(val.tablet) + topologyWatcherOperations.Add(topologyWatcherOpRemoveTablet, 1) + } + } + } + tw.tablets = newTablets + if !tw.firstLoadDone { + tw.firstLoadDone = true + close(tw.firstLoadChan) + } + + // iterate through the tablets in a stable order and compute a + // checksum of the tablet map + sort.Strings(tabletAliasStrs) + var buf bytes.Buffer + for _, alias := range tabletAliasStrs { + tabletInfo, ok := tw.tablets[alias] + if ok { + buf.WriteString(alias) + buf.WriteString(tabletInfo.key) + } + } + tw.topoChecksum = crc32.ChecksumIEEE(buf.Bytes()) + tw.lastRefresh = time.Now() + + tw.mu.Unlock() +} + +// WaitForInitialTopology waits until the watcher reads all of the topology data +// for the first time and transfers the information to LegacyTabletRecorder via its +// AddTablet() method. +func (tw *LegacyTopologyWatcher) WaitForInitialTopology() error { + select { + case <-tw.ctx.Done(): + return tw.ctx.Err() + case <-tw.firstLoadChan: + return nil + } +} + +// Stop stops the watcher. It does not clean up the tablets added to LegacyTabletRecorder. +func (tw *LegacyTopologyWatcher) Stop() { + tw.cancelFunc() + // wait for watch goroutine to finish. + tw.wg.Wait() +} + +// RefreshLag returns the time since the last refresh +func (tw *LegacyTopologyWatcher) RefreshLag() time.Duration { + tw.mu.Lock() + defer tw.mu.Unlock() + + return time.Since(tw.lastRefresh) +} + +// TopoChecksum returns the checksum of the current state of the topo +func (tw *LegacyTopologyWatcher) TopoChecksum() uint32 { + tw.mu.Lock() + defer tw.mu.Unlock() + + return tw.topoChecksum +} + +// LegacyFilterByShard is a LegacyTabletRecorder filter that filters tablets by +// keyspace/shard. +type LegacyFilterByShard struct { + // tr is the underlying LegacyTabletRecorder to forward requests too + tr LegacyTabletRecorder + + // filters is a map of keyspace to filters for shards + filters map[string][]*filterShard +} + +// NewLegacyFilterByShard creates a new LegacyFilterByShard on top of an existing +// LegacyTabletRecorder. Each filter is a keyspace|shard entry, where shard +// can either be a shard name, or a keyrange. All tablets that match +// at least one keyspace|shard tuple will be forwarded to the +// underlying LegacyTabletRecorder. +func NewLegacyFilterByShard(tr LegacyTabletRecorder, filters []string) (*LegacyFilterByShard, error) { + m := make(map[string][]*filterShard) + for _, filter := range filters { + parts := strings.Split(filter, "|") + if len(parts) != 2 { + return nil, fmt.Errorf("invalid LegacyFilterByShard parameter: %v", filter) + } + + keyspace := parts[0] + shard := parts[1] + + // extract keyrange if it's a range + canonical, kr, err := topo.ValidateShardName(shard) + if err != nil { + return nil, fmt.Errorf("error parsing shard name %v: %v", shard, err) + } + + // check for duplicates + for _, c := range m[keyspace] { + if c.shard == canonical { + return nil, fmt.Errorf("duplicate %v/%v entry", keyspace, shard) + } + } + + m[keyspace] = append(m[keyspace], &filterShard{ + keyspace: keyspace, + shard: canonical, + keyRange: kr, + }) + } + + return &LegacyFilterByShard{ + tr: tr, + filters: m, + }, nil +} + +// AddTablet is part of the LegacyTabletRecorder interface. +func (fbs *LegacyFilterByShard) AddTablet(tablet *topodatapb.Tablet, name string) { + if fbs.isIncluded(tablet) { + fbs.tr.AddTablet(tablet, name) + } +} + +// RemoveTablet is part of the LegacyTabletRecorder interface. +func (fbs *LegacyFilterByShard) RemoveTablet(tablet *topodatapb.Tablet) { + if fbs.isIncluded(tablet) { + fbs.tr.RemoveTablet(tablet) + } +} + +// ReplaceTablet is part of the LegacyTabletRecorder interface. +func (fbs *LegacyFilterByShard) ReplaceTablet(old, new *topodatapb.Tablet, name string) { + if fbs.isIncluded(old) && fbs.isIncluded(new) { + fbs.tr.ReplaceTablet(old, new, name) + } +} + +// isIncluded returns true iff the tablet's keyspace and shard should be +// forwarded to the underlying LegacyTabletRecorder. +func (fbs *LegacyFilterByShard) isIncluded(tablet *topodatapb.Tablet) bool { + canonical, kr, err := topo.ValidateShardName(tablet.Shard) + if err != nil { + log.Errorf("Error parsing shard name %v, will ignore tablet: %v", tablet.Shard, err) + return false + } + + for _, c := range fbs.filters[tablet.Keyspace] { + if canonical == c.shard { + // Exact match (probably a non-sharded keyspace). + return true + } + if kr != nil && c.keyRange != nil && key.KeyRangeIncludes(c.keyRange, kr) { + // Our filter's KeyRange includes the provided KeyRange + return true + } + } + return false +} + +// LegacyFilterByKeyspace is a LegacyTabletRecorder filter that filters tablets by +// keyspace +type LegacyFilterByKeyspace struct { + tr LegacyTabletRecorder + + keyspaces map[string]bool +} + +// NewLegacyFilterByKeyspace creates a new LegacyFilterByKeyspace on top of an existing +// LegacyTabletRecorder. Each filter is a keyspace entry. All tablets that match +// a keyspace will be forwarded to the underlying LegacyTabletRecorder. +func NewLegacyFilterByKeyspace(tr LegacyTabletRecorder, selectedKeyspaces []string) *LegacyFilterByKeyspace { + m := make(map[string]bool) + for _, keyspace := range selectedKeyspaces { + m[keyspace] = true + } + + return &LegacyFilterByKeyspace{ + tr: tr, + keyspaces: m, + } +} + +// AddTablet is part of the LegacyTabletRecorder interface. +func (fbk *LegacyFilterByKeyspace) AddTablet(tablet *topodatapb.Tablet, name string) { + if fbk.isIncluded(tablet) { + fbk.tr.AddTablet(tablet, name) + } +} + +// RemoveTablet is part of the LegacyTabletRecorder interface. +func (fbk *LegacyFilterByKeyspace) RemoveTablet(tablet *topodatapb.Tablet) { + if fbk.isIncluded(tablet) { + fbk.tr.RemoveTablet(tablet) + } +} + +// ReplaceTablet is part of the LegacyTabletRecorder interface. +func (fbk *LegacyFilterByKeyspace) ReplaceTablet(old *topodatapb.Tablet, new *topodatapb.Tablet, name string) { + if old.Keyspace != new.Keyspace { + log.Errorf("Error replacing old tablet in %v with new tablet in %v", old.Keyspace, new.Keyspace) + return + } + + if fbk.isIncluded(new) { + fbk.tr.ReplaceTablet(old, new, name) + } +} + +// isIncluded returns true if the tablet's keyspace should be +// forwarded to the underlying LegacyTabletRecorder. +func (fbk *LegacyFilterByKeyspace) isIncluded(tablet *topodatapb.Tablet) bool { + _, exist := fbk.keyspaces[tablet.Keyspace] + return exist +} diff --git a/go/vt/discovery/legacy_topology_watcher_test.go b/go/vt/discovery/legacy_topology_watcher_test.go new file mode 100644 index 00000000000..ad20e104bed --- /dev/null +++ b/go/vt/discovery/legacy_topology_watcher_test.go @@ -0,0 +1,488 @@ +/* +Copyright 2019 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 discovery + +import ( + "math/rand" + "testing" + "time" + + "github.com/golang/protobuf/proto" + "golang.org/x/net/context" + "vitess.io/vitess/go/vt/logutil" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/memorytopo" +) + +func checkLegacyOpCounts(t *testing.T, tw *LegacyTopologyWatcher, prevCounts, deltas map[string]int64) map[string]int64 { + t.Helper() + newCounts := topologyWatcherOperations.Counts() + for key, prevVal := range prevCounts { + delta, ok := deltas[key] + if !ok { + delta = 0 + } + newVal, ok := newCounts[key] + if !ok { + newVal = 0 + } + + if newVal != prevVal+delta { + t.Errorf("expected %v to increase by %v, got %v -> %v", key, delta, prevVal, newVal) + } + } + return newCounts +} + +func checkLegacyChecksum(t *testing.T, tw *LegacyTopologyWatcher, want uint32) { + t.Helper() + got := tw.TopoChecksum() + if want != got { + t.Errorf("want checksum %v got %v", want, got) + } +} + +func TestLegacyCellTabletsWatcher(t *testing.T) { + checkLegacyWatcher(t, true, true) +} + +func TestLegacyCellTabletsWatcherNoRefreshKnown(t *testing.T) { + checkLegacyWatcher(t, true, false) +} + +func TestLegacyShardReplicationWatcher(t *testing.T) { + checkLegacyWatcher(t, false, true) +} + +func checkLegacyWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { + ts := memorytopo.NewServer("aa") + fhc := NewFakeLegacyHealthCheck() + logger := logutil.NewMemoryLogger() + topologyWatcherOperations.ZeroAll() + counts := topologyWatcherOperations.Counts() + var tw *LegacyTopologyWatcher + if cellTablets { + tw = NewLegacyCellTabletsWatcher(context.Background(), ts, fhc, "aa", 10*time.Minute, refreshKnownTablets, 5) + } else { + tw = NewLegacyShardReplicationWatcher(context.Background(), ts, fhc, "aa", "keyspace", "shard", 10*time.Minute, 5) + } + + // Wait for the initial topology load to finish. Otherwise we + // have a background loadTablets() that's running, and it can + // interact with our tests in weird ways. + if err := tw.WaitForInitialTopology(); err != nil { + t.Fatalf("initial WaitForInitialTopology failed") + } + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1}) + checkLegacyChecksum(t, tw, 0) + + // Add a tablet to the topology. + tablet := &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "aa", + Uid: 0, + }, + Hostname: "host1", + PortMap: map[string]int32{ + "vt": 123, + }, + Keyspace: "keyspace", + Shard: "shard", + } + if err := ts.CreateTablet(context.Background(), tablet); err != nil { + t.Fatalf("CreateTablet failed: %v", err) + } + tw.loadTablets() + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "AddTablet": 1}) + checkLegacyChecksum(t, tw, 1261153186) + + // Check the tablet is returned by GetAllTablets(). + allTablets := fhc.GetAllTablets() + key := TabletToMapKey(tablet) + if _, ok := allTablets[key]; !ok || len(allTablets) != 1 || !proto.Equal(allTablets[key], tablet) { + t.Errorf("fhc.GetAllTablets() = %+v; want %+v", allTablets, tablet) + } + + // Add a second tablet to the topology. + tablet2 := &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "aa", + Uid: 2, + }, + Hostname: "host2", + PortMap: map[string]int32{ + "vt": 789, + }, + Keyspace: "keyspace", + Shard: "shard", + } + if err := ts.CreateTablet(context.Background(), tablet2); err != nil { + t.Fatalf("CreateTablet failed: %v", err) + } + tw.loadTablets() + + // If RefreshKnownTablets is disabled, only the new tablet is read + // from the topo + if refreshKnownTablets { + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "AddTablet": 1}) + } else { + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "AddTablet": 1}) + } + checkLegacyChecksum(t, tw, 832404892) + + // Check the new tablet is returned by GetAllTablets(). + allTablets = fhc.GetAllTablets() + key = TabletToMapKey(tablet2) + if _, ok := allTablets[key]; !ok || len(allTablets) != 2 || !proto.Equal(allTablets[key], tablet2) { + t.Errorf("fhc.GetAllTablets() = %+v; want %+v", allTablets, tablet2) + } + + // Load the tablets again to show that when RefreshKnownTablets is disabled, + // only the list is read from the topo and the checksum doesn't change + tw.loadTablets() + if refreshKnownTablets { + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2}) + } else { + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1}) + } + checkLegacyChecksum(t, tw, 832404892) + + // same tablet, different port, should update (previous + // one should go away, new one be added) + // + // if RefreshKnownTablets is disabled, this case is *not* + // detected and the tablet remains in the topo using the + // old key + origTablet := proto.Clone(tablet).(*topodatapb.Tablet) + origKey := TabletToMapKey(tablet) + tablet.PortMap["vt"] = 456 + if _, err := ts.UpdateTabletFields(context.Background(), tablet.Alias, func(t *topodatapb.Tablet) error { + t.PortMap["vt"] = 456 + return nil + }); err != nil { + t.Fatalf("UpdateTabletFields failed: %v", err) + } + tw.loadTablets() + allTablets = fhc.GetAllTablets() + key = TabletToMapKey(tablet) + + if refreshKnownTablets { + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 1}) + + if _, ok := allTablets[key]; !ok || len(allTablets) != 2 || !proto.Equal(allTablets[key], tablet) { + t.Errorf("fhc.GetAllTablets() = %+v; want %+v", allTablets, tablet) + } + if _, ok := allTablets[origKey]; ok { + t.Errorf("fhc.GetAllTablets() = %+v; don't want %v", allTablets, origKey) + } + checkLegacyChecksum(t, tw, 698548794) + } else { + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1}) + + if _, ok := allTablets[origKey]; !ok || len(allTablets) != 2 || !proto.Equal(allTablets[origKey], origTablet) { + t.Errorf("fhc.GetAllTablets() = %+v; want %+v", allTablets, origTablet) + } + if _, ok := allTablets[key]; ok { + t.Errorf("fhc.GetAllTablets() = %+v; don't want %v", allTablets, key) + } + checkLegacyChecksum(t, tw, 832404892) + } + + // Remove the second tablet and re-add with a new uid. This should + // trigger a ReplaceTablet in loadTablets because the uid does not + // match. + // + // This case *is* detected even if RefreshKnownTablets is false + // because the delete tablet / create tablet sequence causes the + // list of tablets to change and therefore the change is detected. + if err := ts.DeleteTablet(context.Background(), tablet2.Alias); err != nil { + t.Fatalf("DeleteTablet failed: %v", err) + } + tablet2.Alias.Uid = 3 + if err := ts.CreateTablet(context.Background(), tablet2); err != nil { + t.Fatalf("CreateTablet failed: %v", err) + } + if err := topo.FixShardReplication(context.Background(), ts, logger, "aa", "keyspace", "shard"); err != nil { + t.Fatalf("FixShardReplication failed: %v", err) + } + tw.loadTablets() + allTablets = fhc.GetAllTablets() + + if refreshKnownTablets { + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 1}) + checkLegacyChecksum(t, tw, 4097170367) + } else { + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "ReplaceTablet": 1}) + checkLegacyChecksum(t, tw, 3960185881) + } + key = TabletToMapKey(tablet2) + if _, ok := allTablets[key]; !ok || len(allTablets) != 2 || !proto.Equal(allTablets[key], tablet2) { + t.Errorf("fhc.GetAllTablets() = %+v; want %v => %+v", allTablets, key, tablet2) + } + + // Both tablets restart on different hosts. + // tablet2 happens to land on the host:port that tablet 1 used to be on. + // This can only be tested when we refresh known tablets. + if refreshKnownTablets { + origTablet := *tablet + origTablet2 := *tablet2 + + if _, err := ts.UpdateTabletFields(context.Background(), tablet2.Alias, func(t *topodatapb.Tablet) error { + t.Hostname = tablet.Hostname + t.PortMap = tablet.PortMap + tablet2 = t + return nil + }); err != nil { + t.Fatalf("UpdateTabletFields failed: %v", err) + } + if _, err := ts.UpdateTabletFields(context.Background(), tablet.Alias, func(t *topodatapb.Tablet) error { + t.Hostname = "host3" + tablet = t + return nil + }); err != nil { + t.Fatalf("UpdateTabletFields failed: %v", err) + } + tw.loadTablets() + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 2}) + allTablets = fhc.GetAllTablets() + key2 := TabletToMapKey(tablet2) + if _, ok := allTablets[key2]; !ok { + t.Fatalf("tablet was lost because it's reusing an address recently used by another tablet: %v", key2) + } + + // Change tablets back to avoid altering later tests. + if _, err := ts.UpdateTabletFields(context.Background(), tablet2.Alias, func(t *topodatapb.Tablet) error { + t.Hostname = origTablet2.Hostname + t.PortMap = origTablet2.PortMap + tablet2 = t + return nil + }); err != nil { + t.Fatalf("UpdateTabletFields failed: %v", err) + } + if _, err := ts.UpdateTabletFields(context.Background(), tablet.Alias, func(t *topodatapb.Tablet) error { + t.Hostname = origTablet.Hostname + tablet = t + return nil + }); err != nil { + t.Fatalf("UpdateTabletFields failed: %v", err) + } + tw.loadTablets() + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 2}) + } + + // Remove the tablet and check that it is detected as being gone. + if err := ts.DeleteTablet(context.Background(), tablet.Alias); err != nil { + t.Fatalf("DeleteTablet failed: %v", err) + } + if err := topo.FixShardReplication(context.Background(), ts, logger, "aa", "keyspace", "shard"); err != nil { + t.Fatalf("FixShardReplication failed: %v", err) + } + tw.loadTablets() + if refreshKnownTablets { + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "RemoveTablet": 1}) + } else { + counts = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "RemoveTablet": 1}) + } + checkLegacyChecksum(t, tw, 1725545897) + + allTablets = fhc.GetAllTablets() + key = TabletToMapKey(tablet) + if _, ok := allTablets[key]; ok || len(allTablets) != 1 { + t.Errorf("fhc.GetAllTablets() = %+v; don't want %v", allTablets, key) + } + key = TabletToMapKey(tablet2) + if _, ok := allTablets[key]; !ok || len(allTablets) != 1 || !proto.Equal(allTablets[key], tablet2) { + t.Errorf("fhc.GetAllTablets() = %+v; want %+v", allTablets, tablet2) + } + + // Remove the other and check that it is detected as being gone. + if err := ts.DeleteTablet(context.Background(), tablet2.Alias); err != nil { + t.Fatalf("DeleteTablet failed: %v", err) + } + if err := topo.FixShardReplication(context.Background(), ts, logger, "aa", "keyspace", "shard"); err != nil { + t.Fatalf("FixShardReplication failed: %v", err) + } + tw.loadTablets() + checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 0, "RemoveTablet": 1}) + checkLegacyChecksum(t, tw, 0) + + allTablets = fhc.GetAllTablets() + key = TabletToMapKey(tablet) + if _, ok := allTablets[key]; ok || len(allTablets) != 0 { + t.Errorf("fhc.GetAllTablets() = %+v; don't want %v", allTablets, key) + } + key = TabletToMapKey(tablet2) + if _, ok := allTablets[key]; ok || len(allTablets) != 0 { + t.Errorf("fhc.GetAllTablets() = %+v; don't want %v", allTablets, key) + } + + tw.Stop() +} + +func TestLegacyFilterByShard(t *testing.T) { + testcases := []struct { + filters []string + keyspace string + shard string + included bool + }{ + // un-sharded keyspaces + { + filters: []string{"ks1|0"}, + keyspace: "ks1", + shard: "0", + included: true, + }, + { + filters: []string{"ks1|0"}, + keyspace: "ks2", + shard: "0", + included: false, + }, + // custom sharding, different shard + { + filters: []string{"ks1|0"}, + keyspace: "ks1", + shard: "1", + included: false, + }, + // keyrange based sharding + { + filters: []string{"ks1|-80"}, + keyspace: "ks1", + shard: "0", + included: false, + }, + { + filters: []string{"ks1|-80"}, + keyspace: "ks1", + shard: "-40", + included: true, + }, + { + filters: []string{"ks1|-80"}, + keyspace: "ks1", + shard: "-80", + included: true, + }, + { + filters: []string{"ks1|-80"}, + keyspace: "ks1", + shard: "80-", + included: false, + }, + { + filters: []string{"ks1|-80"}, + keyspace: "ks1", + shard: "c0-", + included: false, + }, + } + + for _, tc := range testcases { + fbs, err := NewLegacyFilterByShard(nil, tc.filters) + if err != nil { + t.Errorf("cannot create LegacyFilterByShard for filters %v: %v", tc.filters, err) + } + + tablet := &topodatapb.Tablet{ + Keyspace: tc.keyspace, + Shard: tc.shard, + } + + got := fbs.isIncluded(tablet) + if got != tc.included { + t.Errorf("isIncluded(%v,%v) for filters %v returned %v but expected %v", tc.keyspace, tc.shard, tc.filters, got, tc.included) + } + } +} + +func TestLegacyFilterByKeyspace(t *testing.T) { + hc := NewFakeLegacyHealthCheck() + tr := NewLegacyFilterByKeyspace(hc, testKeyspacesToWatch) + ts := memorytopo.NewServer(testCell) + tw := NewLegacyCellTabletsWatcher(context.Background(), ts, tr, testCell, 10*time.Minute, true, 5) + + for _, test := range testFilterByKeyspace { + // Add a new tablet to the topology. + port := rand.Int31n(1000) + tablet := &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: testCell, + Uid: rand.Uint32(), + }, + Hostname: testHostName, + PortMap: map[string]int32{ + "vt": port, + }, + Keyspace: test.keyspace, + Shard: testShard, + } + + got := tr.isIncluded(tablet) + if got != test.expected { + t.Errorf("isIncluded(%v) for keyspace %v returned %v but expected %v", test.keyspace, test.keyspace, got, test.expected) + } + + if err := ts.CreateTablet(context.Background(), tablet); err != nil { + t.Errorf("CreateTablet failed: %v", err) + } + + tw.loadTablets() + key := TabletToMapKey(tablet) + allTablets := hc.GetAllTablets() + + if _, ok := allTablets[key]; ok != test.expected && proto.Equal(allTablets[key], tablet) != test.expected { + t.Errorf("Error adding tablet - got %v; want %v", ok, test.expected) + } + + // Replace the tablet we added above + tabletReplacement := &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: testCell, + Uid: rand.Uint32(), + }, + Hostname: testHostName, + PortMap: map[string]int32{ + "vt": port, + }, + Keyspace: test.keyspace, + Shard: testShard, + } + got = tr.isIncluded(tabletReplacement) + if got != test.expected { + t.Errorf("isIncluded(%v) for keyspace %v returned %v but expected %v", test.keyspace, test.keyspace, got, test.expected) + } + if err := ts.CreateTablet(context.Background(), tabletReplacement); err != nil { + t.Errorf("CreateTablet failed: %v", err) + } + + tw.loadTablets() + key = TabletToMapKey(tabletReplacement) + allTablets = hc.GetAllTablets() + + if _, ok := allTablets[key]; ok != test.expected && proto.Equal(allTablets[key], tabletReplacement) != test.expected { + t.Errorf("Error replacing tablet - got %v; want %v", ok, test.expected) + } + + // Delete the tablet + if err := ts.DeleteTablet(context.Background(), tabletReplacement.Alias); err != nil { + t.Fatalf("DeleteTablet failed: %v", err) + } + } +} diff --git a/go/vt/discovery/replicationlag.go b/go/vt/discovery/replicationlag.go index dea0f5f5152..b527bfb8abf 100644 --- a/go/vt/discovery/replicationlag.go +++ b/go/vt/discovery/replicationlag.go @@ -31,20 +31,20 @@ var ( legacyReplicationLagAlgorithm = flag.Bool("legacy_replication_lag_algorithm", true, "use the legacy algorithm when selecting the vttablets for serving") ) -// IsReplicationLagHigh verifies that the given TabletStats refers to a tablet with high +// IsReplicationLagHigh verifies that the given LegacytabletHealth refers to a tablet with high // replication lag, i.e. higher than the configured discovery_low_replication_lag flag. -func IsReplicationLagHigh(tabletStats *TabletStats) bool { - return float64(tabletStats.Stats.SecondsBehindMaster) > lowReplicationLag.Seconds() +func IsReplicationLagHigh(tabletHealth *TabletHealth) bool { + return float64(tabletHealth.Stats.SecondsBehindMaster) > lowReplicationLag.Seconds() } -// IsReplicationLagVeryHigh verifies that the given TabletStats refers to a tablet with very high +// IsReplicationLagVeryHigh verifies that the given LegacytabletHealth refers to a tablet with very high // replication lag, i.e. higher than the configured discovery_high_replication_lag_minimum_serving flag. -func IsReplicationLagVeryHigh(tabletStats *TabletStats) bool { - return float64(tabletStats.Stats.SecondsBehindMaster) > highReplicationLagMinServing.Seconds() +func IsReplicationLagVeryHigh(tabletHealth *TabletHealth) bool { + return float64(tabletHealth.Stats.SecondsBehindMaster) > highReplicationLagMinServing.Seconds() } -// FilterByReplicationLag filters the list of TabletStats by TabletStats.Stats.SecondsBehindMaster. -// Note that TabletStats that is non-serving or has error is ignored. +// FilterStatsByReplicationLag filters the list of TabletHealth by TabletHealth.Stats.SecondsBehindMaster. +// Note that TabletHealth that is non-serving or has error is ignored. // // The simplified logic: // - Return tablets that have lag <= lowReplicationLag. @@ -68,24 +68,24 @@ func IsReplicationLagVeryHigh(tabletStats *TabletStats) bool { // The default for this is 2h, same as the discovery_high_replication_lag_minimum_serving here. // * degraded_threshold: this is only used by vttablet for display. It should match // discovery_low_replication_lag here, so the vttablet status display matches what vtgate will do of it. -func FilterByReplicationLag(tabletStatsList []*TabletStats) []*TabletStats { +func FilterStatsByReplicationLag(tabletHealthList []*TabletHealth) []*TabletHealth { if !*legacyReplicationLagAlgorithm { - return filterByLag(tabletStatsList) + return filterStatsByLag(tabletHealthList) } - - res := filterByLagWithLegacyAlgorithm(tabletStatsList) + res := filterStatsByLagWithLegacyAlgorithm(tabletHealthList) // run the filter again if exactly one tablet is removed, // and we have spare tablets. - if len(res) > *minNumTablets && len(res) == len(tabletStatsList)-1 { - res = filterByLagWithLegacyAlgorithm(res) + if len(res) > *minNumTablets && len(res) == len(tabletHealthList)-1 { + res = filterStatsByLagWithLegacyAlgorithm(res) } return res + } -func filterByLag(tabletStatsList []*TabletStats) []*TabletStats { - list := make([]tabletLagSnapshot, 0, len(tabletStatsList)) +func filterStatsByLag(tabletHealthList []*TabletHealth) []*TabletHealth { + list := make([]tabletLagSnapshot, 0, len(tabletHealthList)) // filter non-serving tablets and those with very high replication lag - for _, ts := range tabletStatsList { + for _, ts := range tabletHealthList { if !ts.Serving || ts.LastError != nil || ts.Stats == nil || IsReplicationLagVeryHigh(ts) { continue } @@ -96,10 +96,10 @@ func filterByLag(tabletStatsList []*TabletStats) []*TabletStats { } // Sort by replication lag. - sort.Sort(byReplag(list)) + sort.Sort(tabletLagSnapshotList(list)) // Pick those with low replication lag, but at least minNumTablets tablets regardless. - res := make([]*TabletStats, 0, len(list)) + res := make([]*TabletHealth, 0, len(list)) for i := 0; i < len(list); i++ { if !IsReplicationLagHigh(list[i].ts) || i < *minNumTablets { res = append(res, list[i].ts) @@ -108,10 +108,10 @@ func filterByLag(tabletStatsList []*TabletStats) []*TabletStats { return res } -func filterByLagWithLegacyAlgorithm(tabletStatsList []*TabletStats) []*TabletStats { - list := make([]*TabletStats, 0, len(tabletStatsList)) +func filterStatsByLagWithLegacyAlgorithm(tabletHealthList []*TabletHealth) []*TabletHealth { + list := make([]*TabletHealth, 0, len(tabletHealthList)) // filter non-serving tablets - for _, ts := range tabletStatsList { + for _, ts := range tabletHealthList { if !ts.Serving || ts.LastError != nil || ts.Stats == nil { continue } @@ -133,7 +133,7 @@ func filterByLagWithLegacyAlgorithm(tabletStatsList []*TabletStats) []*TabletSta } // filter those affecting "mean" lag significantly // calculate mean for all tablets - res := make([]*TabletStats, 0, len(list)) + res := make([]*TabletHealth, 0, len(list)) m, _ := mean(list, -1) for i, ts := range list { // calculate mean by excluding ith tablet @@ -175,13 +175,29 @@ func filterByLagWithLegacyAlgorithm(tabletStatsList []*TabletStats) []*TabletSta sort.Sort(byReplag(snapshots)) // Pick the first minNumTablets tablets. - res = make([]*TabletStats, 0, *minNumTablets) + res = make([]*TabletHealth, 0, *minNumTablets) for i := 0; i < min(*minNumTablets, len(snapshots)); i++ { res = append(res, snapshots[i].ts) } return res } +type byReplag []tabletLagSnapshot + +func (a byReplag) Len() int { return len(a) } +func (a byReplag) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a byReplag) Less(i, j int) bool { return a[i].replag < a[j].replag } + +type tabletLagSnapshot struct { + ts *TabletHealth + replag uint32 +} +type tabletLagSnapshotList []tabletLagSnapshot + +func (a tabletLagSnapshotList) Len() int { return len(a) } +func (a tabletLagSnapshotList) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a tabletLagSnapshotList) Less(i, j int) bool { return a[i].replag < a[j].replag } + func min(a, b int) int { if a > b { return b @@ -189,22 +205,12 @@ func min(a, b int) int { return a } -type tabletLagSnapshot struct { - ts *TabletStats - replag uint32 -} -type byReplag []tabletLagSnapshot - -func (a byReplag) Len() int { return len(a) } -func (a byReplag) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -func (a byReplag) Less(i, j int) bool { return a[i].replag < a[j].replag } - // mean calculates the mean value over the given list, // while excluding the item with the specified index. -func mean(tabletStatsList []*TabletStats, idxExclude int) (uint64, error) { +func mean(tabletHealthList []*TabletHealth, idxExclude int) (uint64, error) { var sum uint64 var count uint64 - for i, ts := range tabletStatsList { + for i, ts := range tabletHealthList { if i == idxExclude { continue } @@ -216,28 +222,3 @@ func mean(tabletStatsList []*TabletStats, idxExclude int) (uint64, error) { } return sum / count, nil } - -// TrivialStatsUpdate returns true iff the old and new TabletStats -// haven't changed enough to warrant re-calling FilterByReplicationLag. -func TrivialStatsUpdate(o, n *TabletStats) bool { - // Skip replag filter when replag remains in the low rep lag range, - // which should be the case majority of the time. - lowRepLag := lowReplicationLag.Seconds() - oldRepLag := float64(o.Stats.SecondsBehindMaster) - newRepLag := float64(n.Stats.SecondsBehindMaster) - if oldRepLag <= lowRepLag && newRepLag <= lowRepLag { - return true - } - - // Skip replag filter when replag remains in the high rep lag range, - // and did not change beyond +/- 10%. - // when there is a high rep lag, it takes a long time for it to reduce, - // so it is not necessary to re-calculate every time. - // In that case, we won't save the new record, so we still - // remember the original replication lag. - if oldRepLag > lowRepLag && newRepLag > lowRepLag && newRepLag < oldRepLag*1.1 && newRepLag > oldRepLag*0.9 { - return true - } - - return false -} diff --git a/go/vt/discovery/replicationlag_test.go b/go/vt/discovery/replicationlag_test.go index 40e184fc8b6..2c6bc45cc0f 100644 --- a/go/vt/discovery/replicationlag_test.go +++ b/go/vt/discovery/replicationlag_test.go @@ -20,6 +20,8 @@ import ( "fmt" "testing" + "vitess.io/vitess/go/test/utils" + querypb "vitess.io/vitess/go/vt/proto/query" "vitess.io/vitess/go/vt/topo" ) @@ -29,30 +31,21 @@ func testSetMinNumTablets(newMin int) { *minNumTablets = newMin } -// testSetLegacyReplicationLagAlgorithm is a test helper function, if this is used by a production code path, something is wrong. -func testSetLegacyReplicationLagAlgorithm(newLegacy bool) { - *legacyReplicationLagAlgorithm = newLegacy -} - func TestFilterByReplicationLagUnhealthy(t *testing.T) { // 1 healthy serving tablet, 1 not healhty - ts1 := &TabletStats{ + ts1 := &TabletHealth{ Tablet: topo.NewTablet(1, "cell", "host1"), Serving: true, Stats: &querypb.RealtimeStats{}, } - ts2 := &TabletStats{ + ts2 := &TabletHealth{ Tablet: topo.NewTablet(2, "cell", "host2"), Serving: false, Stats: &querypb.RealtimeStats{}, } - got := FilterByReplicationLag([]*TabletStats{ts1, ts2}) - if len(got) != 1 { - t.Errorf("len(FilterByReplicationLag([{Tablet: {Uid: 1}, Serving: true}, {Tablet: {Uid: 2}, Serving: false}])) = %v, want 1", len(got)) - } - if len(got) > 0 && !got[0].DeepEqual(ts1) { - t.Errorf("FilterByReplicationLag([{Tablet: {Uid: 1}, Serving: true}, {Tablet: {Uid: 2}, Serving: false}]) = %+v, want %+v", got[0], ts1) - } + got := FilterStatsByReplicationLag([]*TabletHealth{ts1, ts2}) + want := []*TabletHealth{ts1} + mustMatch(t, want, got, "FilterStatsByReplicationLag") } func TestFilterByReplicationLag(t *testing.T) { @@ -107,22 +100,22 @@ func TestFilterByReplicationLag(t *testing.T) { } for _, tc := range cases { - lts := make([]*TabletStats, len(tc.input)) + lts := make([]*TabletHealth, len(tc.input)) for i, lag := range tc.input { - lts[i] = &TabletStats{ + lts[i] = &TabletHealth{ Tablet: topo.NewTablet(uint32(i+1), "cell", fmt.Sprintf("host-%vs-behind", lag)), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: lag}, } } - got := FilterByReplicationLag(lts) + got := FilterStatsByReplicationLag(lts) if len(got) != len(tc.output) { - t.Errorf("FilterByReplicationLag(%v) failed: got output:\n%v\nExpected: %v", tc.description, got, tc.output) + t.Errorf("FilterStatsByReplicationLag(%v) failed: got output:\n%v\nExpected: %v", tc.description, got, tc.output) continue } for i, elag := range tc.output { if got[i].Stats.SecondsBehindMaster != elag { - t.Errorf("FilterByReplicationLag(%v) failed: got output:\n%v\nExpected value index %v to be %v", tc.description, got, i, elag) + t.Errorf("FilterStatsByReplicationLag(%v) failed: got output:\n%v\nExpected value index %v to be %v", tc.description, got, i, elag) } } } @@ -131,245 +124,94 @@ func TestFilterByReplicationLag(t *testing.T) { testSetLegacyReplicationLagAlgorithm(true) } -func TestFilterByReplicationLagWithLegacyAlgorithm(t *testing.T) { - // Use legacy algorithm by default for now - - cases := []struct { - description string - input []uint32 - output []uint32 - }{ - { - "0 tablet", - []uint32{}, - []uint32{}, - }, - { - "1 serving tablet", - []uint32{1}, - []uint32{1}, - }, - { - "lags of (1s, 1s, 1s, 30s)", - []uint32{1, 1, 1, 30}, - []uint32{1, 1, 1, 30}, - }, - { - "lags of (30m, 35m, 40m, 45m)", - []uint32{30 * 60, 35 * 60, 40 * 60, 45 * 60}, - []uint32{30 * 60, 35 * 60, 40 * 60, 45 * 60}, - }, - { - "lags of (1s, 1s, 1m, 40m, 40m) - not run filter the second time as first run removed two items.", - []uint32{1, 1, 60, 40 * 60, 40 * 60}, - []uint32{1, 1, 60}, - }, - { - "lags of (1s, 1s, 10m, 40m) - run filter twice to remove two items", - []uint32{1, 1, 10 * 60, 40 * 60}, - []uint32{1, 1}, - }, - { - "lags of (1m, 100m) - return at least 2 items to avoid overloading if the 2nd one is not delayed too much.", - []uint32{1 * 60, 100 * 60}, - []uint32{1 * 60, 100 * 60}, - }, - { - "lags of (1m, 3h) - return 1 if the 2nd one is delayed too much.", - []uint32{1 * 60, 3 * 60 * 60}, - []uint32{1 * 60}, - }, - { - "lags of (3h) - return 1 as they're all delayed too much.", - []uint32{3 * 60 * 60}, - []uint32{3 * 60 * 60}, - }, - { - "lags of (3h, 4h) - return 2 as they're all delayed too much, but still in a good group.", - []uint32{3 * 60 * 60, 4 * 60 * 60}, - []uint32{3 * 60 * 60, 4 * 60 * 60}, - }, - { - "lags of (3h, 3h, 4h) - return 3 as they're all delayed too much, but still in a good group.", - []uint32{3 * 60 * 60, 3 * 60 * 60, 4 * 60 * 60}, - []uint32{3 * 60 * 60, 3 * 60 * 60, 4 * 60 * 60}, - }, - { - "lags of (3h, 15h, 18h) - return 3 as they're all delayed too much, but still in a good group." + - "(different test case than above to show how absurb the good group logic is)", - []uint32{3 * 60 * 60, 15 * 60 * 60, 18 * 60 * 60}, - []uint32{3 * 60 * 60, 15 * 60 * 60, 18 * 60 * 60}, - }, - { - "lags of (3h, 12h, 18h) - return 2 as they're all delayed too much, but 18h is now considered an outlier." + - "(different test case than above to show how absurb the good group logic is)", - []uint32{3 * 60 * 60, 12 * 60 * 60, 18 * 60 * 60}, - []uint32{3 * 60 * 60, 12 * 60 * 60}, - }, - { - "lags of (3h, 30h) - return 2 as they're all delayed too much." + - "(different test case that before, as both tablet stats are" + - "widely different, not within 70% of eachother)", - []uint32{3 * 60 * 60, 30 * 60 * 60}, - []uint32{3 * 60 * 60, 30 * 60 * 60}, - }, - } - - for _, tc := range cases { - lts := make([]*TabletStats, len(tc.input)) - for i, lag := range tc.input { - lts[i] = &TabletStats{ - Tablet: topo.NewTablet(uint32(i+1), "cell", fmt.Sprintf("host-%vs-behind", lag)), - Serving: true, - Stats: &querypb.RealtimeStats{SecondsBehindMaster: lag}, - } - } - got := FilterByReplicationLag(lts) - if len(got) != len(tc.output) { - t.Errorf("FilterByReplicationLag(%v) failed: got output:\n%v\nExpected: %v", tc.description, got, tc.output) - continue - } - for i, elag := range tc.output { - if got[i].Stats.SecondsBehindMaster != elag { - t.Errorf("FilterByReplicationLag(%v) failed: got output:\n%v\nExpected value index %v to be %v", tc.description, got, i, elag) - } - } - } -} - func TestFilterByReplicationLagThreeTabletMin(t *testing.T) { // Use at least 3 tablets if possible testSetMinNumTablets(3) // lags of (1s, 1s, 10m, 11m) - returns at least32 items where the slightly delayed ones that are returned are the 10m and 11m ones. - ts1 := &TabletStats{ + ts1 := &TabletHealth{ Tablet: topo.NewTablet(1, "cell", "host1"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1}, } - ts2 := &TabletStats{ + ts2 := &TabletHealth{ Tablet: topo.NewTablet(2, "cell", "host2"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1}, } - ts3 := &TabletStats{ + ts3 := &TabletHealth{ Tablet: topo.NewTablet(3, "cell", "host3"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 10 * 60}, } - ts4 := &TabletStats{ + ts4 := &TabletHealth{ Tablet: topo.NewTablet(4, "cell", "host4"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 11 * 60}, } - got := FilterByReplicationLag([]*TabletStats{ts1, ts2, ts3, ts4}) - if len(got) != 3 || !got[0].DeepEqual(ts1) || !got[1].DeepEqual(ts2) || !got[2].DeepEqual(ts3) { - t.Errorf("FilterByReplicationLag([1s, 1s, 10m, 11m]) = %+v, want [1s, 1s, 10m]", got) - } + got := FilterStatsByReplicationLag([]*TabletHealth{ts1, ts2, ts3, ts4}) + want := []*TabletHealth{ts1, ts2, ts3} + mustMatch(t, want, got, "FilterStatsByReplicationLag") + // lags of (11m, 10m, 1s, 1s) - reordered tablets returns the same 3 items where the slightly delayed one that is returned is the 10m and 11m ones. - ts1 = &TabletStats{ + ts1 = &TabletHealth{ Tablet: topo.NewTablet(1, "cell", "host1"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 11 * 60}, } - ts2 = &TabletStats{ + ts2 = &TabletHealth{ Tablet: topo.NewTablet(2, "cell", "host2"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 10 * 60}, } - ts3 = &TabletStats{ + ts3 = &TabletHealth{ Tablet: topo.NewTablet(3, "cell", "host3"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1}, } - ts4 = &TabletStats{ + ts4 = &TabletHealth{ Tablet: topo.NewTablet(4, "cell", "host4"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1}, } - got = FilterByReplicationLag([]*TabletStats{ts1, ts2, ts3, ts4}) - if len(got) != 3 || !got[0].DeepEqual(ts3) || !got[1].DeepEqual(ts4) || !got[2].DeepEqual(ts2) { - t.Errorf("FilterByReplicationLag([1s, 1s, 10m, 11m]) = %+v, want [1s, 1s, 10m]", got) - } + got = FilterStatsByReplicationLag([]*TabletHealth{ts1, ts2, ts3, ts4}) + want = []*TabletHealth{ts3, ts4, ts2} + mustMatch(t, want, got, "FilterStatsByReplicationLag") // Reset to the default testSetMinNumTablets(2) } -func TestFilterByReplicationLagOneTabletMin(t *testing.T) { +func TestFilterStatsByReplicationLagOneTabletMin(t *testing.T) { // Use at least 1 tablets if possible testSetMinNumTablets(1) // lags of (1s, 100m) - return only healthy tablet if that is all that is available. - ts1 := &TabletStats{ + ts1 := &TabletHealth{ Tablet: topo.NewTablet(1, "cell", "host1"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1}, } - ts2 := &TabletStats{ + ts2 := &TabletHealth{ Tablet: topo.NewTablet(2, "cell", "host2"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 100 * 60}, } - got := FilterByReplicationLag([]*TabletStats{ts1, ts2}) - if len(got) != 1 || !got[0].DeepEqual(ts1) { - t.Errorf("FilterByReplicationLag([1s, 100m]) = %+v, want [1s]", got) - } + got := FilterStatsByReplicationLag([]*TabletHealth{ts1, ts2}) + want := []*TabletHealth{ts1} + mustMatch(t, want, got, "FilterStatsByReplicationLag") + // lags of (1m, 100m) - return only healthy tablet if that is all that is healthy enough. - ts1 = &TabletStats{ + ts1 = &TabletHealth{ Tablet: topo.NewTablet(1, "cell", "host1"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1 * 60}, } - ts2 = &TabletStats{ + ts2 = &TabletHealth{ Tablet: topo.NewTablet(2, "cell", "host2"), Serving: true, Stats: &querypb.RealtimeStats{SecondsBehindMaster: 100 * 60}, } - got = FilterByReplicationLag([]*TabletStats{ts1, ts2}) - if len(got) != 1 || !got[0].DeepEqual(ts1) { - t.Errorf("FilterByReplicationLag([1m, 100m]) = %+v, want [1m]", got) - } + got = FilterStatsByReplicationLag([]*TabletHealth{ts1, ts2}) + want = []*TabletHealth{ts1} + utils.MustMatch(t, want, got, "FilterStatsByReplicationLag") // Reset to the default testSetMinNumTablets(2) } - -func TestTrivialStatsUpdate(t *testing.T) { - // Note the healthy threshold is set to 30s. - cases := []struct { - o uint32 - n uint32 - expected bool - }{ - // both are under 30s - {o: 0, n: 1, expected: true}, - {o: 15, n: 20, expected: true}, - - // one is under 30s, the other isn't - {o: 2, n: 40, expected: false}, - {o: 40, n: 10, expected: false}, - - // both are over 30s, but close enough - {o: 100, n: 100, expected: true}, - {o: 100, n: 105, expected: true}, - {o: 105, n: 100, expected: true}, - - // both are over 30s, but too far - {o: 100, n: 120, expected: false}, - {o: 120, n: 100, expected: false}, - } - - for _, c := range cases { - o := &TabletStats{ - Stats: &querypb.RealtimeStats{ - SecondsBehindMaster: c.o, - }, - } - n := &TabletStats{ - Stats: &querypb.RealtimeStats{ - SecondsBehindMaster: c.n, - }, - } - got := TrivialStatsUpdate(o, n) - if got != c.expected { - t.Errorf("TrivialStatsUpdate(%v, %v) = %v, expected %v", c.o, c.n, got, c.expected) - } - } -} diff --git a/go/vt/discovery/tablet_health.go b/go/vt/discovery/tablet_health.go new file mode 100644 index 00000000000..e3b37e759ff --- /dev/null +++ b/go/vt/discovery/tablet_health.go @@ -0,0 +1,93 @@ +/* +Copyright 2020 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 discovery + +import ( + "bytes" + "strings" + + "vitess.io/vitess/go/vt/vttablet/queryservice" + + "github.com/golang/protobuf/proto" + "vitess.io/vitess/go/netutil" + "vitess.io/vitess/go/vt/proto/query" + "vitess.io/vitess/go/vt/proto/topodata" +) + +// TabletHealth represents simple tablet health data that is returned to users of healthcheck. +// No synchronization is required because we always return a copy. +type TabletHealth struct { + Conn queryservice.QueryService + Tablet *topodata.Tablet + Target *query.Target + Stats *query.RealtimeStats + MasterTermStartTime int64 + LastError error + Serving bool +} + +// DeepEqual compares two TabletHealth. Since we include protos, we +// need to use proto.Equal on these. +func (th *TabletHealth) DeepEqual(other *TabletHealth) bool { + return proto.Equal(th.Tablet, other.Tablet) && + proto.Equal(th.Target, other.Target) && + th.Serving == other.Serving && + th.MasterTermStartTime == other.MasterTermStartTime && + proto.Equal(th.Stats, other.Stats) && + ((th.LastError == nil && other.LastError == nil) || + (th.LastError != nil && other.LastError != nil && th.LastError.Error() == other.LastError.Error())) +} + +// GetTabletHostPort formats a tablet host port address. +func (th *TabletHealth) GetTabletHostPort() string { + hostname := th.Tablet.Hostname + vtPort := th.Tablet.PortMap["vt"] + return netutil.JoinHostPort(hostname, vtPort) +} + +// GetHostNameLevel returns the specified hostname level. If the level does not exist it will pick the closest level. +// This seems unused but can be utilized by certain url formatting templates. See getTabletDebugURL for more details. +func (th *TabletHealth) GetHostNameLevel(level int) string { + hostname := th.Tablet.Hostname + chunkedHostname := strings.Split(hostname, ".") + + if level < 0 { + return chunkedHostname[0] + } else if level >= len(chunkedHostname) { + return chunkedHostname[len(chunkedHostname)-1] + } else { + return chunkedHostname[level] + } +} + +// getTabletDebugURL formats a debug url to the tablet. +// It uses a format string that can be passed into the app to format +// the debug URL to accommodate different network setups. It applies +// the html/template string defined to a tabletHealthCheck object. The +// format string can refer to members and functions of tabletHealthCheck +// like a regular html/template string. +// +// For instance given a tablet with hostname:port of host.dc.domain:22 +// could be configured as follows: +// http://{{.GetTabletHostPort}} -> http://host.dc.domain:22 +// https://{{.Tablet.Hostname}} -> https://host.dc.domain +// https://{{.GetHostNameLevel 0}}.bastion.corp -> https://host.bastion.corp +func (th *TabletHealth) getTabletDebugURL() string { + var buffer bytes.Buffer + tabletURLTemplate.Execute(&buffer, th) + return buffer.String() +} diff --git a/go/vt/discovery/tablet_health_check.go b/go/vt/discovery/tablet_health_check.go new file mode 100644 index 00000000000..39aa9895c0c --- /dev/null +++ b/go/vt/discovery/tablet_health_check.go @@ -0,0 +1,333 @@ +/* +Copyright 2020 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 discovery + +import ( + "context" + "fmt" + "strings" + "time" + + "vitess.io/vitess/go/sync2" + + "vitess.io/vitess/go/vt/grpcclient" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/proto/vtrpc" + "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/topotools" + "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vttablet/queryservice" + "vitess.io/vitess/go/vt/vttablet/tabletconn" + + "github.com/golang/protobuf/proto" + "vitess.io/vitess/go/vt/proto/query" + "vitess.io/vitess/go/vt/proto/topodata" +) + +// tabletHealthCheck maintains the health status of a tablet. A map of this +// structure is maintained in HealthCheck. +type tabletHealthCheck struct { + ctx context.Context + // cancelFunc must be called before discarding tabletHealthCheck. + // This will ensure that the associated checkConn goroutine will terminate. + cancelFunc context.CancelFunc + // Tablet is the tablet object that was sent to HealthCheck.AddTablet. + Tablet *topodata.Tablet + // Conn is the connection associated with the tablet. + Conn queryservice.QueryService + // Target is the current target as returned by the streaming + // StreamHealth RPC. + Target *query.Target + // Serving describes if the tablet can be serving traffic. + Serving bool + // MasterTermStartTime is the last time at which + // this tablet was either elected the master, or received + // a TabletExternallyReparented event. It is set to 0 if the + // tablet doesn't think it's a master. + MasterTermStartTime int64 + // Stats is the current health status, as received by the + // StreamHealth RPC (replication lag, ...). + Stats *query.RealtimeStats + // LastError is the error we last saw when trying to get the + // tablet's healthcheck. + LastError error + // possibly delete both these + loggedServingState bool + lastResponseTimestamp time.Time // timestamp of the last healthcheck response +} + +// String is defined because we want to print a []*tabletHealthCheck array nicely. +func (thc *tabletHealthCheck) String() string { + return fmt.Sprintf("tabletHealthCheck{Tablet: %v,Target: %v,Serving: %v, MasterTermStartTime: %v, Stats: %v, LastError: %v", + thc.Tablet, thc.Target, thc.Serving, thc.MasterTermStartTime, *thc.Stats, thc.LastError) +} + +// SimpleCopy returns a TabletHealth with all the necessary fields copied from tabletHealthCheck. +// Note that this is not a deep copy because we point to the same underlying RealtimeStats. +// That is fine because the RealtimeStats object is never changed after creation. +func (thc *tabletHealthCheck) SimpleCopy() *TabletHealth { + return &TabletHealth{ + Conn: thc.Conn, + Tablet: thc.Tablet, + Target: thc.Target, + Stats: thc.Stats, + LastError: thc.LastError, + MasterTermStartTime: thc.MasterTermStartTime, + Serving: thc.Serving, + } +} + +// setServingState sets the tablet state to the given value. +// +// If the state changes, it logs the change so that failures +// from the health check connection are logged the first time, +// but don't continue to log if the connection stays down. +// +// thc.mu must be locked before calling this function +func (thc *tabletHealthCheck) setServingState(serving bool, reason string) { + if !thc.loggedServingState || (serving != thc.Serving) { + // Emit the log from a separate goroutine to avoid holding + // the th lock while logging is happening + go log.Infof("HealthCheckUpdate(Serving State): tablet: %v serving => %v for %v/%v (%v) reason: %s", + topotools.TabletIdent(thc.Tablet), + serving, + thc.Tablet.GetKeyspace(), + thc.Tablet.GetShard(), + thc.Target.GetTabletType(), + reason, + ) + thc.loggedServingState = true + } + thc.Serving = serving +} + +// stream streams healthcheck responses to callback. +func (thc *tabletHealthCheck) stream(ctx context.Context, callback func(*query.StreamHealthResponse) error) error { + conn := thc.getConnection() + if conn == nil { + // This signals the caller to retry + return nil + } + err := conn.StreamHealth(ctx, callback) + if err != nil { + // Depending on the specific error the caller can take action + thc.closeConnection(ctx, err) + } + return err +} + +func (thc *tabletHealthCheck) getConnection() queryservice.QueryService { + if thc.Conn == nil { + conn, err := tabletconn.GetDialer()(thc.Tablet, grpcclient.FailFast(true)) + if err != nil { + thc.LastError = err + return nil + } + thc.Conn = conn + thc.LastError = nil + } + return thc.Conn +} + +// processResponse reads one health check response, and updates health +func (thc *tabletHealthCheck) processResponse(hc *HealthCheck, shr *query.StreamHealthResponse) error { + select { + case <-thc.ctx.Done(): + return thc.ctx.Err() + default: + } + + // Check for invalid data, better than panicking. + if shr.Target == nil || shr.RealtimeStats == nil { + return fmt.Errorf("health stats is not valid: %v", shr) + } + + // an app-level error from tablet, force serving state. + var healthErr error + serving := shr.Serving + if shr.RealtimeStats.HealthError != "" { + healthErr = fmt.Errorf("vttablet error: %v", shr.RealtimeStats.HealthError) + serving = false + } + + if shr.TabletAlias != nil && !proto.Equal(shr.TabletAlias, thc.Tablet.Alias) { + // TabletAlias change means that the host:port has been taken over by another tablet + // We cancel / exit the healthcheck for this tablet right away + // With the next topo refresh we will get a new tablet with the new host/port + return vterrors.New(vtrpc.Code_FAILED_PRECONDITION, fmt.Sprintf("health stats mismatch, tablet %+v alias does not match response alias %v", thc.Tablet, shr.TabletAlias)) + } + + currentTarget := thc.Target + // check whether this is a trivial update so as to update healthy map + trivialNonMasterUpdate := thc.LastError == nil && thc.Serving && shr.RealtimeStats.HealthError == "" && shr.Serving && + currentTarget.TabletType != topodata.TabletType_MASTER && currentTarget.TabletType == shr.Target.TabletType && thc.isTrivialReplagChange(shr.RealtimeStats) + isMasterUpdate := shr.Target.TabletType == topodata.TabletType_MASTER + isMasterChange := thc.Target.TabletType != topodata.TabletType_MASTER && shr.Target.TabletType == topodata.TabletType_MASTER + thc.lastResponseTimestamp = time.Now() + thc.Target = shr.Target + thc.MasterTermStartTime = shr.TabletExternallyReparentedTimestamp + thc.Stats = shr.RealtimeStats + thc.LastError = healthErr + reason := "healthCheck update" + if healthErr != nil { + reason = "healthCheck update error: " + healthErr.Error() + } + thc.setServingState(serving, reason) + + // notify downstream for master change + hc.updateHealth(thc.SimpleCopy(), shr, currentTarget, trivialNonMasterUpdate, isMasterUpdate, isMasterChange) + return nil +} + +// isTrivialReplagChange returns true iff the old and new RealtimeStats +// haven't changed enough to warrant re-calling FilterLegacyStatsByReplicationLag. +func (thc *tabletHealthCheck) isTrivialReplagChange(newStats *query.RealtimeStats) bool { + // first time always return false + if thc.Stats == nil { + return false + } + // Skip replag filter when replag remains in the low rep lag range, + // which should be the case majority of the time. + lowRepLag := lowReplicationLag.Seconds() + oldRepLag := float64(thc.Stats.SecondsBehindMaster) + newRepLag := float64(newStats.SecondsBehindMaster) + if oldRepLag <= lowRepLag && newRepLag <= lowRepLag { + return true + } + // Skip replag filter when replag remains in the high rep lag range, + // and did not change beyond +/- 10%. + // when there is a high rep lag, it takes a long time for it to reduce, + // so it is not necessary to re-calculate every time. + // In that case, we won't save the new record, so we still + // remember the original replication lag. + if oldRepLag > lowRepLag && newRepLag > lowRepLag && newRepLag < oldRepLag*1.1 && newRepLag > oldRepLag*0.9 { + return true + } + return false +} + +// checkConn performs health checking on the given tablet. +func (thc *tabletHealthCheck) checkConn(hc *HealthCheck) { + defer func() { + // TODO(deepthi): We should ensure any return from this func calls the equivalent of hc.deleteTablet + thc.finalizeConn() + hc.connsWG.Done() + }() + + retryDelay := hc.retryDelay + for { + streamCtx, streamCancel := context.WithCancel(thc.ctx) + + // Setup a watcher that restarts the timer every time an update is received. + // If a timeout occurs for a serving tablet, we make it non-serving and send + // a status update. The stream is also terminated so it can be retried. + // servingStatus feeds into the serving var, which keeps track of the serving + // status transmitted by the tablet. + servingStatus := make(chan bool, 1) + // timedout is accessed atomically because there could be a race + // between the goroutine that sets it and the check for its value + // later. + timedout := sync2.NewAtomicBool(false) + go func() { + for { + select { + case <-servingStatus: + continue + case <-time.After(hc.healthCheckTimeout): + timedout.Set(true) + streamCancel() + return + case <-streamCtx.Done(): + // If the stream is done, stop watching. + return + } + } + }() + + // Read stream health responses. + err := thc.stream(streamCtx, func(shr *query.StreamHealthResponse) error { + // We received a message. Reset the back-off. + retryDelay = hc.retryDelay + // Don't block on send to avoid deadlocks. + select { + case servingStatus <- shr.Serving: + default: + } + return thc.processResponse(hc, shr) + }) + + // streamCancel to make sure the watcher goroutine terminates. + streamCancel() + + if err != nil { + if strings.Contains(err.Error(), "health stats mismatch") { + hc.deleteTablet(thc.Tablet) + return + } + res := thc.SimpleCopy() + hc.broadcast(res) + } + // If there was a timeout send an error. We do this after stream has returned. + // This will ensure that this update prevails over any previous message that + // stream could have sent. + if timedout.Get() { + thc.LastError = fmt.Errorf("healthcheck timed out (latest %v)", thc.lastResponseTimestamp) + thc.setServingState(false, thc.LastError.Error()) + hcErrorCounters.Add([]string{thc.Target.Keyspace, thc.Target.Shard, topoproto.TabletTypeLString(thc.Target.TabletType)}, 1) + hc.broadcast(thc.SimpleCopy()) + } + + // Streaming RPC failed e.g. because vttablet was restarted or took too long. + // Sleep until the next retry is up or the context is done/canceled. + select { + case <-thc.ctx.Done(): + return + case <-time.After(retryDelay): + // Exponentially back-off to prevent tight-loop. + retryDelay *= 2 + // Limit the retry delay backoff to the health check timeout + if retryDelay > hc.healthCheckTimeout { + retryDelay = hc.healthCheckTimeout + } + } + } +} + +func (thc *tabletHealthCheck) closeConnection(ctx context.Context, err error) { + log.Warningf("tablet %v healthcheck stream error: %v", thc.Tablet.Alias, err) + thc.setServingState(false, err.Error()) + thc.LastError = err + _ = thc.Conn.Close(ctx) + thc.Conn = nil +} + +// finalizeConn closes the health checking connection. +// To be called only on exit from checkConn(). +func (thc *tabletHealthCheck) finalizeConn() { + thc.setServingState(false, "finalizeConn closing connection") + // Note: checkConn() exits only when thc.ctx.Done() is closed. Thus it's + // safe to simply get Err() value here and assign to LastError. + thc.LastError = thc.ctx.Err() + if thc.Conn != nil { + // Don't use thc.ctx because it's already closed. + // Use a separate context, and add a timeout to prevent unbounded waits. + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + _ = thc.Conn.Close(ctx) + thc.Conn = nil + } +} diff --git a/go/vt/discovery/tablet_picker.go b/go/vt/discovery/tablet_picker.go index 1cd75ab60c2..2bda68fec42 100644 --- a/go/vt/discovery/tablet_picker.go +++ b/go/vt/discovery/tablet_picker.go @@ -36,9 +36,9 @@ type TabletPicker struct { shard string tabletTypes []topodatapb.TabletType - healthCheck HealthCheck - watcher *TopologyWatcher - statsCache *TabletStatsCache + healthCheck LegacyHealthCheck + watcher *LegacyTopologyWatcher + statsCache *LegacyTabletStatsCache } // NewTabletPicker returns a TabletPicker. @@ -49,9 +49,9 @@ func NewTabletPicker(ctx context.Context, ts *topo.Server, cell, keyspace, shard } // These have to be initialized in the following sequence (watcher must be last). - healthCheck := NewHealthCheck(healthcheckRetryDelay, healthcheckTimeout) - statsCache := NewTabletStatsCache(healthCheck, ts, cell) - watcher := NewShardReplicationWatcher(ctx, ts, healthCheck, cell, keyspace, shard, healthcheckTopologyRefresh, DefaultTopoReadConcurrency) + healthCheck := NewLegacyHealthCheck(healthcheckRetryDelay, healthcheckTimeout) + statsCache := NewLegacyTabletStatsCache(healthCheck, ts, cell) + watcher := NewLegacyShardReplicationWatcher(ctx, ts, healthCheck, cell, keyspace, shard, healthcheckTopologyRefresh, DefaultTopoReadConcurrency) return &TabletPicker{ ts: ts, @@ -73,7 +73,7 @@ func (tp *TabletPicker) PickForStreaming(ctx context.Context) (*topodatapb.Table } // Refilter the tablets list based on the same criteria. - var addrs []TabletStats + var addrs []LegacyTabletStats for _, tabletType := range tp.tabletTypes { list := RemoveUnhealthyTablets(tp.statsCache.GetTabletStats(tp.keyspace, tp.shard, tabletType)) addrs = append(addrs, list...) diff --git a/go/vt/discovery/tablets_cache_status.go b/go/vt/discovery/tablets_cache_status.go new file mode 100644 index 00000000000..46d2fd4ba4d --- /dev/null +++ b/go/vt/discovery/tablets_cache_status.go @@ -0,0 +1,118 @@ +package discovery + +import ( + "fmt" + "html/template" + "sort" + "strings" + + "github.com/gogo/protobuf/proto" + + querypb "vitess.io/vitess/go/vt/proto/query" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + "vitess.io/vitess/go/vt/topo/topoproto" +) + +// TabletsCacheStatus is the current tablets for a cell/target. +type TabletsCacheStatus struct { + Cell string + Target *querypb.Target + TabletsStats TabletStatsList +} + +// TabletStatsList is used for sorting. +type TabletStatsList []*TabletHealth + +// Len is part of sort.Interface. +func (tsl TabletStatsList) Len() int { + return len(tsl) +} + +// Less is part of sort.Interface +func (tsl TabletStatsList) Less(i, j int) bool { + name1 := topoproto.TabletAliasString(tsl[i].Tablet.Alias) + name2 := topoproto.TabletAliasString(tsl[j].Tablet.Alias) + return name1 < name2 +} + +// Swap is part of sort.Interface +func (tsl TabletStatsList) Swap(i, j int) { + tsl[i], tsl[j] = tsl[j], tsl[i] +} + +func (tsl TabletStatsList) deepEqual(other TabletStatsList) bool { + if len(tsl) != len(other) { + return false + } + for i, th := range tsl { + o := other[i] + if !th.DeepEqual(o) { + return false + } + } + return true +} + +// StatusAsHTML returns an HTML version of the status. +func (tcs *TabletsCacheStatus) StatusAsHTML() template.HTML { + tLinks := make([]string, 0, 1) + if tcs.TabletsStats != nil { + sort.Sort(tcs.TabletsStats) + } + for _, ts := range tcs.TabletsStats { + color := "green" + extra := "" + if ts.LastError != nil { + color = "red" + extra = fmt.Sprintf(" (%v)", ts.LastError) + } else if !ts.Serving { + color = "red" + extra = " (Not Serving)" + } else if ts.Target.TabletType == topodatapb.TabletType_MASTER { + extra = fmt.Sprintf(" (MasterTS: %v)", ts.MasterTermStartTime) + } else { + extra = fmt.Sprintf(" (RepLag: %v)", ts.Stats.SecondsBehindMaster) + } + name := topoproto.TabletAliasString(ts.Tablet.Alias) + tLinks = append(tLinks, fmt.Sprintf(`%v%v`, ts.getTabletDebugURL(), color, name, extra)) + } + return template.HTML(strings.Join(tLinks, "
")) +} + +func (tcs *TabletsCacheStatus) deepEqual(otcs *TabletsCacheStatus) bool { + return tcs.Cell == otcs.Cell && + proto.Equal(tcs.Target, otcs.Target) && + tcs.TabletsStats.deepEqual(otcs.TabletsStats) +} + +// TabletsCacheStatusList is used for sorting. +type TabletsCacheStatusList []*TabletsCacheStatus + +// Len is part of sort.Interface. +func (tcsl TabletsCacheStatusList) Len() int { + return len(tcsl) +} + +// Less is part of sort.Interface +func (tcsl TabletsCacheStatusList) Less(i, j int) bool { + return tcsl[i].Cell+"."+tcsl[i].Target.Keyspace+"."+tcsl[i].Target.Shard+"."+string(tcsl[i].Target.TabletType) < + tcsl[j].Cell+"."+tcsl[j].Target.Keyspace+"."+tcsl[j].Target.Shard+"."+string(tcsl[j].Target.TabletType) +} + +// Swap is part of sort.Interface +func (tcsl TabletsCacheStatusList) Swap(i, j int) { + tcsl[i], tcsl[j] = tcsl[j], tcsl[i] +} + +func (tcsl TabletsCacheStatusList) deepEqual(other TabletsCacheStatusList) bool { + if len(tcsl) != len(other) { + return false + } + for i, tcs := range tcsl { + otcs := other[i] + if !tcs.deepEqual(otcs) { + return false + } + } + return true +} diff --git a/go/vt/discovery/topology_watcher.go b/go/vt/discovery/topology_watcher.go index b623457663c..802ec4d4a83 100644 --- a/go/vt/discovery/topology_watcher.go +++ b/go/vt/discovery/topology_watcher.go @@ -25,16 +25,17 @@ import ( "sync" "time" + "vitess.io/vitess/go/vt/topo/topoproto" + + "vitess.io/vitess/go/vt/key" + "golang.org/x/net/context" "vitess.io/vitess/go/stats" "vitess.io/vitess/go/trace" - "vitess.io/vitess/go/vt/key" "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/topo" - "vitess.io/vitess/go/vt/topo/topoproto" - - topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) const ( @@ -52,70 +53,24 @@ var ( "Operation", topologyWatcherOpListTablets, topologyWatcherOpGetTablet) ) -// TabletRecorder is the part of the HealthCheck interface that can -// add or remove tablets. We define it as a sub-interface here so we -// can add filters on tablets if needed. -type TabletRecorder interface { - // AddTablet adds the tablet. - // Name is an alternate name, like an address. - AddTablet(tablet *topodatapb.Tablet, name string) - - // RemoveTablet removes the tablet. - RemoveTablet(tablet *topodatapb.Tablet) - - // ReplaceTablet does an AddTablet and RemoveTablet in one call, effectively replacing the old tablet with the new. - ReplaceTablet(old, new *topodatapb.Tablet, name string) -} - -// NewCellTabletsWatcher returns a TopologyWatcher that monitors all -// the tablets in a cell, and starts refreshing. -func NewCellTabletsWatcher(ctx context.Context, topoServer *topo.Server, tr TabletRecorder, cell string, refreshInterval time.Duration, refreshKnownTablets bool, topoReadConcurrency int) *TopologyWatcher { - return NewTopologyWatcher(ctx, topoServer, tr, cell, refreshInterval, refreshKnownTablets, topoReadConcurrency, func(tw *TopologyWatcher) ([]*topodatapb.TabletAlias, error) { - return tw.topoServer.GetTabletsByCell(ctx, tw.cell) - }) -} - -// NewShardReplicationWatcher returns a TopologyWatcher that -// monitors the tablets in a cell/keyspace/shard, and starts refreshing. -func NewShardReplicationWatcher(ctx context.Context, topoServer *topo.Server, tr TabletRecorder, cell, keyspace, shard string, refreshInterval time.Duration, topoReadConcurrency int) *TopologyWatcher { - return NewTopologyWatcher(ctx, topoServer, tr, cell, refreshInterval, true /* refreshKnownTablets */, topoReadConcurrency, func(tw *TopologyWatcher) ([]*topodatapb.TabletAlias, error) { - sri, err := tw.topoServer.GetShardReplication(ctx, tw.cell, keyspace, shard) - switch { - case err == nil: - // we handle this case after this switch block - case topo.IsErrType(err, topo.NoNode): - // this is not an error - return nil, nil - default: - return nil, err - } - - result := make([]*topodatapb.TabletAlias, len(sri.Nodes)) - for i, node := range sri.Nodes { - result[i] = node.TabletAlias - } - return result, nil - }) -} - // tabletInfo is used internally by the TopologyWatcher class type tabletInfo struct { alias string - key string - tablet *topodatapb.Tablet + tablet *topodata.Tablet } // TopologyWatcher polls tablet from a configurable set of tablets // periodically. When tablets are added / removed, it calls -// the TabletRecorder AddTablet / RemoveTablet interface appropriately. +// the LegacyTabletRecorder AddTablet / RemoveTablet interface appropriately. type TopologyWatcher struct { // set at construction time topoServer *topo.Server - tr TabletRecorder + tabletRecorder TabletRecorder + tabletFilter TabletFilter cell string refreshInterval time.Duration refreshKnownTablets bool - getTablets func(tw *TopologyWatcher) ([]*topodatapb.TabletAlias, error) + getTablets func(tw *TopologyWatcher) ([]*topodata.TabletAlias, error) sem chan int ctx context.Context cancelFunc context.CancelFunc @@ -138,10 +93,11 @@ type TopologyWatcher struct { // NewTopologyWatcher returns a TopologyWatcher that monitors all // the tablets in a cell, and starts refreshing. -func NewTopologyWatcher(ctx context.Context, topoServer *topo.Server, tr TabletRecorder, cell string, refreshInterval time.Duration, refreshKnownTablets bool, topoReadConcurrency int, getTablets func(tw *TopologyWatcher) ([]*topodatapb.TabletAlias, error)) *TopologyWatcher { +func NewTopologyWatcher(ctx context.Context, topoServer *topo.Server, tr TabletRecorder, filter TabletFilter, cell string, refreshInterval time.Duration, refreshKnownTablets bool, topoReadConcurrency int, getTablets func(tw *TopologyWatcher) ([]*topodata.TabletAlias, error)) *TopologyWatcher { tw := &TopologyWatcher{ topoServer: topoServer, - tr: tr, + tabletRecorder: tr, + tabletFilter: filter, cell: cell, refreshInterval: refreshInterval, refreshKnownTablets: refreshKnownTablets, @@ -154,13 +110,20 @@ func NewTopologyWatcher(ctx context.Context, topoServer *topo.Server, tr TabletR // We want the span from the context, but not the cancelation that comes with it spanContext := trace.CopySpan(context.Background(), ctx) tw.ctx, tw.cancelFunc = context.WithCancel(spanContext) - tw.wg.Add(1) - go tw.watch() return tw } -// watch polls all tablets and notifies TabletRecorder by adding/removing tablets. -func (tw *TopologyWatcher) watch() { +// NewCellTabletsWatcher returns a TopologyWatcher that monitors all +// the tablets in a cell, and starts refreshing. +func NewCellTabletsWatcher(ctx context.Context, topoServer *topo.Server, tr TabletRecorder, f TabletFilter, cell string, refreshInterval time.Duration, refreshKnownTablets bool, topoReadConcurrency int) *TopologyWatcher { + return NewTopologyWatcher(ctx, topoServer, tr, f, cell, refreshInterval, refreshKnownTablets, topoReadConcurrency, func(tw *TopologyWatcher) ([]*topodata.TabletAlias, error) { + return tw.topoServer.GetTabletsByCell(ctx, tw.cell) + }) +} + +// Start starts the topology watcher +func (tw *TopologyWatcher) Start() { + tw.wg.Add(1) defer tw.wg.Done() ticker := time.NewTicker(tw.refreshInterval) defer ticker.Stop() @@ -174,12 +137,18 @@ func (tw *TopologyWatcher) watch() { } } -// loadTablets reads all tablets from topology, and updates TabletRecorder. +// Stop stops the watcher. It does not clean up the tablets added to LegacyTabletRecorder. +func (tw *TopologyWatcher) Stop() { + tw.cancelFunc() + // wait for watch goroutine to finish. + tw.wg.Wait() +} + func (tw *TopologyWatcher) loadTablets() { var wg sync.WaitGroup newTablets := make(map[string]*tabletInfo) - replacedTablets := make(map[string]*tabletInfo) + // first get the list of relevant tabletAliases tabletAliases, err := tw.getTablets(tw) topologyWatcherOperations.Add(topologyWatcherOpListTablets, 1) if err != nil { @@ -203,6 +172,7 @@ func (tw *TopologyWatcher) loadTablets() { tabletAliasStrs = append(tabletAliasStrs, aliasStr) if !tw.refreshKnownTablets { + // we already have a tabletInfo for this and the flag tells us to not refresh if val, ok := tw.tablets[aliasStr]; ok { newTablets[aliasStr] = val continue @@ -210,7 +180,7 @@ func (tw *TopologyWatcher) loadTablets() { } wg.Add(1) - go func(alias *topodatapb.TabletAlias) { + go func(alias *topodata.TabletAlias) { defer wg.Done() tw.sem <- 1 // Wait for active queue to drain. tablet, err := tw.topoServer.GetTablet(tw.ctx, alias) @@ -226,11 +196,13 @@ func (tw *TopologyWatcher) loadTablets() { log.Errorf("cannot get tablet for alias %v: %v", alias, err) return } + if !(tw.tabletFilter == nil || tw.tabletFilter.IsIncluded(tablet.Tablet)) { + return + } tw.mu.Lock() aliasStr := topoproto.TabletAliasString(alias) newTablets[aliasStr] = &tabletInfo{ alias: aliasStr, - key: TabletToMapKey(tablet.Tablet), tablet: tablet.Tablet, } tw.mu.Unlock() @@ -242,39 +214,27 @@ func (tw *TopologyWatcher) loadTablets() { tw.mu.Lock() for alias, newVal := range newTablets { + // trust the alias from topo and add it if it doesn't exist if val, ok := tw.tablets[alias]; !ok { - // Check if there's a tablet with the same address key but a - // different alias. If so, replace it and keep track of the - // replaced alias to make sure it isn't removed later. - found := false - for _, otherVal := range tw.tablets { - if newVal.key == otherVal.key { - found = true - tw.tr.ReplaceTablet(otherVal.tablet, newVal.tablet, alias) - topologyWatcherOperations.Add(topologyWatcherOpReplaceTablet, 1) - replacedTablets[otherVal.alias] = newVal - } - } - if !found { - tw.tr.AddTablet(newVal.tablet, alias) - topologyWatcherOperations.Add(topologyWatcherOpAddTablet, 1) + tw.tabletRecorder.AddTablet(newVal.tablet) + topologyWatcherOperations.Add(topologyWatcherOpAddTablet, 1) + } else { + // check if the host and port have changed. If yes, replace tablet + oldKey := TabletToMapKey(val.tablet) + newKey := TabletToMapKey(newVal.tablet) + if oldKey != newKey { + // This is the case where the same tablet alias is now reporting + // a different address key. + tw.tabletRecorder.ReplaceTablet(val.tablet, newVal.tablet) + topologyWatcherOperations.Add(topologyWatcherOpReplaceTablet, 1) } - - } else if val.key != newVal.key { - // Handle the case where the same tablet alias is now reporting - // a different address key. - replacedTablets[alias] = newVal - tw.tr.ReplaceTablet(val.tablet, newVal.tablet, alias) - topologyWatcherOperations.Add(topologyWatcherOpReplaceTablet, 1) } } for _, val := range tw.tablets { if _, ok := newTablets[val.alias]; !ok { - if _, ok2 := replacedTablets[val.alias]; !ok2 { - tw.tr.RemoveTablet(val.tablet) - topologyWatcherOperations.Add(topologyWatcherOpRemoveTablet, 1) - } + tw.tabletRecorder.RemoveTablet(val.tablet) + topologyWatcherOperations.Add(topologyWatcherOpRemoveTablet, 1) } } tw.tablets = newTablets @@ -288,35 +248,16 @@ func (tw *TopologyWatcher) loadTablets() { sort.Strings(tabletAliasStrs) var buf bytes.Buffer for _, alias := range tabletAliasStrs { - tabletInfo, ok := tw.tablets[alias] + _, ok := tw.tablets[alias] if ok { buf.WriteString(alias) - buf.WriteString(tabletInfo.key) } } tw.topoChecksum = crc32.ChecksumIEEE(buf.Bytes()) tw.lastRefresh = time.Now() tw.mu.Unlock() -} -// WaitForInitialTopology waits until the watcher reads all of the topology data -// for the first time and transfers the information to TabletRecorder via its -// AddTablet() method. -func (tw *TopologyWatcher) WaitForInitialTopology() error { - select { - case <-tw.ctx.Done(): - return tw.ctx.Err() - case <-tw.firstLoadChan: - return nil - } -} - -// Stop stops the watcher. It does not clean up the tablets added to TabletRecorder. -func (tw *TopologyWatcher) Stop() { - tw.cancelFunc() - // wait for watch goroutine to finish. - tw.wg.Wait() } // RefreshLag returns the time since the last refresh @@ -335,12 +276,16 @@ func (tw *TopologyWatcher) TopoChecksum() uint32 { return tw.topoChecksum } -// FilterByShard is a TabletRecorder filter that filters tablets by +// TabletFilter is an interface that can be given to a TopologyWatcher +// to be applied as an additional filter on the list of tablets returned by its getTablets function +type TabletFilter interface { + // IsIncluded returns whether tablet is included in this filter + IsIncluded(tablet *topodata.Tablet) bool +} + +// FilterByShard is a filter that filters tablets by // keyspace/shard. type FilterByShard struct { - // tr is the underlying TabletRecorder to forward requests too - tr TabletRecorder - // filters is a map of keyspace to filters for shards filters map[string][]*filterShard } @@ -350,15 +295,15 @@ type FilterByShard struct { type filterShard struct { keyspace string shard string - keyRange *topodatapb.KeyRange // only set if shard is also a KeyRange + keyRange *topodata.KeyRange // only set if shard is also a KeyRange } // NewFilterByShard creates a new FilterByShard on top of an existing -// TabletRecorder. Each filter is a keyspace|shard entry, where shard +// LegacyTabletRecorder. Each filter is a keyspace|shard entry, where shard // can either be a shard name, or a keyrange. All tablets that match // at least one keyspace|shard tuple will be forwarded to the -// underlying TabletRecorder. -func NewFilterByShard(tr TabletRecorder, filters []string) (*FilterByShard, error) { +// underlying LegacyTabletRecorder. +func NewFilterByShard(filters []string) (*FilterByShard, error) { m := make(map[string][]*filterShard) for _, filter := range filters { parts := strings.Split(filter, "|") @@ -390,35 +335,13 @@ func NewFilterByShard(tr TabletRecorder, filters []string) (*FilterByShard, erro } return &FilterByShard{ - tr: tr, filters: m, }, nil } -// AddTablet is part of the TabletRecorder interface. -func (fbs *FilterByShard) AddTablet(tablet *topodatapb.Tablet, name string) { - if fbs.isIncluded(tablet) { - fbs.tr.AddTablet(tablet, name) - } -} - -// RemoveTablet is part of the TabletRecorder interface. -func (fbs *FilterByShard) RemoveTablet(tablet *topodatapb.Tablet) { - if fbs.isIncluded(tablet) { - fbs.tr.RemoveTablet(tablet) - } -} - -// ReplaceTablet is part of the TabletRecorder interface. -func (fbs *FilterByShard) ReplaceTablet(old, new *topodatapb.Tablet, name string) { - if fbs.isIncluded(old) && fbs.isIncluded(new) { - fbs.tr.ReplaceTablet(old, new, name) - } -} - -// isIncluded returns true iff the tablet's keyspace and shard should be -// forwarded to the underlying TabletRecorder. -func (fbs *FilterByShard) isIncluded(tablet *topodatapb.Tablet) bool { +// IsIncluded returns true iff the tablet's keyspace and shard should be +// forwarded to the underlying LegacyTabletRecorder. +func (fbs *FilterByShard) IsIncluded(tablet *topodata.Tablet) bool { canonical, kr, err := topo.ValidateShardName(tablet.Shard) if err != nil { log.Errorf("Error parsing shard name %v, will ignore tablet: %v", tablet.Shard, err) @@ -438,58 +361,29 @@ func (fbs *FilterByShard) isIncluded(tablet *topodatapb.Tablet) bool { return false } -// FilterByKeyspace is a TabletRecorder filter that filters tablets by +// FilterByKeyspace is a filter that filters tablets by // keyspace type FilterByKeyspace struct { - tr TabletRecorder - keyspaces map[string]bool } -// NewFilterByKeyspace creates a new FilterByKeyspace on top of an existing -// TabletRecorder. Each filter is a keyspace entry. All tablets that match -// a keyspace will be forwarded to the underlying TabletRecorder. -func NewFilterByKeyspace(tr TabletRecorder, selectedKeyspaces []string) *FilterByKeyspace { +// NewFilterByKeyspace creates a new FilterByKeyspace. +// Each filter is a keyspace entry. All tablets that match +// a keyspace will be forwarded to the underlying LegacyTabletRecorder. +func NewFilterByKeyspace(selectedKeyspaces []string) *FilterByKeyspace { m := make(map[string]bool) for _, keyspace := range selectedKeyspaces { m[keyspace] = true } return &FilterByKeyspace{ - tr: tr, keyspaces: m, } } -// AddTablet is part of the TabletRecorder interface. -func (fbk *FilterByKeyspace) AddTablet(tablet *topodatapb.Tablet, name string) { - if fbk.isIncluded(tablet) { - fbk.tr.AddTablet(tablet, name) - } -} - -// RemoveTablet is part of the TabletRecorder interface. -func (fbk *FilterByKeyspace) RemoveTablet(tablet *topodatapb.Tablet) { - if fbk.isIncluded(tablet) { - fbk.tr.RemoveTablet(tablet) - } -} - -// ReplaceTablet is part of the TabletRecorder interface. -func (fbk *FilterByKeyspace) ReplaceTablet(old *topodatapb.Tablet, new *topodatapb.Tablet, name string) { - if old.Keyspace != new.Keyspace { - log.Errorf("Error replacing old tablet in %v with new tablet in %v", old.Keyspace, new.Keyspace) - return - } - - if fbk.isIncluded(new) { - fbk.tr.ReplaceTablet(old, new, name) - } -} - -// isIncluded returns true if the tablet's keyspace should be -// forwarded to the underlying TabletRecorder. -func (fbk *FilterByKeyspace) isIncluded(tablet *topodatapb.Tablet) bool { +// IsIncluded returns true if the tablet's keyspace should be +// forwarded to the underlying LegacyTabletRecorder. +func (fbk *FilterByKeyspace) IsIncluded(tablet *topodata.Tablet) bool { _, exist := fbk.keyspaces[tablet.Keyspace] return exist } diff --git a/go/vt/discovery/topology_watcher_test.go b/go/vt/discovery/topology_watcher_test.go index 02c664fa15b..86dd652c7cf 100644 --- a/go/vt/discovery/topology_watcher_test.go +++ b/go/vt/discovery/topology_watcher_test.go @@ -29,7 +29,7 @@ import ( "vitess.io/vitess/go/vt/topo/memorytopo" ) -func checkOpCounts(t *testing.T, tw *TopologyWatcher, prevCounts, deltas map[string]int64) map[string]int64 { +func checkOpCounts(t *testing.T, prevCounts, deltas map[string]int64) map[string]int64 { t.Helper() newCounts := topologyWatcherOperations.Counts() for key, prevVal := range prevCounts { @@ -58,37 +58,22 @@ func checkChecksum(t *testing.T, tw *TopologyWatcher, want uint32) { } func TestCellTabletsWatcher(t *testing.T) { - checkWatcher(t, true, true) + checkWatcher(t, true) } func TestCellTabletsWatcherNoRefreshKnown(t *testing.T) { - checkWatcher(t, true, false) + checkWatcher(t, false) } -func TestShardReplicationWatcher(t *testing.T) { - checkWatcher(t, false, true) -} - -func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { +func checkWatcher(t *testing.T, refreshKnownTablets bool) { ts := memorytopo.NewServer("aa") fhc := NewFakeHealthCheck() logger := logutil.NewMemoryLogger() topologyWatcherOperations.ZeroAll() counts := topologyWatcherOperations.Counts() - var tw *TopologyWatcher - if cellTablets { - tw = NewCellTabletsWatcher(context.Background(), ts, fhc, "aa", 10*time.Minute, refreshKnownTablets, 5) - } else { - tw = NewShardReplicationWatcher(context.Background(), ts, fhc, "aa", "keyspace", "shard", 10*time.Minute, 5) - } + tw := NewCellTabletsWatcher(context.Background(), ts, fhc, nil, "aa", 10*time.Minute, refreshKnownTablets, 5) - // Wait for the initial topology load to finish. Otherwise we - // have a background loadTablets() that's running, and it can - // interact with our tests in weird ways. - if err := tw.WaitForInitialTopology(); err != nil { - t.Fatalf("initial WaitForInitialTopology failed") - } - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1}) + counts = checkOpCounts(t, counts, map[string]int64{}) checkChecksum(t, tw, 0) // Add a tablet to the topology. @@ -108,8 +93,8 @@ func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { t.Fatalf("CreateTablet failed: %v", err) } tw.loadTablets() - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "AddTablet": 1}) - checkChecksum(t, tw, 1261153186) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "AddTablet": 1}) + checkChecksum(t, tw, 3238442862) // Check the tablet is returned by GetAllTablets(). allTablets := fhc.GetAllTablets() @@ -136,14 +121,14 @@ func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { } tw.loadTablets() - // If refreshKnownTablets is disabled, only the new tablet is read + // If RefreshKnownTablets is disabled, only the new tablet is read // from the topo if refreshKnownTablets { - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "AddTablet": 1}) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "AddTablet": 1}) } else { - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "AddTablet": 1}) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "AddTablet": 1}) } - checkChecksum(t, tw, 832404892) + checkChecksum(t, tw, 2762153755) // Check the new tablet is returned by GetAllTablets(). allTablets = fhc.GetAllTablets() @@ -152,20 +137,20 @@ func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { t.Errorf("fhc.GetAllTablets() = %+v; want %+v", allTablets, tablet2) } - // Load the tablets again to show that when refreshKnownTablets is disabled, + // Load the tablets again to show that when RefreshKnownTablets is disabled, // only the list is read from the topo and the checksum doesn't change tw.loadTablets() if refreshKnownTablets { - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2}) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2}) } else { - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1}) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1}) } - checkChecksum(t, tw, 832404892) + checkChecksum(t, tw, 2762153755) // same tablet, different port, should update (previous // one should go away, new one be added) // - // if refreshKnownTablets is disabled, this case is *not* + // if RefreshKnownTablets is disabled, this case is *not* // detected and the tablet remains in the topo using the // old key origTablet := proto.Clone(tablet).(*topodatapb.Tablet) @@ -182,7 +167,7 @@ func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { key = TabletToMapKey(tablet) if refreshKnownTablets { - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 1}) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 1}) if _, ok := allTablets[key]; !ok || len(allTablets) != 2 || !proto.Equal(allTablets[key], tablet) { t.Errorf("fhc.GetAllTablets() = %+v; want %+v", allTablets, tablet) @@ -190,9 +175,9 @@ func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { if _, ok := allTablets[origKey]; ok { t.Errorf("fhc.GetAllTablets() = %+v; don't want %v", allTablets, origKey) } - checkChecksum(t, tw, 698548794) + checkChecksum(t, tw, 2762153755) } else { - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1}) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1}) if _, ok := allTablets[origKey]; !ok || len(allTablets) != 2 || !proto.Equal(allTablets[origKey], origTablet) { t.Errorf("fhc.GetAllTablets() = %+v; want %+v", allTablets, origTablet) @@ -200,39 +185,7 @@ func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { if _, ok := allTablets[key]; ok { t.Errorf("fhc.GetAllTablets() = %+v; don't want %v", allTablets, key) } - checkChecksum(t, tw, 832404892) - } - - // Remove the second tablet and re-add with a new uid. This should - // trigger a ReplaceTablet in loadTablets because the uid does not - // match. - // - // This case *is* detected even if refreshKnownTablets is false - // because the delete tablet / create tablet sequence causes the - // list of tablets to change and therefore the change is detected. - if err := ts.DeleteTablet(context.Background(), tablet2.Alias); err != nil { - t.Fatalf("DeleteTablet failed: %v", err) - } - tablet2.Alias.Uid = 3 - if err := ts.CreateTablet(context.Background(), tablet2); err != nil { - t.Fatalf("CreateTablet failed: %v", err) - } - if err := topo.FixShardReplication(context.Background(), ts, logger, "aa", "keyspace", "shard"); err != nil { - t.Fatalf("FixShardReplication failed: %v", err) - } - tw.loadTablets() - allTablets = fhc.GetAllTablets() - - if refreshKnownTablets { - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 1}) - checkChecksum(t, tw, 4097170367) - } else { - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "ReplaceTablet": 1}) - checkChecksum(t, tw, 3960185881) - } - key = TabletToMapKey(tablet2) - if _, ok := allTablets[key]; !ok || len(allTablets) != 2 || !proto.Equal(allTablets[key], tablet2) { - t.Errorf("fhc.GetAllTablets() = %+v; want %v => %+v", allTablets, key, tablet2) + checkChecksum(t, tw, 2762153755) } // Both tablets restart on different hosts. @@ -258,7 +211,7 @@ func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { t.Fatalf("UpdateTabletFields failed: %v", err) } tw.loadTablets() - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 2}) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 2}) allTablets = fhc.GetAllTablets() key2 := TabletToMapKey(tablet2) if _, ok := allTablets[key2]; !ok { @@ -282,7 +235,7 @@ func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { t.Fatalf("UpdateTabletFields failed: %v", err) } tw.loadTablets() - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 2}) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "ReplaceTablet": 2}) } // Remove the tablet and check that it is detected as being gone. @@ -294,11 +247,11 @@ func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { } tw.loadTablets() if refreshKnownTablets { - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "RemoveTablet": 1}) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1, "GetTablet": 1, "RemoveTablet": 1}) } else { - counts = checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "RemoveTablet": 1}) + counts = checkOpCounts(t, counts, map[string]int64{"ListTablets": 1, "RemoveTablet": 1}) } - checkChecksum(t, tw, 1725545897) + checkChecksum(t, tw, 789108290) allTablets = fhc.GetAllTablets() key = TabletToMapKey(tablet) @@ -318,7 +271,7 @@ func checkWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { t.Fatalf("FixShardReplication failed: %v", err) } tw.loadTablets() - checkOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 0, "RemoveTablet": 1}) + checkOpCounts(t, counts, map[string]int64{"ListTablets": 1, "GetTablet": 0, "RemoveTablet": 1}) checkChecksum(t, tw, 0) allTablets = fhc.GetAllTablets() @@ -395,7 +348,7 @@ func TestFilterByShard(t *testing.T) { } for _, tc := range testcases { - fbs, err := NewFilterByShard(nil, tc.filters) + fbs, err := NewFilterByShard(tc.filters) if err != nil { t.Errorf("cannot create FilterByShard for filters %v: %v", tc.filters, err) } @@ -405,7 +358,7 @@ func TestFilterByShard(t *testing.T) { Shard: tc.shard, } - got := fbs.isIncluded(tablet) + got := fbs.IsIncluded(tablet) if got != tc.included { t.Errorf("isIncluded(%v,%v) for filters %v returned %v but expected %v", tc.keyspace, tc.shard, tc.filters, got, tc.included) } @@ -433,9 +386,9 @@ var ( func TestFilterByKeyspace(t *testing.T) { hc := NewFakeHealthCheck() - tr := NewFilterByKeyspace(hc, testKeyspacesToWatch) + f := NewFilterByKeyspace(testKeyspacesToWatch) ts := memorytopo.NewServer(testCell) - tw := NewCellTabletsWatcher(context.Background(), ts, tr, testCell, 10*time.Minute, true, 5) + tw := NewCellTabletsWatcher(context.Background(), ts, hc, f, testCell, 10*time.Minute, true, 5) for _, test := range testFilterByKeyspace { // Add a new tablet to the topology. @@ -453,7 +406,7 @@ func TestFilterByKeyspace(t *testing.T) { Shard: testShard, } - got := tr.isIncluded(tablet) + got := f.IsIncluded(tablet) if got != test.expected { t.Errorf("isIncluded(%v) for keyspace %v returned %v but expected %v", test.keyspace, test.keyspace, got, test.expected) } @@ -483,7 +436,7 @@ func TestFilterByKeyspace(t *testing.T) { Keyspace: test.keyspace, Shard: testShard, } - got = tr.isIncluded(tabletReplacement) + got = f.IsIncluded(tabletReplacement) if got != test.expected { t.Errorf("isIncluded(%v) for keyspace %v returned %v but expected %v", test.keyspace, test.keyspace, got, test.expected) } diff --git a/go/vt/discovery/utils.go b/go/vt/discovery/utils.go index ac57b032ded..7a9e94f019b 100644 --- a/go/vt/discovery/utils.go +++ b/go/vt/discovery/utils.go @@ -17,20 +17,20 @@ limitations under the License. package discovery // This file contains helper filter methods to process the unfiltered list of -// tablets returned by HealthCheck.GetTabletStatsFrom*. -// See also replicationlag.go for a more sophisicated filter used by vtgate. +// tablets returned by LegacyHealthCheck.GetTabletStatsFrom*. +// See also legacy_replicationlag.go for a more sophisicated filter used by vtgate. // RemoveUnhealthyTablets filters all unhealthy tablets out. // NOTE: Non-serving tablets are considered healthy. -func RemoveUnhealthyTablets(tabletStatsList []TabletStats) []TabletStats { - result := make([]TabletStats, 0, len(tabletStatsList)) +func RemoveUnhealthyTablets(tabletStatsList []LegacyTabletStats) []LegacyTabletStats { + result := make([]LegacyTabletStats, 0, len(tabletStatsList)) for _, ts := range tabletStatsList { // Note we do not check the 'Serving' flag here. // This is mainly to avoid the case where we run a vtworker Diff between a // source and destination, and the source is not serving (disabled by // TabletControl). When we switch the tablet to 'worker', it will // go back to serving state. - if ts.Stats == nil || ts.Stats.HealthError != "" || ts.LastError != nil || IsReplicationLagHigh(&ts) { + if ts.Stats == nil || ts.Stats.HealthError != "" || ts.LastError != nil || LegacyIsReplicationLagHigh(&ts) { continue } result = append(result, ts) diff --git a/go/vt/discovery/utils_test.go b/go/vt/discovery/utils_test.go index 4a0266072c9..c9db72e28be 100644 --- a/go/vt/discovery/utils_test.go +++ b/go/vt/discovery/utils_test.go @@ -27,36 +27,36 @@ import ( func TestRemoveUnhealthyTablets(t *testing.T) { var testcases = []struct { desc string - input []TabletStats - want []TabletStats + input []LegacyTabletStats + want []LegacyTabletStats }{{ desc: "tablets missing Stats", - input: []TabletStats{replica(1), replica(2)}, - want: []TabletStats{}, + input: []LegacyTabletStats{replica(1), replica(2)}, + want: []LegacyTabletStats{}, }, { desc: "all tablets healthy", - input: []TabletStats{healthy(replica(1)), healthy(replica(2))}, - want: []TabletStats{healthy(replica(1)), healthy(replica(2))}, + input: []LegacyTabletStats{healthy(replica(1)), healthy(replica(2))}, + want: []LegacyTabletStats{healthy(replica(1)), healthy(replica(2))}, }, { desc: "one unhealthy tablet (error)", - input: []TabletStats{healthy(replica(1)), unhealthyError(replica(2))}, - want: []TabletStats{healthy(replica(1))}, + input: []LegacyTabletStats{healthy(replica(1)), unhealthyError(replica(2))}, + want: []LegacyTabletStats{healthy(replica(1))}, }, { desc: "one error tablet", - input: []TabletStats{healthy(replica(1)), unhealthyLastError(replica(2))}, - want: []TabletStats{healthy(replica(1))}, + input: []LegacyTabletStats{healthy(replica(1)), unhealthyLastError(replica(2))}, + want: []LegacyTabletStats{healthy(replica(1))}, }, { desc: "one unhealthy tablet (lag)", - input: []TabletStats{healthy(replica(1)), unhealthyLag(replica(2))}, - want: []TabletStats{healthy(replica(1))}, + input: []LegacyTabletStats{healthy(replica(1)), unhealthyLag(replica(2))}, + want: []LegacyTabletStats{healthy(replica(1))}, }, { desc: "no filtering by tablet type", - input: []TabletStats{healthy(master(1)), healthy(replica(2)), healthy(rdonly(3))}, - want: []TabletStats{healthy(master(1)), healthy(replica(2)), healthy(rdonly(3))}, + input: []LegacyTabletStats{healthy(master(1)), healthy(replica(2)), healthy(rdonly(3))}, + want: []LegacyTabletStats{healthy(master(1)), healthy(replica(2)), healthy(rdonly(3))}, }, { desc: "non-serving tablets won't be removed", - input: []TabletStats{notServing(healthy(replica(1)))}, - want: []TabletStats{notServing(healthy(replica(1)))}, + input: []LegacyTabletStats{notServing(healthy(replica(1)))}, + want: []LegacyTabletStats{notServing(healthy(replica(1)))}, }} for _, tc := range testcases { @@ -73,20 +73,20 @@ func TestRemoveUnhealthyTablets(t *testing.T) { } } -func master(uid uint32) TabletStats { +func master(uid uint32) LegacyTabletStats { return minimalTabletStats(uid, topodatapb.TabletType_MASTER) } -func replica(uid uint32) TabletStats { +func replica(uid uint32) LegacyTabletStats { return minimalTabletStats(uid, topodatapb.TabletType_REPLICA) } -func rdonly(uid uint32) TabletStats { +func rdonly(uid uint32) LegacyTabletStats { return minimalTabletStats(uid, topodatapb.TabletType_RDONLY) } -func minimalTabletStats(uid uint32, tabletType topodatapb.TabletType) TabletStats { - return TabletStats{ +func minimalTabletStats(uid uint32, tabletType topodatapb.TabletType) LegacyTabletStats { + return LegacyTabletStats{ Tablet: &topodatapb.Tablet{ Alias: &topodatapb.TabletAlias{ Uid: uid}, @@ -100,33 +100,33 @@ func minimalTabletStats(uid uint32, tabletType topodatapb.TabletType) TabletStat } } -func healthy(ts TabletStats) TabletStats { +func healthy(ts LegacyTabletStats) LegacyTabletStats { ts.Stats = &querypb.RealtimeStats{ SecondsBehindMaster: uint32(1), } return ts } -func unhealthyLag(ts TabletStats) TabletStats { +func unhealthyLag(ts LegacyTabletStats) LegacyTabletStats { ts.Stats = &querypb.RealtimeStats{ SecondsBehindMaster: uint32(3600), } return ts } -func unhealthyError(ts TabletStats) TabletStats { +func unhealthyError(ts LegacyTabletStats) LegacyTabletStats { ts.Stats = &querypb.RealtimeStats{ HealthError: "unhealthy", } return ts } -func unhealthyLastError(ts TabletStats) TabletStats { +func unhealthyLastError(ts LegacyTabletStats) LegacyTabletStats { ts.LastError = errors.New("err") return ts } -func notServing(ts TabletStats) TabletStats { +func notServing(ts LegacyTabletStats) LegacyTabletStats { ts.Serving = false return ts } diff --git a/go/vt/schemamanager/schemaswap/schema_swap.go b/go/vt/schemamanager/schemaswap/schema_swap.go index 14e7161d919..95302ca1e44 100644 --- a/go/vt/schemamanager/schemaswap/schema_swap.go +++ b/go/vt/schemamanager/schemaswap/schema_swap.go @@ -136,17 +136,17 @@ type shardSchemaSwap struct { numTabletsSwapped int // tabletHealthCheck watches after the healthiness of all tablets in the shard. - tabletHealthCheck discovery.HealthCheck + tabletHealthCheck discovery.LegacyHealthCheck // tabletWatchers contains list of topology watchers monitoring changes in the shard // topology. There are several of them because the watchers are per-cell. - tabletWatchers []*discovery.TopologyWatcher + tabletWatchers []*discovery.LegacyTopologyWatcher // allTabletsLock is a mutex protecting access to contents of health check related // variables below. allTabletsLock sync.RWMutex // allTablets is the list of all tablets on the shard mapped by the key provided // by discovery. The contents of the map is guarded by allTabletsLock. - allTablets map[string]*discovery.TabletStats + allTablets map[string]*discovery.LegacyTabletStats // healthWaitingTablet is a key (the same key as used in allTablets) of a tablet that // is currently being waited on to become healthy and to catch up with replication. // The variable is guarded by allTabletsLock. @@ -686,9 +686,9 @@ func (shardSwap *shardSchemaSwap) writeFinishedSwap() error { // all tablets on the shard. Function should be called before the start of the schema // swap process. func (shardSwap *shardSchemaSwap) startHealthWatchers(ctx context.Context) error { - shardSwap.allTablets = make(map[string]*discovery.TabletStats) + shardSwap.allTablets = make(map[string]*discovery.LegacyTabletStats) - shardSwap.tabletHealthCheck = discovery.NewHealthCheck(*vtctl.HealthcheckRetryDelay, *vtctl.HealthCheckTimeout) + shardSwap.tabletHealthCheck = discovery.NewLegacyHealthCheck(*vtctl.HealthcheckRetryDelay, *vtctl.HealthCheckTimeout) shardSwap.tabletHealthCheck.SetListener(shardSwap, true /* sendDownEvents */) topoServer := shardSwap.parent.topoServer @@ -697,7 +697,7 @@ func (shardSwap *shardSchemaSwap) startHealthWatchers(ctx context.Context) error return err } for _, cell := range cellList { - watcher := discovery.NewShardReplicationWatcher( + watcher := discovery.NewLegacyShardReplicationWatcher( ctx, topoServer, shardSwap.tabletHealthCheck, @@ -748,10 +748,10 @@ func (shardSwap *shardSchemaSwap) stopHealthWatchers() { } } -// isTabletHealthy verifies that the given TabletStats represents a healthy tablet that is +// isTabletHealthy verifies that the given LegacyTabletStats represents a healthy tablet that is // caught up with replication to a serving level. -func isTabletHealthy(tabletStats *discovery.TabletStats) bool { - return tabletStats.Stats.HealthError == "" && !discovery.IsReplicationLagHigh(tabletStats) +func isTabletHealthy(tabletStats *discovery.LegacyTabletStats) bool { + return tabletStats.Stats.HealthError == "" && !discovery.LegacyIsReplicationLagHigh(tabletStats) } // startWaitingOnUnhealthyTablet registers the tablet as being waited on in a way that @@ -778,11 +778,11 @@ func (shardSwap *shardSchemaSwap) startWaitingOnUnhealthyTablet(tablet *topodata return shardSwap.healthWaitingChannel, nil } -// checkWaitingTabletHealthiness verifies whether the provided TabletStats represent the +// checkWaitingTabletHealthiness verifies whether the provided LegacyTabletStats represent the // tablet that is being waited to become healthy, and notifies the waiting go routine if // it is the tablet and if it is healthy now. // The function should be called with shardSwap.allTabletsLock mutex locked. -func (shardSwap *shardSchemaSwap) checkWaitingTabletHealthiness(tabletStats *discovery.TabletStats) { +func (shardSwap *shardSchemaSwap) checkWaitingTabletHealthiness(tabletStats *discovery.LegacyTabletStats) { if shardSwap.healthWaitingTablet == tabletStats.Key && isTabletHealthy(tabletStats) { close(*shardSwap.healthWaitingChannel) shardSwap.healthWaitingChannel = nil @@ -790,11 +790,11 @@ func (shardSwap *shardSchemaSwap) checkWaitingTabletHealthiness(tabletStats *dis } } -// StatsUpdate is the part of discovery.HealthCheckStatsListener interface. It makes sure +// StatsUpdate is the part of discovery.LegacyHealthCheckStatsListener interface. It makes sure // that when a change of tablet health happens it's recorded in allTablets list, and if // this is the tablet that is being waited for after restore, the function wakes up the // waiting go routine. -func (shardSwap *shardSchemaSwap) StatsUpdate(newTabletStats *discovery.TabletStats) { +func (shardSwap *shardSchemaSwap) StatsUpdate(newTabletStats *discovery.LegacyTabletStats) { shardSwap.allTabletsLock.Lock() defer shardSwap.allTabletsLock.Unlock() @@ -813,21 +813,21 @@ func (shardSwap *shardSchemaSwap) StatsUpdate(newTabletStats *discovery.TabletSt // getTabletList returns the list of all known tablets in the shard so that the caller // could operate with it without holding the allTabletsLock. -func (shardSwap *shardSchemaSwap) getTabletList() []discovery.TabletStats { +func (shardSwap *shardSchemaSwap) getTabletList() []discovery.LegacyTabletStats { shardSwap.allTabletsLock.RLock() defer shardSwap.allTabletsLock.RUnlock() - tabletList := make([]discovery.TabletStats, 0, len(shardSwap.allTablets)) + tabletList := make([]discovery.LegacyTabletStats, 0, len(shardSwap.allTablets)) for _, tabletStats := range shardSwap.allTablets { tabletList = append(tabletList, *tabletStats) } return tabletList } -// orderTabletsForSwap is an alias for the slice of TabletStats. It implements +// orderTabletsForSwap is an alias for the slice of LegacyTabletStats. It implements // sort.Interface interface so that it's possible to sort the array in the order // in which schema swap will propagate. -type orderTabletsForSwap []discovery.TabletStats +type orderTabletsForSwap []discovery.LegacyTabletStats // Len is part of sort.Interface interface. func (array orderTabletsForSwap) Len() int { @@ -839,11 +839,11 @@ func (array orderTabletsForSwap) Swap(i, j int) { array[i], array[j] = array[j], array[i] } -// getTabletTypeFromStats returns the tablet type saved in the TabletStats object. If there is Target -// data in the TabletStats object then the function returns TabletType from it because it will be more +// getTabletTypeFromStats returns the tablet type saved in the LegacyTabletStats object. If there is Target +// data in the LegacyTabletStats object then the function returns TabletType from it because it will be more // up-to-date. But if that's not available then it returns Tablet.Type which will contain data read // from the topology during initialization of health watchers. -func getTabletTypeFromStats(tabletStats *discovery.TabletStats) topodatapb.TabletType { +func getTabletTypeFromStats(tabletStats *discovery.LegacyTabletStats) topodatapb.TabletType { if tabletStats.Target == nil || tabletStats.Target.TabletType == topodatapb.TabletType_UNKNOWN { return tabletStats.Tablet.Type } @@ -857,7 +857,7 @@ func getTabletTypeFromStats(tabletStats *discovery.TabletStats) topodatapb.Table // then will go 'replica' tablets, and the first will be 'rdonly' and all other // non-replica and non-master types. The sorting order within each of those 5 buckets // doesn't matter. -func tabletSortIndex(tabletStats *discovery.TabletStats) int { +func tabletSortIndex(tabletStats *discovery.LegacyTabletStats) int { tabletType := getTabletTypeFromStats(tabletStats) switch { case tabletType == topodatapb.TabletType_MASTER: diff --git a/go/vt/srvtopo/target_stats.go b/go/vt/srvtopo/target_stats.go deleted file mode 100644 index e492bd59991..00000000000 --- a/go/vt/srvtopo/target_stats.go +++ /dev/null @@ -1,87 +0,0 @@ -/* -Copyright 2019 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 srvtopo - -import ( - "fmt" - - querypb "vitess.io/vitess/go/vt/proto/query" -) - -// TargetStatsEntry has the updated information for a Target. -type TargetStatsEntry struct { - // Target is what this entry applies to. - Target *querypb.Target - - // TabletExternallyReparentedTimestamp is the latest timestamp - // that was reported for this entry. It applies to masters only. - TabletExternallyReparentedTimestamp int64 -} - -// TargetStatsMultiplexer is a helper class to help broadcast stats updates. -// It doesn't have any synchronization, as the container class will already -// have some and this can just use it. -type TargetStatsMultiplexer struct { - // listeners has the map of channels to send updates to. - listeners map[int]chan (*TargetStatsEntry) - - // nextIndex has the next map id. - nextIndex int -} - -// NewTargetStatsMultiplexer returns an initialized TargetStatsMultiplexer. -func NewTargetStatsMultiplexer() TargetStatsMultiplexer { - return TargetStatsMultiplexer{ - listeners: make(map[int]chan (*TargetStatsEntry)), - } -} - -// Subscribe adds a channel to the list. -// Will change the list. -func (tsm *TargetStatsMultiplexer) Subscribe() (int, <-chan (*TargetStatsEntry)) { - i := tsm.nextIndex - tsm.nextIndex++ - c := make(chan (*TargetStatsEntry), 100) - tsm.listeners[i] = c - return i, c -} - -// Unsubscribe removes a channel from the list. -// Will change the list. -func (tsm *TargetStatsMultiplexer) Unsubscribe(i int) error { - c, ok := tsm.listeners[i] - if !ok { - return fmt.Errorf("TargetStatsMultiplexer.Unsubscribe(%v): not suc channel", i) - } - delete(tsm.listeners, i) - close(c) - return nil -} - -// HasSubscribers returns true if we have registered subscribers. -// Will read the list. -func (tsm *TargetStatsMultiplexer) HasSubscribers() bool { - return len(tsm.listeners) > 0 -} - -// Broadcast sends an update to the list. -// Will read the list. -func (tsm *TargetStatsMultiplexer) Broadcast(tse *TargetStatsEntry) { - for _, c := range tsm.listeners { - c <- tse - } -} diff --git a/go/vt/throttler/demo/throttler_demo.go b/go/vt/throttler/demo/throttler_demo.go index 783a27b8afd..b19512a8d00 100644 --- a/go/vt/throttler/demo/throttler_demo.go +++ b/go/vt/throttler/demo/throttler_demo.go @@ -213,7 +213,7 @@ func (r *replica) stop() { type client struct { master *master - healthCheck discovery.HealthCheck + healthCheck discovery.LegacyHealthCheck throttler *throttler.Throttler stopChan chan struct{} @@ -226,7 +226,7 @@ func newClient(master *master, replica *replica) *client { log.Fatal(err) } - healthCheck := discovery.NewHealthCheck(5*time.Second, 1*time.Minute) + healthCheck := discovery.NewLegacyHealthCheck(5*time.Second, 1*time.Minute) c := &client{ master: master, healthCheck: healthCheck, @@ -273,10 +273,10 @@ func (c *client) stop() { c.throttler.Close() } -// StatsUpdate implements discovery.HealthCheckStatsListener. +// StatsUpdate implements discovery.LegacyHealthCheckStatsListener. // It gets called by the healthCheck instance every time a tablet broadcasts // a health update. -func (c *client) StatsUpdate(ts *discovery.TabletStats) { +func (c *client) StatsUpdate(ts *discovery.LegacyTabletStats) { // Ignore unless REPLICA or RDONLY. if ts.Target.TabletType != topodatapb.TabletType_REPLICA && ts.Target.TabletType != topodatapb.TabletType_RDONLY { return diff --git a/go/vt/throttler/max_replication_lag_module.go b/go/vt/throttler/max_replication_lag_module.go index e0219d40b76..857e8ba52ed 100644 --- a/go/vt/throttler/max_replication_lag_module.go +++ b/go/vt/throttler/max_replication_lag_module.go @@ -54,7 +54,7 @@ const ( // i.e. we'll ignore lag records with lower lag from other replicas while we're // waiting for the next record of this replica under test. type replicaUnderTest struct { - // key holds the discovery.TabletStats.Key value for the replica. + // key holds the discovery.LegacyTabletStats.Key value for the replica. key string alias string tabletType topodatapb.TabletType @@ -114,7 +114,7 @@ type MaxReplicationLagModule struct { // max rate calculation has changed. The field is immutable (set in Start().) rateUpdateChan chan<- struct{} - // lagRecords buffers the replication lag records received by the HealthCheck + // lagRecords buffers the replication lag records received by the LegacyHealthCheck // listener. ProcessRecords() will process them. lagRecords chan replicationLagRecord wg sync.WaitGroup @@ -240,7 +240,7 @@ func (m *MaxReplicationLagModule) resetConfiguration() { } // RecordReplicationLag records the current replication lag for processing. -func (m *MaxReplicationLagModule) RecordReplicationLag(t time.Time, ts *discovery.TabletStats) { +func (m *MaxReplicationLagModule) RecordReplicationLag(t time.Time, ts *discovery.LegacyTabletStats) { m.mutableConfigMu.Lock() if m.mutableConfig.MaxReplicationLagSec == ReplicationLagModuleDisabled { m.mutableConfigMu.Unlock() @@ -248,7 +248,7 @@ func (m *MaxReplicationLagModule) RecordReplicationLag(t time.Time, ts *discover } m.mutableConfigMu.Unlock() - // Buffer data point for now to unblock the HealthCheck listener and process + // Buffer data point for now to unblock the LegacyHealthCheck listener and process // it asynchronously in ProcessRecords(). m.lagRecords <- replicationLagRecord{t, *ts} } @@ -404,7 +404,7 @@ func (m *MaxReplicationLagModule) clearReplicaUnderTest(now time.Time, testedSta return true, "it is no longer actively tracked" } if lr.LastError != nil { - // LastError is set i.e. HealthCheck module cannot connect and the cached + // LastError is set i.e. LegacyHealthCheck module cannot connect and the cached // data for the replica might be outdated. return true, "it has LastError set i.e. is no longer correctly tracked" } diff --git a/go/vt/throttler/max_replication_lag_module_test.go b/go/vt/throttler/max_replication_lag_module_test.go index 100a76f7726..812cde0a0ad 100644 --- a/go/vt/throttler/max_replication_lag_module_test.go +++ b/go/vt/throttler/max_replication_lag_module_test.go @@ -223,7 +223,7 @@ func TestMaxReplicationLagModule_ReplicaUnderTest_LastErrorOrNotUp(t *testing.T) // r2 @ 75s, 0s lag, LastError set rError := lagRecord(sinceZero(75*time.Second), r2, 0) - rError.LastError = errors.New("HealthCheck reporting broken") + rError.LastError = errors.New("LegacyHealthCheck reporting broken") tf.m.replicaLagCache.add(rError) // r1 @ 110s, 0s lag @@ -945,13 +945,13 @@ func TestMaxReplicationLagModule_NoIncreaseIfMaxRateWasNotApproached(t *testing. } } -// lagRecord creates a fake record using a fake TabletStats object. +// lagRecord creates a fake record using a fake LegacyTabletStats object. func lagRecord(t time.Time, uid, lag uint32) replicationLagRecord { return replicationLagRecord{t, tabletStats(uid, lag)} } // tabletStats creates fake tablet health data. -func tabletStats(uid, lag uint32) discovery.TabletStats { +func tabletStats(uid, lag uint32) discovery.LegacyTabletStats { typ := topodatapb.TabletType_REPLICA if uid == rdonly1 || uid == rdonly2 { typ = topodatapb.TabletType_RDONLY @@ -963,7 +963,7 @@ func tabletStats(uid, lag uint32) discovery.TabletStats { Type: typ, PortMap: map[string]int32{"vt": int32(uid)}, } - return discovery.TabletStats{ + return discovery.LegacyTabletStats{ Tablet: tablet, Key: discovery.TabletToMapKey(tablet), Target: &querypb.Target{ diff --git a/go/vt/throttler/replication_lag_cache.go b/go/vt/throttler/replication_lag_cache.go index 5e14f5b86b6..9ab61eb81a8 100644 --- a/go/vt/throttler/replication_lag_cache.go +++ b/go/vt/throttler/replication_lag_cache.go @@ -27,11 +27,11 @@ import ( // replicationlagRecord entries. type replicationLagCache struct { // entries maps from the replica to its history. - // The map key is replicationLagRecord.TabletStats.Key. + // The map key is replicationLagRecord.LegacyTabletStats.Key. entries map[string]*replicationLagHistory // slowReplicas is a set of slow replicas. - // The map key is replicationLagRecord.TabletStats.Key. + // The map key is replicationLagRecord.LegacyTabletStats.Key. // This map will always be recomputed by sortByLag() and must not be modified // from other methods. slowReplicas map[string]bool @@ -43,7 +43,7 @@ type replicationLagCache struct { // becomes the new slowest replica. This set is used to detect such a chain. // The set will be cleared if ignoreSlowReplica() returns false. // - // The map key is replicationLagRecord.TabletStats.Key. + // The map key is replicationLagRecord.LegacyTabletStats.Key. // If an entry is deleted from "entries", it must be deleted here as well. ignoredSlowReplicasInARow map[string]bool @@ -76,7 +76,7 @@ func (c *replicationLagCache) add(r replicationLagRecord) { entry.add(r) } -// latest returns the current lag record for the given TabletStats.Key string. +// latest returns the current lag record for the given LegacyTabletStats.Key string. // A zero record is returned if there is no latest entry. func (c *replicationLagCache) latest(key string) replicationLagRecord { entry, ok := c.entries[key] @@ -114,7 +114,7 @@ func (c *replicationLagCache) sortByLag(ignoreNSlowestReplicas int, minimumRepli for _, v := range c.entries { record := v.latest() if int64(record.Stats.SecondsBehindMaster) >= minimumReplicationLag { - list = append(list, record.TabletStats) + list = append(list, record.LegacyTabletStats) i++ } } @@ -126,9 +126,9 @@ func (c *replicationLagCache) sortByLag(ignoreNSlowestReplicas int, minimumRepli } } -// byLagAndTabletUID is a slice of discovery.TabletStats elements that +// byLagAndTabletUID is a slice of discovery.LegacyTabletStats elements that // implements sort.Interface to sort by replication lag and tablet Uid. -type byLagAndTabletUID []discovery.TabletStats +type byLagAndTabletUID []discovery.LegacyTabletStats func (a byLagAndTabletUID) Swap(i, j int) { a[i], a[j] = a[j], a[i] } func (a byLagAndTabletUID) Len() int { return len(a) } @@ -140,7 +140,7 @@ func (a byLagAndTabletUID) Less(i, j int) bool { // ignoreSlowReplica returns true if the MaxReplicationLagModule should ignore // this slow replica. -// "key" refers to ReplicationLagRecord.TabletStats.Key. +// "key" refers to ReplicationLagRecord.LegacyTabletStats.Key. func (c *replicationLagCache) ignoreSlowReplica(key string) bool { if len(c.slowReplicas) == 0 { // No slow replicas at all. @@ -169,7 +169,7 @@ func (c *replicationLagCache) ignoreSlowReplica(key string) bool { } // isIgnored returns true if the given replica is a slow, ignored replica. -// "key" refers to ReplicationLagRecord.TabletStats.Key. +// "key" refers to ReplicationLagRecord.LegacyTabletStats.Key. // Note: Unlike ignoreSlowReplica(key), this method does not update the count // how many replicas in a row have been ignored. Instead, it's meant to find out // when a replica is ignored and therefore the module should not wait for it. diff --git a/go/vt/throttler/replication_lag_record.go b/go/vt/throttler/replication_lag_record.go index bd233ec8803..b0dff1e0a27 100644 --- a/go/vt/throttler/replication_lag_record.go +++ b/go/vt/throttler/replication_lag_record.go @@ -23,13 +23,13 @@ import ( ) // replicationLagRecord stores the tablet health data for a given point in time. -// This data is obtained via the HealthCheck module. +// This data is obtained via the LegacyHealthCheck module. type replicationLagRecord struct { // time is the time at which "value" was observed. time time.Time - // TabletStats holds a copy of the current health data of the tablet. - discovery.TabletStats + // LegacyTabletStats holds a copy of the current health data of the tablet. + discovery.LegacyTabletStats } func (r replicationLagRecord) isZero() bool { diff --git a/go/vt/throttler/throttler.go b/go/vt/throttler/throttler.go index 7de6ca65fdc..ea0096bc537 100644 --- a/go/vt/throttler/throttler.go +++ b/go/vt/throttler/throttler.go @@ -295,7 +295,7 @@ func (t *Throttler) SetMaxRate(rate int64) { // RecordReplicationLag must be called by users to report the "ts" tablet health // data observed at "time". // Note: After Close() is called, this method must not be called anymore. -func (t *Throttler) RecordReplicationLag(time time.Time, ts *discovery.TabletStats) { +func (t *Throttler) RecordReplicationLag(time time.Time, ts *discovery.LegacyTabletStats) { t.maxReplicationLagModule.RecordReplicationLag(time, ts) } diff --git a/go/vt/vtctld/realtime_status.go b/go/vt/vtctld/realtime_status.go index c3e04dfcb0c..43598921a62 100644 --- a/go/vt/vtctld/realtime_status.go +++ b/go/vt/vtctld/realtime_status.go @@ -28,13 +28,13 @@ import ( // realtimeStats holds the objects needed to obtain realtime health stats of tablets. type realtimeStats struct { - healthCheck discovery.HealthCheck + healthCheck discovery.LegacyHealthCheck *tabletStatsCache - cellWatchers []*discovery.TopologyWatcher + cellWatchers []*discovery.LegacyTopologyWatcher } func newRealtimeStats(ts *topo.Server) (*realtimeStats, error) { - hc := discovery.NewHealthCheck(*vtctl.HealthcheckRetryDelay, *vtctl.HealthCheckTimeout) + hc := discovery.NewLegacyHealthCheck(*vtctl.HealthcheckRetryDelay, *vtctl.HealthCheckTimeout) tabletStatsCache := newTabletStatsCache() // sendDownEvents is set to true here, as we want to receive // Up=False events for a tablet. @@ -49,9 +49,9 @@ func newRealtimeStats(ts *topo.Server) (*realtimeStats, error) { if err != nil { return r, fmt.Errorf("error when getting cells: %v", err) } - var watchers []*discovery.TopologyWatcher + var watchers []*discovery.LegacyTopologyWatcher for _, cell := range cells { - watcher := discovery.NewCellTabletsWatcher(context.Background(), ts, hc, cell, *vtctl.HealthCheckTopologyRefresh, true /* refreshKnownTablets */, discovery.DefaultTopoReadConcurrency) + watcher := discovery.NewLegacyCellTabletsWatcher(context.Background(), ts, hc, cell, *vtctl.HealthCheckTopologyRefresh, true /* refreshKnownTablets */, discovery.DefaultTopoReadConcurrency) watchers = append(watchers, watcher) } r.cellWatchers = watchers diff --git a/go/vt/vtctld/realtime_status_test.go b/go/vt/vtctld/realtime_status_test.go index 4df601a955e..4186ada084e 100644 --- a/go/vt/vtctld/realtime_status_test.go +++ b/go/vt/vtctld/realtime_status_test.go @@ -38,7 +38,7 @@ import ( ) // TestRealtimeStatsWithQueryService uses fakeTablets and the fakeQueryService to -// copy the environment needed for the HealthCheck object. +// copy the environment needed for the LegacyHealthCheck object. func TestRealtimeStatsWithQueryService(t *testing.T) { // Set up testing keyspace with 2 tablets within 2 cells. keyspace := "ks" @@ -113,7 +113,7 @@ func TestRealtimeStatsWithQueryService(t *testing.T) { } } -// checkStats ensures that the HealthCheck object received an update and passed +// checkStats ensures that the LegacyHealthCheck object received an update and passed // that information to the correct tablet. func checkStats(realtimeStats *realtimeStats, tablet *testlib.FakeTablet, want *querypb.RealtimeStats) error { deadline := time.Now().Add(time.Second * 5) @@ -122,7 +122,7 @@ func checkStats(realtimeStats *realtimeStats, tablet *testlib.FakeTablet, want * if err != nil { continue } - if result.DeepEqual(&discovery.TabletStats{}) { + if result.DeepEqual(&discovery.LegacyTabletStats{}) { continue } got := result.Stats @@ -134,7 +134,7 @@ func checkStats(realtimeStats *realtimeStats, tablet *testlib.FakeTablet, want * return fmt.Errorf("timeout error when getting tabletStatuses") } -// newRealtimeStatsForTesting creates a new realtimeStats object without creating a HealthCheck object. +// newRealtimeStatsForTesting creates a new realtimeStats object without creating a LegacyHealthCheck object. func newRealtimeStatsForTesting() *realtimeStats { tabletStatsCache := newTabletStatsCache() return &realtimeStats{ diff --git a/go/vt/vtctld/tablet_stats_cache.go b/go/vt/vtctld/tablet_stats_cache.go index 9e7e0fa5dbd..88719d14198 100644 --- a/go/vt/vtctld/tablet_stats_cache.go +++ b/go/vt/vtctld/tablet_stats_cache.go @@ -66,7 +66,7 @@ type heatmap struct { YGridLines []float64 } -type byTabletUID []*discovery.TabletStats +type byTabletUID []*discovery.LegacyTabletStats func (a byTabletUID) Len() int { return len(a) } func (a byTabletUID) Swap(i, j int) { a[i], a[j] = a[j], a[i] } @@ -78,18 +78,18 @@ var availableTabletTypes = []topodatapb.TabletType{topodatapb.TabletType_MASTER, // tabletStatsCache holds the most recent status update received for // each tablet. The tablets are indexed by uid, so it is different -// than discovery.TabletStatsCache. +// than discovery.LegacyTabletStatsCache. type tabletStatsCache struct { // mu guards access to the fields below. mu sync.Mutex - // statuses keeps a map of TabletStats. + // statuses keeps a map of LegacyTabletStats. // The first key is the keyspace, the second key is the shard, // the third key is the cell, the last key is the tabletType. // The keys are strings to allow exposing this map as a JSON object in api.go. - statuses map[string]map[string]map[string]map[topodatapb.TabletType][]*discovery.TabletStats + statuses map[string]map[string]map[string]map[topodatapb.TabletType][]*discovery.LegacyTabletStats // statusesByAlias is a copy of statuses and will be updated simultaneously. // The first key is the string representation of the tablet alias. - statusesByAlias map[string]*discovery.TabletStats + statusesByAlias map[string]*discovery.LegacyTabletStats } type topologyInfo struct { @@ -100,14 +100,14 @@ type topologyInfo struct { func newTabletStatsCache() *tabletStatsCache { return &tabletStatsCache{ - statuses: make(map[string]map[string]map[string]map[topodatapb.TabletType][]*discovery.TabletStats), - statusesByAlias: make(map[string]*discovery.TabletStats), + statuses: make(map[string]map[string]map[string]map[topodatapb.TabletType][]*discovery.LegacyTabletStats), + statusesByAlias: make(map[string]*discovery.LegacyTabletStats), } } -// StatsUpdate is part of the discovery.HealthCheckStatsListener interface. -// Upon receiving a new TabletStats, it updates the two maps in tablet_stats_cache. -func (c *tabletStatsCache) StatsUpdate(stats *discovery.TabletStats) { +// StatsUpdate is part of the discovery.LegacyHealthCheckStatsListener interface. +// Upon receiving a new LegacyTabletStats, it updates the two maps in tablet_stats_cache. +func (c *tabletStatsCache) StatsUpdate(stats *discovery.LegacyTabletStats) { c.mu.Lock() defer c.mu.Unlock() @@ -133,25 +133,25 @@ func (c *tabletStatsCache) StatsUpdate(stats *discovery.TabletStats) { // Tablet isn't tracked yet so just add it. _, ok := c.statuses[keyspace] if !ok { - shards := make(map[string]map[string]map[topodatapb.TabletType][]*discovery.TabletStats) + shards := make(map[string]map[string]map[topodatapb.TabletType][]*discovery.LegacyTabletStats) c.statuses[keyspace] = shards } _, ok = c.statuses[keyspace][shard] if !ok { - cells := make(map[string]map[topodatapb.TabletType][]*discovery.TabletStats) + cells := make(map[string]map[topodatapb.TabletType][]*discovery.LegacyTabletStats) c.statuses[keyspace][shard] = cells } _, ok = c.statuses[keyspace][shard][cell] if !ok { - types := make(map[topodatapb.TabletType][]*discovery.TabletStats) + types := make(map[topodatapb.TabletType][]*discovery.LegacyTabletStats) c.statuses[keyspace][shard][cell] = types } _, ok = c.statuses[keyspace][shard][cell][tabletType] if !ok { - tablets := make([]*discovery.TabletStats, 0) + tablets := make([]*discovery.LegacyTabletStats, 0) c.statuses[keyspace][shard][cell][tabletType] = tablets } @@ -165,13 +165,13 @@ func (c *tabletStatsCache) StatsUpdate(stats *discovery.TabletStats) { *ts = *stats } -func tabletToMapKey(stats *discovery.TabletStats) string { +func tabletToMapKey(stats *discovery.LegacyTabletStats) string { return stats.Tablet.Alias.String() } // remove takes in an array and returns it with the specified element removed // (leaves the array unchanged if element isn't in the array). -func remove(tablets []*discovery.TabletStats, tabletAlias *topodatapb.TabletAlias) []*discovery.TabletStats { +func remove(tablets []*discovery.LegacyTabletStats, tabletAlias *topodatapb.TabletAlias) []*discovery.LegacyTabletStats { filteredTablets := tablets[:0] for _, tablet := range tablets { if !topoproto.TabletAliasEqual(tablet.Tablet.Alias, tabletAlias) { @@ -312,7 +312,7 @@ func (c *tabletStatsCache) heatmapData(selectedKeyspace, selectedCell, selectedT defer c.mu.Unlock() // Get the metric data. - var metricFunc func(stats *discovery.TabletStats) float64 + var metricFunc func(stats *discovery.LegacyTabletStats) float64 switch selectedMetric { case "lag": metricFunc = replicationLag @@ -399,7 +399,7 @@ func (c *tabletStatsCache) heatmapData(selectedKeyspace, selectedCell, selectedT return heatmaps, nil } -func (c *tabletStatsCache) unaggregatedData(keyspace, cell, selectedType string, metricFunc func(stats *discovery.TabletStats) float64) ([][]float64, [][]*topodatapb.TabletAlias, yLabel) { +func (c *tabletStatsCache) unaggregatedData(keyspace, cell, selectedType string, metricFunc func(stats *discovery.LegacyTabletStats) float64) ([][]float64, [][]*topodatapb.TabletAlias, yLabel) { // This loop goes through every nested label (in this case, tablet type). var cellData [][]float64 var cellAliases [][]*topodatapb.TabletAlias @@ -459,7 +459,7 @@ func (c *tabletStatsCache) unaggregatedData(keyspace, cell, selectedType string, // aggregatedData gets heatmapData by taking the average of the metric value of all tablets within the keyspace and cell of the // specified type (or from all types if 'all' was selected). -func (c *tabletStatsCache) aggregatedData(keyspace, cell, selectedType, selectedMetric string, metricFunc func(stats *discovery.TabletStats) float64) ([][]float64, [][]*topodatapb.TabletAlias, yLabel) { +func (c *tabletStatsCache) aggregatedData(keyspace, cell, selectedType, selectedMetric string, metricFunc func(stats *discovery.LegacyTabletStats) float64) ([][]float64, [][]*topodatapb.TabletAlias, yLabel) { shards := c.shards(keyspace) tabletTypes := c.tabletTypesLocked(keyspace, cell, selectedType) @@ -508,18 +508,18 @@ func (c *tabletStatsCache) aggregatedData(keyspace, cell, selectedType, selected return cellData, nil, cellLabel } -func (c *tabletStatsCache) tabletStats(tabletAlias *topodatapb.TabletAlias) (discovery.TabletStats, error) { +func (c *tabletStatsCache) tabletStats(tabletAlias *topodatapb.TabletAlias) (discovery.LegacyTabletStats, error) { c.mu.Lock() defer c.mu.Unlock() ts, ok := c.statusesByAlias[tabletAlias.String()] if !ok { - return discovery.TabletStats{}, fmt.Errorf("could not find tablet: %v", tabletAlias) + return discovery.LegacyTabletStats{}, fmt.Errorf("could not find tablet: %v", tabletAlias) } return *ts, nil } -func health(stat *discovery.TabletStats) float64 { +func health(stat *discovery.LegacyTabletStats) float64 { // The tablet is unhealthy if there is an health error. if stat.Stats.HealthError != "" { return tabletUnhealthy @@ -548,13 +548,13 @@ func health(stat *discovery.TabletStats) float64 { return tabletHealthy } -func replicationLag(stat *discovery.TabletStats) float64 { +func replicationLag(stat *discovery.LegacyTabletStats) float64 { return float64(stat.Stats.SecondsBehindMaster) } -func qps(stat *discovery.TabletStats) float64 { +func qps(stat *discovery.LegacyTabletStats) float64 { return stat.Stats.Qps } // compile-time interface check -var _ discovery.HealthCheckStatsListener = (*tabletStatsCache)(nil) +var _ discovery.LegacyHealthCheckStatsListener = (*tabletStatsCache)(nil) diff --git a/go/vt/vtctld/tablet_stats_cache_test.go b/go/vt/vtctld/tablet_stats_cache_test.go index 781d0c98f77..eb4d4415456 100644 --- a/go/vt/vtctld/tablet_stats_cache_test.go +++ b/go/vt/vtctld/tablet_stats_cache_test.go @@ -342,7 +342,7 @@ func TestTabletStats(t *testing.T) { tabletStatsCache.StatsUpdate(ts1) tabletStatsCache.StatsUpdate(ts2) - // Test 1: tablet1 and tablet2 are updated with the stats received by the HealthCheck module. + // Test 1: tablet1 and tablet2 are updated with the stats received by the LegacyHealthCheck module. got1, err := tabletStatsCache.tabletStats(ts1.Tablet.Alias) want1 := ts1 if err != nil || !got1.DeepEqual(want1) { @@ -426,8 +426,8 @@ func TestTopologyInfo(t *testing.T) { } } -// tabletStats will create a discovery.TabletStats object. -func tabletStats(keyspace, cell, shard string, tabletType topodatapb.TabletType, uid uint32) *discovery.TabletStats { +// tabletStats will create a discovery.LegacyTabletStats object. +func tabletStats(keyspace, cell, shard string, tabletType topodatapb.TabletType, uid uint32) *discovery.LegacyTabletStats { target := &querypb.Target{ Keyspace: keyspace, Shard: shard, @@ -445,7 +445,7 @@ func tabletStats(keyspace, cell, shard string, tabletType topodatapb.TabletType, // uid is used for SecondsBehindMaster to give it a unique value. SecondsBehindMaster: uid, } - stats := &discovery.TabletStats{ + stats := &discovery.LegacyTabletStats{ Tablet: tablet, Target: target, Up: true, diff --git a/go/vt/vtexplain/vtexplain_flaky_test.go b/go/vt/vtexplain/vtexplain_flaky_test.go index 36b84ba9351..6b22a09c243 100644 --- a/go/vt/vtexplain/vtexplain_flaky_test.go +++ b/go/vt/vtexplain/vtexplain_flaky_test.go @@ -24,6 +24,8 @@ import ( "strings" "testing" + "vitess.io/vitess/go/vt/vtgate" + "github.com/google/go-cmp/cmp" "github.com/stretchr/testify/require" ) @@ -40,6 +42,8 @@ func defaultTestOpts() *Options { } func initTest(mode string, opts *Options, t *testing.T) { + // Use legacy gateway until we can rewrite these tests to use new tabletgateway + *vtgate.GatewayImplementation = vtgate.GatewayImplementationDiscovery schema, err := ioutil.ReadFile("testdata/test-schema.sql") require.NoError(t, err) diff --git a/go/vt/vtexplain/vtexplain_vtgate.go b/go/vt/vtexplain/vtexplain_vtgate.go index 95259873f67..a59c8b2d0cf 100644 --- a/go/vt/vtexplain/vtexplain_vtgate.go +++ b/go/vt/vtexplain/vtexplain_vtgate.go @@ -42,7 +42,7 @@ import ( var ( explainTopo *ExplainTopo vtgateExecutor *vtgate.Executor - healthCheck *discovery.FakeHealthCheck + healthCheck *discovery.FakeLegacyHealthCheck vtgateSession = &vtgatepb.Session{ TargetString: "", @@ -52,7 +52,7 @@ var ( func initVtgateExecutor(vSchemaStr string, opts *Options) error { explainTopo = &ExplainTopo{NumShards: opts.NumShards} - healthCheck = discovery.NewFakeHealthCheck() + healthCheck = discovery.NewFakeLegacyHealthCheck() resolver := newFakeResolver(opts, healthCheck, explainTopo, vtexplainCell) @@ -70,7 +70,7 @@ func initVtgateExecutor(vSchemaStr string, opts *Options) error { return nil } -func newFakeResolver(opts *Options, hc discovery.HealthCheck, serv srvtopo.Server, cell string) *vtgate.Resolver { +func newFakeResolver(opts *Options, hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string) *vtgate.Resolver { ctx := context.Background() gw := vtgate.GatewayCreator()(ctx, hc, serv, cell, 3) gw.WaitForTablets(ctx, []topodatapb.TabletType{topodatapb.TabletType_REPLICA}) @@ -80,7 +80,7 @@ func newFakeResolver(opts *Options, hc discovery.HealthCheck, serv srvtopo.Serve txMode = vtgatepb.TransactionMode_TWOPC } tc := vtgate.NewTxConn(gw, txMode) - sc := vtgate.NewScatterConn("", tc, gw, hc) + sc := vtgate.LegacyNewScatterConn("", tc, gw, hc) srvResolver := srvtopo.NewResolver(serv, gw, cell) return vtgate.NewResolver(srvResolver, serv, cell, sc) } diff --git a/go/vt/vtgate/api.go b/go/vt/vtgate/api.go index c2a6f36e747..bf4d1ca5ef5 100644 --- a/go/vt/vtgate/api.go +++ b/go/vt/vtgate/api.go @@ -88,7 +88,7 @@ func getItemPath(url string) string { return parts[1] } -func initAPI(ctx context.Context, hc discovery.HealthCheck) { +func initAPI(ctx context.Context, hc *discovery.HealthCheck) { // Healthcheck real time status per (cell, keyspace, tablet type, metric). handleCollection("health-check", func(r *http.Request) (interface{}, error) { cacheStatus := hc.CacheStatus() @@ -103,7 +103,7 @@ func initAPI(ctx context.Context, hc discovery.HealthCheck) { return cacheStatus, nil } if len(parts) != 2 { - return nil, fmt.Errorf("invalid health-check path: %q expected path: / or /cell/ or /keyspace/ or /tablet/", itemPath) + return nil, fmt.Errorf("invalid health-check path: %q expected path: / or /cell/ or /keyspace/ or /tablet/mysql_hostname", itemPath) } value := parts[1] @@ -128,6 +128,62 @@ func initAPI(ctx context.Context, hc discovery.HealthCheck) { } return filteredStatus, nil } + case "tablet": + { + // Return a _specific tablet_ + for _, tabletCacheStatus := range cacheStatus { + for _, tabletStats := range tabletCacheStatus.TabletsStats { + if tabletStats.Tablet.MysqlHostname == value { + return tabletStats, nil + } + } + } + } + } + return nil, fmt.Errorf("cannot find health for: %s", itemPath) + }) +} + +func legacyInitAPI(ctx context.Context, hc discovery.LegacyHealthCheck) { + // Healthcheck real time status per (cell, keyspace, tablet type, metric). + handleCollection("health-check", func(r *http.Request) (interface{}, error) { + cacheStatus := hc.CacheStatus() + + itemPath := getItemPath(r.URL.Path) + if itemPath == "" { + return cacheStatus, nil + } + parts := strings.SplitN(itemPath, "/", 2) + collectionFilter := parts[0] + if collectionFilter == "" { + return cacheStatus, nil + } + if len(parts) != 2 { + return nil, fmt.Errorf("invalid health-check path: %q expected path: / or /cell/ or /keyspace/ or /tablet/", itemPath) + } + value := parts[1] + + switch collectionFilter { + case "cell": + { + filteredStatus := make(discovery.LegacyTabletsCacheStatusList, 0) + for _, tabletCacheStatus := range cacheStatus { + if tabletCacheStatus.Cell == value { + filteredStatus = append(filteredStatus, tabletCacheStatus) + } + } + return filteredStatus, nil + } + case "keyspace": + { + filteredStatus := make(discovery.LegacyTabletsCacheStatusList, 0) + for _, tabletCacheStatus := range cacheStatus { + if tabletCacheStatus.Target.Keyspace == value { + filteredStatus = append(filteredStatus, tabletCacheStatus) + } + } + return filteredStatus, nil + } case "tablet": { // Return a _specific tablet_ diff --git a/go/vt/vtgate/buffer/buffer.go b/go/vt/vtgate/buffer/buffer.go index 3cfe150930b..84ab628cfc9 100644 --- a/go/vt/vtgate/buffer/buffer.go +++ b/go/vt/vtgate/buffer/buffer.go @@ -88,7 +88,7 @@ type Buffer struct { // In particular, it is used to serialize the following Go routines: // - 1. Requests which may buffer (RLock, can be run in parallel) // - 2. Request which starts buffering (based on the seen error) - // - 3. HealthCheck listener ("StatsUpdate") which stops buffering + // - 3. LegacyHealthCheck listener ("StatsUpdate") which stops buffering // - 4. Timer which may stop buffering after -buffer_max_failover_duration mu sync.RWMutex // buffers holds a shardBuffer object per shard, even if no failover is in @@ -213,12 +213,33 @@ func (b *Buffer) WaitForFailoverEnd(ctx context.Context, keyspace, shard string, return sb.waitForFailoverEnd(ctx, keyspace, shard, err) } +// ProcessMasterHealth notifies the buffer to record a new master +// and end any failover buffering that may be in progress +func (b *Buffer) ProcessMasterHealth(th *discovery.TabletHealth) { + if th.Target.TabletType != topodatapb.TabletType_MASTER { + panic(fmt.Sprintf("BUG: non MASTER TabletHealth object must not be forwarded: %#v", th)) + } + timestamp := th.MasterTermStartTime + if timestamp == 0 { + // Masters where TabletExternallyReparented was never called will return 0. + // Ignore them. + return + } + + sb := b.getOrCreateBuffer(th.Target.Keyspace, th.Target.Shard) + if sb == nil { + // Buffer is shut down. Ignore all calls. + return + } + sb.recordExternallyReparentedTimestamp(timestamp, th.Tablet.Alias) +} + // StatsUpdate keeps track of the "tablet_externally_reparented_timestamp" of // each master. This way we can detect the end of a failover. -// It is part of the discovery.HealthCheckStatsListener interface. -func (b *Buffer) StatsUpdate(ts *discovery.TabletStats) { +// It is part of the discovery.LegacyHealthCheckStatsListener interface. +func (b *Buffer) StatsUpdate(ts *discovery.LegacyTabletStats) { if ts.Target.TabletType != topodatapb.TabletType_MASTER { - panic(fmt.Sprintf("BUG: non MASTER TabletStats object must not be forwarded: %#v", ts)) + panic(fmt.Sprintf("BUG: non MASTER LegacyTabletStats object must not be forwarded: %#v", ts)) } timestamp := ts.TabletExternallyReparentedTimestamp diff --git a/go/vt/vtgate/buffer/buffer_test.go b/go/vt/vtgate/buffer/buffer_test.go index 59a2d79ed95..b2c1c7f9c2f 100644 --- a/go/vt/vtgate/buffer/buffer_test.go +++ b/go/vt/vtgate/buffer/buffer_test.go @@ -93,7 +93,7 @@ func TestBuffer(t *testing.T) { // after this. If the TabletExternallyReparented RPC is called regularly by // an external failover tool, the timestamp will be increased (even though // the master did not change.) - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: oldMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: now.Unix(), @@ -123,7 +123,7 @@ func TestBuffer(t *testing.T) { // Mimic the failover end. now = now.Add(1 * time.Second) - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: newMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: now.Unix(), @@ -184,7 +184,7 @@ func TestBuffer(t *testing.T) { t.Fatalf("buffering start was not tracked: got = %v, want = %v", got, want) } // Stop buffering. - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: oldMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: now.Unix(), @@ -321,7 +321,7 @@ func TestDryRun(t *testing.T) { } // End of failover is tracked as well. - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: newMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: 1, // Use any value > 0. @@ -373,7 +373,7 @@ func TestLastReparentTooRecent_BufferingSkipped(t *testing.T) { // Simulate that the old master notified us about its reparented timestamp // very recently (time.Now()). // vtgate should see this immediately after the start. - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: oldMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: now.Unix(), @@ -382,7 +382,7 @@ func TestLastReparentTooRecent_BufferingSkipped(t *testing.T) { // Failover to new master. Its end is detected faster than the beginning. // Do not start buffering. now = now.Add(1 * time.Second) - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: newMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: now.Unix(), @@ -417,7 +417,7 @@ func TestLastReparentTooRecent_Buffering(t *testing.T) { // Simulate that the old master notified us about its reparented timestamp // very recently (time.Now()). // vtgate should see this immediately after the start. - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: oldMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: now.Unix(), @@ -426,7 +426,7 @@ func TestLastReparentTooRecent_Buffering(t *testing.T) { // Failover to new master. Do not issue any requests before or after i.e. // there was 0 QPS traffic and no buffering was started. now = now.Add(1 * time.Second) - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: newMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: now.Unix(), @@ -441,7 +441,7 @@ func TestLastReparentTooRecent_Buffering(t *testing.T) { t.Fatal(err) } // And then the failover end. - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: newMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: now.Unix(), @@ -480,7 +480,7 @@ func TestPassthroughDuringDrain(t *testing.T) { } // Stop buffering and trigger drain. - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: newMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: 1, // Use any value > 0. @@ -595,7 +595,7 @@ func testRequestCanceled(t *testing.T, explicitEnd bool) { } if explicitEnd { - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: newMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: 1, // Use any value > 0. @@ -614,7 +614,7 @@ func testRequestCanceled(t *testing.T, explicitEnd bool) { // If buffering stopped implicitly, the explicit signal will still happen // shortly after. In that case, the buffer should ignore it. if !explicitEnd { - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: newMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: 1, // Use any value > 0. @@ -660,7 +660,7 @@ func TestEviction(t *testing.T) { } // End of failover. Stop buffering. - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: newMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: 1, // Use any value > 0. @@ -743,7 +743,7 @@ func TestEvictionNotPossible(t *testing.T) { } // End of failover. Stop buffering. - b.StatsUpdate(&discovery.TabletStats{ + b.StatsUpdate(&discovery.LegacyTabletStats{ Tablet: newMaster, Target: &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: topodatapb.TabletType_MASTER}, TabletExternallyReparentedTimestamp: 1, // Use any value > 0. diff --git a/go/vt/vtgate/discoverygateway.go b/go/vt/vtgate/discoverygateway.go index 474ec99d35f..057e620d7f2 100644 --- a/go/vt/vtgate/discoverygateway.go +++ b/go/vt/vtgate/discoverygateway.go @@ -17,7 +17,6 @@ limitations under the License. package vtgate import ( - "flag" "fmt" "math/rand" "sort" @@ -28,7 +27,6 @@ import ( "golang.org/x/net/context" "vitess.io/vitess/go/vt/topotools" - "vitess.io/vitess/go/flagutil" "vitess.io/vitess/go/stats" "vitess.io/vitess/go/vt/discovery" "vitess.io/vitess/go/vt/log" @@ -44,38 +42,28 @@ import ( "vitess.io/vitess/go/vt/topo/topoproto" ) -var ( - cellsToWatch = flag.String("cells_to_watch", "", "comma-separated list of cells for watching tablets") - refreshInterval = flag.Duration("tablet_refresh_interval", 1*time.Minute, "tablet refresh interval") - refreshKnownTablets = flag.Bool("tablet_refresh_known_tablets", true, "tablet refresh reloads the tablet address/port map from topo in case it changes") - topoReadConcurrency = flag.Int("topo_read_concurrency", 32, "concurrent topo reads") - - allowedTabletTypes []topodatapb.TabletType - - tabletFilters flagutil.StringListValue -) - const ( - gatewayImplementationDiscovery = "discoverygateway" + // GatewayImplementationDiscovery defines the string value used as the implementation key for DiscoveryGateway + GatewayImplementationDiscovery = "discoverygateway" ) func init() { - flag.Var(&tabletFilters, "tablet_filters", "Specifies a comma-separated list of 'keyspace|shard_name or keyrange' values to filter the tablets to watch") - topoproto.TabletTypeListVar(&allowedTabletTypes, "allowed_tablet_types", "Specifies the tablet types this vtgate is allowed to route queries to") - RegisterGatewayCreator(gatewayImplementationDiscovery, createDiscoveryGateway) + RegisterGatewayCreator(GatewayImplementationDiscovery, createDiscoveryGateway) } -type discoveryGateway struct { +// DiscoveryGateway is the default Gateway implementation. +// This implementation uses the legacy healthcheck module. +type DiscoveryGateway struct { queryservice.QueryService - hc discovery.HealthCheck - tsc *discovery.TabletStatsCache + hc discovery.LegacyHealthCheck + tsc *discovery.LegacyTabletStatsCache srvTopoServer srvtopo.Server localCell string retryCount int // tabletsWatchers contains a list of all the watchers we use. // We create one per cell. - tabletsWatchers []*discovery.TopologyWatcher + tabletsWatchers []*discovery.LegacyTopologyWatcher // mu protects the fields of this group. mu sync.RWMutex @@ -87,11 +75,15 @@ type discoveryGateway struct { buffer *buffer.Buffer } -func createDiscoveryGateway(ctx context.Context, hc discovery.HealthCheck, serv srvtopo.Server, cell string, retryCount int) Gateway { +func createDiscoveryGateway(ctx context.Context, hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string, retryCount int) Gateway { return NewDiscoveryGateway(ctx, hc, serv, cell, retryCount) } -func NewDiscoveryGateway(ctx context.Context, hc discovery.HealthCheck, serv srvtopo.Server, cell string, retryCount int) *discoveryGateway { +// NewDiscoveryGateway creates a new DiscoveryGateway using the provided healthcheck and toposerver. +// cell is the cell where the gateway is located a.k.a localCell. +// This gateway can route to MASTER in any cell provided by the cells_to_watch command line argument. +// Other tablet type requests (REPLICA/RDONLY) are only routed to tablets in the same cell. +func NewDiscoveryGateway(ctx context.Context, hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string, retryCount int) *DiscoveryGateway { var topoServer *topo.Server if serv != nil { var err error @@ -101,42 +93,43 @@ func NewDiscoveryGateway(ctx context.Context, hc discovery.HealthCheck, serv srv } } - dg := &discoveryGateway{ + dg := &DiscoveryGateway{ hc: hc, tsc: discovery.NewTabletStatsCacheDoNotSetListener(topoServer, cell), srvTopoServer: serv, localCell: cell, retryCount: retryCount, - tabletsWatchers: make([]*discovery.TopologyWatcher, 0, 1), + tabletsWatchers: make([]*discovery.LegacyTopologyWatcher, 0, 1), statusAggregators: make(map[string]*TabletStatusAggregator), buffer: buffer.New(), } - // Set listener which will update TabletStatsCache and MasterBuffer. - // We set sendDownEvents=true because it's required by TabletStatsCache. + // Set listener which will update LegacyTabletStatsCache and MasterBuffer. + // We set sendDownEvents=true because it's required by LegacyTabletStatsCache. hc.SetListener(dg, true /* sendDownEvents */) - log.Infof("loading tablets for cells: %v", *cellsToWatch) - for _, c := range strings.Split(*cellsToWatch, ",") { + cells := *discovery.CellsToWatch + log.Infof("loading tablets for cells: %v", cells) + for _, c := range strings.Split(cells, ",") { if c == "" { continue } - var tr discovery.TabletRecorder = dg.hc - if len(tabletFilters) > 0 { - if len(KeyspacesToWatch) > 0 { + var recorder discovery.LegacyTabletRecorder = dg.hc + if len(discovery.TabletFilters) > 0 { + if len(discovery.KeyspacesToWatch) > 0 { log.Exitf("Only one of -keyspaces_to_watch and -tablet_filters may be specified at a time") } - fbs, err := discovery.NewFilterByShard(dg.hc, tabletFilters) + fbs, err := discovery.NewLegacyFilterByShard(recorder, discovery.TabletFilters) if err != nil { log.Exitf("Cannot parse tablet_filters parameter: %v", err) } - tr = fbs - } else if len(KeyspacesToWatch) > 0 { - tr = discovery.NewFilterByKeyspace(dg.hc, KeyspacesToWatch) + recorder = fbs + } else if len(discovery.KeyspacesToWatch) > 0 { + recorder = discovery.NewLegacyFilterByKeyspace(recorder, discovery.KeyspacesToWatch) } - ctw := discovery.NewCellTabletsWatcher(ctx, topoServer, tr, c, *refreshInterval, *refreshKnownTablets, *topoReadConcurrency) + ctw := discovery.NewLegacyCellTabletsWatcher(ctx, topoServer, recorder, c, *discovery.RefreshInterval, *discovery.RefreshKnownTablets, *discovery.TopoReadConcurrency) dg.tabletsWatchers = append(dg.tabletsWatchers, ctw) } dg.QueryService = queryservice.Wrap(nil, dg.withRetry) @@ -145,7 +138,7 @@ func NewDiscoveryGateway(ctx context.Context, hc discovery.HealthCheck, serv srv // RegisterStats registers the stats to export the lag since the last refresh // and the checksum of the topology -func (dg *discoveryGateway) RegisterStats() { +func (dg *DiscoveryGateway) RegisterStats() { stats.NewGaugeDurationFunc( "TopologyWatcherMaxRefreshLag", "maximum time since the topology watcher refreshed a cell", @@ -161,7 +154,7 @@ func (dg *discoveryGateway) RegisterStats() { // topologyWatcherMaxRefreshLag returns the maximum lag since the watched // cells were refreshed from the topo server -func (dg *discoveryGateway) topologyWatcherMaxRefreshLag() time.Duration { +func (dg *DiscoveryGateway) topologyWatcherMaxRefreshLag() time.Duration { var lag time.Duration for _, tw := range dg.tabletsWatchers { cellLag := tw.RefreshLag() @@ -173,7 +166,7 @@ func (dg *discoveryGateway) topologyWatcherMaxRefreshLag() time.Duration { } // topologyWatcherChecksum returns a checksum of the topology watcher state -func (dg *discoveryGateway) topologyWatcherChecksum() int64 { +func (dg *DiscoveryGateway) topologyWatcherChecksum() int64 { var checksum int64 for _, tw := range dg.tabletsWatchers { checksum = checksum ^ int64(tw.TopoChecksum()) @@ -181,9 +174,9 @@ func (dg *discoveryGateway) topologyWatcherChecksum() int64 { return checksum } -// StatsUpdate forwards HealthCheck updates to TabletStatsCache and MasterBuffer. -// It is part of the discovery.HealthCheckStatsListener interface. -func (dg *discoveryGateway) StatsUpdate(ts *discovery.TabletStats) { +// StatsUpdate forwards LegacyHealthCheck updates to LegacyTabletStatsCache and MasterBuffer. +// It is part of the discovery.LegacyHealthCheckStatsListener interface. +func (dg *DiscoveryGateway) StatsUpdate(ts *discovery.LegacyTabletStats) { dg.tsc.StatsUpdate(ts) if ts.Target.TabletType == topodatapb.TabletType_MASTER { @@ -192,7 +185,7 @@ func (dg *discoveryGateway) StatsUpdate(ts *discovery.TabletStats) { } // WaitForTablets is part of the gateway.Gateway interface. -func (dg *discoveryGateway) WaitForTablets(ctx context.Context, tabletTypesToWait []topodatapb.TabletType) error { +func (dg *DiscoveryGateway) WaitForTablets(ctx context.Context, tabletTypesToWait []topodatapb.TabletType) error { // Skip waiting for tablets if we are not told to do so. if len(tabletTypesToWait) == 0 { return nil @@ -209,7 +202,7 @@ func (dg *discoveryGateway) WaitForTablets(ctx context.Context, tabletTypesToWai // Close shuts down underlying connections. // This function hides the inner implementation. -func (dg *discoveryGateway) Close(ctx context.Context) error { +func (dg *DiscoveryGateway) Close(ctx context.Context) error { dg.buffer.Shutdown() for _, ctw := range dg.tabletsWatchers { ctw.Stop() @@ -219,7 +212,7 @@ func (dg *discoveryGateway) Close(ctx context.Context) error { // CacheStatus returns a list of TabletCacheStatus per // keyspace/shard/tablet_type. -func (dg *discoveryGateway) CacheStatus() TabletCacheStatusList { +func (dg *DiscoveryGateway) CacheStatus() TabletCacheStatusList { dg.mu.RLock() res := make(TabletCacheStatusList, 0, len(dg.statusAggregators)) for _, aggr := range dg.statusAggregators { @@ -235,21 +228,21 @@ func (dg *discoveryGateway) CacheStatus() TabletCacheStatusList { // the middle of a transaction. While returning the error check if it maybe a result of // a resharding event, and set the re-resolve bit and let the upper layers // re-resolve and retry. -func (dg *discoveryGateway) withRetry(ctx context.Context, target *querypb.Target, unused queryservice.QueryService, name string, inTransaction bool, inner func(ctx context.Context, target *querypb.Target, conn queryservice.QueryService) (bool, error)) error { +func (dg *DiscoveryGateway) withRetry(ctx context.Context, target *querypb.Target, unused queryservice.QueryService, name string, inTransaction bool, inner func(ctx context.Context, target *querypb.Target, conn queryservice.QueryService) (bool, error)) error { var tabletLastUsed *topodatapb.Tablet var err error invalidTablets := make(map[string]bool) - if len(allowedTabletTypes) > 0 { + if len(discovery.AllowedTabletTypes) > 0 { var match bool - for _, allowed := range allowedTabletTypes { + for _, allowed := range discovery.AllowedTabletTypes { if allowed == target.TabletType { match = true break } } if !match { - return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "requested tablet type %v is not part of the allowed tablet types for this vtgate: %+v", target.TabletType.String(), allowedTabletTypes) + return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "requested tablet type %v is not part of the allowed tablet types for this vtgate: %+v", target.TabletType.String(), discovery.AllowedTabletTypes) } } @@ -290,7 +283,7 @@ func (dg *discoveryGateway) withRetry(ctx context.Context, target *querypb.Targe shuffleTablets(dg.localCell, tablets) // skip tablets we tried before - var ts *discovery.TabletStats + var ts *discovery.LegacyTabletStats for _, t := range tablets { if _, ok := invalidTablets[t.Key]; !ok { ts = &t @@ -327,7 +320,7 @@ func (dg *discoveryGateway) withRetry(ctx context.Context, target *querypb.Targe return NewShardError(err, target, tabletLastUsed) } -func shuffleTablets(cell string, tablets []discovery.TabletStats) { +func shuffleTablets(cell string, tablets []discovery.LegacyTabletStats) { sameCell, diffCell, sameCellMax := 0, 0, -1 length := len(tablets) @@ -365,7 +358,7 @@ func shuffleTablets(cell string, tablets []discovery.TabletStats) { } } -func nextTablet(cell string, tablets []discovery.TabletStats, offset, length int, sameCell bool) int { +func nextTablet(cell string, tablets []discovery.LegacyTabletStats, offset, length int, sameCell bool) int { for ; offset < length; offset++ { if (tablets[offset].Tablet.Alias.Cell == cell) == sameCell { return offset @@ -374,13 +367,13 @@ func nextTablet(cell string, tablets []discovery.TabletStats, offset, length int return -1 } -func (dg *discoveryGateway) updateStats(target *querypb.Target, startTime time.Time, err error) { +func (dg *DiscoveryGateway) updateStats(target *querypb.Target, startTime time.Time, err error) { elapsed := time.Since(startTime) aggr := dg.getStatsAggregator(target) aggr.UpdateQueryInfo("", target.TabletType, elapsed, err != nil) } -func (dg *discoveryGateway) getStatsAggregator(target *querypb.Target) *TabletStatusAggregator { +func (dg *DiscoveryGateway) getStatsAggregator(target *querypb.Target) *TabletStatusAggregator { key := fmt.Sprintf("%v/%v/%v", target.Keyspace, target.Shard, target.TabletType.String()) // get existing aggregator @@ -415,3 +408,9 @@ func NewShardError(in error, target *querypb.Target, tablet *topodatapb.Tablet) } return in } + +// HealthCheck should never be called on a DiscoveryGateway +// This exists only to satisfy the interface +func (dg *DiscoveryGateway) HealthCheck() *discovery.HealthCheck { + return nil +} diff --git a/go/vt/vtgate/discoverygateway_test.go b/go/vt/vtgate/discoverygateway_test.go index ce7508f96e1..b02ec1fa471 100644 --- a/go/vt/vtgate/discoverygateway_test.go +++ b/go/vt/vtgate/discoverygateway_test.go @@ -37,23 +37,23 @@ import ( ) func TestDiscoveryGatewayExecute(t *testing.T) { - testDiscoveryGatewayGeneric(t, func(dg *discoveryGateway, target *querypb.Target) error { + testDiscoveryGatewayGeneric(t, func(dg *DiscoveryGateway, target *querypb.Target) error { _, err := dg.Execute(context.Background(), target, "query", nil, 0, nil) return err }) - testDiscoveryGatewayTransact(t, func(dg *discoveryGateway, target *querypb.Target) error { + testDiscoveryGatewayTransact(t, func(dg *DiscoveryGateway, target *querypb.Target) error { _, err := dg.Execute(context.Background(), target, "query", nil, 1, nil) return err }) } func TestDiscoveryGatewayExecuteBatch(t *testing.T) { - testDiscoveryGatewayGeneric(t, func(dg *discoveryGateway, target *querypb.Target) error { + testDiscoveryGatewayGeneric(t, func(dg *DiscoveryGateway, target *querypb.Target) error { queries := []*querypb.BoundQuery{{Sql: "query", BindVariables: nil}} _, err := dg.ExecuteBatch(context.Background(), target, queries, false, 0, nil) return err }) - testDiscoveryGatewayTransact(t, func(dg *discoveryGateway, target *querypb.Target) error { + testDiscoveryGatewayTransact(t, func(dg *DiscoveryGateway, target *querypb.Target) error { queries := []*querypb.BoundQuery{{Sql: "query", BindVariables: nil}} _, err := dg.ExecuteBatch(context.Background(), target, queries, false, 1, nil) return err @@ -61,7 +61,7 @@ func TestDiscoveryGatewayExecuteBatch(t *testing.T) { } func TestDiscoveryGatewayExecuteStream(t *testing.T) { - testDiscoveryGatewayGeneric(t, func(dg *discoveryGateway, target *querypb.Target) error { + testDiscoveryGatewayGeneric(t, func(dg *DiscoveryGateway, target *querypb.Target) error { err := dg.StreamExecute(context.Background(), target, "query", nil, 0, nil, func(qr *sqltypes.Result) error { return nil }) @@ -70,33 +70,33 @@ func TestDiscoveryGatewayExecuteStream(t *testing.T) { } func TestDiscoveryGatewayBegin(t *testing.T) { - testDiscoveryGatewayGeneric(t, func(dg *discoveryGateway, target *querypb.Target) error { + testDiscoveryGatewayGeneric(t, func(dg *DiscoveryGateway, target *querypb.Target) error { _, err := dg.Begin(context.Background(), target, nil) return err }) } func TestDiscoveryGatewayCommit(t *testing.T) { - testDiscoveryGatewayTransact(t, func(dg *discoveryGateway, target *querypb.Target) error { + testDiscoveryGatewayTransact(t, func(dg *DiscoveryGateway, target *querypb.Target) error { return dg.Commit(context.Background(), target, 1) }) } func TestDiscoveryGatewayRollback(t *testing.T) { - testDiscoveryGatewayTransact(t, func(dg *discoveryGateway, target *querypb.Target) error { + testDiscoveryGatewayTransact(t, func(dg *DiscoveryGateway, target *querypb.Target) error { return dg.Rollback(context.Background(), target, 1) }) } func TestDiscoveryGatewayBeginExecute(t *testing.T) { - testDiscoveryGatewayGeneric(t, func(dg *discoveryGateway, target *querypb.Target) error { + testDiscoveryGatewayGeneric(t, func(dg *DiscoveryGateway, target *querypb.Target) error { _, _, err := dg.BeginExecute(context.Background(), target, "query", nil, nil) return err }) } func TestDiscoveryGatewayBeginExecuteBatch(t *testing.T) { - testDiscoveryGatewayGeneric(t, func(dg *discoveryGateway, target *querypb.Target) error { + testDiscoveryGatewayGeneric(t, func(dg *DiscoveryGateway, target *querypb.Target) error { queries := []*querypb.BoundQuery{{Sql: "query", BindVariables: nil}} _, _, err := dg.BeginExecuteBatch(context.Background(), target, queries, false, nil) return err @@ -106,7 +106,7 @@ func TestDiscoveryGatewayBeginExecuteBatch(t *testing.T) { func TestDiscoveryGatewayGetTablets(t *testing.T) { keyspace := "ks" shard := "0" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() dg := NewDiscoveryGateway(context.Background(), hc, nil, "local", 2) // replica should only use local ones @@ -131,7 +131,7 @@ func TestDiscoveryGatewayGetTablets(t *testing.T) { } func TestShuffleTablets(t *testing.T) { - ts1 := discovery.TabletStats{ + ts1 := discovery.LegacyTabletStats{ Key: "t1", Tablet: topo.NewTablet(10, "cell1", "host1"), Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -140,7 +140,7 @@ func TestShuffleTablets(t *testing.T) { Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1, CpuUsage: 0.2}, } - ts2 := discovery.TabletStats{ + ts2 := discovery.LegacyTabletStats{ Key: "t2", Tablet: topo.NewTablet(10, "cell1", "host2"), Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -149,7 +149,7 @@ func TestShuffleTablets(t *testing.T) { Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1, CpuUsage: 0.2}, } - ts3 := discovery.TabletStats{ + ts3 := discovery.LegacyTabletStats{ Key: "t3", Tablet: topo.NewTablet(10, "cell2", "host3"), Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -158,7 +158,7 @@ func TestShuffleTablets(t *testing.T) { Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1, CpuUsage: 0.2}, } - ts4 := discovery.TabletStats{ + ts4 := discovery.LegacyTabletStats{ Key: "t4", Tablet: topo.NewTablet(10, "cell2", "host4"), Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, @@ -167,9 +167,9 @@ func TestShuffleTablets(t *testing.T) { Stats: &querypb.RealtimeStats{SecondsBehindMaster: 1, CpuUsage: 0.2}, } - sameCellTablets := []discovery.TabletStats{ts1, ts2} - diffCellTablets := []discovery.TabletStats{ts3, ts4} - mixedTablets := []discovery.TabletStats{ts1, ts2, ts3, ts4} + sameCellTablets := []discovery.LegacyTabletStats{ts1, ts2} + diffCellTablets := []discovery.LegacyTabletStats{ts3, ts4} + mixedTablets := []discovery.LegacyTabletStats{ts1, ts2, ts3, ts4} // repeat shuffling 10 times and every time the same cell tablets should be in the front for i := 0; i < 10; i++ { shuffleTablets("cell1", sameCellTablets) @@ -206,7 +206,7 @@ func TestShuffleTablets(t *testing.T) { func TestDiscoveryGatewayGetTabletsInRegion(t *testing.T) { keyspace := "ks" shard := "0" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() ts := memorytopo.NewServer("local-west", "local-east", "local", "remote") srvTopo := srvtopotest.NewPassthroughSrvTopoServer() srvTopo.TopoServer = ts @@ -236,7 +236,7 @@ func TestDiscoveryGatewayGetTabletsInRegion(t *testing.T) { func TestDiscoveryGatewayGetTabletsWithRegion(t *testing.T) { keyspace := "ks" shard := "0" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() ts := memorytopo.NewServer("local-west", "local-east", "local", "remote") srvTopo := srvtopotest.NewPassthroughSrvTopoServer() srvTopo.TopoServer = ts @@ -264,7 +264,7 @@ func TestDiscoveryGatewayGetTabletsWithRegion(t *testing.T) { } } -func testDiscoveryGatewayGeneric(t *testing.T, f func(dg *discoveryGateway, target *querypb.Target) error) { +func testDiscoveryGatewayGeneric(t *testing.T, f func(dg *DiscoveryGateway, target *querypb.Target) error) { keyspace := "ks" shard := "0" tabletType := topodatapb.TabletType_REPLICA @@ -273,7 +273,7 @@ func testDiscoveryGatewayGeneric(t *testing.T, f func(dg *discoveryGateway, targ Shard: shard, TabletType: tabletType, } - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() dg := NewDiscoveryGateway(context.Background(), hc, nil, "cell", 2) // no tablet @@ -347,7 +347,7 @@ func testDiscoveryGatewayGeneric(t *testing.T, f func(dg *discoveryGateway, targ } } -func testDiscoveryGatewayTransact(t *testing.T, f func(dg *discoveryGateway, target *querypb.Target) error) { +func testDiscoveryGatewayTransact(t *testing.T, f func(dg *DiscoveryGateway, target *querypb.Target) error) { keyspace := "ks" shard := "0" tabletType := topodatapb.TabletType_REPLICA @@ -356,7 +356,7 @@ func testDiscoveryGatewayTransact(t *testing.T, f func(dg *discoveryGateway, tar Shard: shard, TabletType: tabletType, } - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() dg := NewDiscoveryGateway(context.Background(), hc, nil, "cell", 2) // retry error - no retry diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index d0e2acf5986..196399ce2e8 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -841,28 +841,56 @@ func (e *Executor) handleShow(ctx context.Context, safeSession *SafeSession, sql }, nil case "vitess_tablets": var rows [][]sqltypes.Value - stats := e.scatterConn.healthCheck.CacheStatus() - for _, s := range stats { - for _, ts := range s.TabletsStats { - state := "SERVING" - if !ts.Serving { - state = "NOT_SERVING" + if *GatewayImplementation == GatewayImplementationDiscovery { + status := e.scatterConn.GetLegacyHealthCheckCacheStatus() + for _, s := range status { + for _, ts := range s.TabletsStats { + state := "SERVING" + if !ts.Serving { + state = "NOT_SERVING" + } + mtst := ts.Tablet.MasterTermStartTime + mtstStr := "" + if mtst != nil && mtst.Seconds > 0 { + mtstStr = logutil.ProtoToTime(ts.Tablet.MasterTermStartTime).Format(time.RFC3339) + } + rows = append(rows, buildVarCharRow( + s.Cell, + s.Target.Keyspace, + s.Target.Shard, + ts.Target.TabletType.String(), + state, + topoproto.TabletAliasString(ts.Tablet.Alias), + ts.Tablet.Hostname, + mtstStr, + )) } - mtst := ts.Tablet.MasterTermStartTime - mtstStr := "" - if mtst != nil && mtst.Seconds > 0 { - mtstStr = logutil.ProtoToTime(ts.Tablet.MasterTermStartTime).Format(time.RFC3339) + } + } + if *GatewayImplementation == tabletGatewayImplementation { + status := e.scatterConn.GetHealthCheckCacheStatus() + for _, s := range status { + for _, ts := range s.TabletsStats { + state := "SERVING" + if !ts.Serving { + state = "NOT_SERVING" + } + mtst := ts.Tablet.MasterTermStartTime + mtstStr := "" + if mtst != nil && mtst.Seconds > 0 { + mtstStr = logutil.ProtoToTime(ts.Tablet.MasterTermStartTime).Format(time.RFC3339) + } + rows = append(rows, buildVarCharRow( + s.Cell, + s.Target.Keyspace, + s.Target.Shard, + ts.Target.TabletType.String(), + state, + topoproto.TabletAliasString(ts.Tablet.Alias), + ts.Tablet.Hostname, + mtstStr, + )) } - rows = append(rows, buildVarCharRow( - s.Cell, - s.Target.Keyspace, - s.Target.Shard, - ts.Target.TabletType.String(), - state, - topoproto.TabletAliasString(ts.Tablet.Alias), - ts.Tablet.Hostname, - mtstStr, - )) } } return &sqltypes.Result{ diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go index f8621cea6c0..061591fba18 100644 --- a/go/vt/vtgate/executor_framework_test.go +++ b/go/vt/vtgate/executor_framework_test.go @@ -342,8 +342,10 @@ const ( ) func createExecutorEnvUsing(t executorType) (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn.SandboxConn) { + // Use legacy gateway until we can rewrite these tests to use new tabletgateway + *GatewayImplementation = GatewayImplementationDiscovery cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema serv := newSandboxForCells([]string{cell}) @@ -387,7 +389,7 @@ func createExecutorEnv() (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn func createCustomExecutor(vschema string) (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn.SandboxConn) { cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = vschema serv := newSandboxForCells([]string{cell}) @@ -576,7 +578,7 @@ func testQueryLog(t *testing.T, logChan chan interface{}, method, stmtType, sql return logStats } -func newTestResolver(hc discovery.HealthCheck, serv srvtopo.Server, cell string) *Resolver { +func newTestResolver(hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string) *Resolver { sc := newTestScatterConn(hc, serv, cell) srvResolver := srvtopo.NewResolver(serv, sc.gateway, cell) return NewResolver(srvResolver, serv, cell, sc) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index 163d661e78d..398b2f7180a 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -894,7 +894,7 @@ func TestStreamSelectIN(t *testing.T) { func TestSelectScatter(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -927,7 +927,7 @@ func TestSelectScatter(t *testing.T) { func TestSelectScatterPartial(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -987,7 +987,7 @@ func TestSelectScatterPartial(t *testing.T) { func TestStreamSelectScatter(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1024,7 +1024,7 @@ func TestStreamSelectScatter(t *testing.T) { func TestSelectScatterOrderBy(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1094,7 +1094,7 @@ func TestSelectScatterOrderBy(t *testing.T) { func TestSelectScatterOrderByVarChar(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1164,7 +1164,7 @@ func TestSelectScatterOrderByVarChar(t *testing.T) { func TestStreamSelectScatterOrderBy(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1225,7 +1225,7 @@ func TestStreamSelectScatterOrderBy(t *testing.T) { func TestStreamSelectScatterOrderByVarChar(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1288,7 +1288,7 @@ func TestStreamSelectScatterOrderByVarChar(t *testing.T) { func TestSelectScatterAggregate(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1351,7 +1351,7 @@ func TestSelectScatterAggregate(t *testing.T) { func TestStreamSelectScatterAggregate(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1414,7 +1414,7 @@ func TestStreamSelectScatterAggregate(t *testing.T) { func TestSelectScatterLimit(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1486,7 +1486,7 @@ func TestSelectScatterLimit(t *testing.T) { func TestStreamSelectScatterLimit(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema diff --git a/go/vt/vtgate/executor_stream_test.go b/go/vt/vtgate/executor_stream_test.go index ba708f4b724..45cc8068c81 100644 --- a/go/vt/vtgate/executor_stream_test.go +++ b/go/vt/vtgate/executor_stream_test.go @@ -47,7 +47,7 @@ func TestStreamSQLUnsharded(t *testing.T) { func TestStreamSQLSharded(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema diff --git a/go/vt/vtgate/gateway.go b/go/vt/vtgate/gateway.go index 8bf30833372..fe7468efe67 100644 --- a/go/vt/vtgate/gateway.go +++ b/go/vt/vtgate/gateway.go @@ -18,7 +18,6 @@ import ( "time" "golang.org/x/net/context" - "vitess.io/vitess/go/flagutil" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/discovery" @@ -34,19 +33,14 @@ import ( // a query targeted to a keyspace/shard/tablet_type and send it off. var ( - implementation = flag.String("gateway_implementation", "discoverygateway", "The implementation of gateway") - initialTabletTimeout = flag.Duration("gateway_initial_tablet_timeout", 30*time.Second, "At startup, the gateway will wait up to that duration to get one tablet per keyspace/shard/tablettype") - - // KeyspacesToWatch - if provided this specifies which keyspaces should be - // visible to a vtgate. By default the vtgate will allow access to any - // keyspace. - KeyspacesToWatch flagutil.StringListValue + // GatewayImplementation allows you to choose which gateway to use for vtgate routing. Defaults to discoverygateway, other option is tabletgateway + GatewayImplementation = flag.String("gateway_implementation", "tabletgateway", "Allowed values: discoverygateway (deprecated), tabletgateway (default)") + initialTabletTimeout = flag.Duration("gateway_initial_tablet_timeout", 30*time.Second, "At startup, the gateway will wait up to that duration to get one tablet per keyspace/shard/tablettype") + // RetryCount is the number of times a query will be retried on error + // Make this unexported after DiscoveryGateway is deprecated + RetryCount = flag.Int("retry-count", 2, "retry count") ) -func init() { - flag.Var(&KeyspacesToWatch, "keyspaces_to_watch", "Specifies which keyspaces this vtgate should have access to while routing queries or accessing the vschema") -} - // A Gateway is the query processing module for each shard, // which is used by ScatterConn. type Gateway interface { @@ -68,10 +62,13 @@ type Gateway interface { // CacheStatus returns a list of TabletCacheStatus per shard / tablet type. CacheStatus() TabletCacheStatusList + + // HealthCheck returns a reference to the healthCheck being used by this gateway + HealthCheck() *discovery.HealthCheck } // Creator is the factory method which can create the actual gateway object. -type Creator func(ctx context.Context, hc discovery.HealthCheck, serv srvtopo.Server, cell string, retryCount int) Gateway +type Creator func(ctx context.Context, hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string, retryCount int) Gateway var creators = make(map[string]Creator) @@ -85,9 +82,9 @@ func RegisterGatewayCreator(name string, gc Creator) { // GatewayCreator returns the Creator specified by the gateway_implementation flag. func GatewayCreator() Creator { - gc, ok := creators[*implementation] + gc, ok := creators[*GatewayImplementation] if !ok { - log.Exitf("No gateway registered as %s", *implementation) + log.Exitf("No gateway registered as %s", *GatewayImplementation) } return gc } diff --git a/go/vt/vtgate/grpc_discovery_test.go b/go/vt/vtgate/grpc_discovery_test.go index 03d22a810ad..61b12aa47e2 100644 --- a/go/vt/vtgate/grpc_discovery_test.go +++ b/go/vt/vtgate/grpc_discovery_test.go @@ -62,7 +62,7 @@ func TestGRPCDiscovery(t *testing.T) { // VTGate: create the discovery healthcheck, and the gateway. // Wait for the right tablets to be present. - hc := discovery.NewHealthCheck(10*time.Second, 2*time.Minute) + hc := discovery.NewLegacyHealthCheck(10*time.Second, 2*time.Minute) rs := srvtopo.NewResilientServer(ts, "TestGRPCDiscovery") dg := NewDiscoveryGateway(context.Background(), hc, rs, cell, 2) hc.AddTablet(&topodatapb.Tablet{ diff --git a/go/vt/vtgate/plan_executor_select_test.go b/go/vt/vtgate/plan_executor_select_test.go index 4a8caea274e..5c625698ec6 100644 --- a/go/vt/vtgate/plan_executor_select_test.go +++ b/go/vt/vtgate/plan_executor_select_test.go @@ -871,7 +871,7 @@ func TestPlanStreamSelectIN(t *testing.T) { func TestPlanSelectScatter(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -904,7 +904,7 @@ func TestPlanSelectScatter(t *testing.T) { func TestPlanSelectScatterPartial(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -964,7 +964,7 @@ func TestPlanSelectScatterPartial(t *testing.T) { func TestPlanStreamSelectScatter(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1001,7 +1001,7 @@ func TestPlanStreamSelectScatter(t *testing.T) { func TestPlanSelectScatterOrderBy(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1071,7 +1071,7 @@ func TestPlanSelectScatterOrderBy(t *testing.T) { func TestPlanSelectScatterOrderByVarChar(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1141,7 +1141,7 @@ func TestPlanSelectScatterOrderByVarChar(t *testing.T) { func TestPlanStreamSelectScatterOrderBy(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1202,7 +1202,7 @@ func TestPlanStreamSelectScatterOrderBy(t *testing.T) { func TestPlanStreamSelectScatterOrderByVarChar(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1265,7 +1265,7 @@ func TestPlanStreamSelectScatterOrderByVarChar(t *testing.T) { func TestPlanSelectScatterAggregate(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1328,7 +1328,7 @@ func TestPlanSelectScatterAggregate(t *testing.T) { func TestPlanStreamSelectScatterAggregate(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1391,7 +1391,7 @@ func TestPlanStreamSelectScatterAggregate(t *testing.T) { func TestPlanSelectScatterLimit(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -1463,7 +1463,7 @@ func TestPlanSelectScatterLimit(t *testing.T) { func TestPlanStreamSelectScatterLimit(t *testing.T) { // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema diff --git a/go/vt/vtgate/scatter_conn.go b/go/vt/vtgate/scatter_conn.go index d3657aab4c7..663bf20ac41 100644 --- a/go/vt/vtgate/scatter_conn.go +++ b/go/vt/vtgate/scatter_conn.go @@ -48,7 +48,7 @@ type ScatterConn struct { tabletCallErrorCount *stats.CountersWithMultiLabels txConn *TxConn gateway Gateway - healthCheck discovery.HealthCheck + legacyHealthCheck discovery.LegacyHealthCheck } // shardActionFunc defines the contract for a shard action @@ -68,8 +68,30 @@ type shardActionFunc func(rs *srvtopo.ResolvedShard, i int) error // the results and errors for the caller. type shardActionTransactionFunc func(rs *srvtopo.ResolvedShard, i int, shouldBegin bool, transactionID int64) (int64, error) +// LegacyNewScatterConn creates a new ScatterConn. +func LegacyNewScatterConn(statsName string, txConn *TxConn, gw Gateway, hc discovery.LegacyHealthCheck) *ScatterConn { + tabletCallErrorCountStatsName := "" + if statsName != "" { + tabletCallErrorCountStatsName = statsName + "ErrorCount" + } + return &ScatterConn{ + timings: stats.NewMultiTimings( + statsName, + "Scatter connection timings", + []string{"Operation", "Keyspace", "ShardName", "DbType"}), + tabletCallErrorCount: stats.NewCountersWithMultiLabels( + tabletCallErrorCountStatsName, + "Error count from tablet calls in scatter conns", + []string{"Operation", "Keyspace", "ShardName", "DbType"}), + txConn: txConn, + gateway: gw, + legacyHealthCheck: hc, + } +} + // NewScatterConn creates a new ScatterConn. -func NewScatterConn(statsName string, txConn *TxConn, gw Gateway, hc discovery.HealthCheck) *ScatterConn { +func NewScatterConn(statsName string, txConn *TxConn, gw *TabletGateway) *ScatterConn { + // this only works with TabletGateway tabletCallErrorCountStatsName := "" if statsName != "" { tabletCallErrorCountStatsName = statsName + "ErrorCount" @@ -83,9 +105,10 @@ func NewScatterConn(statsName string, txConn *TxConn, gw Gateway, hc discovery.H tabletCallErrorCountStatsName, "Error count from tablet calls in scatter conns", []string{"Operation", "Keyspace", "ShardName", "DbType"}), - txConn: txConn, - gateway: gw, - healthCheck: hc, + txConn: txConn, + gateway: gw, + // gateway has a reference to healthCheck so we don't need this any more + legacyHealthCheck: nil, } } @@ -419,6 +442,23 @@ func (stc *ScatterConn) GetGatewayCacheStatus() TabletCacheStatusList { return stc.gateway.CacheStatus() } +// GetLegacyHealthCheckCacheStatus returns a displayable version of the HealthCheck cache. +func (stc *ScatterConn) GetLegacyHealthCheckCacheStatus() discovery.LegacyTabletsCacheStatusList { + if stc.legacyHealthCheck != nil { + return stc.legacyHealthCheck.CacheStatus() + } + return nil +} + +// GetHealthCheckCacheStatus returns a displayable version of the HealthCheck cache. +func (stc *ScatterConn) GetHealthCheckCacheStatus() discovery.TabletsCacheStatusList { + gw, ok := stc.gateway.(*TabletGateway) + if ok { + return gw.HealthCheck().CacheStatus() + } + return nil +} + // multiGo performs the requested 'action' on the specified // shards in parallel. This does not handle any transaction state. // The action function must match the shardActionFunc2 signature. diff --git a/go/vt/vtgate/scatter_conn_test.go b/go/vt/vtgate/scatter_conn_test.go index ede85cc5cfc..b14deea5890 100644 --- a/go/vt/vtgate/scatter_conn_test.go +++ b/go/vt/vtgate/scatter_conn_test.go @@ -119,7 +119,7 @@ func verifyScatterConnError(t *testing.T, err error, wantErr string, wantCode vt } func testScatterConnGeneric(t *testing.T, name string, f func(sc *ScatterConn, shards []string) (*sqltypes.Result, error)) { - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() // no shard s := createSandbox(name) @@ -230,7 +230,7 @@ func TestMaxMemoryRows(t *testing.T) { defer func() { *maxMemoryRows = save }() createSandbox("TestMaxMemoryRows") - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() sc := newTestScatterConn(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "0", 1, "TestMaxMemoryRows", "0", topodatapb.TabletType_REPLICA, true, 1, nil) sbc1 := hc.AddTestTablet("aa", "1", 1, "TestMaxMemoryRows", "1", topodatapb.TabletType_REPLICA, true, 1, nil) @@ -277,7 +277,7 @@ func TestMaxMemoryRows(t *testing.T) { func TestMultiExecs(t *testing.T) { createSandbox("TestMultiExecs") - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() sc := newTestScatterConn(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "0", 1, "TestMultiExecs", "0", topodatapb.TabletType_REPLICA, true, 1, nil) sbc1 := hc.AddTestTablet("aa", "1", 1, "TestMultiExecs", "1", topodatapb.TabletType_REPLICA, true, 1, nil) @@ -369,7 +369,7 @@ func TestMultiExecs(t *testing.T) { func TestScatterConnStreamExecuteSendError(t *testing.T) { createSandbox("TestScatterConnStreamExecuteSendError") - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() sc := newTestScatterConn(hc, new(sandboxTopo), "aa") hc.AddTestTablet("aa", "0", 1, "TestScatterConnStreamExecuteSendError", "0", topodatapb.TabletType_REPLICA, true, 1, nil) res := srvtopo.NewResolver(&sandboxTopo{}, sc.gateway, "aa") @@ -389,7 +389,7 @@ func TestScatterConnStreamExecuteSendError(t *testing.T) { func TestScatterConnQueryNotInTransaction(t *testing.T) { s := createSandbox("TestScatterConnQueryNotInTransaction") - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() // case 1: read query (not in transaction) followed by write query, not in the same shard. hc.Reset() @@ -543,7 +543,7 @@ func TestScatterConnQueryNotInTransaction(t *testing.T) { func TestScatterConnSingleDB(t *testing.T) { createSandbox("TestScatterConnSingleDB") - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() hc.Reset() sc := newTestScatterConn(hc, new(sandboxTopo), "aa") @@ -648,11 +648,11 @@ func TestAppendResult(t *testing.T) { } } -func newTestScatterConn(hc discovery.HealthCheck, serv srvtopo.Server, cell string) *ScatterConn { +func newTestScatterConn(hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string) *ScatterConn { // The topo.Server is used to start watching the cells described // in '-cells_to_watch' command line parameter, which is // empty by default. So it's unused in this test, set to nil. gw := GatewayCreator()(context.Background(), hc, serv, cell, 3) tc := NewTxConn(gw, vtgatepb.TransactionMode_TWOPC) - return NewScatterConn("", tc, gw, hc) + return LegacyNewScatterConn("", tc, gw, hc) } diff --git a/go/vt/vtgate/tabletgateway.go b/go/vt/vtgate/tabletgateway.go new file mode 100644 index 00000000000..255c492326a --- /dev/null +++ b/go/vt/vtgate/tabletgateway.go @@ -0,0 +1,332 @@ +/* +Copyright 2019 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 vtgate + +import ( + "fmt" + "math/rand" + "sort" + "sync" + "time" + + "vitess.io/vitess/go/vt/topo/topoproto" + + "golang.org/x/net/context" + + "vitess.io/vitess/go/vt/discovery" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/srvtopo" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vtgate/buffer" + "vitess.io/vitess/go/vt/vttablet/queryservice" + + querypb "vitess.io/vitess/go/vt/proto/query" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" +) + +const ( + tabletGatewayImplementation = "tabletgateway" +) + +func init() { + RegisterGatewayCreator(tabletGatewayImplementation, createTabletGateway) +} + +// TabletGateway implements the Gateway interface. +// This implementation uses the new healthcheck module. +type TabletGateway struct { + queryservice.QueryService + hc *discovery.HealthCheck + srvTopoServer srvtopo.Server + localCell string + retryCount int + + // mu protects the fields of this group. + mu sync.Mutex + // statusAggregators is a map indexed by the key + // keyspace/shard/tablet_type. + statusAggregators map[string]*TabletStatusAggregator + + // buffer, if enabled, buffers requests during a detected MASTER failover. + buffer *buffer.Buffer +} + +func createTabletGateway(ctx context.Context, unused discovery.LegacyHealthCheck, serv srvtopo.Server, + cell string, unused2 int) Gateway { + return NewTabletGateway(ctx, serv, cell) +} + +// NewTabletGateway creates and returns a new TabletGateway +func NewTabletGateway(ctx context.Context, serv srvtopo.Server, localCell string) *TabletGateway { + var topoServer *topo.Server + if serv != nil { + var err error + topoServer, err = serv.GetTopoServer() + if err != nil { + log.Exitf("Unable to create new TabletGateway: %v", err) + } + } + hc := discovery.NewHealthCheck(ctx, *HealthCheckRetryDelay, *HealthCheckTimeout, topoServer, localCell) + + gw := &TabletGateway{ + hc: hc, + srvTopoServer: serv, + localCell: localCell, + retryCount: *RetryCount, + statusAggregators: make(map[string]*TabletStatusAggregator), + buffer: buffer.New(), + } + // subscribe to healthcheck updates so that buffer can be notified if needed + // we run this in a separate goroutine so that normal processing doesn't need to block + hcChan := hc.Subscribe() + bufferCtx, bufferCancel := context.WithCancel(ctx) + go func(ctx context.Context, c chan *discovery.TabletHealth, buffer *buffer.Buffer) { + for { + select { + case <-ctx.Done(): + return + case result := <-hcChan: + if result == nil { + // If result is nil it must mean the channel has been closed. Stop goroutine in that case + bufferCancel() + return + } + if result.Target.TabletType == topodatapb.TabletType_MASTER { + buffer.ProcessMasterHealth(result) + } + } + } + }(bufferCtx, hcChan, gw.buffer) + gw.QueryService = queryservice.Wrap(nil, gw.withRetry) + return gw +} + +// RegisterStats registers the stats to export the lag since the last refresh +// and the checksum of the topology +func (gw *TabletGateway) RegisterStats() { + gw.hc.RegisterStats() +} + +// WaitForTablets is part of the Gateway interface. +func (gw *TabletGateway) WaitForTablets(ctx context.Context, tabletTypesToWait []topodatapb.TabletType) error { + // Skip waiting for tablets if we are not told to do so. + if len(tabletTypesToWait) == 0 { + return nil + } + + // Finds the targets to look for. + targets, err := srvtopo.FindAllTargets(ctx, gw.srvTopoServer, gw.localCell, tabletTypesToWait) + if err != nil { + return err + } + return gw.hc.WaitForAllServingTablets(ctx, targets) +} + +// Close shuts down underlying connections. +// This function hides the inner implementation. +func (gw *TabletGateway) Close(ctx context.Context) error { + gw.buffer.Shutdown() + return gw.hc.Close() +} + +// CacheStatus returns a list of TabletCacheStatus per +// keyspace/shard/tablet_type. +func (gw *TabletGateway) CacheStatus() TabletCacheStatusList { + gw.mu.Lock() + res := make(TabletCacheStatusList, 0, len(gw.statusAggregators)) + for _, aggr := range gw.statusAggregators { + res = append(res, aggr.GetCacheStatus()) + } + gw.mu.Unlock() + sort.Sort(res) + return res +} + +// withRetry gets available connections and executes the action. If there are retryable errors, +// it retries retryCount times before failing. It does not retry if the connection is in +// the middle of a transaction. While returning the error check if it maybe a result of +// a resharding event, and set the re-resolve bit and let the upper layers +// re-resolve and retry. +func (gw *TabletGateway) withRetry(ctx context.Context, target *querypb.Target, unused queryservice.QueryService, + name string, inTransaction bool, inner func(ctx context.Context, target *querypb.Target, conn queryservice.QueryService) (bool, error)) error { + var tabletLastUsed *topodatapb.Tablet + var err error + invalidTablets := make(map[string]bool) + + if len(discovery.AllowedTabletTypes) > 0 { + var match bool + for _, allowed := range discovery.AllowedTabletTypes { + if allowed == target.TabletType { + match = true + break + } + } + if !match { + return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "requested tablet type %v is not part of the allowed tablet types for this vtgate: %+v", target.TabletType.String(), discovery.AllowedTabletTypes) + } + } + + bufferedOnce := false + for i := 0; i < gw.retryCount+1; i++ { + // Check if we should buffer MASTER queries which failed due to an ongoing + // failover. + // Note: We only buffer once and only "!inTransaction" queries i.e. + // a) no transaction is necessary (e.g. critical reads) or + // b) no transaction was created yet. + if !bufferedOnce && !inTransaction && target.TabletType == topodatapb.TabletType_MASTER { + // The next call blocks if we should buffer during a failover. + retryDone, bufferErr := gw.buffer.WaitForFailoverEnd(ctx, target.Keyspace, target.Shard, err) + if bufferErr != nil { + // Buffering failed e.g. buffer is already full. Do not retry. + err = vterrors.Errorf( + vterrors.Code(bufferErr), + "failed to automatically buffer and retry failed request during failover: %v original err (type=%T): %v", + bufferErr, err, err) + break + } + + // Request may have been buffered. + if retryDone != nil { + // We're going to retry this request as part of a buffer drain. + // Notify the buffer after we retried. + defer retryDone() + bufferedOnce = true + } + } + + tablets := gw.hc.GetHealthyTabletStats(target) + if len(tablets) == 0 { + // fail fast if there is no tablet + err = vterrors.New(vtrpcpb.Code_UNAVAILABLE, "no valid tablet") + break + } + gw.shuffleTablets(gw.localCell, tablets) + + var th *discovery.TabletHealth + // skip tablets we tried before + for _, t := range tablets { + tabletLastUsed = t.Tablet + if _, ok := invalidTablets[topoproto.TabletAliasString(tabletLastUsed.Alias)]; !ok { + th = t + break + } else { + tabletLastUsed = nil + } + } + if tabletLastUsed == nil { + // do not override error from last attempt. + if err == nil { + err = vterrors.New(vtrpcpb.Code_UNAVAILABLE, "no available connection") + } + break + } + + // execute + if th.Conn == nil { + err = vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "no connection for tablet %v", tabletLastUsed) + invalidTablets[topoproto.TabletAliasString(tabletLastUsed.Alias)] = true + continue + } + + startTime := time.Now() + var canRetry bool + canRetry, err = inner(ctx, target, th.Conn) + gw.updateStats(target, startTime, err) + if canRetry { + invalidTablets[topoproto.TabletAliasString(tabletLastUsed.Alias)] = true + continue + } + break + } + return NewShardError(err, target, tabletLastUsed) +} + +func (gw *TabletGateway) updateStats(target *querypb.Target, startTime time.Time, err error) { + elapsed := time.Since(startTime) + aggr := gw.getStatsAggregator(target) + aggr.UpdateQueryInfo("", target.TabletType, elapsed, err != nil) +} + +func (gw *TabletGateway) getStatsAggregator(target *querypb.Target) *TabletStatusAggregator { + key := fmt.Sprintf("%v/%v/%v", target.Keyspace, target.Shard, target.TabletType.String()) + + // get existing aggregator + gw.mu.Lock() + defer gw.mu.Unlock() + aggr, ok := gw.statusAggregators[key] + if ok { + return aggr + } + // create a new one if it doesn't exist yet + aggr = NewTabletStatusAggregator(target.Keyspace, target.Shard, target.TabletType, key) + gw.statusAggregators[key] = aggr + return aggr +} + +func (gw *TabletGateway) shuffleTablets(cell string, tablets []*discovery.TabletHealth) { + sameCell, diffCell, sameCellMax := 0, 0, -1 + length := len(tablets) + + // move all same cell tablets to the front, this is O(n) + for { + sameCellMax = diffCell - 1 + sameCell = gw.nextTablet(cell, tablets, sameCell, length, true) + diffCell = gw.nextTablet(cell, tablets, diffCell, length, false) + // either no more diffs or no more same cells should stop the iteration + if sameCell < 0 || diffCell < 0 { + break + } + + if sameCell < diffCell { + // fast forward the `sameCell` lookup to `diffCell + 1`, `diffCell` unchanged + sameCell = diffCell + 1 + } else { + // sameCell > diffCell, swap needed + tablets[sameCell], tablets[diffCell] = tablets[diffCell], tablets[sameCell] + sameCell++ + diffCell++ + } + } + + //shuffle in same cell tablets + for i := sameCellMax; i > 0; i-- { + swap := rand.Intn(i + 1) + tablets[i], tablets[swap] = tablets[swap], tablets[i] + } + + //shuffle in diff cell tablets + for i, diffCellMin := length-1, sameCellMax+1; i > diffCellMin; i-- { + swap := rand.Intn(i-sameCellMax) + diffCellMin + tablets[i], tablets[swap] = tablets[swap], tablets[i] + } +} + +func (gw *TabletGateway) nextTablet(cell string, tablets []*discovery.TabletHealth, offset, length int, sameCell bool) int { + for ; offset < length; offset++ { + if (tablets[offset].Tablet.Alias.Cell == cell) == sameCell { + return offset + } + } + return -1 +} + +// HealthCheck satisfies the Gateway interface +func (gw *TabletGateway) HealthCheck() *discovery.HealthCheck { + return gw.hc +} diff --git a/go/vt/vtgate/tx_conn_test.go b/go/vt/vtgate/tx_conn_test.go index b7a945a2642..e45d207d946 100644 --- a/go/vt/vtgate/tx_conn_test.go +++ b/go/vt/vtgate/tx_conn_test.go @@ -914,7 +914,7 @@ func TestTxConnMultiGoTargets(t *testing.T) { func newTestTxConnEnv(t *testing.T, name string) (sc *ScatterConn, sbc0, sbc1 *sandboxconn.SandboxConn, rss0, rss1, rss01 []*srvtopo.ResolvedShard) { createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() sc = newTestScatterConn(hc, new(sandboxTopo), "aa") sbc0 = hc.AddTestTablet("aa", "0", 1, name, "0", topodatapb.TabletType_MASTER, true, 1, nil) sbc1 = hc.AddTestTablet("aa", "1", 1, name, "1", topodatapb.TabletType_MASTER, true, 1, nil) diff --git a/go/vt/vtgate/vstream_manager_test.go b/go/vt/vtgate/vstream_manager_test.go index 94aeea2a1ed..f13e4f71309 100644 --- a/go/vt/vtgate/vstream_manager_test.go +++ b/go/vt/vtgate/vstream_manager_test.go @@ -40,7 +40,7 @@ func TestVStreamEvents(t *testing.T) { name := "TestVStream" _ = createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() vsm := newTestVStreamManager(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil) @@ -114,7 +114,7 @@ func TestVStreamChunks(t *testing.T) { name := "TestVStream" _ = createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() vsm := newTestVStreamManager(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil) sbc1 := hc.AddTestTablet("aa", "1.1.1.1", 1002, name, "20-40", topodatapb.TabletType_MASTER, true, 1, nil) @@ -184,7 +184,7 @@ func TestVStreamMulti(t *testing.T) { name := "TestVStream" _ = createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() vsm := newTestVStreamManager(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil) sbc1 := hc.AddTestTablet("aa", "1.1.1.1", 1002, name, "20-40", topodatapb.TabletType_MASTER, true, 1, nil) @@ -243,7 +243,7 @@ func TestVStreamRetry(t *testing.T) { name := "TestVStream" _ = createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() vsm := newTestVStreamManager(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil) @@ -282,7 +282,7 @@ func TestVStreamHeartbeat(t *testing.T) { name := "TestVStream" _ = createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() vsm := newTestVStreamManager(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil) @@ -330,7 +330,7 @@ func TestVStreamJournalOneToMany(t *testing.T) { name := "TestVStream" _ = createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() vsm := newTestVStreamManager(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil) sbc1 := hc.AddTestTablet("aa", "1.1.1.1", 1002, name, "-10", topodatapb.TabletType_MASTER, true, 1, nil) @@ -435,7 +435,7 @@ func TestVStreamJournalManyToOne(t *testing.T) { // Variable names are maintained like in OneToMany, but order is different. name := "TestVStream" _ = createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() vsm := newTestVStreamManager(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil) sbc1 := hc.AddTestTablet("aa", "1.1.1.1", 1002, name, "-10", topodatapb.TabletType_MASTER, true, 1, nil) @@ -544,7 +544,7 @@ func TestVStreamJournalNoMatch(t *testing.T) { name := "TestVStream" _ = createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() vsm := newTestVStreamManager(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil) @@ -670,7 +670,7 @@ func TestVStreamJournalPartialMatch(t *testing.T) { // Variable names are maintained like in OneToMany, but order is different.1 name := "TestVStream" _ = createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() vsm := newTestVStreamManager(hc, new(sandboxTopo), "aa") _ = hc.AddTestTablet("aa", "1.1.1.1", 1002, name, "-10", topodatapb.TabletType_MASTER, true, 1, nil) sbc2 := hc.AddTestTablet("aa", "1.1.1.1", 1003, name, "10-20", topodatapb.TabletType_MASTER, true, 1, nil) @@ -748,7 +748,7 @@ func TestVStreamJournalPartialMatch(t *testing.T) { func TestResolveVStreamParams(t *testing.T) { name := "TestVStream" _ = createSandbox(name) - hc := discovery.NewFakeHealthCheck() + hc := discovery.NewFakeLegacyHealthCheck() vsm := newTestVStreamManager(hc, new(sandboxTopo), "aa") testcases := []struct { input *binlogdatapb.VGtid @@ -872,7 +872,7 @@ func TestResolveVStreamParams(t *testing.T) { } } -func newTestVStreamManager(hc discovery.HealthCheck, serv srvtopo.Server, cell string) *vstreamManager { +func newTestVStreamManager(hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string) *vstreamManager { gw := NewDiscoveryGateway(context.Background(), hc, serv, cell, 3) srvResolver := srvtopo.NewResolver(serv, gw, cell) return newVStreamManager(srvResolver, serv, cell) diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index 0a92c6dbf95..bf4560241d4 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -60,6 +60,13 @@ var ( _ = flag.Bool("disable_local_gateway", false, "deprecated: if specified, this process will not route any queries to local tablets in the local cell") maxMemoryRows = flag.Int("max_memory_rows", 300000, "Maximum number of rows that will be held in memory for intermediate results as well as the final result.") warnMemoryRows = flag.Int("warn_memory_rows", 30000, "Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented.") + + // TODO(deepthi): change these two vars to unexported and move to healthcheck.go when LegacyHealthcheck is removed + + // HealthCheckRetryDelay is the time to wait before retrying healthcheck + HealthCheckRetryDelay = flag.Duration("healthcheck_retry_delay", 2*time.Millisecond, "health check retry delay") + // HealthCheckTimeout is the timeout on the RPC call to tablets + HealthCheckTimeout = flag.Duration("healthcheck_timeout", time.Minute, "the health check timeout period") ) func getTxMode() vtgatepb.TransactionMode { @@ -121,7 +128,7 @@ type RegisterVTGate func(vtgateservice.VTGateService) var RegisterVTGates []RegisterVTGate // Init initializes VTGate server. -func Init(ctx context.Context, hc discovery.HealthCheck, serv srvtopo.Server, cell string, retryCount int, tabletTypesToWait []topodatapb.TabletType) *VTGate { +func Init(ctx context.Context, serv srvtopo.Server, cell string, tabletTypesToWait []topodatapb.TabletType) *VTGate { if rpcVTGate != nil { log.Fatalf("VTGate already initialized") } @@ -133,7 +140,7 @@ func Init(ctx context.Context, hc discovery.HealthCheck, serv srvtopo.Server, ce // Build objects from low to high level. // Start with the gateway. If we can't reach the topology service, // we can't go on much further, so we log.Fatal out. - gw := GatewayCreator()(ctx, hc, serv, cell, retryCount) + gw := NewTabletGateway(ctx, serv, cell) gw.RegisterStats() if err := WaitForTablets(gw, tabletTypesToWait); err != nil { log.Fatalf("gateway.WaitForTablets failed: %v", err) @@ -141,10 +148,10 @@ func Init(ctx context.Context, hc discovery.HealthCheck, serv srvtopo.Server, ce // If we want to filter keyspaces replace the srvtopo.Server with a // filtering server - if len(KeyspacesToWatch) > 0 { - log.Infof("Keyspace filtering enabled, selecting %v", KeyspacesToWatch) + if len(discovery.KeyspacesToWatch) > 0 { + log.Infof("Keyspace filtering enabled, selecting %v", discovery.KeyspacesToWatch) var err error - serv, err = srvtopo.NewKeyspaceFilteringServer(serv, KeyspacesToWatch) + serv, err = srvtopo.NewKeyspaceFilteringServer(serv, discovery.KeyspacesToWatch) if err != nil { log.Fatalf("Unable to construct SrvTopo server: %v", err.Error()) } @@ -152,7 +159,7 @@ func Init(ctx context.Context, hc discovery.HealthCheck, serv srvtopo.Server, ce tc := NewTxConn(gw, getTxMode()) // ScatterConn depends on TxConn to perform forced rollbacks. - sc := NewScatterConn("VttabletCall", tc, gw, hc) + sc := NewScatterConn("VttabletCall", tc, gw) srvResolver := srvtopo.NewResolver(serv, gw, cell) resolver := NewResolver(srvResolver, serv, cell, sc) vsm := newVStreamManager(srvResolver, serv, cell) @@ -200,7 +207,7 @@ func Init(ctx context.Context, hc discovery.HealthCheck, serv srvtopo.Server, ce log.Fatalf("error initializing query logger: %v", err) } - initAPI(ctx, hc) + initAPI(ctx, gw.hc) return rpcVTGate } @@ -451,3 +458,88 @@ func (vtg *VTGate) HandlePanic(err *error) { errorCounts.Add([]string{"Panic", "Unknown", "Unknown", vtrpcpb.Code_INTERNAL.String()}, 1) } } + +// LegacyInit initializes VTGate server with LegacyHealthCheck +func LegacyInit(ctx context.Context, hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string, retryCount int, tabletTypesToWait []topodatapb.TabletType) *VTGate { + if rpcVTGate != nil { + log.Fatalf("VTGate already initialized") + } + + // vschemaCounters needs to be initialized before planner to + // catch the initial load stats. + vschemaCounters = stats.NewCountersWithSingleLabel("VtgateVSchemaCounts", "Vtgate vschema counts", "changes") + + // Build objects from low to high level. + // Start with the gateway. If we can't reach the topology service, + // we can't go on much further, so we log.Fatal out. + gw := GatewayCreator()(ctx, hc, serv, cell, retryCount) + gw.RegisterStats() + if err := WaitForTablets(gw, tabletTypesToWait); err != nil { + log.Fatalf("gateway.WaitForTablets failed: %v", err) + } + + // If we want to filter keyspaces replace the srvtopo.Server with a + // filtering server + if len(discovery.KeyspacesToWatch) > 0 { + log.Infof("Keyspace filtering enabled, selecting %v", discovery.KeyspacesToWatch) + var err error + serv, err = srvtopo.NewKeyspaceFilteringServer(serv, discovery.KeyspacesToWatch) + if err != nil { + log.Fatalf("Unable to construct SrvTopo server: %v", err.Error()) + } + } + + tc := NewTxConn(gw, getTxMode()) + // ScatterConn depends on TxConn to perform forced rollbacks. + sc := LegacyNewScatterConn("VttabletCall", tc, gw, hc) + srvResolver := srvtopo.NewResolver(serv, gw, cell) + resolver := NewResolver(srvResolver, serv, cell, sc) + vsm := newVStreamManager(srvResolver, serv, cell) + + rpcVTGate = &VTGate{ + executor: NewExecutor(ctx, serv, cell, resolver, *normalizeQueries, *streamBufferSize, *queryPlanCacheSize), + resolver: resolver, + vsm: vsm, + txConn: tc, + gw: gw, + timings: stats.NewMultiTimings( + "VtgateApi", + "VtgateApi timings", + []string{"Operation", "Keyspace", "DbType"}), + rowsReturned: stats.NewCountersWithMultiLabels( + "VtgateApiRowsReturned", + "Rows returned through the VTgate API", + []string{"Operation", "Keyspace", "DbType"}), + + logExecute: logutil.NewThrottledLogger("Execute", 5*time.Second), + logStreamExecute: logutil.NewThrottledLogger("StreamExecute", 5*time.Second), + } + + errorCounts = stats.NewCountersWithMultiLabels("VtgateApiErrorCounts", "Vtgate API error counts per error type", []string{"Operation", "Keyspace", "DbType", "Code"}) + + _ = stats.NewRates("QPSByOperation", stats.CounterForDimension(rpcVTGate.timings, "Operation"), 15, 1*time.Minute) + _ = stats.NewRates("QPSByKeyspace", stats.CounterForDimension(rpcVTGate.timings, "Keyspace"), 15, 1*time.Minute) + _ = stats.NewRates("QPSByDbType", stats.CounterForDimension(rpcVTGate.timings, "DbType"), 15*60/5, 5*time.Second) + + _ = stats.NewRates("ErrorsByOperation", stats.CounterForDimension(errorCounts, "Operation"), 15, 1*time.Minute) + _ = stats.NewRates("ErrorsByKeyspace", stats.CounterForDimension(errorCounts, "Keyspace"), 15, 1*time.Minute) + _ = stats.NewRates("ErrorsByDbType", stats.CounterForDimension(errorCounts, "DbType"), 15, 1*time.Minute) + _ = stats.NewRates("ErrorsByCode", stats.CounterForDimension(errorCounts, "Code"), 15, 1*time.Minute) + + warnings = stats.NewCountersWithSingleLabel("VtGateWarnings", "Vtgate warnings", "type", "IgnoredSet", "ResultsExceeded") + + servenv.OnRun(func() { + for _, f := range RegisterVTGates { + f(rpcVTGate) + } + }) + rpcVTGate.registerDebugHealthHandler() + err := initQueryLogger(rpcVTGate) + if err != nil { + log.Fatalf("error initializing query logger: %v", err) + } + + legacyInitAPI(ctx, hc) + + return rpcVTGate +} diff --git a/go/vt/vtgate/vtgate_test.go b/go/vt/vtgate/vtgate_test.go index 72d30c25851..9d0702c4cf3 100644 --- a/go/vt/vtgate/vtgate_test.go +++ b/go/vt/vtgate/vtgate_test.go @@ -37,7 +37,7 @@ import ( // This file uses the sandbox_test framework. -var hcVTGateTest *discovery.FakeHealthCheck +var hcVTGateTest *discovery.FakeLegacyHealthCheck var executeOptions = &querypb.ExecuteOptions{ IncludedFields: querypb.ExecuteOptions_TYPE_ONLY, @@ -69,12 +69,14 @@ func init() { } } ` - hcVTGateTest = discovery.NewFakeHealthCheck() + hcVTGateTest = discovery.NewFakeLegacyHealthCheck() *transactionMode = "MULTI" + // Use legacy gateway until we can rewrite these tests to use new tabletgateway + *GatewayImplementation = GatewayImplementationDiscovery // The topo.Server is used to start watching the cells described // in '-cells_to_watch' command line parameter, which is // empty by default. So it's unused in this test, set to nil. - Init(context.Background(), hcVTGateTest, new(sandboxTopo), "aa", 10, nil) + LegacyInit(context.Background(), hcVTGateTest, new(sandboxTopo), "aa", 10, nil) *mysqlServerPort = 0 *mysqlAuthServerImpl = "none" diff --git a/go/vt/vttablet/tabletserver/txthrottler/mock_healthcheck_test.go b/go/vt/vttablet/tabletserver/txthrottler/mock_healthcheck_test.go index c29c63a40aa..ecded2d0bc6 100644 --- a/go/vt/vttablet/tabletserver/txthrottler/mock_healthcheck_test.go +++ b/go/vt/vttablet/tabletserver/txthrottler/mock_healthcheck_test.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: vitess.io/vitess/go/vt/discovery (interfaces: HealthCheck) +// Source: vitess.io/vitess/go/vt/discovery (interfaces: LegacyHealthCheck) // Package txthrottler is a generated GoMock package. package txthrottler @@ -13,7 +13,7 @@ import ( queryservice "vitess.io/vitess/go/vt/vttablet/queryservice" ) -// MockHealthCheck is a mock of HealthCheck interface +// MockHealthCheck is a mock of LegacyHealthCheck interface type MockHealthCheck struct { ctrl *gomock.Controller recorder *MockHealthCheckMockRecorder @@ -47,9 +47,9 @@ func (mr *MockHealthCheckMockRecorder) AddTablet(arg0, arg1 interface{}) *gomock } // CacheStatus mocks base method -func (m *MockHealthCheck) CacheStatus() discovery.TabletsCacheStatusList { +func (m *MockHealthCheck) CacheStatus() discovery.LegacyTabletsCacheStatusList { ret := m.ctrl.Call(m, "CacheStatus") - ret0, _ := ret[0].(discovery.TabletsCacheStatusList) + ret0, _ := ret[0].(discovery.LegacyTabletsCacheStatusList) return ret0 } @@ -113,7 +113,7 @@ func (mr *MockHealthCheckMockRecorder) ReplaceTablet(arg0, arg1, arg2 interface{ } // SetListener mocks base method -func (m *MockHealthCheck) SetListener(arg0 discovery.HealthCheckStatsListener, arg1 bool) { +func (m *MockHealthCheck) SetListener(arg0 discovery.LegacyHealthCheckStatsListener, arg1 bool) { m.ctrl.Call(m, "SetListener", arg0, arg1) } diff --git a/go/vt/vttablet/tabletserver/txthrottler/mock_throttler_test.go b/go/vt/vttablet/tabletserver/txthrottler/mock_throttler_test.go index 7604b9b1f43..b760b1a88ae 100644 --- a/go/vt/vttablet/tabletserver/txthrottler/mock_throttler_test.go +++ b/go/vt/vttablet/tabletserver/txthrottler/mock_throttler_test.go @@ -71,7 +71,7 @@ func (mr *MockThrottlerInterfaceMockRecorder) MaxRate() *gomock.Call { } // RecordReplicationLag mocks base method -func (m *MockThrottlerInterface) RecordReplicationLag(arg0 time.Time, arg1 *discovery.TabletStats) { +func (m *MockThrottlerInterface) RecordReplicationLag(arg0 time.Time, arg1 *discovery.LegacyTabletStats) { m.ctrl.Call(m, "RecordReplicationLag", arg0, arg1) } diff --git a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go index 10683619133..001435e92f5 100644 --- a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go +++ b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go @@ -36,7 +36,7 @@ import ( // TxThrottler throttles transactions based on replication lag. // It's a thin wrapper around the throttler found in vitess/go/vt/throttler. -// It uses a discovery.HealthCheck to send replication-lag updates to the wrapped throttler. +// It uses a discovery.LegacyHealthCheck to send replication-lag updates to the wrapped throttler. // // Intended Usage: // // Assuming topoServer is a topo.Server variable pointing to a Vitess topology server. @@ -137,15 +137,15 @@ type ThrottlerInterface interface { Close() MaxRate() int64 SetMaxRate(rate int64) - RecordReplicationLag(time time.Time, ts *discovery.TabletStats) + RecordReplicationLag(time time.Time, ts *discovery.LegacyTabletStats) GetConfiguration() *throttlerdatapb.Configuration UpdateConfiguration(configuration *throttlerdatapb.Configuration, copyZeroValues bool) error ResetConfiguration() } // TopologyWatcherInterface defines the public interface that is implemented by -// discovery.TopologyWatcher. It is only used here to allow mocking out -// go/vt/discovery.TopologyWatcher. +// discovery.LegacyTopologyWatcher. It is only used here to allow mocking out +// go/vt/discovery.LegacyTopologyWatcher. type TopologyWatcherInterface interface { WaitForInitialTopology() error Stop() @@ -158,15 +158,15 @@ type txThrottlerState struct { throttleMu sync.Mutex throttler ThrottlerInterface - healthCheck discovery.HealthCheck + healthCheck discovery.LegacyHealthCheck topologyWatchers []TopologyWatcherInterface } // These vars store the functions used to create the topo server, healthcheck, // topology watchers and go/vt/throttler. These are provided here so that they can be overridden // in tests to generate mocks. -type healthCheckFactoryFunc func() discovery.HealthCheck -type topologyWatcherFactoryFunc func(topoServer *topo.Server, tr discovery.TabletRecorder, cell, keyspace, shard string, refreshInterval time.Duration, topoReadConcurrency int) TopologyWatcherInterface +type healthCheckFactoryFunc func() discovery.LegacyHealthCheck +type topologyWatcherFactoryFunc func(topoServer *topo.Server, tr discovery.LegacyTabletRecorder, cell, keyspace, shard string, refreshInterval time.Duration, topoReadConcurrency int) TopologyWatcherInterface type throttlerFactoryFunc func(name, unit string, threadCount int, maxRate, maxReplicationLag int64) (ThrottlerInterface, error) var ( @@ -180,9 +180,9 @@ func init() { } func resetTxThrottlerFactories() { - healthCheckFactory = discovery.NewDefaultHealthCheck - topologyWatcherFactory = func(topoServer *topo.Server, tr discovery.TabletRecorder, cell, keyspace, shard string, refreshInterval time.Duration, topoReadConcurrency int) TopologyWatcherInterface { - return discovery.NewShardReplicationWatcher(context.Background(), topoServer, tr, cell, keyspace, shard, refreshInterval, topoReadConcurrency) + healthCheckFactory = discovery.NewLegacyDefaultHealthCheck + topologyWatcherFactory = func(topoServer *topo.Server, tr discovery.LegacyTabletRecorder, cell, keyspace, shard string, refreshInterval time.Duration, topoReadConcurrency int) TopologyWatcherInterface { + return discovery.NewLegacyShardReplicationWatcher(context.Background(), topoServer, tr, cell, keyspace, shard, refreshInterval, topoReadConcurrency) } throttlerFactory = func(name, unit string, threadCount int, maxRate, maxReplicationLag int64) (ThrottlerInterface, error) { return throttler.NewThrottler(name, unit, threadCount, maxRate, maxReplicationLag) @@ -278,7 +278,7 @@ func newTxThrottlerState(config *txThrottlerConfig, keyspace, shard string, result.topologyWatchers, topologyWatcherFactory( config.topoServer, - result.healthCheck, /* TabletRecorder */ + result.healthCheck, /* LegacyTabletRecorder */ cell, keyspace, shard, @@ -316,8 +316,8 @@ func (ts *txThrottlerState) deallocateResources() { ts.throttler = nil } -// StatsUpdate is part of the HealthCheckStatsListener interface. -func (ts *txThrottlerState) StatsUpdate(tabletStats *discovery.TabletStats) { +// StatsUpdate is part of the LegacyHealthCheckStatsListener interface. +func (ts *txThrottlerState) StatsUpdate(tabletStats *discovery.LegacyTabletStats) { // Ignore MASTER and RDONLY stats. // We currently do not monitor RDONLY tablets for replication lag. RDONLY tablets are not // candidates for becoming master during failover, and it's acceptable to serve somewhat diff --git a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go index bca23f8b48f..6fea2c65379 100644 --- a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go +++ b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go @@ -17,7 +17,7 @@ limitations under the License. package txthrottler // Commands to generate the mocks for this test. -//go:generate mockgen -destination mock_healthcheck_test.go -package txthrottler vitess.io/vitess/go/vt/discovery HealthCheck +//go:generate mockgen -destination mock_healthcheck_test.go -package txthrottler vitess.io/vitess/go/vt/discovery LegacyHealthCheck //go:generate mockgen -destination mock_throttler_test.go -package txthrottler vitess.io/vitess/go/vt/vttablet/tabletserver/txthrottler ThrottlerInterface //go:generate mockgen -destination mock_topology_watcher_test.go -package txthrottler vitess.io/vitess/go/vt/vttablet/tabletserver/txthrottler TopologyWatcherInterface @@ -56,17 +56,17 @@ func TestEnabledThrottler(t *testing.T) { ts := memorytopo.NewServer("cell1", "cell2") mockHealthCheck := NewMockHealthCheck(mockCtrl) - var hcListener discovery.HealthCheckStatsListener + var hcListener discovery.LegacyHealthCheckStatsListener hcCall1 := mockHealthCheck.EXPECT().SetListener(gomock.Any(), false /* sendDownEvents */) - hcCall1.Do(func(listener discovery.HealthCheckStatsListener, sendDownEvents bool) { + hcCall1.Do(func(listener discovery.LegacyHealthCheckStatsListener, sendDownEvents bool) { // Record the listener we're given. hcListener = listener }) hcCall2 := mockHealthCheck.EXPECT().Close() hcCall2.After(hcCall1) - healthCheckFactory = func() discovery.HealthCheck { return mockHealthCheck } + healthCheckFactory = func() discovery.LegacyHealthCheck { return mockHealthCheck } - topologyWatcherFactory = func(topoServer *topo.Server, tr discovery.TabletRecorder, cell, keyspace, shard string, refreshInterval time.Duration, topoReadConcurrency int) TopologyWatcherInterface { + topologyWatcherFactory = func(topoServer *topo.Server, tr discovery.LegacyTabletRecorder, cell, keyspace, shard string, refreshInterval time.Duration, topoReadConcurrency int) TopologyWatcherInterface { if ts != topoServer { t.Errorf("want: %v, got: %v", ts, topoServer) } @@ -95,7 +95,7 @@ func TestEnabledThrottler(t *testing.T) { call0 := mockThrottler.EXPECT().UpdateConfiguration(gomock.Any(), true /* copyZeroValues */) call1 := mockThrottler.EXPECT().Throttle(0) call1.Return(0 * time.Second) - tabletStats := &discovery.TabletStats{ + tabletStats := &discovery.LegacyTabletStats{ Target: &querypb.Target{ TabletType: topodatapb.TabletType_REPLICA, }, @@ -124,7 +124,7 @@ func TestEnabledThrottler(t *testing.T) { t.Errorf("want: false, got: %v", result) } hcListener.StatsUpdate(tabletStats) - rdonlyTabletStats := &discovery.TabletStats{ + rdonlyTabletStats := &discovery.LegacyTabletStats{ Target: &querypb.Target{ TabletType: topodatapb.TabletType_RDONLY, }, diff --git a/go/vt/worker/executor.go b/go/vt/worker/executor.go index f0e14db21de..9e559552111 100644 --- a/go/vt/worker/executor.go +++ b/go/vt/worker/executor.go @@ -41,7 +41,7 @@ import ( // executor is also used for executing vreplication and RefreshState commands. type executor struct { wr *wrangler.Wrangler - tsc *discovery.TabletStatsCache + tsc *discovery.LegacyTabletStatsCache throttler *throttler.Throttler keyspace string shard string @@ -51,7 +51,7 @@ type executor struct { statsKey []string } -func newExecutor(wr *wrangler.Wrangler, tsc *discovery.TabletStatsCache, throttler *throttler.Throttler, keyspace, shard string, threadID int) *executor { +func newExecutor(wr *wrangler.Wrangler, tsc *discovery.LegacyTabletStatsCache, throttler *throttler.Throttler, keyspace, shard string, threadID int) *executor { return &executor{ wr: wr, tsc: tsc, @@ -113,7 +113,7 @@ func (e *executor) refreshState(ctx context.Context) error { // it fails due to a timeout or a retriable application error. // // executeFetchWithRetries will always get the current MASTER tablet from the -// TabletStatsCache instance. If no MASTER is available, it will keep retrying. +// LegacyTabletStatsCache instance. If no MASTER is available, it will keep retrying. func (e *executor) fetchWithRetries(ctx context.Context, action func(ctx context.Context, tablet *topodatapb.Tablet) error) error { retryDuration := *retryDuration // We should keep retrying up until the retryCtx runs out. @@ -122,10 +122,10 @@ func (e *executor) fetchWithRetries(ctx context.Context, action func(ctx context // Is this current attempt a retry of a previous attempt? isRetry := false for { - var master *discovery.TabletStats + var master *discovery.LegacyTabletStats var err error - // Get the current master from the TabletStatsCache. + // Get the current master from the LegacyTabletStatsCache. masters := e.tsc.GetHealthyTabletStats(e.keyspace, e.shard, topodatapb.TabletType_MASTER) if len(masters) == 0 { e.wr.Logger().Warningf("ExecuteFetch failed for keyspace/shard %v/%v because no MASTER is available; will retry until there is MASTER again", e.keyspace, e.shard) @@ -185,7 +185,7 @@ func (e *executor) fetchWithRetries(ctx context.Context, action func(ctx context } return vterrors.Wrapf(err, "interrupted while trying to run a command on tablet %v", tabletString) case <-time.After(*executeFetchRetryTime): - // Retry 30s after the failure using the current master seen by the HealthCheck. + // Retry 30s after the failure using the current master seen by the LegacyHealthCheck. } isRetry = true } @@ -194,7 +194,7 @@ func (e *executor) fetchWithRetries(ctx context.Context, action func(ctx context // checkError returns true if the error can be ignored and the command // succeeded, false if the error is retryable and a non-nil error if the // command must not be retried. -func (e *executor) checkError(ctx context.Context, err error, isRetry bool, master *discovery.TabletStats) (bool, error) { +func (e *executor) checkError(ctx context.Context, err error, isRetry bool, master *discovery.LegacyTabletStats) (bool, error) { tabletString := fmt.Sprintf("%v (%v/%v)", topoproto.TabletAliasString(master.Tablet.Alias), e.keyspace, e.shard) // first see if it was a context timeout. diff --git a/go/vt/worker/legacy_split_clone.go b/go/vt/worker/legacy_split_clone.go index dd9e568e47a..0cba9d92ce2 100644 --- a/go/vt/worker/legacy_split_clone.go +++ b/go/vt/worker/legacy_split_clone.go @@ -76,13 +76,13 @@ type LegacySplitCloneWorker struct { sourceTablets []*topodatapb.Tablet // healthCheck tracks the health of all MASTER and REPLICA tablets. // It must be closed at the end of the command. - healthCheck discovery.HealthCheck - tsc *discovery.TabletStatsCache - // destinationShardWatchers contains a TopologyWatcher for each destination + healthCheck discovery.LegacyHealthCheck + tsc *discovery.LegacyTabletStatsCache + // destinationShardWatchers contains a LegacyTopologyWatcher for each destination // shard. It updates the list of tablets in the healthcheck if replicas are // added/removed. // Each watcher must be stopped at the end of the command. - destinationShardWatchers []*discovery.TopologyWatcher + destinationShardWatchers []*discovery.LegacyTopologyWatcher // destinationDbNames stores for each destination keyspace/shard the MySQL // database name. // Example Map Entry: test_keyspace/-80 => vt_test_keyspace @@ -222,7 +222,7 @@ func (scw *LegacySplitCloneWorker) Run(ctx context.Context) error { } if scw.healthCheck != nil { if err := scw.healthCheck.Close(); err != nil { - scw.wr.Logger().Errorf2(err, "HealthCheck.Close() failed") + scw.wr.Logger().Errorf2(err, "LegacyHealthCheck.Close() failed") } } @@ -386,10 +386,10 @@ func (scw *LegacySplitCloneWorker) findTargets(ctx context.Context) error { } // Initialize healthcheck and add destination shards to it. - scw.healthCheck = discovery.NewHealthCheck(*healthcheckRetryDelay, *healthCheckTimeout) - scw.tsc = discovery.NewTabletStatsCache(scw.healthCheck, scw.wr.TopoServer(), scw.cell) + scw.healthCheck = discovery.NewLegacyHealthCheck(*healthcheckRetryDelay, *healthCheckTimeout) + scw.tsc = discovery.NewLegacyTabletStatsCache(scw.healthCheck, scw.wr.TopoServer(), scw.cell) for _, si := range scw.destinationShards { - watcher := discovery.NewShardReplicationWatcher(ctx, scw.wr.TopoServer(), scw.healthCheck, + watcher := discovery.NewLegacyShardReplicationWatcher(ctx, scw.wr.TopoServer(), scw.healthCheck, scw.cell, si.Keyspace(), si.ShardName(), *healthCheckTopologyRefresh, discovery.DefaultTopoReadConcurrency) scw.destinationShardWatchers = append(scw.destinationShardWatchers, watcher) @@ -405,7 +405,7 @@ func (scw *LegacySplitCloneWorker) findTargets(ctx context.Context) error { } masters := scw.tsc.GetHealthyTabletStats(si.Keyspace(), si.ShardName(), topodatapb.TabletType_MASTER) if len(masters) == 0 { - return fmt.Errorf("cannot find MASTER tablet for destination shard for %v/%v in HealthCheck: empty TabletStats list", si.Keyspace(), si.ShardName()) + return fmt.Errorf("cannot find MASTER tablet for destination shard for %v/%v in LegacyHealthCheck: empty LegacyTabletStats list", si.Keyspace(), si.ShardName()) } master := masters[0] @@ -421,7 +421,7 @@ func (scw *LegacySplitCloneWorker) findTargets(ctx context.Context) error { scw.wr.Logger().Infof("Using tablet %v as destination master for %v/%v", topoproto.TabletAliasString(master.Tablet.Alias), si.Keyspace(), si.ShardName()) } - scw.wr.Logger().Infof("NOTE: The used master of a destination shard might change over the course of the copy e.g. due to a reparent. The HealthCheck module will track and log master changes and any error message will always refer the actually used master address.") + scw.wr.Logger().Infof("NOTE: The used master of a destination shard might change over the course of the copy e.g. due to a reparent. The LegacyHealthCheck module will track and log master changes and any error message will always refer the actually used master address.") // Set up the throttler for each destination shard. for _, si := range scw.destinationShards { diff --git a/go/vt/worker/split_clone.go b/go/vt/worker/split_clone.go index ce042c7d351..a3422dea6b6 100644 --- a/go/vt/worker/split_clone.go +++ b/go/vt/worker/split_clone.go @@ -97,19 +97,19 @@ type SplitCloneWorker struct { // MASTER tablet, b) get the list of healthy RDONLY tablets and c) track the // replication lag of all REPLICA tablets. // It must be closed at the end of the command. - healthCheck discovery.HealthCheck - tsc *discovery.TabletStatsCache + healthCheck discovery.LegacyHealthCheck + tsc *discovery.LegacyTabletStatsCache // populated during WorkerStateFindTargets, read-only after that sourceTablets []*topodatapb.Tablet lastPos string // contains the GTID position for the source transactions []int64 - // shardWatchers contains a TopologyWatcher for each source and destination + // shardWatchers contains a LegacyTopologyWatcher for each source and destination // shard. It updates the list of tablets in the healthcheck if replicas are // added/removed. // Each watcher must be stopped at the end of the command. - shardWatchers []*discovery.TopologyWatcher + shardWatchers []*discovery.LegacyTopologyWatcher // destinationDbNames stores for each destination keyspace/shard the MySQL // database name. // Example Map Entry: test_keyspace/-80 => vt_test_keyspace @@ -420,7 +420,7 @@ func (scw *SplitCloneWorker) Run(ctx context.Context) error { // After Close returned, we can be sure that it won't call our listener // implementation (method StatsUpdate) anymore. if err := scw.healthCheck.Close(); err != nil { - scw.wr.Logger().Errorf2(err, "HealthCheck.Close() failed") + scw.wr.Logger().Errorf2(err, "LegacyHealthCheck.Close() failed") } } @@ -558,15 +558,15 @@ func (scw *SplitCloneWorker) init(ctx context.Context) error { } // Initialize healthcheck and add destination shards to it. - scw.healthCheck = discovery.NewHealthCheck(*healthcheckRetryDelay, *healthCheckTimeout) + scw.healthCheck = discovery.NewLegacyHealthCheck(*healthcheckRetryDelay, *healthCheckTimeout) scw.tsc = discovery.NewTabletStatsCacheDoNotSetListener(scw.wr.TopoServer(), scw.cell) - // We set sendDownEvents=true because it's required by TabletStatsCache. + // We set sendDownEvents=true because it's required by LegacyTabletStatsCache. scw.healthCheck.SetListener(scw, true /* sendDownEvents */) // Start watchers to get tablets added automatically to healthCheck. allShards := append(scw.sourceShards, scw.destinationShards...) for _, si := range allShards { - watcher := discovery.NewShardReplicationWatcher(ctx, scw.wr.TopoServer(), scw.healthCheck, + watcher := discovery.NewLegacyShardReplicationWatcher(ctx, scw.wr.TopoServer(), scw.healthCheck, scw.cell, si.Keyspace(), si.ShardName(), *healthCheckTopologyRefresh, discovery.DefaultTopoReadConcurrency) scw.shardWatchers = append(scw.shardWatchers, watcher) @@ -851,7 +851,7 @@ func (scw *SplitCloneWorker) findDestinationMasters(ctx context.Context) error { } masters := scw.tsc.GetHealthyTabletStats(si.Keyspace(), si.ShardName(), topodatapb.TabletType_MASTER) if len(masters) == 0 { - return vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "cannot find MASTER tablet for destination shard for %v/%v (in cell: %v) in HealthCheck: empty TabletStats list", si.Keyspace(), si.ShardName(), scw.cell) + return vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "cannot find MASTER tablet for destination shard for %v/%v (in cell: %v) in LegacyHealthCheck: empty LegacyTabletStats list", si.Keyspace(), si.ShardName(), scw.cell) } master := masters[0] @@ -861,7 +861,7 @@ func (scw *SplitCloneWorker) findDestinationMasters(ctx context.Context) error { scw.wr.Logger().Infof("Using tablet %v as destination master for %v/%v", topoproto.TabletAliasString(master.Tablet.Alias), si.Keyspace(), si.ShardName()) } - scw.wr.Logger().Infof("NOTE: The used master of a destination shard might change over the course of the copy e.g. due to a reparent. The HealthCheck module will track and log master changes and any error message will always refer the actually used master address.") + scw.wr.Logger().Infof("NOTE: The used master of a destination shard might change over the course of the copy e.g. due to a reparent. The LegacyHealthCheck module will track and log master changes and any error message will always refer the actually used master address.") return nil } @@ -1355,9 +1355,9 @@ func (scw *SplitCloneWorker) createKeyResolver(td *tabletmanagerdatapb.TableDefi // StatsUpdate receives replication lag updates for each destination master // and forwards them to the respective throttler instance. -// It also forwards any update to the TabletStatsCache to keep it up to date. -// It is part of the discovery.HealthCheckStatsListener interface. -func (scw *SplitCloneWorker) StatsUpdate(ts *discovery.TabletStats) { +// It also forwards any update to the LegacyTabletStatsCache to keep it up to date. +// It is part of the discovery.LegacyHealthCheckStatsListener interface. +func (scw *SplitCloneWorker) StatsUpdate(ts *discovery.LegacyTabletStats) { scw.tsc.StatsUpdate(ts) // Ignore unless REPLICA or RDONLY. diff --git a/go/vt/worker/tablet_provider.go b/go/vt/worker/tablet_provider.go index 2eaddf9e8b3..248eba1de84 100644 --- a/go/vt/worker/tablet_provider.go +++ b/go/vt/worker/tablet_provider.go @@ -73,16 +73,16 @@ func (p *singleTabletProvider) description() string { } // shardTabletProvider returns a random healthy RDONLY tablet for a given -// keyspace and shard. It uses the HealthCheck module to retrieve the tablets. +// keyspace and shard. It uses the LegacyHealthCheck module to retrieve the tablets. type shardTabletProvider struct { - tsc *discovery.TabletStatsCache + tsc *discovery.LegacyTabletStatsCache tracker *TabletTracker keyspace string shard string tabletType topodatapb.TabletType } -func newShardTabletProvider(tsc *discovery.TabletStatsCache, tracker *TabletTracker, keyspace, shard string, tabletType topodatapb.TabletType) *shardTabletProvider { +func newShardTabletProvider(tsc *discovery.LegacyTabletStatsCache, tracker *TabletTracker, keyspace, shard string, tabletType topodatapb.TabletType) *shardTabletProvider { return &shardTabletProvider{tsc, tracker, keyspace, shard, tabletType} } diff --git a/go/vt/worker/tablet_tracker.go b/go/vt/worker/tablet_tracker.go index ba85e4ab092..3e2dee1ba61 100644 --- a/go/vt/worker/tablet_tracker.go +++ b/go/vt/worker/tablet_tracker.go @@ -50,7 +50,7 @@ func NewTabletTracker() *TabletTracker { // Track will pick the least used tablet from "stats", increment its usage by 1 // and return it. // "stats" must not be empty. -func (t *TabletTracker) Track(stats []discovery.TabletStats) *topodatapb.Tablet { +func (t *TabletTracker) Track(stats []discovery.LegacyTabletStats) *topodatapb.Tablet { if len(stats) == 0 { panic("stats must not be empty") } diff --git a/go/vt/worker/tablet_tracker_test.go b/go/vt/worker/tablet_tracker_test.go index cf91138cbce..887ce538f6d 100644 --- a/go/vt/worker/tablet_tracker_test.go +++ b/go/vt/worker/tablet_tracker_test.go @@ -27,25 +27,25 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) -var ts1 = discovery.TabletStats{ +var ts1 = discovery.LegacyTabletStats{ Tablet: topo.NewTablet(10, "cell", "host1"), Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, } -var ts2 = discovery.TabletStats{ +var ts2 = discovery.LegacyTabletStats{ Tablet: topo.NewTablet(20, "cell", "host1"), Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, } -var allTs = []discovery.TabletStats{ts1, ts2} +var allTs = []discovery.LegacyTabletStats{ts1, ts2} func TestTabletsInUse(t *testing.T) { tt := NewTabletTracker() - tt.Track([]discovery.TabletStats{ts1}) + tt.Track([]discovery.LegacyTabletStats{ts1}) if got, want := tt.TabletsInUse(), "cell-0000000010"; got != want { t.Fatalf("TabletsInUse() = %v, want = %v", got, want) } - tt.Track([]discovery.TabletStats{ts2}) + tt.Track([]discovery.LegacyTabletStats{ts2}) if got, want := tt.TabletsInUse(), "cell-0000000010 cell-0000000020"; got != want { t.Fatalf("TabletsInUse() = %v, want = %v", got, want) } diff --git a/go/vt/worker/topo_utils.go b/go/vt/worker/topo_utils.go index c83f90a2fa1..a125910a9eb 100644 --- a/go/vt/worker/topo_utils.go +++ b/go/vt/worker/topo_utils.go @@ -47,12 +47,12 @@ var ( // Since we don't want to use them all, we require at least // minHealthyRdonlyTablets servers to be healthy. // May block up to -wait_for_healthy_rdonly_tablets_timeout. -func FindHealthyTablet(ctx context.Context, wr *wrangler.Wrangler, tsc *discovery.TabletStatsCache, cell, keyspace, shard string, minHealthyRdonlyTablets int, tabletType topodatapb.TabletType) (*topodatapb.TabletAlias, error) { +func FindHealthyTablet(ctx context.Context, wr *wrangler.Wrangler, tsc *discovery.LegacyTabletStatsCache, cell, keyspace, shard string, minHealthyRdonlyTablets int, tabletType topodatapb.TabletType) (*topodatapb.TabletAlias, error) { if tsc == nil { // No healthcheck instance provided. Create one. - healthCheck := discovery.NewHealthCheck(*healthcheckRetryDelay, *healthCheckTimeout) - tsc = discovery.NewTabletStatsCache(healthCheck, wr.TopoServer(), cell) - watcher := discovery.NewShardReplicationWatcher(ctx, wr.TopoServer(), healthCheck, cell, keyspace, shard, *healthCheckTopologyRefresh, discovery.DefaultTopoReadConcurrency) + healthCheck := discovery.NewLegacyHealthCheck(*healthcheckRetryDelay, *healthCheckTimeout) + tsc = discovery.NewLegacyTabletStatsCache(healthCheck, wr.TopoServer(), cell) + watcher := discovery.NewLegacyShardReplicationWatcher(ctx, wr.TopoServer(), healthCheck, cell, keyspace, shard, *healthCheckTopologyRefresh, discovery.DefaultTopoReadConcurrency) defer watcher.Stop() defer healthCheck.Close() } @@ -67,7 +67,7 @@ func FindHealthyTablet(ctx context.Context, wr *wrangler.Wrangler, tsc *discover return healthyTablets[index].Tablet.Alias, nil } -func waitForHealthyTablets(ctx context.Context, wr *wrangler.Wrangler, tsc *discovery.TabletStatsCache, cell, keyspace, shard string, minHealthyRdonlyTablets int, timeout time.Duration, tabletType topodatapb.TabletType) ([]discovery.TabletStats, error) { +func waitForHealthyTablets(ctx context.Context, wr *wrangler.Wrangler, tsc *discovery.LegacyTabletStatsCache, cell, keyspace, shard string, minHealthyRdonlyTablets int, timeout time.Duration, tabletType topodatapb.TabletType) ([]discovery.LegacyTabletStats, error) { busywaitCtx, busywaitCancel := context.WithTimeout(ctx, timeout) defer busywaitCancel() @@ -81,7 +81,7 @@ func waitForHealthyTablets(ctx context.Context, wr *wrangler.Wrangler, tsc *disc return nil, vterrors.Wrapf(err, "error waiting for %v tablets for (%v,%v/%v)", tabletType, cell, keyspace, shard) } - var healthyTablets []discovery.TabletStats + var healthyTablets []discovery.LegacyTabletStats for { select { case <-busywaitCtx.Done(): @@ -115,7 +115,7 @@ func waitForHealthyTablets(ctx context.Context, wr *wrangler.Wrangler, tsc *disc // - find a tabletType instance in the keyspace / shard // - mark it as worker // - tag it with our worker process -func FindWorkerTablet(ctx context.Context, wr *wrangler.Wrangler, cleaner *wrangler.Cleaner, tsc *discovery.TabletStatsCache, cell, keyspace, shard string, minHealthyTablets int, tabletType topodatapb.TabletType) (*topodatapb.TabletAlias, error) { +func FindWorkerTablet(ctx context.Context, wr *wrangler.Wrangler, cleaner *wrangler.Cleaner, tsc *discovery.LegacyTabletStatsCache, cell, keyspace, shard string, minHealthyTablets int, tabletType topodatapb.TabletType) (*topodatapb.TabletAlias, error) { tabletAlias, err := FindHealthyTablet(ctx, wr, tsc, cell, keyspace, shard, minHealthyTablets, tabletType) if err != nil { return nil, err diff --git a/go/vt/wrangler/keyspace.go b/go/vt/wrangler/keyspace.go index f4d901c1421..56350348dc1 100644 --- a/go/vt/wrangler/keyspace.go +++ b/go/vt/wrangler/keyspace.go @@ -989,12 +989,12 @@ func (wr *Wrangler) waitForDrainInCell(ctx context.Context, cell, keyspace, shar retryDelay, healthCheckTopologyRefresh, healthcheckRetryDelay, healthCheckTimeout, initialWait time.Duration) error { // Create the healthheck module, with a cache. - hc := discovery.NewHealthCheck(healthcheckRetryDelay, healthCheckTimeout) + hc := discovery.NewLegacyHealthCheck(healthcheckRetryDelay, healthCheckTimeout) defer hc.Close() - tsc := discovery.NewTabletStatsCache(hc, wr.TopoServer(), cell) + tsc := discovery.NewLegacyTabletStatsCache(hc, wr.TopoServer(), cell) // Create a tablet watcher. - watcher := discovery.NewShardReplicationWatcher(ctx, wr.TopoServer(), hc, cell, keyspace, shard, healthCheckTopologyRefresh, discovery.DefaultTopoReadConcurrency) + watcher := discovery.NewLegacyShardReplicationWatcher(ctx, wr.TopoServer(), hc, cell, keyspace, shard, healthCheckTopologyRefresh, discovery.DefaultTopoReadConcurrency) defer watcher.Stop() // Wait for at least one tablet. @@ -1016,8 +1016,8 @@ func (wr *Wrangler) waitForDrainInCell(ctx context.Context, cell, keyspace, shar startTime := time.Now() for { // map key: tablet uid - drainedHealthyTablets := make(map[uint32]*discovery.TabletStats) - notDrainedHealtyTablets := make(map[uint32]*discovery.TabletStats) + drainedHealthyTablets := make(map[uint32]*discovery.LegacyTabletStats) + notDrainedHealtyTablets := make(map[uint32]*discovery.LegacyTabletStats) healthyTablets := tsc.GetHealthyTabletStats(keyspace, shard, servedType) for _, ts := range healthyTablets { @@ -1060,7 +1060,7 @@ func (wr *Wrangler) waitForDrainInCell(ctx context.Context, cell, keyspace, shar return nil } -func formatTabletStats(ts *discovery.TabletStats) string { +func formatTabletStats(ts *discovery.LegacyTabletStats) string { webURL := "unknown http port" if webPort, ok := ts.Tablet.PortMap["vt"]; ok { webURL = fmt.Sprintf("http://%v:%d/", ts.Tablet.Hostname, webPort) diff --git a/test/config.json b/test/config.json index dfc4d677438..67fad2afdce 100644 --- a/test/config.json +++ b/test/config.json @@ -324,6 +324,33 @@ "RetryMax": 0, "Tags": [] }, + "tabletgateway_buffer": { + "File": "unused.go", + "Args": ["vitess.io/vitess/go/test/endtoend/tabletgateway/buffer"], + "Command": [], + "Manual": false, + "Shard": 14, + "RetryMax": 0, + "Tags": [] + }, + "tabletgateway_cellalias": { + "File": "unused.go", + "Args": ["vitess.io/vitess/go/test/endtoend/tabletgateway/cellalias"], + "Command": [], + "Manual": false, + "Shard": 14, + "RetryMax": 0, + "Tags": [] + }, + "tabletgateway_healthcheck": { + "File": "unused.go", + "Args": ["vitess.io/vitess/go/test/endtoend/tabletgateway/healthcheck"], + "Command": [], + "Manual": false, + "Shard": 14, + "RetryMax": 0, + "Tags": [] + }, "tabletmanager": { "File": "unused.go", "Args": ["vitess.io/vitess/go/test/endtoend/tabletmanager"],