From 18234019398bac30dfe4040f529cb5a7fe68bc00 Mon Sep 17 00:00:00 2001 From: deepthi Date: Mon, 10 Jan 2022 11:44:37 -0800 Subject: [PATCH 01/10] delete discoverygateway Signed-off-by: deepthi --- go/cmd/vtcombo/main.go | 3 +- go/cmd/vtgate/status.go | 12 +- go/cmd/vtgate/vtgate.go | 17 +- go/vt/discovery/fake_healthcheck.go | 14 +- go/vt/vtgate/api.go | 56 --- go/vt/vtgate/autocommit_test.go | 34 +- go/vt/vtgate/discoverygateway.go | 416 ------------------ go/vt/vtgate/discoverygateway_test.go | 455 -------------------- go/vt/vtgate/executor.go | 260 ++++------- go/vt/vtgate/executor_ddl_test.go | 2 +- go/vt/vtgate/executor_dml_test.go | 72 ++-- go/vt/vtgate/executor_framework_test.go | 54 +-- go/vt/vtgate/executor_scatter_stats_test.go | 6 +- go/vt/vtgate/executor_select_test.go | 171 ++++---- go/vt/vtgate/executor_set_test.go | 10 +- go/vt/vtgate/executor_stream_test.go | 8 +- go/vt/vtgate/executor_test.go | 78 ++-- go/vt/vtgate/executor_vschema_ddl_test.go | 14 +- go/vt/vtgate/gateway.go | 33 +- go/vt/vtgate/grpc_discovery_test.go | 6 +- go/vt/vtgate/legacy_scatter_conn_test.go | 43 +- go/vt/vtgate/queryz_test.go | 2 +- go/vt/vtgate/scatter_conn.go | 43 +- go/vt/vtgate/tabletgateway.go | 13 - go/vt/vtgate/tx_conn.go | 5 +- go/vt/vtgate/tx_conn_test.go | 66 ++- go/vt/vtgate/vcursor_impl.go | 4 - go/vt/vtgate/vstream_manager_test.go | 8 +- go/vt/vtgate/vtgate.go | 98 +---- go/vt/vtgate/vtgate_test.go | 11 +- 30 files changed, 355 insertions(+), 1659 deletions(-) delete mode 100644 go/vt/vtgate/discoverygateway.go delete mode 100644 go/vt/vtgate/discoverygateway_test.go diff --git a/go/cmd/vtcombo/main.go b/go/cmd/vtcombo/main.go index a6ae267bf86..ff7afdeafb7 100644 --- a/go/cmd/vtcombo/main.go +++ b/go/cmd/vtcombo/main.go @@ -242,7 +242,8 @@ func main() { vtgate.QueryLogHandler = "/debug/vtgate/querylog" vtgate.QueryLogzHandler = "/debug/vtgate/querylogz" vtgate.QueryzHandler = "/debug/vtgate/queryz" - vtg := vtgate.Init(context.Background(), resilientServer, tpb.Cells[0], tabletTypesToWait) + // pass nil for healthcheck, it will get created + vtg := vtgate.Init(context.Background(), nil, resilientServer, tpb.Cells[0], tabletTypesToWait) // vtctld configuration and init err = vtctld.InitVtctld(ts) diff --git a/go/cmd/vtgate/status.go b/go/cmd/vtgate/status.go index ee0ff6788a3..a6e9e8f0e79 100644 --- a/go/cmd/vtgate/status.go +++ b/go/cmd/vtgate/status.go @@ -38,13 +38,7 @@ func addStatusParts(vtg *vtgate.VTGate) { servenv.AddStatusPart("Gateway Status", vtgate.StatusTemplate, func() interface{} { return vtg.GetGatewayCacheStatus() }) - if vtgate.UsingLegacyGateway() { - 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().TabletsCacheStatus() - }) - } + servenv.AddStatusPart("Health Check Cache", discovery.HealthCheckTemplate, func() interface{} { + return vtg.Gateway().TabletsCacheStatus() + }) } diff --git a/go/cmd/vtgate/vtgate.go b/go/cmd/vtgate/vtgate.go index 4853da2e132..3a8b73f9c33 100644 --- a/go/cmd/vtgate/vtgate.go +++ b/go/cmd/vtgate/vtgate.go @@ -44,7 +44,6 @@ var ( ) var resilientServer *srvtopo.ResilientServer -var legacyHealthCheck discovery.LegacyHealthCheck func init() { rand.Seed(time.Now().UnixNano()) @@ -143,17 +142,8 @@ func main() { log.Exitf("cells_to_watch validation failed: %v", err) } - 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) - } + // pass nil for HealthCheck and it will be created + vtg := vtgate.Init(context.Background(), nil, resilientServer, *cell, tabletTypes) servenv.OnRun(func() { // Flags are parsed now. Parse the template using the actual flag value and overwrite the current template. @@ -162,9 +152,6 @@ func main() { }) servenv.OnClose(func() { _ = vtg.Gateway().Close(context.Background()) - if legacyHealthCheck != nil { - _ = legacyHealthCheck.Close() - } }) servenv.RunDefault() } diff --git a/go/vt/discovery/fake_healthcheck.go b/go/vt/discovery/fake_healthcheck.go index 02e2038db4f..aad55923448 100644 --- a/go/vt/discovery/fake_healthcheck.go +++ b/go/vt/discovery/fake_healthcheck.go @@ -35,10 +35,6 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) -var ( - currentTabletUID sync2.AtomicInt32 -) - // This file contains the definitions for a FakeHealthCheck class to // simulate a HealthCheck module. Note it is not in a sub-package because // otherwise it couldn't be used in this package's tests because of @@ -55,9 +51,9 @@ func NewFakeHealthCheck(ch chan *TabletHealth) *FakeHealthCheck { // FakeHealthCheck implements discovery.HealthCheck. type FakeHealthCheck struct { // mu protects the items map - mu sync.RWMutex - items map[string]*fhcItem - + mu sync.RWMutex + items map[string]*fhcItem + currentTabletUID sync2.AtomicInt32 // channel to return on subscribe. Pass nil if no subscribe should not return a channel ch chan *TabletHealth } @@ -259,8 +255,8 @@ func (fhc *FakeHealthCheck) Reset() { // For flexibility the connection is created via a connFactory callback func (fhc *FakeHealthCheck) 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 { // tabletUID must be unique - currentTabletUID.Add(1) - uid := currentTabletUID.Get() + fhc.currentTabletUID.Add(1) + uid := fhc.currentTabletUID.Get() t := topo.NewTablet(uint32(uid), cell, host) t.Keyspace = keyspace t.Shard = shard diff --git a/go/vt/vtgate/api.go b/go/vt/vtgate/api.go index 2446968b85e..34e3cd07987 100644 --- a/go/vt/vtgate/api.go +++ b/go/vt/vtgate/api.go @@ -142,59 +142,3 @@ func initAPI(hc discovery.HealthCheck) { return nil, fmt.Errorf("cannot find health for: %s", itemPath) }) } - -func legacyInitAPI(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_ - for _, tabletCacheStatus := range cacheStatus { - for _, tabletStats := range tabletCacheStatus.TabletsStats { - if tabletStats.Name == value || tabletStats.Tablet.MysqlHostname == value { - return tabletStats, nil - } - } - } - } - } - return nil, fmt.Errorf("cannot find health for: %s", itemPath) - }) -} diff --git a/go/vt/vtgate/autocommit_test.go b/go/vt/vtgate/autocommit_test.go index 9ccc4243a6b..f09d852a32e 100644 --- a/go/vt/vtgate/autocommit_test.go +++ b/go/vt/vtgate/autocommit_test.go @@ -36,7 +36,7 @@ import ( // TestAutocommitUpdateSharded: instant-commit. func TestAutocommitUpdateSharded(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := autocommitExec(executor, "update user set a=2 where id = 1") require.NoError(t, err) @@ -53,7 +53,7 @@ func TestAutocommitUpdateSharded(t *testing.T) { // TestAutocommitUpdateLookup: transaction: select before update. func TestAutocommitUpdateLookup(t *testing.T) { - executor, sbc1, _, sbclookup := createLegacyExecutorEnv() + executor, sbc1, _, sbclookup := createExecutorEnv() sbclookup.SetResults([]*sqltypes.Result{sqltypes.MakeTestResult( sqltypes.MakeTestFields("b|a", "int64|varbinary"), "2|1", @@ -82,7 +82,7 @@ func TestAutocommitUpdateLookup(t *testing.T) { // TestAutocommitUpdateVindexChange: transaction: select & update before final update. func TestAutocommitUpdateVindexChange(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() sbc.SetResults([]*sqltypes.Result{sqltypes.MakeTestResult( sqltypes.MakeTestFields("id|name|lastname|name_lastname_keyspace_id_map", "int64|int32|varchar|int64"), "1|1|foo|0", @@ -121,7 +121,7 @@ func TestAutocommitUpdateVindexChange(t *testing.T) { // TestAutocommitDeleteSharded: instant-commit. func TestAutocommitDeleteSharded(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := autocommitExec(executor, "delete from user_extra where user_id = 1") require.NoError(t, err) @@ -138,7 +138,7 @@ func TestAutocommitDeleteSharded(t *testing.T) { // TestAutocommitDeleteLookup: transaction: select before update. func TestAutocommitDeleteLookup(t *testing.T) { - executor, sbc1, _, sbclookup := createLegacyExecutorEnv() + executor, sbc1, _, sbclookup := createExecutorEnv() sbc1.SetResults([]*sqltypes.Result{sqltypes.MakeTestResult( sqltypes.MakeTestFields("id|name|lastname", "int64|int32|varchar"), "1|1|foo", @@ -180,7 +180,7 @@ func TestAutocommitDeleteLookup(t *testing.T) { // TestAutocommitDeleteIn: instant-commit. func TestAutocommitDeleteIn(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := autocommitExec(executor, "delete from user_extra where user_id in (1, 2)") require.NoError(t, err) @@ -197,7 +197,7 @@ func TestAutocommitDeleteIn(t *testing.T) { // TestAutocommitDeleteMultiShard: instant-commit. func TestAutocommitDeleteMultiShard(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := autocommitExec(executor, "delete from user_extra where user_id = user_id + 1") require.NoError(t, err) @@ -217,7 +217,7 @@ func TestAutocommitDeleteMultiShard(t *testing.T) { // TestAutocommitDeleteMultiShardAutoCommit: instant-commit. func TestAutocommitDeleteMultiShardAutoCommit(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := autocommitExec(executor, "delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from user_extra where user_id = user_id + 1") require.NoError(t, err) @@ -237,7 +237,7 @@ func TestAutocommitDeleteMultiShardAutoCommit(t *testing.T) { // TestAutocommitInsertSharded: instant-commit. func TestAutocommitInsertSharded(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := autocommitExec(executor, "insert into user_extra(user_id, v) values (1, 2)") require.NoError(t, err) @@ -256,7 +256,7 @@ func TestAutocommitInsertSharded(t *testing.T) { // TestAutocommitInsertLookup: transaction: select before update. func TestAutocommitInsertLookup(t *testing.T) { - executor, sbc1, _, sbclookup := createLegacyExecutorEnv() + executor, sbc1, _, sbclookup := createExecutorEnv() _, err := autocommitExec(executor, "insert into user(id, v, name) values (1, 2, 'myname')") require.NoError(t, err) @@ -283,7 +283,7 @@ func TestAutocommitInsertLookup(t *testing.T) { // TestAutocommitInsertShardAutoCommit: instant-commit. func TestAutocommitInsertMultishardAutoCommit(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := autocommitExec(executor, "insert /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ into user_extra(user_id, v) values (1, 2), (3, 4)") require.NoError(t, err) @@ -306,7 +306,7 @@ func TestAutocommitInsertMultishardAutoCommit(t *testing.T) { }}) testCommitCount(t, "sbc2", sbc2, 0) - executor, sbc1, sbc2, _ = createLegacyExecutorEnv() + executor, sbc1, sbc2, _ = createExecutorEnv() // Make the first shard fail - the second completes anyway sbc1.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1 _, err = autocommitExec(executor, "insert /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ into user_extra(user_id, v) values (1, 2), (3, 4)") @@ -327,7 +327,7 @@ func TestAutocommitInsertMultishardAutoCommit(t *testing.T) { } func TestAutocommitInsertMultishard(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := autocommitExec(executor, "insert into user_extra(user_id, v) values (1, 2), (3, 4)") require.NoError(t, err) @@ -353,7 +353,7 @@ func TestAutocommitInsertMultishard(t *testing.T) { // TestAutocommitInsertAutoinc: instant-commit: sequence fetch is not transactional. func TestAutocommitInsertAutoinc(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() _, err := autocommitExec(executor, "insert into main1(id, name) values (null, 'myname')") require.NoError(t, err) @@ -372,7 +372,7 @@ func TestAutocommitInsertAutoinc(t *testing.T) { // TestAutocommitTransactionStarted: no instant-commit. func TestAutocommitTransactionStarted(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() session := &vtgatepb.Session{ TargetString: "@primary", @@ -394,7 +394,7 @@ func TestAutocommitTransactionStarted(t *testing.T) { // TestAutocommitDirectTarget: instant-commit. func TestAutocommitDirectTarget(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() session := &vtgatepb.Session{ TargetString: "TestUnsharded/0@primary", @@ -415,7 +415,7 @@ func TestAutocommitDirectTarget(t *testing.T) { // TestAutocommitDirectRangeTarget: no instant-commit. func TestAutocommitDirectRangeTarget(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() session := &vtgatepb.Session{ TargetString: "TestExecutor[-]@primary", diff --git a/go/vt/vtgate/discoverygateway.go b/go/vt/vtgate/discoverygateway.go deleted file mode 100644 index 3b4ef2c068f..00000000000 --- a/go/vt/vtgate/discoverygateway.go +++ /dev/null @@ -1,416 +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 vtgate - -import ( - "context" - "fmt" - "math/rand" - "sort" - "strings" - "sync" - "time" - - "vitess.io/vitess/go/stats" - "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 ( - // GatewayImplementationDiscovery defines the string value used as the implementation key for DiscoveryGateway - GatewayImplementationDiscovery = "discoverygateway" -) - -//UsingLegacyGateway returns true when legacy -func UsingLegacyGateway() bool { - return *GatewayImplementation == GatewayImplementationDiscovery -} - -func init() { - RegisterGatewayCreator(GatewayImplementationDiscovery, createDiscoveryGateway) -} - -// DiscoveryGateway is not the default Gateway implementation anymore. -// This implementation uses the legacy healthcheck module. -type DiscoveryGateway struct { - queryservice.QueryService - 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.LegacyTopologyWatcher - - // mu protects the fields of this group. - mu sync.RWMutex - // statusAggregators is a map indexed by the key - // keyspace/shard/tablet_type. - statusAggregators map[string]*TabletStatusAggregator - - // buffer, if enabled, buffers requests during a detected PRIMARY failover. - buffer *buffer.Buffer -} - -//TabletsCacheStatus is not implemented for this struct -func (dg *DiscoveryGateway) TabletsCacheStatus() discovery.TabletsCacheStatusList { - return nil -} - -var _ Gateway = (*DiscoveryGateway)(nil) - -func createDiscoveryGateway(ctx context.Context, hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string, retryCount int) Gateway { - return NewDiscoveryGateway(ctx, hc, serv, cell, retryCount) -} - -// 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 PRIMARY 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 - topoServer, err = serv.GetTopoServer() - if err != nil { - log.Exitf("Unable to create new discoverygateway: %v", err) - } - } - - bufferCfg := buffer.NewConfigFromFlags() - - dg := &DiscoveryGateway{ - hc: hc, - tsc: discovery.NewTabletStatsCacheDoNotSetListener(topoServer, cell), - srvTopoServer: serv, - localCell: cell, - retryCount: retryCount, - tabletsWatchers: make([]*discovery.LegacyTopologyWatcher, 0, 1), - statusAggregators: make(map[string]*TabletStatusAggregator), - buffer: buffer.New(bufferCfg), - } - - // Set listener which will update LegacyTabletStatsCache and PrimaryBuffer. - // We set sendDownEvents=true because it's required by LegacyTabletStatsCache. - hc.SetListener(dg, true /* sendDownEvents */) - - cells := *CellsToWatch - log.Infof("loading tablets for cells: %v", cells) - for _, c := range strings.Split(cells, ",") { - if c == "" { - continue - } - var recorder discovery.LegacyTabletRecorder = dg.hc - if len(discovery.TabletFilters) > 0 { - if discovery.FilteringKeyspaces() { - log.Exitf("Only one of -keyspaces_to_watch and -tablet_filters may be specified at a time") - } - - fbs, err := discovery.NewLegacyFilterByShard(recorder, discovery.TabletFilters) - if err != nil { - log.Exitf("Cannot parse tablet_filters parameter: %v", err) - } - recorder = fbs - } else if discovery.FilteringKeyspaces() { - recorder = discovery.NewLegacyFilterByKeyspace(recorder, discovery.KeyspacesToWatch) - } - - 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) - return dg -} - -// RegisterStats registers the stats to export the lag since the last refresh -// and the checksum of the topology -func (dg *DiscoveryGateway) RegisterStats() { - stats.NewGaugeDurationFunc( - "TopologyWatcherMaxRefreshLag", - "maximum time since the topology watcher refreshed a cell", - dg.topologyWatcherMaxRefreshLag, - ) - - stats.NewGaugeFunc( - "TopologyWatcherChecksum", - "crc32 checksum of the topology watcher state", - dg.topologyWatcherChecksum, - ) -} - -// topologyWatcherMaxRefreshLag returns the maximum lag since the watched -// cells were refreshed from the topo server -func (dg *DiscoveryGateway) topologyWatcherMaxRefreshLag() time.Duration { - var lag time.Duration - for _, tw := range dg.tabletsWatchers { - cellLag := tw.RefreshLag() - if cellLag > lag { - lag = cellLag - } - } - return lag -} - -// topologyWatcherChecksum returns a checksum of the topology watcher state -func (dg *DiscoveryGateway) topologyWatcherChecksum() int64 { - var checksum int64 - for _, tw := range dg.tabletsWatchers { - checksum = checksum ^ int64(tw.TopoChecksum()) - } - return checksum -} - -// StatsUpdate forwards LegacyHealthCheck updates to LegacyTabletStatsCache and PrimaryBuffer. -// 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_PRIMARY { - dg.buffer.StatsUpdate(ts) - } -} - -// WaitForTablets is part of the gateway.Gateway interface. -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 - } - - // Finds the targets to look for. - targets, err := srvtopo.FindAllTargets(ctx, dg.srvTopoServer, dg.localCell, tabletTypesToWait) - if err != nil { - return err - } - - filteredTargets := discovery.FilterTargetsByKeyspaces(discovery.KeyspacesToWatch, targets) - return dg.tsc.WaitForAllServingTablets(ctx, filteredTargets) -} - -// Close shuts down underlying connections. -// This function hides the inner implementation. -func (dg *DiscoveryGateway) Close(ctx context.Context) error { - dg.buffer.Shutdown() - for _, ctw := range dg.tabletsWatchers { - ctw.Stop() - } - return nil -} - -// CacheStatus returns a list of TabletCacheStatus per -// keyspace/shard/tablet_type. -func (dg *DiscoveryGateway) CacheStatus() TabletCacheStatusList { - dg.mu.RLock() - res := make(TabletCacheStatusList, 0, len(dg.statusAggregators)) - for _, aggr := range dg.statusAggregators { - res = append(res, aggr.GetCacheStatus()) - } - dg.mu.RUnlock() - 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 (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 err error - invalidTablets := make(map[string]bool) - - if target == nil { - err = fmt.Errorf("withRetry called with nil target") - log.Errorf(err.Error()) - return err - } - 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 < dg.retryCount+1; i++ { - // Check if we should buffer PRIMARY 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_PRIMARY { - // The next call blocks if we should buffer during a failover. - retryDone, bufferErr := dg.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 := dg.tsc.GetHealthyTabletStats(target.Keyspace, target.Shard, target.TabletType) - if len(tablets) == 0 { - // fail fast if there is no tablet - err = vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "no healthy tablet available for '%s'", target.String()) - break - } - shuffleTablets(dg.localCell, tablets) - - // skip tablets we tried before - var ts *discovery.LegacyTabletStats - for _, t := range tablets { - if _, ok := invalidTablets[t.Key]; !ok { - ts = &t - break - } - } - if ts == nil { - if err == nil { - // do not override error from last attempt. - err = vterrors.New(vtrpcpb.Code_UNAVAILABLE, "no available connection") - } - break - } - - // execute - conn := dg.hc.GetConnection(ts.Key) - if conn == nil { - err = vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "no connection for key %v tablet %+v", ts.Key, ts.Tablet) - invalidTablets[ts.Key] = true - continue - } - - startTime := time.Now() - var canRetry bool - canRetry, err = inner(ctx, ts.Target, conn) - dg.updateStats(target, startTime, err) - if canRetry { - invalidTablets[ts.Key] = true - continue - } - break - } - return NewShardError(err, target) -} - -func shuffleTablets(cell string, tablets []discovery.LegacyTabletStats) { - 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 = nextTablet(cell, tablets, sameCell, length, true) - diffCell = 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 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 - } - } - return -1 -} - -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 { - key := fmt.Sprintf("%v/%v/%v", target.Keyspace, target.Shard, target.TabletType.String()) - - // get existing aggregator - dg.mu.RLock() - aggr, ok := dg.statusAggregators[key] - dg.mu.RUnlock() - if ok { - return aggr - } - // create a new one, but check again before the creation - dg.mu.Lock() - defer dg.mu.Unlock() - aggr, ok = dg.statusAggregators[key] - if ok { - return aggr - } - aggr = NewTabletStatusAggregator(target.Keyspace, target.Shard, target.TabletType, key) - dg.statusAggregators[key] = aggr - return aggr -} - -// QueryServiceByAlias satisfies the Gateway interface -func (dg *DiscoveryGateway) QueryServiceByAlias(_ *topodatapb.TabletAlias, _ *querypb.Target) (queryservice.QueryService, error) { - return nil, vterrors.New(vtrpcpb.Code_UNIMPLEMENTED, "DiscoveryGateway does not implement QueryServiceByAlias") -} diff --git a/go/vt/vtgate/discoverygateway_test.go b/go/vt/vtgate/discoverygateway_test.go deleted file mode 100644 index 36d55f4d963..00000000000 --- a/go/vt/vtgate/discoverygateway_test.go +++ /dev/null @@ -1,455 +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 vtgate - -import ( - "fmt" - "testing" - "time" - - "vitess.io/vitess/go/vt/log" - - "context" - - "vitess.io/vitess/go/sqltypes" - "vitess.io/vitess/go/vt/discovery" - "vitess.io/vitess/go/vt/srvtopo/srvtopotest" - "vitess.io/vitess/go/vt/topo" - "vitess.io/vitess/go/vt/topo/memorytopo" - - querypb "vitess.io/vitess/go/vt/proto/query" - "vitess.io/vitess/go/vt/proto/topodata" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" - vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" -) - -func TestDiscoveryGatewayExecute(t *testing.T) { - testDiscoveryGatewayGeneric(t, func(dg *DiscoveryGateway, target *querypb.Target) error { - _, err := dg.Execute(context.Background(), target, "query", nil, 0, 0, nil) - return err - }) - testDiscoveryGatewayTransact(t, func(dg *DiscoveryGateway, target *querypb.Target) error { - _, err := dg.Execute(context.Background(), target, "query", nil, 1, 0, nil) - return err - }) -} - -func TestDiscoveryGatewayExecuteBatch(t *testing.T) { - 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 { - queries := []*querypb.BoundQuery{{Sql: "query", BindVariables: nil}} - _, err := dg.ExecuteBatch(context.Background(), target, queries, false, 1, nil) - return err - }) -} - -func TestDiscoveryGatewayExecuteStream(t *testing.T) { - testDiscoveryGatewayGeneric(t, func(dg *DiscoveryGateway, target *querypb.Target) error { - err := dg.StreamExecute(context.Background(), target, "query", nil, 0, 0, nil, func(qr *sqltypes.Result) error { - return nil - }) - return err - }) -} - -func TestDiscoveryGatewayBegin(t *testing.T) { - 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 { - _, err := dg.Commit(context.Background(), target, 1) - return err - }) -} - -func TestDiscoveryGatewayRollback(t *testing.T) { - testDiscoveryGatewayTransact(t, func(dg *DiscoveryGateway, target *querypb.Target) error { - _, err := dg.Rollback(context.Background(), target, 1) - return err - }) -} - -func TestDiscoveryGatewayBeginExecute(t *testing.T) { - testDiscoveryGatewayGeneric(t, func(dg *DiscoveryGateway, target *querypb.Target) error { - _, _, _, err := dg.BeginExecute(context.Background(), target, nil, "query", nil, 0, nil) - return err - }) -} - -func TestDiscoveryGatewayBeginExecuteBatch(t *testing.T) { - 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 - }) -} - -func TestDiscoveryGatewayGetTablets(t *testing.T) { - keyspace := "ks" - shard := "0" - hc := discovery.NewFakeLegacyHealthCheck() - dg := NewDiscoveryGateway(context.Background(), hc, nil, "local", 2) - - // replica should only use local ones - hc.Reset() - dg.tsc.ResetForTesting() - hc.AddTestTablet("remote", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) - ep1 := hc.AddTestTablet("local", "2.2.2.2", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil).Tablet() - tsl := dg.tsc.GetHealthyTabletStats(keyspace, shard, topodatapb.TabletType_REPLICA) - if len(tsl) != 1 || !topo.TabletEquality(tsl[0].Tablet, ep1) { - t.Errorf("want %+v, got %+v", ep1, tsl) - } - - // primary should use the one with newer timestamp regardless of cell - hc.Reset() - dg.tsc.ResetForTesting() - hc.AddTestTablet("remote", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_PRIMARY, true, 5, nil) - ep1 = hc.AddTestTablet("remote", "2.2.2.2", 1001, keyspace, shard, topodatapb.TabletType_PRIMARY, true, 10, nil).Tablet() - tsl = dg.tsc.GetHealthyTabletStats(keyspace, shard, topodatapb.TabletType_PRIMARY) - if len(tsl) != 1 || !topo.TabletEquality(tsl[0].Tablet, ep1) { - t.Errorf("want %+v, got %+v", ep1, tsl) - } -} - -func TestDiscoveryGatewayWaitForTablets(t *testing.T) { - keyspace := "ks" - shard := "0" - cell := "local" - hc := discovery.NewFakeLegacyHealthCheck() - ts := memorytopo.NewServer("local") - srvTopo := srvtopotest.NewPassthroughSrvTopoServer() - srvTopo.TopoServer = ts - srvTopo.SrvKeyspaceNames = []string{keyspace} - srvTopo.SrvKeyspace = &topodatapb.SrvKeyspace{ - Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - { - ServedType: topodata.TabletType_PRIMARY, - ShardReferences: []*topodatapb.ShardReference{ - { - Name: shard, - }, - }, - }, - { - ServedType: topodata.TabletType_REPLICA, - ShardReferences: []*topodatapb.ShardReference{ - { - Name: shard, - }, - }, - }, - { - ServedType: topodata.TabletType_RDONLY, - ShardReferences: []*topodatapb.ShardReference{ - { - Name: shard, - }, - }, - }, - }, - } - dg := NewDiscoveryGateway(context.Background(), hc, srvTopo, "local", 2) - - // replica should only use local ones - hc.Reset() - dg.tsc.ResetForTesting() - hc.AddTestTablet(cell, "2.2.2.2", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) - hc.AddTestTablet(cell, "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_PRIMARY, true, 5, nil) - { - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) //nolint - defer cancel() - err := dg.WaitForTablets(ctx, []topodatapb.TabletType{topodatapb.TabletType_REPLICA, topodatapb.TabletType_PRIMARY}) - if err != nil { - t.Errorf("want %+v, got %+v", nil, err) - } - - // fails if there are no available tablets for the desired TabletType - err = dg.WaitForTablets(ctx, []topodatapb.TabletType{topodatapb.TabletType_RDONLY}) - if err == nil { - t.Errorf("expected error, got nil") - } - } - { - // errors because there is no primary on ks2 - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) //nolint - defer cancel() - srvTopo.SrvKeyspaceNames = []string{keyspace, "ks2"} - err := dg.WaitForTablets(ctx, []topodatapb.TabletType{topodatapb.TabletType_PRIMARY}) - if err == nil { - t.Errorf("expected error, got nil") - } - } - discovery.KeyspacesToWatch = []string{keyspace} - // does not wait for ks2 if it's not part of the filter - { - ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) //nolint - defer cancel() - err := dg.WaitForTablets(ctx, []topodatapb.TabletType{topodatapb.TabletType_PRIMARY}) - if err != nil { - t.Errorf("want %+v, got %+v", nil, err) - } - } - discovery.KeyspacesToWatch = []string{} -} - -func TestShuffleTablets(t *testing.T) { - ts1 := discovery.LegacyTabletStats{ - Key: "t1", - Tablet: topo.NewTablet(10, "cell1", "host1"), - Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, - Up: true, - Serving: true, - Stats: &querypb.RealtimeStats{ReplicationLagSeconds: 1, CpuUsage: 0.2}, - } - - ts2 := discovery.LegacyTabletStats{ - Key: "t2", - Tablet: topo.NewTablet(10, "cell1", "host2"), - Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, - Up: true, - Serving: true, - Stats: &querypb.RealtimeStats{ReplicationLagSeconds: 1, CpuUsage: 0.2}, - } - - ts3 := discovery.LegacyTabletStats{ - Key: "t3", - Tablet: topo.NewTablet(10, "cell2", "host3"), - Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, - Up: true, - Serving: true, - Stats: &querypb.RealtimeStats{ReplicationLagSeconds: 1, CpuUsage: 0.2}, - } - - ts4 := discovery.LegacyTabletStats{ - Key: "t4", - Tablet: topo.NewTablet(10, "cell2", "host4"), - Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}, - Up: true, - Serving: true, - Stats: &querypb.RealtimeStats{ReplicationLagSeconds: 1, CpuUsage: 0.2}, - } - - 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) - if (len(sameCellTablets) != 2) || - (sameCellTablets[0].Key != "t1" && sameCellTablets[0].Key != "t2") || - (sameCellTablets[1].Key != "t1" && sameCellTablets[1].Key != "t2") { - t.Errorf("should shuffle in only same cell tablets, got %+v", sameCellTablets) - } - - shuffleTablets("cell1", diffCellTablets) - if (len(diffCellTablets) != 2) || - (diffCellTablets[0].Key != "t3" && diffCellTablets[0].Key != "t4") || - (diffCellTablets[1].Key != "t3" && diffCellTablets[1].Key != "t4") { - t.Errorf("should shuffle in only diff cell tablets, got %+v", diffCellTablets) - } - - shuffleTablets("cell1", mixedTablets) - if len(mixedTablets) != 4 { - t.Errorf("should have 4 tablets, got %+v", mixedTablets) - } - - if (mixedTablets[0].Key != "t1" && mixedTablets[0].Key != "t2") || - (mixedTablets[1].Key != "t1" && mixedTablets[1].Key != "t2") { - t.Errorf("should have same cell tablets in the front, got %+v", mixedTablets) - } - - if (mixedTablets[2].Key != "t3" && mixedTablets[2].Key != "t4") || - (mixedTablets[3].Key != "t3" && mixedTablets[3].Key != "t4") { - t.Errorf("should have diff cell tablets in the rear, got %+v", mixedTablets) - } - } -} - -func TestDiscoveryGatewayGetTabletsInRegion(t *testing.T) { - keyspace := "ks" - shard := "0" - hc := discovery.NewFakeLegacyHealthCheck() - ts := memorytopo.NewServer("local-west", "local-east", "local", "remote") - srvTopo := srvtopotest.NewPassthroughSrvTopoServer() - srvTopo.TopoServer = ts - - cellsAlias := &topodatapb.CellsAlias{ - Cells: []string{"local-west", "local-east"}, - } - - dg := NewDiscoveryGateway(context.Background(), hc, srvTopo, "local-west", 2) - - ts.CreateCellsAlias(context.Background(), "local", cellsAlias) - - defer ts.DeleteCellsAlias(context.Background(), "local") - - // this is a test - // replica should only use local ones - hc.Reset() - dg.tsc.ResetForTesting() - hc.AddTestTablet("remote", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) - ep1 := hc.AddTestTablet("local-west", "2.2.2.2", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil).Tablet() - ep2 := hc.AddTestTablet("local-east", "3.3.3.3", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil).Tablet() - tsl := dg.tsc.GetHealthyTabletStats(keyspace, shard, topodatapb.TabletType_REPLICA) - if len(tsl) != 2 || (!topo.TabletEquality(tsl[0].Tablet, ep1) && !topo.TabletEquality(tsl[0].Tablet, ep2)) { - t.Fatalf("want %+v or %+v, got %+v", ep1, ep2, tsl) - } -} -func TestDiscoveryGatewayGetTabletsWithRegion(t *testing.T) { - keyspace := "ks" - shard := "0" - hc := discovery.NewFakeLegacyHealthCheck() - ts := memorytopo.NewServer("local-west", "local-east", "local", "remote") - srvTopo := srvtopotest.NewPassthroughSrvTopoServer() - srvTopo.TopoServer = ts - - cellsAlias := &topodatapb.CellsAlias{ - Cells: []string{"local-west", "local-east"}, - } - - dg := NewDiscoveryGateway(context.Background(), hc, srvTopo, "local", 2) - - if err := ts.CreateCellsAlias(context.Background(), "local", cellsAlias); err != nil { - log.Errorf("ts.CreateCellsAlias(context.Background()... %v", err) - } - - defer ts.DeleteCellsAlias(context.Background(), "local") - - // this is a test - // replica should only use local ones - hc.Reset() - dg.tsc.ResetForTesting() - hc.AddTestTablet("remote", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) - ep1 := hc.AddTestTablet("local-west", "2.2.2.2", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil).Tablet() - ep2 := hc.AddTestTablet("local-east", "3.3.3.3", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil).Tablet() - tsl := dg.tsc.GetHealthyTabletStats(keyspace, shard, topodatapb.TabletType_REPLICA) - if len(tsl) != 2 || (!topo.TabletEquality(tsl[0].Tablet, ep1) && !topo.TabletEquality(tsl[0].Tablet, ep2)) { - t.Fatalf("want %+v or %+v, got %+v", ep1, ep2, tsl) - } -} - -func testDiscoveryGatewayGeneric(t *testing.T, f func(dg *DiscoveryGateway, target *querypb.Target) error) { - keyspace := "ks" - shard := "0" - tabletType := topodatapb.TabletType_REPLICA - target := &querypb.Target{ - Keyspace: keyspace, - Shard: shard, - TabletType: tabletType, - } - hc := discovery.NewFakeLegacyHealthCheck() - dg := NewDiscoveryGateway(context.Background(), hc, nil, "cell", 2) - - // no tablet - hc.Reset() - dg.tsc.ResetForTesting() - want := []string{"target: ks.0.replica", `no healthy tablet available for 'keyspace:"ks" shard:"0" tablet_type:REPLICA`} - err := f(dg, target) - verifyShardErrors(t, err, want, vtrpcpb.Code_UNAVAILABLE) - - // tablet with error - hc.Reset() - dg.tsc.ResetForTesting() - hc.AddTestTablet("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, false, 10, fmt.Errorf("no connection")) - err = f(dg, target) - verifyShardErrors(t, err, want, vtrpcpb.Code_UNAVAILABLE) - - // tablet without connection - hc.Reset() - dg.tsc.ResetForTesting() - _ = hc.AddTestTablet("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, false, 10, nil).Tablet() - err = f(dg, target) - verifyShardErrors(t, err, want, vtrpcpb.Code_UNAVAILABLE) - - // retry error - hc.Reset() - dg.tsc.ResetForTesting() - sc1 := hc.AddTestTablet("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, true, 10, nil) - sc2 := hc.AddTestTablet("cell", "1.1.1.1", 1002, keyspace, shard, tabletType, true, 10, nil) - sc1.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1 - sc2.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1 - - err = f(dg, target) - verifyContainsError(t, err, "target: ks.0.replica", vtrpcpb.Code_FAILED_PRECONDITION) - - // fatal error - hc.Reset() - dg.tsc.ResetForTesting() - sc1 = hc.AddTestTablet("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, true, 10, nil) - sc2 = hc.AddTestTablet("cell", "1.1.1.1", 1002, keyspace, shard, tabletType, true, 10, nil) - sc1.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1 - sc2.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1 - err = f(dg, target) - verifyContainsError(t, err, "target: ks.0.replica", vtrpcpb.Code_FAILED_PRECONDITION) - - // server error - no retry - hc.Reset() - dg.tsc.ResetForTesting() - sc1 = hc.AddTestTablet("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, true, 10, nil) - sc1.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1 - err = f(dg, target) - verifyContainsError(t, err, "target: ks.0.replica", vtrpcpb.Code_INVALID_ARGUMENT) - - // no failure - hc.Reset() - dg.tsc.ResetForTesting() - hc.AddTestTablet("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, true, 10, nil) - err = f(dg, target) - if err != nil { - t.Errorf("want nil, got %v", err) - } -} - -func testDiscoveryGatewayTransact(t *testing.T, f func(dg *DiscoveryGateway, target *querypb.Target) error) { - keyspace := "ks" - shard := "0" - tabletType := topodatapb.TabletType_REPLICA - target := &querypb.Target{ - Keyspace: keyspace, - Shard: shard, - TabletType: tabletType, - } - hc := discovery.NewFakeLegacyHealthCheck() - dg := NewDiscoveryGateway(context.Background(), hc, nil, "cell", 2) - - // retry error - no retry - hc.Reset() - dg.tsc.ResetForTesting() - sc1 := hc.AddTestTablet("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, true, 10, nil) - sc2 := hc.AddTestTablet("cell", "1.1.1.1", 1002, keyspace, shard, tabletType, true, 10, nil) - sc1.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1 - sc2.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1 - - err := f(dg, target) - verifyContainsError(t, err, "target: ks.0.replica", vtrpcpb.Code_FAILED_PRECONDITION) - - // server error - no retry - hc.Reset() - dg.tsc.ResetForTesting() - sc1 = hc.AddTestTablet("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, true, 10, nil) - sc1.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1 - err = f(dg, target) - verifyContainsError(t, err, "target: ks.0.replica", vtrpcpb.Code_INVALID_ARGUMENT) -} diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index 322a6f3953a..30bf5a1ca7e 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -381,10 +381,6 @@ func (e *Executor) legacyExecute(ctx context.Context, safeSession *SafeSession, logStats.Keyspace = destKeyspace logStats.TabletType = destTabletType.String() - // Legacy gateway allows transactions only on PRIMARY - if UsingLegacyGateway() && safeSession.InTransaction() && destTabletType != topodatapb.TabletType_PRIMARY { - return 0, nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "transaction is supported only for primary tablet type, current type: %v", destTabletType) - } if bindVars == nil { bindVars = make(map[string]*querypb.BindVariable) } @@ -1083,83 +1079,42 @@ func (e *Executor) showTablets(show *sqlparser.ShowLegacy) (*sqltypes.Result, er tabletFilters := getTabletFilters(show) rows := [][]sqltypes.Value{} - if UsingLegacyGateway() { - status := e.scatterConn.GetLegacyHealthCheckCacheStatus() - for _, s := range status { - for _, ts := range s.TabletsStats { - state := "SERVING" - if !ts.Serving { - state = "NOT_SERVING" - } - mtst := ts.TabletExternallyReparentedTimestamp - mtstStr := "" - if mtst > 0 { - // this code depends on the fact that TabletExternallyReparentedTimestamp is the seconds since epoch start - mtstStr = time.Unix(mtst, 0).UTC().Format(time.RFC3339) - } - - skipTablet := false - for _, filter := range tabletFilters { - if !filter(ts.Tablet, state, mtst) { - skipTablet = true - break - } - } - - if skipTablet { - continue - } - - 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, - )) + status := e.scatterConn.GetHealthCheckCacheStatus() + for _, s := range status { + for _, ts := range s.TabletsStats { + state := "SERVING" + if !ts.Serving { + state = "NOT_SERVING" + } + mtst := ts.PrimaryTermStartTime + mtstStr := "" + if mtst > 0 { + // this code depends on the fact that PrimaryTermStartTime is the seconds since epoch start + mtstStr = time.Unix(mtst, 0).UTC().Format(time.RFC3339) } - } - } else { - status := e.scatterConn.GetHealthCheckCacheStatus() - for _, s := range status { - for _, ts := range s.TabletsStats { - state := "SERVING" - if !ts.Serving { - state = "NOT_SERVING" - } - mtst := ts.PrimaryTermStartTime - mtstStr := "" - if mtst > 0 { - // this code depends on the fact that PrimaryTermStartTime is the seconds since epoch start - mtstStr = time.Unix(mtst, 0).UTC().Format(time.RFC3339) - } - - skipTablet := false - for _, filter := range tabletFilters { - if !filter(ts.Tablet, state, mtst) { - skipTablet = true - break - } - } - if skipTablet { - continue + skipTablet := false + for _, filter := range tabletFilters { + if !filter(ts.Tablet, state, mtst) { + skipTablet = true + break } + } - 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, - )) + if skipTablet { + continue } + + 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{ @@ -1173,117 +1128,63 @@ func (e *Executor) showVitessReplicationStatus(ctx context.Context, show *sqlpar defer cancel() rows := [][]sqltypes.Value{} - // This is only used for tests - if UsingLegacyGateway() { - status := e.scatterConn.GetLegacyHealthCheckCacheStatus() - - for _, s := range status { - for _, ts := range s.TabletsStats { - // We only want to show REPLICA and RDONLY tablets - if ts.Tablet.Type != topodatapb.TabletType_REPLICA && ts.Tablet.Type != topodatapb.TabletType_RDONLY { - continue - } + status := e.scatterConn.GetHealthCheckCacheStatus() - tabletHostPort := ts.GetTabletHostPort() - throttlerStatus, err := getTabletThrottlerStatus(tabletHostPort) - if err != nil { - log.Warningf("Could not get throttler status from %s: %v", tabletHostPort, err) - } - - replSourceHost := "" - replSourcePort := int64(0) - replIOThreadHealth := "" - replSQLThreadHealth := "" - replLastError := "" - replLag := int64(-1) - sql := "show slave status" - results, err := e.txConn.gateway.Execute(ctx, ts.Target, sql, nil, 0, 0, nil) - if err != nil { - log.Warningf("Could not get replication status from %s: %v", tabletHostPort, err) - } else if results != nil && len(results.Rows) == 1 { - replSourceHost = results.Rows[0][1].ToString() - replSourcePort, _ = results.Rows[0][3].ToInt64() - replIOThreadHealth = results.Rows[0][10].ToString() - replSQLThreadHealth = results.Rows[0][11].ToString() - replLastError = results.Rows[0][19].ToString() - if ts.Stats != nil { - replLag = int64(ts.Stats.ReplicationLagSeconds) - } - } - replicationHealth := fmt.Sprintf("{\"EventStreamRunning\":\"%s\",\"EventApplierRunning\":\"%s\",\"LastError\":\"%s\"}", replIOThreadHealth, replSQLThreadHealth, replLastError) - - rows = append(rows, buildVarCharRow( - s.Target.Keyspace, - s.Target.Shard, - ts.Target.TabletType.String(), - topoproto.TabletAliasString(ts.Tablet.Alias), - ts.Tablet.Hostname, - fmt.Sprintf("%s:%d", replSourceHost, replSourcePort), - replicationHealth, - fmt.Sprintf("%d", replLag), - throttlerStatus, - )) + for _, s := range status { + for _, ts := range s.TabletsStats { + // We only want to show REPLICA and RDONLY tablets + if ts.Tablet.Type != topodatapb.TabletType_REPLICA && ts.Tablet.Type != topodatapb.TabletType_RDONLY { + continue } - } - } else { - status := e.scatterConn.GetHealthCheckCacheStatus() - for _, s := range status { - for _, ts := range s.TabletsStats { - // We only want to show REPLICA and RDONLY tablets - if ts.Tablet.Type != topodatapb.TabletType_REPLICA && ts.Tablet.Type != topodatapb.TabletType_RDONLY { + // Allow people to filter by Keyspace and Shard using a LIKE clause + if show.ShowTablesOpt != nil && show.ShowTablesOpt.Filter != nil { + ksFilterRegex := sqlparser.LikeToRegexp(show.ShowTablesOpt.Filter.Like) + keyspaceShardStr := fmt.Sprintf("%s/%s", ts.Tablet.Keyspace, ts.Tablet.Shard) + if !ksFilterRegex.MatchString(keyspaceShardStr) { continue } + } - // Allow people to filter by Keyspace and Shard using a LIKE clause - if show.ShowTablesOpt != nil && show.ShowTablesOpt.Filter != nil { - ksFilterRegex := sqlparser.LikeToRegexp(show.ShowTablesOpt.Filter.Like) - keyspaceShardStr := fmt.Sprintf("%s/%s", ts.Tablet.Keyspace, ts.Tablet.Shard) - if !ksFilterRegex.MatchString(keyspaceShardStr) { - continue - } - } - - tabletHostPort := ts.GetTabletHostPort() - throttlerStatus, err := getTabletThrottlerStatus(tabletHostPort) - if err != nil { - log.Warningf("Could not get throttler status from %s: %v", tabletHostPort, err) - } + tabletHostPort := ts.GetTabletHostPort() + throttlerStatus, err := getTabletThrottlerStatus(tabletHostPort) + if err != nil { + log.Warningf("Could not get throttler status from %s: %v", tabletHostPort, err) + } - replSourceHost := "" - replSourcePort := int64(0) - replIOThreadHealth := "" - replSQLThreadHealth := "" - replLastError := "" - replLag := int64(-1) - sql := "show slave status" - results, err := e.txConn.gateway.Execute(ctx, ts.Target, sql, nil, 0, 0, nil) - if err != nil || results == nil { - log.Warningf("Could not get replication status from %s: %v", tabletHostPort, err) - } else if row := results.Named().Row(); row != nil { - replSourceHost = row["Master_Host"].ToString() - replSourcePort, _ = row["Master_Port"].ToInt64() - replIOThreadHealth = row["Slave_IO_Running"].ToString() - replSQLThreadHealth = row["Slave_SQL_Running"].ToString() - replLastError = row["Last_Error"].ToString() - if ts.Stats != nil { - replLag = int64(ts.Stats.ReplicationLagSeconds) - } + replSourceHost := "" + replSourcePort := int64(0) + replIOThreadHealth := "" + replSQLThreadHealth := "" + replLastError := "" + replLag := int64(-1) + sql := "show slave status" + results, err := e.txConn.gateway.Execute(ctx, ts.Target, sql, nil, 0, 0, nil) + if err != nil || results == nil { + log.Warningf("Could not get replication status from %s: %v", tabletHostPort, err) + } else if row := results.Named().Row(); row != nil { + replSourceHost = row["Master_Host"].ToString() + replSourcePort, _ = row["Master_Port"].ToInt64() + replIOThreadHealth = row["Slave_IO_Running"].ToString() + replSQLThreadHealth = row["Slave_SQL_Running"].ToString() + replLastError = row["Last_Error"].ToString() + if ts.Stats != nil { + replLag = int64(ts.Stats.ReplicationLagSeconds) } - replicationHealth := fmt.Sprintf("{\"EventStreamRunning\":\"%s\",\"EventApplierRunning\":\"%s\",\"LastError\":\"%s\"}", replIOThreadHealth, replSQLThreadHealth, replLastError) - - rows = append(rows, buildVarCharRow( - s.Target.Keyspace, - s.Target.Shard, - ts.Target.TabletType.String(), - topoproto.TabletAliasString(ts.Tablet.Alias), - ts.Tablet.Hostname, - fmt.Sprintf("%s:%d", replSourceHost, replSourcePort), - replicationHealth, - fmt.Sprintf("%d", replLag), - throttlerStatus, - )) } + replicationHealth := fmt.Sprintf("{\"EventStreamRunning\":\"%s\",\"EventApplierRunning\":\"%s\",\"LastError\":\"%s\"}", replIOThreadHealth, replSQLThreadHealth, replLastError) + + rows = append(rows, buildVarCharRow( + s.Target.Keyspace, + s.Target.Shard, + ts.Target.TabletType.String(), + topoproto.TabletAliasString(ts.Tablet.Alias), + ts.Tablet.Hostname, + fmt.Sprintf("%s:%d", replSourceHost, replSourcePort), + replicationHealth, + fmt.Sprintf("%d", replLag), + throttlerStatus, + )) } } return &sqltypes.Result{ @@ -1602,9 +1503,6 @@ func (e *Executor) prepare(ctx context.Context, safeSession *SafeSession, sql st return nil, err } - if UsingLegacyGateway() && safeSession.InTransaction() && destTabletType != topodatapb.TabletType_PRIMARY { - return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "transaction is supported only for primary tablet type, current type: %v", destTabletType) - } if bindVars == nil { bindVars = make(map[string]*querypb.BindVariable) } diff --git a/go/vt/vtgate/executor_ddl_test.go b/go/vt/vtgate/executor_ddl_test.go index 23463846819..c89eac910d1 100644 --- a/go/vt/vtgate/executor_ddl_test.go +++ b/go/vt/vtgate/executor_ddl_test.go @@ -26,7 +26,7 @@ import ( ) func TestDDLFlags(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}) defer func() { *enableOnlineDDL = true diff --git a/go/vt/vtgate/executor_dml_test.go b/go/vt/vtgate/executor_dml_test.go index 2415b7e91c2..8666776b41f 100644 --- a/go/vt/vtgate/executor_dml_test.go +++ b/go/vt/vtgate/executor_dml_test.go @@ -42,7 +42,7 @@ import ( ) func TestUpdateEqual(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -249,7 +249,7 @@ func TestUpdateMultiOwned(t *testing.T) { } func TestUpdateComments(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := executorExec(executor, "update user set a=2 where id = 1 /* trailing */", nil) require.NoError(t, err) @@ -262,7 +262,7 @@ func TestUpdateComments(t *testing.T) { } func TestUpdateNormalize(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() executor.normalize = true _, err := executorExec(executor, "/* leading */ update user set a=2 where id = 1 /* trailing */", nil) @@ -296,7 +296,7 @@ func TestUpdateNormalize(t *testing.T) { } func TestDeleteEqual(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() sbc.SetResults([]*sqltypes.Result{{ Fields: []*querypb.Field{ @@ -409,7 +409,7 @@ func TestDeleteEqual(t *testing.T) { } func TestUpdateScatter(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := executorExec(executor, "update user_extra set col = 2", nil) require.NoError(t, err) // Queries get annotatted. @@ -422,7 +422,7 @@ func TestUpdateScatter(t *testing.T) { } func TestDeleteScatter(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := executorExec(executor, "delete from user_extra", nil) require.NoError(t, err) // Queries get annotatted. @@ -744,7 +744,7 @@ func TestDeleteUseHigherCostVindexIfBackfilling(t *testing.T) { } func TestDeleteByDestination(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() // This query is not supported in v3, so we know for sure is taking the DeleteByDestination route _, err := executorExec(executor, "delete from `TestExecutor[-]`.user_extra limit 10", nil) require.NoError(t, err) @@ -758,7 +758,7 @@ func TestDeleteByDestination(t *testing.T) { } func TestDeleteComments(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() sbc.SetResults([]*sqltypes.Result{{ Fields: []*querypb.Field{ @@ -794,7 +794,7 @@ func TestDeleteComments(t *testing.T) { } func TestInsertSharded(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -897,7 +897,7 @@ func TestInsertSharded(t *testing.T) { } func TestInsertShardedKeyrange(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() // If a unique vindex returns a keyrange, we fail the insert _, err := executorExec(executor, "insert into keyrange_table(krcol_unique, krcol) values(1, 1)", nil) @@ -976,7 +976,7 @@ func TestInsertShardedAutocommitLookup(t *testing.T) { } func TestInsertShardedIgnore(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() // Build the sequence of responses for sbclookup. This should // match the sequence of queries we validate below. @@ -1128,7 +1128,7 @@ func TestInsertShardedIgnore(t *testing.T) { func TestInsertOnDupKey(t *testing.T) { // This test just sanity checks that the statement is getting passed through // correctly. The full set of use cases are covered by TestInsertShardedIgnore. - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() sbclookup.SetResults([]*sqltypes.Result{sqltypes.MakeTestResult( sqltypes.MakeTestFields("b|a", "int64|varbinary"), "1|1", @@ -1170,7 +1170,7 @@ func TestInsertOnDupKey(t *testing.T) { } func TestAutocommitFail(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() query := "insert into user (id) values (1)" sbc1.MustFailCodes[vtrpcpb.Code_ALREADY_EXISTS] = 1 @@ -1187,7 +1187,7 @@ func TestAutocommitFail(t *testing.T) { } func TestInsertComments(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() _, err := executorExec(executor, "insert into user(id, v, name) values (1, 2, 'myname') /* trailing */", nil) require.NoError(t, err) @@ -1212,7 +1212,7 @@ func TestInsertComments(t *testing.T) { } func TestInsertGeneratorSharded(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() sbclookup.SetResults([]*sqltypes.Result{{ Rows: [][]sqltypes.Value{{ @@ -1251,7 +1251,7 @@ func TestInsertGeneratorSharded(t *testing.T) { } func TestInsertAutoincSharded(t *testing.T) { - router, sbc, _, _ := createLegacyExecutorEnv() + router, sbc, _, _ := createExecutorEnv() // Fake a mysql auto-inc response. wantResult := &sqltypes.Result{ @@ -1278,7 +1278,7 @@ func TestInsertAutoincSharded(t *testing.T) { } func TestInsertGeneratorUnsharded(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() result, err := executorExec(executor, "insert into main1(id, name) values (null, 'myname')", nil) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{ @@ -1299,7 +1299,7 @@ func TestInsertGeneratorUnsharded(t *testing.T) { } func TestInsertAutoincUnsharded(t *testing.T) { - router, _, _, sbclookup := createLegacyExecutorEnv() + router, _, _, sbclookup := createExecutorEnv() // Fake a mysql auto-inc response. query := "insert into simple(val) values ('val')" @@ -1319,13 +1319,11 @@ func TestInsertAutoincUnsharded(t *testing.T) { BindVariables: map[string]*querypb.BindVariable{}, }} assertQueries(t, sbclookup, wantQueries) - if !result.Equal(wantResult) { - t.Errorf("result: %+v, want %+v", result, wantResult) - } + assert.Equal(t, result, wantResult) } func TestInsertLookupOwned(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() _, err := executorExec(executor, "insert into music(user_id, id) values (2, 3)", nil) require.NoError(t, err) @@ -1349,7 +1347,7 @@ func TestInsertLookupOwned(t *testing.T) { } func TestInsertLookupOwnedGenerator(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() sbclookup.SetResults([]*sqltypes.Result{{ Rows: [][]sqltypes.Value{{ @@ -1388,7 +1386,7 @@ func TestInsertLookupOwnedGenerator(t *testing.T) { } func TestInsertLookupUnowned(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() _, err := executorExec(executor, "insert into music_extra(user_id, music_id) values (2, 3)", nil) require.NoError(t, err) @@ -1411,7 +1409,7 @@ func TestInsertLookupUnowned(t *testing.T) { } func TestInsertLookupUnownedUnsupplied(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() sbclookup.SetResults([]*sqltypes.Result{sqltypes.MakeTestResult( sqltypes.MakeTestFields("b|a", "int64|varbinary"), "3|1", @@ -1440,7 +1438,7 @@ func TestInsertLookupUnownedUnsupplied(t *testing.T) { // If a statement gets broken up into two, and the first one fails, // then an error should be returned normally. func TestInsertPartialFail1(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() // Make the first DML fail, there should be no rollback. sbclookup.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1 @@ -1459,7 +1457,7 @@ func TestInsertPartialFail1(t *testing.T) { // after successful execution of the first, then the transaction must // be rolled back due to partial execution. func TestInsertPartialFail2(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() // Make the second DML fail, it should result in a rollback. sbc1.MustFailExecute[sqlparser.StmtInsert] = 1 @@ -1498,7 +1496,7 @@ func TestInsertPartialFail2(t *testing.T) { } func TestMultiInsertSharded(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() _, err := executorExec(executor, "insert into user(id, v, name) values (1, 1, 'myname1'),(3, 3, 'myname3')", nil) require.NoError(t, err) @@ -1602,7 +1600,7 @@ func TestMultiInsertSharded(t *testing.T) { } func TestMultiInsertGenerator(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() sbclookup.SetResults([]*sqltypes.Result{{ Rows: [][]sqltypes.Value{{ @@ -1647,7 +1645,7 @@ func TestMultiInsertGenerator(t *testing.T) { } func TestMultiInsertGeneratorSparse(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() sbclookup.SetResults([]*sqltypes.Result{{ Rows: [][]sqltypes.Value{{ @@ -1791,7 +1789,7 @@ func TestKeyDestRangeQuery(t *testing.T) { for _, tc := range tests { t.Run(tc.targetString+" - "+tc.inputQuery, func(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() primarySession.TargetString = tc.targetString _, err := executorExec(executor, tc.inputQuery, nil) @@ -1812,7 +1810,7 @@ func TestKeyDestRangeQuery(t *testing.T) { } // it does not work for inserts - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() primarySession.TargetString = "TestExecutor[-]" _, err := executorExec(executor, insertInput, nil) @@ -1832,7 +1830,7 @@ func assertQueriesContain(t *testing.T, sql, sbcName string, sbc *sandboxconn.Sa // Prepared statement tests func TestUpdateEqualWithPrepare(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -1850,7 +1848,7 @@ func TestUpdateEqualWithPrepare(t *testing.T) { assertQueries(t, sbc1, nil) } func TestInsertShardedWithPrepare(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -1871,7 +1869,7 @@ func TestInsertShardedWithPrepare(t *testing.T) { } func TestDeleteEqualWithPrepare(t *testing.T) { - executor, sbc, _, sbclookup := createLegacyExecutorEnv() + executor, sbc, _, sbclookup := createExecutorEnv() _, err := executorPrepare(executor, "delete from user where id = :id0", map[string]*querypb.BindVariable{ "id0": sqltypes.Int64BindVariable(1), }) @@ -1885,7 +1883,7 @@ func TestDeleteEqualWithPrepare(t *testing.T) { } func TestUpdateLastInsertID(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() executor.normalize = true sql := "update user set a = last_insert_id() where id = 1" @@ -1903,7 +1901,7 @@ func TestUpdateLastInsertID(t *testing.T) { } func TestDeleteLookupOwnedEqual(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() sbc1.SetResults([]*sqltypes.Result{ sqltypes.MakeTestResult(sqltypes.MakeTestFields("uniq_col|keyspace_id", "int64|varbinary"), "1|N±\u0090ɢú\u0016\u009C"), diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go index fd7cf80db23..c0199a898ab 100644 --- a/go/vt/vtgate/executor_framework_test.go +++ b/go/vt/vtgate/executor_framework_test.go @@ -438,49 +438,10 @@ func newKeyRangeLookuperUnique(name string, params map[string]string) (vindexes. func init() { vindexes.Register("keyrange_lookuper", newKeyRangeLookuper) vindexes.Register("keyrange_lookuper_unique", newKeyRangeLookuperUnique) - // Use legacy gateway until we can rewrite these tests to use new tabletgateway - *GatewayImplementation = GatewayImplementationDiscovery -} - -func createLegacyExecutorEnv() (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.NewFakeLegacyHealthCheck() - s := createSandbox("TestExecutor") - s.VSchema = executorVSchema - serv := newSandboxForCells([]string{cell}) - resolver := newTestLegacyResolver(hc, serv, cell) - sbc1 = hc.AddTestTablet(cell, "-20", 1, "TestExecutor", "-20", topodatapb.TabletType_PRIMARY, true, 1, nil) - sbc2 = hc.AddTestTablet(cell, "40-60", 1, "TestExecutor", "40-60", topodatapb.TabletType_PRIMARY, true, 1, nil) - // Create these connections so scatter queries don't fail. - _ = hc.AddTestTablet(cell, "20-40", 1, "TestExecutor", "20-40", topodatapb.TabletType_PRIMARY, true, 1, nil) - _ = hc.AddTestTablet(cell, "60-80", 1, "TestExecutor", "60-80", topodatapb.TabletType_PRIMARY, true, 1, nil) - _ = hc.AddTestTablet(cell, "80-a0", 1, "TestExecutor", "80-a0", topodatapb.TabletType_PRIMARY, true, 1, nil) - _ = hc.AddTestTablet(cell, "a0-c0", 1, "TestExecutor", "a0-c0", topodatapb.TabletType_PRIMARY, true, 1, nil) - _ = hc.AddTestTablet(cell, "c0-e0", 1, "TestExecutor", "c0-e0", topodatapb.TabletType_PRIMARY, true, 1, nil) - _ = hc.AddTestTablet(cell, "e0-", 1, "TestExecutor", "e0-", topodatapb.TabletType_PRIMARY, true, 1, nil) - // Below is needed so that SendAnyWherePlan doesn't fail - _ = hc.AddTestTablet(cell, "e0-", 1, "TestXBadVSchema", "-20", topodatapb.TabletType_PRIMARY, true, 1, nil) - - createSandbox(KsTestUnsharded) - sbclookup = hc.AddTestTablet(cell, "0", 1, KsTestUnsharded, "0", topodatapb.TabletType_PRIMARY, true, 1, nil) - - // Ues the 'X' in the name to ensure it's not alphabetically first. - // Otherwise, it would become the default keyspace for the dual table. - bad := createSandbox("TestXBadSharding") - bad.VSchema = badVSchema - - getSandbox(KsTestUnsharded).VSchema = unshardedVSchema - executor = NewExecutor(context.Background(), serv, cell, resolver, false, false, testBufferSize, cache.DefaultConfig, nil, false) - - key.AnyShardPicker = DestinationAnyShardPickerFirstShard{} - return executor, sbc1, sbc2, sbclookup } func createExecutorEnv() (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn.SandboxConn) { // Use legacy gateway until we can rewrite these tests to use new tabletgateway - *GatewayImplementation = tabletGatewayImplementation cell := "aa" hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") @@ -496,6 +457,8 @@ func createExecutorEnv() (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn _ = hc.AddTestTablet(cell, "a0-c0", 1, "TestExecutor", "a0-c0", topodatapb.TabletType_PRIMARY, true, 1, nil) _ = hc.AddTestTablet(cell, "c0-e0", 1, "TestExecutor", "c0-e0", topodatapb.TabletType_PRIMARY, true, 1, nil) _ = hc.AddTestTablet(cell, "e0-", 1, "TestExecutor", "e0-", topodatapb.TabletType_PRIMARY, true, 1, nil) + // Below is needed so that SendAnyWherePlan doesn't fail + _ = hc.AddTestTablet(cell, "e0-", 1, "TestXBadVSchema", "-20", topodatapb.TabletType_PRIMARY, true, 1, nil) createSandbox(KsTestUnsharded) _ = topo.NewShardInfo(KsTestUnsharded, "0", &topodatapb.Shard{}, nil) @@ -531,11 +494,11 @@ func createExecutorEnv() (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn func createCustomExecutor(vschema string) (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn.SandboxConn) { cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = vschema serv := newSandboxForCells([]string{cell}) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) sbc1 = hc.AddTestTablet(cell, "-20", 1, "TestExecutor", "-20", topodatapb.TabletType_PRIMARY, true, 1, nil) sbc2 = hc.AddTestTablet(cell, "40-60", 1, "TestExecutor", "40-60", topodatapb.TabletType_PRIMARY, true, 1, nil) @@ -549,11 +512,11 @@ func createCustomExecutor(vschema string) (executor *Executor, sbc1, sbc2, sbclo func createCustomExecutorSetValues(vschema string, values []*sqltypes.Result) (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn.SandboxConn) { cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = vschema serv := newSandboxForCells([]string{cell}) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} sbcs := []*sandboxconn.SandboxConn{} for _, shard := range shards { @@ -782,11 +745,6 @@ func testQueryLogWithSavepoint(t *testing.T, logChan chan interface{}, method, s return logStats } -func newTestLegacyResolver(hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string) *Resolver { - sc := newTestLegacyScatterConn(hc, serv, cell) - srvResolver := srvtopo.NewResolver(serv, sc.gateway, cell) - return NewResolver(srvResolver, serv, cell, sc) -} func newTestResolver(hc discovery.HealthCheck, serv srvtopo.Server, cell string) *Resolver { sc := newTestScatterConn(hc, serv, cell) srvResolver := srvtopo.NewResolver(serv, sc.gateway, cell) diff --git a/go/vt/vtgate/executor_scatter_stats_test.go b/go/vt/vtgate/executor_scatter_stats_test.go index a51fa7e64ee..eff8d4f6c77 100644 --- a/go/vt/vtgate/executor_scatter_stats_test.go +++ b/go/vt/vtgate/executor_scatter_stats_test.go @@ -28,7 +28,7 @@ import ( ) func TestScatterStatsWithNoScatterQuery(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) _, err := executor.Execute(context.Background(), "TestExecutorResultsExceeded", session, "select * from main1", nil) @@ -40,7 +40,7 @@ func TestScatterStatsWithNoScatterQuery(t *testing.T) { } func TestScatterStatsWithSingleScatterQuery(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) _, err := executor.Execute(context.Background(), "TestExecutorResultsExceeded", session, "select * from user", nil) @@ -52,7 +52,7 @@ func TestScatterStatsWithSingleScatterQuery(t *testing.T) { } func TestScatterStatsHttpWriting(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) _, err := executor.Execute(context.Background(), "TestExecutorResultsExceeded", session, "select * from user", nil) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index fe451b2eb22..b26f2ca61b4 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -45,7 +45,7 @@ import ( ) func TestSelectNext(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() query := "select next :n values from user_seq" bv := map[string]*querypb.BindVariable{"n": sqltypes.Int64BindVariable(2)} @@ -60,7 +60,7 @@ func TestSelectNext(t *testing.T) { } func TestSelectDBA(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() query := "select * from INFORMATION_SCHEMA.foo" _, err := executor.Execute(context.Background(), "TestSelectDBA", @@ -195,7 +195,7 @@ func TestGen4SelectDBA(t *testing.T) { } func TestUnsharded(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() _, err := executorExec(executor, "select id from music_user_map where id = 1", nil) require.NoError(t, err) @@ -207,7 +207,7 @@ func TestUnsharded(t *testing.T) { } func TestUnshardedComments(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() _, err := executorExec(executor, "/* leading */ select id from music_user_map where id = 1 /* trailing */", nil) require.NoError(t, err) @@ -248,7 +248,7 @@ func TestUnshardedComments(t *testing.T) { } func TestStreamUnsharded(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -264,7 +264,7 @@ func TestStreamUnsharded(t *testing.T) { } func TestStreamBuffering(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() // This test is similar to TestStreamUnsharded except that it returns a Result > 10 bytes, // such that the splitting of the Result into multiple Result responses gets tested. @@ -315,7 +315,7 @@ func TestStreamBuffering(t *testing.T) { } func TestStreamLimitOffset(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() // This test is similar to TestStreamUnsharded except that it returns a Result > 10 bytes, // such that the splitting of the Result into multiple Result responses gets tested. @@ -390,7 +390,7 @@ func TestStreamLimitOffset(t *testing.T) { func TestSelectLastInsertId(t *testing.T) { primarySession.LastInsertId = 52 - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.normalize = true logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -415,7 +415,7 @@ func TestSelectSystemVariables(t *testing.T) { ReadAfterWriteTimeout: 13, SessionTrackGtids: true, } - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.normalize = true logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -462,7 +462,7 @@ func TestSelectSystemVariables(t *testing.T) { } func TestSelectInitializedVitessAwareVariable(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.normalize = true logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -493,7 +493,7 @@ func TestSelectInitializedVitessAwareVariable(t *testing.T) { } func TestSelectUserDefindVariable(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.normalize = true logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -526,7 +526,7 @@ func TestSelectUserDefindVariable(t *testing.T) { } func TestFoundRows(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.normalize = true logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -550,7 +550,7 @@ func TestFoundRows(t *testing.T) { } func TestRowCount(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.normalize = true logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -580,7 +580,7 @@ func testRowCount(t *testing.T, executor *Executor, wantRowCount int64) { } func TestSelectLastInsertIdInUnion(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.normalize = true sql := "select last_insert_id() as id union select id from user" _, err := executorExec(executor, sql, map[string]*querypb.BindVariable{}) @@ -589,7 +589,7 @@ func TestSelectLastInsertIdInUnion(t *testing.T) { } func TestSelectLastInsertIdInWhere(t *testing.T) { - executor, _, _, lookup := createLegacyExecutorEnv() + executor, _, _, lookup := createExecutorEnv() executor.normalize = true logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -606,7 +606,7 @@ func TestSelectLastInsertIdInWhere(t *testing.T) { } func TestLastInsertIDInVirtualTable(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() executor.normalize = true result1 := []*sqltypes.Result{{ Fields: []*querypb.Field{ @@ -631,7 +631,7 @@ func TestLastInsertIDInVirtualTable(t *testing.T) { } func TestLastInsertIDInSubQueryExpression(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() executor.normalize = true primarySession.LastInsertId = 12345 defer func() { @@ -656,7 +656,7 @@ func TestLastInsertIDInSubQueryExpression(t *testing.T) { } func TestSelectDatabase(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.normalize = true sql := "select database()" newSession := proto.Clone(primarySession).(*vtgatepb.Session) @@ -682,7 +682,7 @@ func TestSelectDatabase(t *testing.T) { } func TestSelectBindvars(t *testing.T) { - executor, sbc1, sbc2, lookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, lookup := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -792,7 +792,7 @@ func TestSelectBindvars(t *testing.T) { } func TestSelectEqual(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() _, err := executorExec(executor, "select id from user where id = 1", nil) require.NoError(t, err) @@ -859,7 +859,7 @@ func TestSelectEqual(t *testing.T) { } func TestSelectDual(t *testing.T) { - executor, sbc1, _, lookup := createLegacyExecutorEnv() + executor, sbc1, _, lookup := createExecutorEnv() _, err := executorExec(executor, "select @@aa.bb from dual", nil) require.NoError(t, err) @@ -875,7 +875,7 @@ func TestSelectDual(t *testing.T) { } func TestSelectComments(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := executorExec(executor, "/* leading */ select id from user where id = 1 /* trailing */", nil) require.NoError(t, err) @@ -891,7 +891,7 @@ func TestSelectComments(t *testing.T) { } func TestSelectNormalize(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() executor.normalize = true _, err := executorExec(executor, "/* leading */ select id from user where id = 1 /* trailing */", nil) @@ -925,7 +925,7 @@ func TestSelectNormalize(t *testing.T) { } func TestSelectCaseSensitivity(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := executorExec(executor, "select Id from user where iD = 1", nil) require.NoError(t, err) @@ -941,7 +941,7 @@ func TestSelectCaseSensitivity(t *testing.T) { } func TestStreamSelectEqual(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() sql := "select id from user where id = 1" result, err := executorStream(executor, sql) @@ -953,7 +953,7 @@ func TestStreamSelectEqual(t *testing.T) { } func TestSelectKeyRange(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := executorExec(executor, "select krcol_unique, krcol from keyrange_table where krcol = 1", nil) require.NoError(t, err) @@ -969,7 +969,7 @@ func TestSelectKeyRange(t *testing.T) { } func TestSelectKeyRangeUnique(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() _, err := executorExec(executor, "select krcol_unique, krcol from keyrange_table where krcol_unique = 1", nil) require.NoError(t, err) @@ -985,7 +985,7 @@ func TestSelectKeyRangeUnique(t *testing.T) { } func TestSelectIN(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() // Constant in IN clause is just a number, not a bind variable. _, err := executorExec(executor, "select id from user where id in (1)", nil) @@ -1073,7 +1073,7 @@ func TestSelectIN(t *testing.T) { } func TestStreamSelectIN(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() sql := "select id from user where id in (1)" result, err := executorStream(executor, sql) @@ -1122,14 +1122,14 @@ func createExecutor(serv *sandboxTopo, cell string, resolver *Resolver) *Executo } func TestSelectScatter(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for _, shard := range shards { @@ -1154,14 +1154,14 @@ func TestSelectScatter(t *testing.T) { } func TestSelectScatterPartial(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for _, shard := range shards { @@ -1214,14 +1214,14 @@ func TestSelectScatterPartial(t *testing.T) { } func TestSelectScatterPartialOLAP(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for _, shard := range shards { @@ -1265,7 +1265,7 @@ func TestSelectScatterPartialOLAP(t *testing.T) { } func TestSelectScatterPartialOLAP2(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") @@ -1321,14 +1321,14 @@ func TestSelectScatterPartialOLAP2(t *testing.T) { } func TestStreamSelectScatter(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} for _, shard := range shards { _ = hc.AddTestTablet(cell, shard, 1, "TestExecutor", shard, topodatapb.TabletType_PRIMARY, true, 1, nil) @@ -1356,14 +1356,14 @@ func TestStreamSelectScatter(t *testing.T) { // TestSelectScatterOrderBy will run an ORDER BY query that will scatter out to 8 shards and return the 8 rows (one per shard) sorted. func TestSelectScatterOrderBy(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for i, shard := range shards { @@ -1422,14 +1422,14 @@ func TestSelectScatterOrderBy(t *testing.T) { // TestSelectScatterOrderByVarChar will run an ORDER BY query that will scatter out to 8 shards and return the 8 rows (one per shard) sorted. func TestSelectScatterOrderByVarChar(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for i, shard := range shards { @@ -1486,14 +1486,14 @@ func TestSelectScatterOrderByVarChar(t *testing.T) { } func TestStreamSelectScatterOrderBy(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for i, shard := range shards { @@ -1544,14 +1544,14 @@ func TestStreamSelectScatterOrderBy(t *testing.T) { } func TestStreamSelectScatterOrderByVarChar(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for i, shard := range shards { @@ -1602,14 +1602,14 @@ func TestStreamSelectScatterOrderByVarChar(t *testing.T) { // TestSelectScatterAggregate will run an aggregate query that will scatter out to 8 shards and return 4 aggregated rows. func TestSelectScatterAggregate(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for i, shard := range shards { @@ -1661,14 +1661,14 @@ func TestSelectScatterAggregate(t *testing.T) { } func TestStreamSelectScatterAggregate(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for i, shard := range shards { @@ -1721,14 +1721,14 @@ func TestStreamSelectScatterAggregate(t *testing.T) { // TestSelectScatterLimit will run a limit query (ordered for consistency) against // a scatter route and verify that the limit primitive works as intended. func TestSelectScatterLimit(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for i, shard := range shards { @@ -1789,14 +1789,14 @@ func TestSelectScatterLimit(t *testing.T) { // TestStreamSelectScatterLimit will run a streaming limit query (ordered for consistency) against // a scatter route and verify that the limit primitive works as intended. func TestStreamSelectScatterLimit(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} var conns []*sandboxconn.SandboxConn for i, shard := range shards { @@ -1856,7 +1856,7 @@ func TestStreamSelectScatterLimit(t *testing.T) { // TODO(sougou): stream and non-stream testing are very similar. // Could reuse code, func TestSimpleJoin(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -1893,7 +1893,7 @@ func TestSimpleJoin(t *testing.T) { } func TestJoinComments(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -1915,7 +1915,7 @@ func TestJoinComments(t *testing.T) { } func TestSimpleJoinStream(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -1953,7 +1953,7 @@ func TestSimpleJoinStream(t *testing.T) { } func TestVarJoin(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -1988,7 +1988,7 @@ func TestVarJoin(t *testing.T) { } func TestVarJoinStream(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -2023,7 +2023,7 @@ func TestVarJoinStream(t *testing.T) { } func TestLeftJoin(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) result1 := []*sqltypes.Result{{ @@ -2066,7 +2066,7 @@ func TestLeftJoin(t *testing.T) { } func TestLeftJoinStream(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() result1 := []*sqltypes.Result{{ Fields: []*querypb.Field{ {Name: "id", Type: sqltypes.Int32}, @@ -2106,7 +2106,7 @@ func TestLeftJoinStream(t *testing.T) { } func TestEmptyJoin(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() // Empty result requires a field query for the second part of join, // which is sent to shard 0. sbc1.SetResults([]*sqltypes.Result{{ @@ -2142,7 +2142,7 @@ func TestEmptyJoin(t *testing.T) { } func TestEmptyJoinStream(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() // Empty result requires a field query for the second part of join, // which is sent to shard 0. sbc1.SetResults([]*sqltypes.Result{{ @@ -2178,7 +2178,7 @@ func TestEmptyJoinStream(t *testing.T) { } func TestEmptyJoinRecursive(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() // Make sure it also works recursively. sbc1.SetResults([]*sqltypes.Result{{ Fields: []*querypb.Field{ @@ -2222,7 +2222,7 @@ func TestEmptyJoinRecursive(t *testing.T) { } func TestEmptyJoinRecursiveStream(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() // Make sure it also works recursively. sbc1.SetResults([]*sqltypes.Result{{ Fields: []*querypb.Field{ @@ -2266,7 +2266,7 @@ func TestEmptyJoinRecursiveStream(t *testing.T) { } func TestCrossShardSubquery(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() result1 := []*sqltypes.Result{{ Fields: []*querypb.Field{ {Name: "id", Type: sqltypes.Int32}, @@ -2349,7 +2349,7 @@ func TestSubQueryAndQueryWithLimit(t *testing.T) { } func TestCrossShardSubqueryStream(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() result1 := []*sqltypes.Result{{ Fields: []*querypb.Field{ {Name: "id", Type: sqltypes.Int32}, @@ -2390,7 +2390,7 @@ func TestCrossShardSubqueryStream(t *testing.T) { } func TestCrossShardSubqueryGetFields(t *testing.T) { - executor, sbc1, _, sbclookup := createLegacyExecutorEnv() + executor, sbc1, _, sbclookup := createExecutorEnv() sbclookup.SetResults([]*sqltypes.Result{{ Fields: []*querypb.Field{ {Name: "col", Type: sqltypes.Int32}, @@ -2428,7 +2428,7 @@ func TestCrossShardSubqueryGetFields(t *testing.T) { } func TestSelectBindvarswithPrepare(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -2460,7 +2460,7 @@ func TestSelectDatabasePrepare(t *testing.T) { } func TestSelectWithUnionAll(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() executor.normalize = true sql := "select id from user where id in (1, 2, 3) union all select id from user where id in (1, 2, 3)" bv, _ := sqltypes.BuildBindVariable([]int64{1, 2, 3}) @@ -2588,7 +2588,7 @@ func TestSelectFromInformationSchema(t *testing.T) { } func TestStreamOrderByLimitWithMultipleResults(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") @@ -2928,9 +2928,8 @@ func TestGen4MultiColMultiEqual(t *testing.T) { } func TestRegionRange(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. - *GatewayImplementation = tabletGatewayImplementation *plannerVersion = "gen4" defer func() { *plannerVersion = "v3" @@ -2987,7 +2986,6 @@ func TestRegionRange(t *testing.T) { func TestMultiCol(t *testing.T) { // Special setup: Don't use createLegacyExecutorEnv. - *GatewayImplementation = tabletGatewayImplementation *plannerVersion = "gen4" defer func() { *plannerVersion = "v3" @@ -3045,7 +3043,6 @@ func TestMultiCol(t *testing.T) { func TestMultiColPartial(t *testing.T) { // Special setup: Don't use createLegacyExecutorEnv. - *GatewayImplementation = tabletGatewayImplementation *plannerVersion = "gen4" defer func() { *plannerVersion = "v3" diff --git a/go/vt/vtgate/executor_set_test.go b/go/vt/vtgate/executor_set_test.go index a26174cb6de..c8e64906345 100644 --- a/go/vt/vtgate/executor_set_test.go +++ b/go/vt/vtgate/executor_set_test.go @@ -264,7 +264,7 @@ func TestExecutorSet(t *testing.T) { } func TestExecutorSetOp(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() *sysVarSetEnabled = true returnResult := func(columnName, typ, value string) *sqltypes.Result { @@ -366,7 +366,7 @@ func TestExecutorSetOp(t *testing.T) { } func TestExecutorSetMetadata(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true}) set := "set @@vitess_metadata.app_keyspace_v1= '1'" @@ -378,7 +378,7 @@ func TestExecutorSetMetadata(t *testing.T) { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, _, _, _ = createLegacyExecutorEnv() + executor, _, _, _ = createExecutorEnv() session = NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true}) set = "set @@vitess_metadata.app_keyspace_v1= '1'" @@ -423,7 +423,7 @@ func TestExecutorSetMetadata(t *testing.T) { } func TestPlanExecutorSetUDV(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() testcases := []struct { in string @@ -453,7 +453,7 @@ func TestPlanExecutorSetUDV(t *testing.T) { } func TestSetUDVFromTabletInput(t *testing.T) { - executor, sbc1, _, _ := createLegacyExecutorEnv() + executor, sbc1, _, _ := createExecutorEnv() fields := sqltypes.MakeTestFields("some", "VARBINARY") sbc1.SetResults([]*sqltypes.Result{ diff --git a/go/vt/vtgate/executor_stream_test.go b/go/vt/vtgate/executor_stream_test.go index 548e8ab5122..e1926cb0250 100644 --- a/go/vt/vtgate/executor_stream_test.go +++ b/go/vt/vtgate/executor_stream_test.go @@ -34,7 +34,7 @@ import ( ) func TestStreamSQLUnsharded(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -48,14 +48,14 @@ func TestStreamSQLUnsharded(t *testing.T) { } func TestStreamSQLSharded(t *testing.T) { - // Special setup: Don't use createLegacyExecutorEnv. + // Special setup: Don't use createExecutorEnv. cell := "aa" - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") s.VSchema = executorVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := newSandboxForCells([]string{cell}) - resolver := newTestLegacyResolver(hc, serv, cell) + resolver := newTestResolver(hc, serv, cell) shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"} for _, shard := range shards { _ = hc.AddTestTablet(cell, shard, 1, "TestExecutor", shard, topodatapb.TabletType_PRIMARY, true, 1, nil) diff --git a/go/vt/vtgate/executor_test.go b/go/vt/vtgate/executor_test.go index 0f22d9a7fbf..afb1e72a73a 100644 --- a/go/vt/vtgate/executor_test.go +++ b/go/vt/vtgate/executor_test.go @@ -60,7 +60,7 @@ func TestExecutorResultsExceeded(t *testing.T) { *warnMemoryRows = 3 defer func() { *warnMemoryRows = save }() - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) initial := warnings.Counts()["ResultsExceeded"] @@ -83,7 +83,7 @@ func TestExecutorMaxMemoryRowsExceeded(t *testing.T) { *maxMemoryRows = 3 defer func() { *maxMemoryRows = save }() - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) result := sqltypes.MakeTestResult(sqltypes.MakeTestFields("col", "int64"), "1", "2", "3", "4") fn := func(r *sqltypes.Result) error { @@ -116,7 +116,7 @@ func TestExecutorMaxMemoryRowsExceeded(t *testing.T) { } func TestLegacyExecutorTransactionsNoAutoCommit(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) logChan := QueryLogger.Subscribe("Test") @@ -198,7 +198,7 @@ func TestLegacyExecutorTransactionsNoAutoCommit(t *testing.T) { } func TestDirectTargetRewrites(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() executor.normalize = true session := &vtgatepb.Session{ @@ -217,7 +217,7 @@ func TestDirectTargetRewrites(t *testing.T) { } func TestExecutorTransactionsAutoCommit(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true}) logChan := QueryLogger.Subscribe("Test") @@ -327,7 +327,7 @@ func TestExecutorDeleteMetadata(t *testing.T) { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true}) set := "set @@vitess_metadata.app_v1= '1'" @@ -354,7 +354,7 @@ func TestExecutorDeleteMetadata(t *testing.T) { } func TestExecutorAutocommit(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) logChan := QueryLogger.Subscribe("Test") @@ -454,7 +454,7 @@ func TestExecutorAutocommit(t *testing.T) { } func TestExecutorShowColumns(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: ""}) queries := []string{ @@ -492,7 +492,7 @@ func TestExecutorShowColumns(t *testing.T) { } func TestExecutorShow(t *testing.T) { - executor, _, _, sbclookup := createLegacyExecutorEnv() + executor, _, _, sbclookup := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) for _, query := range []string{"show vitess_keyspaces", "show keyspaces"} { @@ -1037,7 +1037,7 @@ func TestExecutorShow(t *testing.T) { } func TestExecutorShowTargeted(t *testing.T) { - executor, _, sbc2, _ := createLegacyExecutorEnv() + executor, _, sbc2, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor/40-60"}) queries := []string{ @@ -1063,7 +1063,7 @@ func TestExecutorShowTargeted(t *testing.T) { } func TestExecutorUse(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{Autocommit: true, TargetString: "@primary"}) stmts := []string{ @@ -1094,7 +1094,7 @@ func TestExecutorUse(t *testing.T) { } func TestExecutorComment(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() stmts := []string{ "/*! SET autocommit=1*/", @@ -1114,7 +1114,7 @@ func TestExecutorComment(t *testing.T) { } func TestExecutorOther(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() type cnts struct { Sbc1Cnt int64 @@ -1209,7 +1209,7 @@ func TestExecutorDDL(t *testing.T) { logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() type cnts struct { Sbc1Cnt int64 @@ -1360,7 +1360,7 @@ func TestExecutorAlterVSchemaKeyspace(t *testing.T) { defer func() { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true}) vschemaUpdates := make(chan *vschemapb.SrvVSchema, 2) @@ -1388,7 +1388,7 @@ func TestExecutorCreateVindexDDL(t *testing.T) { defer func() { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() ks := "TestExecutor" vschemaUpdates := make(chan *vschemapb.SrvVSchema, 4) @@ -1460,7 +1460,7 @@ func TestExecutorAddDropVschemaTableDDL(t *testing.T) { defer func() { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() ks := KsTestUnsharded vschemaUpdates := make(chan *vschemapb.SrvVSchema, 4) @@ -1508,7 +1508,7 @@ func TestExecutorAddDropVschemaTableDDL(t *testing.T) { } func TestExecutorVindexDDLACL(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() ks := "TestExecutor" session := NewSafeSession(&vtgatepb.Session{TargetString: ks}) @@ -1553,7 +1553,7 @@ func TestExecutorVindexDDLACL(t *testing.T) { } func TestExecutorUnrecognized(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() _, err := executor.Execute(ctx, "TestExecute", NewSafeSession(&vtgatepb.Session{}), "invalid statement", nil) require.Error(t, err, "unrecognized statement: invalid statement'") } @@ -1561,7 +1561,7 @@ func TestExecutorUnrecognized(t *testing.T) { // TestVSchemaStats makes sure the building and displaying of the // VSchemaStats works. func TestVSchemaStats(t *testing.T) { - r, _, _, _ := createLegacyExecutorEnv() + r, _, _, _ := createExecutorEnv() stats := r.VSchemaStats() @@ -1582,7 +1582,7 @@ func TestVSchemaStats(t *testing.T) { } func TestGetPlanUnnormalized(t *testing.T) { - r, _, _, _ := createLegacyExecutorEnv() + r, _, _, _ := createExecutorEnv() emptyvc, _ := newVCursorImpl(ctx, NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false) unshardedvc, _ := newVCursorImpl(ctx, NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false) @@ -1659,7 +1659,7 @@ func getPlanCached(t *testing.T, e *Executor, vcursor *vcursorImpl, sql string, } func TestGetPlanCacheUnnormalized(t *testing.T) { - r, _, _, _ := createLegacyExecutorEnv() + r, _, _, _ := createExecutorEnv() emptyvc, _ := newVCursorImpl(ctx, NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false) query1 := "select * from music_user_map where id = 1" @@ -1680,7 +1680,7 @@ func TestGetPlanCacheUnnormalized(t *testing.T) { } // Skip cache using directive - r, _, _, _ = createLegacyExecutorEnv() + r, _, _, _ = createExecutorEnv() unshardedvc, _ := newVCursorImpl(ctx, NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false) query1 = "insert /*vt+ SKIP_QUERY_PLAN_CACHE=1 */ into user(id) values (1), (2)" @@ -1703,7 +1703,7 @@ func TestGetPlanCacheUnnormalized(t *testing.T) { } func TestGetPlanCacheNormalized(t *testing.T) { - r, _, _, _ := createLegacyExecutorEnv() + r, _, _, _ := createExecutorEnv() r.normalize = true emptyvc, _ := newVCursorImpl(ctx, NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false) @@ -1722,7 +1722,7 @@ func TestGetPlanCacheNormalized(t *testing.T) { } // Skip cache using directive - r, _, _, _ = createLegacyExecutorEnv() + r, _, _, _ = createExecutorEnv() r.normalize = true unshardedvc, _ := newVCursorImpl(ctx, NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false) @@ -1746,7 +1746,7 @@ func TestGetPlanCacheNormalized(t *testing.T) { } func TestGetPlanNormalized(t *testing.T) { - r, _, _, _ := createLegacyExecutorEnv() + r, _, _, _ := createExecutorEnv() r.normalize = true emptyvc, _ := newVCursorImpl(ctx, NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false) unshardedvc, _ := newVCursorImpl(ctx, NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false) @@ -1822,7 +1822,7 @@ func TestGetPlanNormalized(t *testing.T) { } func TestPassthroughDDL(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() primarySession.TargetString = "TestExecutor" alterDDL := "/* leading */ alter table passthrough_ddl add columne col bigint default 123 /* trailing */" @@ -1871,7 +1871,7 @@ func TestPassthroughDDL(t *testing.T) { } func TestParseEmptyTargetSingleKeyspace(t *testing.T) { - r, _, _, _ := createLegacyExecutorEnv() + r, _, _, _ := createExecutorEnv() altVSchema := &vindexes.VSchema{ Keyspaces: map[string]*vindexes.KeyspaceSchema{ KsTestUnsharded: r.vschema.Keyspaces[KsTestUnsharded], @@ -1893,7 +1893,7 @@ func TestParseEmptyTargetSingleKeyspace(t *testing.T) { } func TestParseEmptyTargetMultiKeyspace(t *testing.T) { - r, _, _, _ := createLegacyExecutorEnv() + r, _, _, _ := createExecutorEnv() altVSchema := &vindexes.VSchema{ Keyspaces: map[string]*vindexes.KeyspaceSchema{ KsTestUnsharded: r.vschema.Keyspaces[KsTestUnsharded], @@ -1916,7 +1916,7 @@ func TestParseEmptyTargetMultiKeyspace(t *testing.T) { } func TestParseTargetSingleKeyspace(t *testing.T) { - r, _, _, _ := createLegacyExecutorEnv() + r, _, _, _ := createExecutorEnv() altVSchema := &vindexes.VSchema{ Keyspaces: map[string]*vindexes.KeyspaceSchema{ KsTestUnsharded: r.vschema.Keyspaces[KsTestUnsharded], @@ -1941,7 +1941,7 @@ func TestDebugVSchema(t *testing.T) { resp := httptest.NewRecorder() req, _ := http.NewRequest("GET", "/debug/vschema", nil) - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.ServeHTTP(resp, req) v := make(map[string]interface{}) if err := json.Unmarshal(resp.Body.Bytes(), &v); err != nil { @@ -1965,7 +1965,7 @@ func TestExecutorMaxPayloadSizeExceeded(t *testing.T) { *warnPayloadSize = saveWarn }() - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) warningCount := warnings.Counts()["WarnPayloadSizeExceeded"] testMaxPayloadSizeExceeded := []string{ @@ -2002,7 +2002,7 @@ func TestExecutorMaxPayloadSizeExceeded(t *testing.T) { } func TestOlapSelectDatabase(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.normalize = true session := &vtgatepb.Session{Autocommit: true} @@ -2019,7 +2019,7 @@ func TestOlapSelectDatabase(t *testing.T) { } func TestExecutorClearsWarnings(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{ Warnings: []*querypb.QueryWarning{{Code: 234, Message: "oh noes"}}, }) @@ -2029,7 +2029,7 @@ func TestExecutorClearsWarnings(t *testing.T) { } func TestExecutorOtherRead(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() type cnts struct { Sbc1Cnt int64 @@ -2104,7 +2104,7 @@ func TestExecutorOtherRead(t *testing.T) { } func TestExecutorExplain(t *testing.T) { - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() executor.normalize = true logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -2130,7 +2130,7 @@ func TestExecutorExplain(t *testing.T) { } func TestExecutorOtherAdmin(t *testing.T) { - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() type cnts struct { Sbc1Cnt int64 @@ -2204,7 +2204,7 @@ func TestExecutorOtherAdmin(t *testing.T) { } func TestExecutorSavepointInTx(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("TestExecutorSavepoint") defer QueryLogger.Unsubscribe(logChan) @@ -2350,7 +2350,7 @@ func TestExecutorSavepointInTxWithReservedConn(t *testing.T) { } func TestExecutorSavepointWithoutTx(t *testing.T) { - executor, sbc1, sbc2, _ := createLegacyExecutorEnv() + executor, sbc1, sbc2, _ := createExecutorEnv() logChan := QueryLogger.Subscribe("TestExecutorSavepoint") defer QueryLogger.Unsubscribe(logChan) diff --git a/go/vt/vtgate/executor_vschema_ddl_test.go b/go/vt/vtgate/executor_vschema_ddl_test.go index 697f8ecdf74..fdc291197b1 100644 --- a/go/vt/vtgate/executor_vschema_ddl_test.go +++ b/go/vt/vtgate/executor_vschema_ddl_test.go @@ -134,7 +134,7 @@ func TestPlanExecutorAlterVSchemaKeyspace(t *testing.T) { defer func() { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true}) vschemaUpdates := make(chan *vschemapb.SrvVSchema, 2) @@ -162,7 +162,7 @@ func TestPlanExecutorCreateVindexDDL(t *testing.T) { defer func() { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() ks := "TestExecutor" vschemaUpdates := make(chan *vschemapb.SrvVSchema, 4) @@ -204,7 +204,7 @@ func TestPlanExecutorDropVindexDDL(t *testing.T) { defer func() { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() ks := "TestExecutor" vschemaUpdates := make(chan *vschemapb.SrvVSchema, 4) @@ -273,7 +273,7 @@ func TestPlanExecutorAddDropVschemaTableDDL(t *testing.T) { defer func() { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() + executor, sbc1, sbc2, sbclookup := createExecutorEnv() ks := KsTestUnsharded vschemaUpdates := make(chan *vschemapb.SrvVSchema, 4) @@ -330,7 +330,7 @@ func TestExecutorAddSequenceDDL(t *testing.T) { defer func() { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() ks := KsTestUnsharded vschema := executor.vm.GetCurrentSrvVschema() @@ -390,7 +390,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { defer func() { *vschemaacl.AuthorizedDDLUsers = "" }() - executor, sbc1, sbc2, sbclookup := createLegacyExecutorEnv() //nolint + executor, sbc1, sbc2, sbclookup := createExecutorEnv() //nolint ks := "TestExecutor" session := NewSafeSession(&vtgatepb.Session{TargetString: ks}) vschemaUpdates := make(chan *vschemapb.SrvVSchema, 4) @@ -721,7 +721,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { func TestPlanExecutorVindexDDLACL(t *testing.T) { //t.Skip("not yet planned") - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() ks := "TestExecutor" session := NewSafeSession(&vtgatepb.Session{TargetString: ks}) diff --git a/go/vt/vtgate/gateway.go b/go/vt/vtgate/gateway.go index dc899cc6550..e34eaebdfd6 100644 --- a/go/vt/vtgate/gateway.go +++ b/go/vt/vtgate/gateway.go @@ -23,7 +23,6 @@ import ( "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/discovery" - "vitess.io/vitess/go/vt/srvtopo" "vitess.io/vitess/go/vt/vttablet/queryservice" topodatapb "vitess.io/vitess/go/vt/proto/topodata" @@ -35,12 +34,10 @@ import ( // a query targeted to a keyspace/shard/tablet_type and send it off. var ( - _ = flag.String("gateway_implementation", "", "Deprecated. Only tabletgateway is now supported, discoverygateway is no longer available") - // We cannot reference tabletGatewayImplementation directly because it is const - defaultGatewayImplementation = tabletGatewayImplementation - GatewayImplementation = &defaultGatewayImplementation - bufferImplementation = flag.String("buffer_implementation", "keyspace_events", "Allowed values: healthcheck (legacy implementation), keyspace_events (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") + // Deprecated GatewayImplementation allows you to choose which gateway to use for vtgate routing. Defaults to tabletgateway, other option is discoverygateway + _ = flag.String("gateway_implementation", "tabletgateway", "Deprecated. The only available gateway_implementation is tabletgateway") + bufferImplementation = flag.String("buffer_implementation", "keyspace_events", "Allowed values: healthcheck (legacy implementation), keyspace_events (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") @@ -75,28 +72,6 @@ type Gateway interface { QueryServiceByAlias(alias *topodatapb.TabletAlias, target *querypb.Target) (queryservice.QueryService, error) } -// Creator is the factory method which can create the actual gateway object. -type Creator func(ctx context.Context, hc discovery.LegacyHealthCheck, serv srvtopo.Server, cell string, retryCount int) Gateway - -var creators = make(map[string]Creator) - -// RegisterGatewayCreator registers a Creator with given name. -func RegisterGatewayCreator(name string, gc Creator) { - if _, ok := creators[name]; ok { - log.Fatalf("Gateway %s already exists", name) - } - creators[name] = gc -} - -// GatewayCreator returns the Creator specified by the gateway_implementation flag. -func GatewayCreator() Creator { - gc, ok := creators[*GatewayImplementation] - if !ok { - log.Exitf("No gateway registered as %s", *GatewayImplementation) - } - return gc -} - // WaitForTablets is a helper method to wait for the provided tablets, // up until the *initialTabletTimeout. It will log what it is doing. // Note it has the same name as the Gateway's interface method, as it diff --git a/go/vt/vtgate/grpc_discovery_test.go b/go/vt/vtgate/grpc_discovery_test.go index 13cd7e1644e..7ba6014be63 100644 --- a/go/vt/vtgate/grpc_discovery_test.go +++ b/go/vt/vtgate/grpc_discovery_test.go @@ -62,9 +62,9 @@ func TestGRPCDiscovery(t *testing.T) { // VTGate: create the discovery healthcheck, and the gateway. // Wait for the right tablets to be present. - hc := discovery.NewLegacyHealthCheck(10*time.Second, 2*time.Minute) + hc := discovery.NewHealthCheck(context.Background(), 10*time.Second, 2*time.Minute, ts, cell, cell) rs := srvtopo.NewResilientServer(ts, "TestGRPCDiscovery") - dg := NewDiscoveryGateway(context.Background(), hc, rs, cell, 2) + dg := NewTabletGateway(context.Background(), hc, rs, cell) hc.AddTablet(&topodatapb.Tablet{ Alias: tabletconntest.TestAlias, Keyspace: tabletconntest.TestTarget.Keyspace, @@ -74,7 +74,7 @@ func TestGRPCDiscovery(t *testing.T) { PortMap: map[string]int32{ "grpc": int32(port), }, - }, "test_tablet") + }) err = WaitForTablets(dg, []topodatapb.TabletType{tabletconntest.TestTarget.TabletType}) if err != nil { t.Fatalf("WaitForTablets failed: %v", err) diff --git a/go/vt/vtgate/legacy_scatter_conn_test.go b/go/vt/vtgate/legacy_scatter_conn_test.go index 51f6053fc30..a5cdf7e32fc 100644 --- a/go/vt/vtgate/legacy_scatter_conn_test.go +++ b/go/vt/vtgate/legacy_scatter_conn_test.go @@ -44,8 +44,8 @@ import ( func TestLegacyExecuteFailOnAutocommit(t *testing.T) { createSandbox("TestExecuteFailOnAutocommit") - hc := discovery.NewFakeLegacyHealthCheck() - sc := newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + hc := discovery.NewFakeHealthCheck(nil) + sc := newTestScatterConn(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "0", 1, "TestExecuteFailOnAutocommit", "0", topodatapb.TabletType_PRIMARY, true, 1, nil) sbc1 := hc.AddTestTablet("aa", "1", 1, "TestExecuteFailOnAutocommit", "1", topodatapb.TabletType_PRIMARY, true, 1, nil) @@ -154,11 +154,11 @@ 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.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) // no shard s := createSandbox(name) - sc := newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + sc := newTestScatterConn(hc, new(sandboxTopo), "aa") qr, err := f(sc, nil) require.NoError(t, err) if qr.RowsAffected != 0 { @@ -167,7 +167,7 @@ func testScatterConnGeneric(t *testing.T, name string, f func(sc *ScatterConn, s // single shard s.Reset() - sc = newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + sc = newTestScatterConn(hc, new(sandboxTopo), "aa") sbc := hc.AddTestTablet("aa", "0", 1, name, "0", topodatapb.TabletType_REPLICA, true, 1, nil) sbc.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1 _, err = f(sc, []string{"0"}) @@ -184,7 +184,7 @@ func testScatterConnGeneric(t *testing.T, name string, f func(sc *ScatterConn, s // two shards s.Reset() hc.Reset() - sc = newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + sc = newTestScatterConn(hc, new(sandboxTopo), "aa") sbc0 := hc.AddTestTablet("aa", "0", 1, name, "0", topodatapb.TabletType_REPLICA, true, 1, nil) sbc1 := hc.AddTestTablet("aa", "1", 1, name, "1", topodatapb.TabletType_REPLICA, true, 1, nil) sbc0.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1 @@ -204,7 +204,7 @@ func testScatterConnGeneric(t *testing.T, name string, f func(sc *ScatterConn, s // two shards with different errors s.Reset() hc.Reset() - sc = newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + sc = newTestScatterConn(hc, new(sandboxTopo), "aa") sbc0 = hc.AddTestTablet("aa", "0", 1, name, "0", topodatapb.TabletType_REPLICA, true, 1, nil) sbc1 = hc.AddTestTablet("aa", "1", 1, name, "1", topodatapb.TabletType_REPLICA, true, 1, nil) sbc0.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1 @@ -225,7 +225,7 @@ func testScatterConnGeneric(t *testing.T, name string, f func(sc *ScatterConn, s // duplicate shards s.Reset() hc.Reset() - sc = newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + sc = newTestScatterConn(hc, new(sandboxTopo), "aa") sbc = hc.AddTestTablet("aa", "0", 1, name, "0", topodatapb.TabletType_REPLICA, true, 1, nil) _, _ = f(sc, []string{"0", "0"}) // Ensure that we executed only once. @@ -236,7 +236,7 @@ func testScatterConnGeneric(t *testing.T, name string, f func(sc *ScatterConn, s // no errors s.Reset() hc.Reset() - sc = newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + sc = newTestScatterConn(hc, new(sandboxTopo), "aa") sbc0 = hc.AddTestTablet("aa", "0", 1, name, "0", topodatapb.TabletType_REPLICA, true, 1, nil) sbc1 = hc.AddTestTablet("aa", "1", 1, name, "1", topodatapb.TabletType_REPLICA, true, 1, nil) qr, err = f(sc, []string{"0", "1"}) @@ -263,8 +263,8 @@ func TestMaxMemoryRows(t *testing.T) { defer func() { *maxMemoryRows = save }() createSandbox("TestMaxMemoryRows") - hc := discovery.NewFakeLegacyHealthCheck() - sc := newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + hc := discovery.NewFakeHealthCheck(nil) + 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) @@ -315,8 +315,8 @@ func TestMaxMemoryRows(t *testing.T) { func TestLegaceHealthCheckFailsOnReservedConnections(t *testing.T) { keyspace := "keyspace" createSandbox(keyspace) - hc := discovery.NewFakeLegacyHealthCheck() - sc := newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + hc := discovery.NewFakeHealthCheck(nil) + sc := newTestScatterConn(hc, new(sandboxTopo), "aa") res := srvtopo.NewResolver(&sandboxTopo{}, sc.gateway, "aa") @@ -363,8 +363,8 @@ func executeOnShardsReturnsErr(t *testing.T, res *srvtopo.Resolver, keyspace str func TestMultiExecs(t *testing.T) { createSandbox("TestMultiExecs") - hc := discovery.NewFakeLegacyHealthCheck() - sc := newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + hc := discovery.NewFakeHealthCheck(nil) + 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) @@ -459,10 +459,10 @@ func TestMultiExecs(t *testing.T) { func TestScatterConnSingleDB(t *testing.T) { createSandbox("TestScatterConnSingleDB") - hc := discovery.NewFakeLegacyHealthCheck() + hc := discovery.NewFakeHealthCheck(nil) hc.Reset() - sc := newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") + sc := newTestScatterConn(hc, new(sandboxTopo), "aa") hc.AddTestTablet("aa", "0", 1, "TestScatterConnSingleDB", "0", topodatapb.TabletType_PRIMARY, true, 1, nil) hc.AddTestTablet("aa", "1", 1, "TestScatterConnSingleDB", "1", topodatapb.TabletType_PRIMARY, true, 1, nil) @@ -578,15 +578,6 @@ func TestReservePrequeries(t *testing.T) { assert.Equal(t, 2+1, len(sbc0.StringQueries())) } -func newTestLegacyScatterConn(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()(ctx, hc, serv, cell, 3) - tc := NewTxConn(gw, vtgatepb.TransactionMode_TWOPC) - return NewLegacyScatterConn("", tc, gw, hc) -} - func newTestScatterConn(hc discovery.HealthCheck, 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 diff --git a/go/vt/vtgate/queryz_test.go b/go/vt/vtgate/queryz_test.go index 1478997d017..f45d1fa7eb8 100644 --- a/go/vt/vtgate/queryz_test.go +++ b/go/vt/vtgate/queryz_test.go @@ -37,7 +37,7 @@ func TestQueryzHandler(t *testing.T) { resp := httptest.NewRecorder() req, _ := http.NewRequest("GET", "/schemaz", nil) - executor, _, _, _ := createLegacyExecutorEnv() + executor, _, _, _ := createExecutorEnv() // single shard query sql := "select id from user where id = 1" diff --git a/go/vt/vtgate/scatter_conn.go b/go/vt/vtgate/scatter_conn.go index 43ed3f04462..a2ba9d305f6 100644 --- a/go/vt/vtgate/scatter_conn.go +++ b/go/vt/vtgate/scatter_conn.go @@ -54,7 +54,6 @@ type ScatterConn struct { tabletCallErrorCount *stats.CountersWithMultiLabels txConn *TxConn gateway Gateway - legacyHealthCheck discovery.LegacyHealthCheck } // shardActionFunc defines the contract for a shard action @@ -74,27 +73,6 @@ type shardActionFunc func(rs *srvtopo.ResolvedShard, i int) error // the results and errors for the caller. type shardActionTransactionFunc func(rs *srvtopo.ResolvedShard, i int, shardActionInfo *shardActionInfo) (*shardActionInfo, error) -// NewLegacyScatterConn creates a new ScatterConn. -func NewLegacyScatterConn(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 *TabletGateway) *ScatterConn { // this only works with TabletGateway @@ -113,8 +91,6 @@ func NewScatterConn(statsName string, txConn *TxConn, gw *TabletGateway) *Scatte []string{"Operation", "Keyspace", "ShardName", "DbType"}), txConn: txConn, gateway: gw, - // gateway has a reference to healthCheck so we don't need this any more - legacyHealthCheck: nil, } } @@ -298,12 +274,7 @@ func checkAndResetShardSession(info *shardActionInfo, err error, session *SafeSe } func getQueryService(rs *srvtopo.ResolvedShard, info *shardActionInfo) (queryservice.QueryService, error) { - _, usingLegacyGw := rs.Gateway.(*DiscoveryGateway) - if usingLegacyGw && - (info.actionNeeded == reserve || info.actionNeeded == reserveBegin) { - return nil, vterrors.New(vtrpcpb.Code_UNIMPLEMENTED, "reserved connections are not supported on old gen gateway") - } - if usingLegacyGw || info.alias == nil { + if info.alias == nil { return rs.Gateway, nil } return rs.Gateway.QueryServiceByAlias(info.alias, rs.Target) @@ -528,20 +499,8 @@ 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 { - if UsingLegacyGateway() { - panic("this should never be called") - } - return stc.gateway.TabletsCacheStatus() } diff --git a/go/vt/vtgate/tabletgateway.go b/go/vt/vtgate/tabletgateway.go index 49a6bc01dec..068f4a97de0 100644 --- a/go/vt/vtgate/tabletgateway.go +++ b/go/vt/vtgate/tabletgateway.go @@ -41,14 +41,6 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) -const ( - tabletGatewayImplementation = "tabletgateway" -) - -func init() { - RegisterGatewayCreator(tabletGatewayImplementation, createTabletGateway) -} - var ( _ discovery.HealthCheck = (*discovery.HealthCheckImpl)(nil) // CellsToWatch is the list of cells the healthcheck operates over. If it is empty, only the local cell is watched @@ -76,11 +68,6 @@ type TabletGateway struct { buffer *buffer.Buffer } -func createTabletGateway(ctx context.Context, _ discovery.LegacyHealthCheck, serv srvtopo.Server, cell string, _ int) Gateway { - // we ignore the passed in LegacyHealthCheck and let TabletGateway create it's own HealthCheck - return NewTabletGateway(ctx, nil /*discovery.Healthcheck*/, serv, cell) -} - func createHealthCheck(ctx context.Context, retryDelay, timeout time.Duration, ts *topo.Server, cell, cellsToWatch string) discovery.HealthCheck { return discovery.NewHealthCheck(ctx, retryDelay, timeout, ts, cell, cellsToWatch) } diff --git a/go/vt/vtgate/tx_conn.go b/go/vt/vtgate/tx_conn.go index b056b8db508..961d200d948 100644 --- a/go/vt/vtgate/tx_conn.go +++ b/go/vt/vtgate/tx_conn.go @@ -83,9 +83,8 @@ func (txc *TxConn) Commit(ctx context.Context, session *SafeSession) error { } func (txc *TxConn) queryService(alias *topodatapb.TabletAlias) (queryservice.QueryService, error) { - qs, _ := txc.gateway.(*DiscoveryGateway) - if qs != nil { - return qs, nil + if alias == nil { + return txc.gateway, nil } return txc.gateway.QueryServiceByAlias(alias, nil) } diff --git a/go/vt/vtgate/tx_conn_test.go b/go/vt/vtgate/tx_conn_test.go index 6bed357b12f..ec91b7da5cc 100644 --- a/go/vt/vtgate/tx_conn_test.go +++ b/go/vt/vtgate/tx_conn_test.go @@ -43,7 +43,7 @@ var queries = []*querypb.BoundQuery{{Sql: "query1"}} var twoQueries = []*querypb.BoundQuery{{Sql: "query1"}, {Sql: "query1"}} func TestTxConnBegin(t *testing.T) { - sc, sbc0, _, rss0, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, _, rss0, _, _ := newTestTxConnEnv(t, "TestTxConn") session := &vtgatepb.Session{} // begin @@ -63,7 +63,7 @@ func TestTxConnBegin(t *testing.T) { } func TestTxConnCommitSuccess(t *testing.T) { - sc, sbc0, sbc1, rss0, _, rss01 := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, "TestTxConn") sc.txConn.mode = vtgatepb.TransactionMode_MULTI // Sequence the executes to ensure commit order @@ -382,7 +382,7 @@ func TestTxConnReservedOn2ShardTxOn1ShardAndRollback(t *testing.T) { } func TestTxConnCommitOrderFailure1(t *testing.T) { - sc, sbc0, sbc1, rss0, rss1, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, rss0, rss1, _ := newTestTxConnEnv(t, "TestTxConn") sc.txConn.mode = vtgatepb.TransactionMode_MULTI queries := []*querypb.BoundQuery{{Sql: "query1"}} @@ -413,7 +413,7 @@ func TestTxConnCommitOrderFailure1(t *testing.T) { } func TestTxConnCommitOrderFailure2(t *testing.T) { - sc, sbc0, sbc1, rss0, rss1, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, rss0, rss1, _ := newTestTxConnEnv(t, "TestTxConn") sc.txConn.mode = vtgatepb.TransactionMode_MULTI queries := []*querypb.BoundQuery{{ @@ -445,7 +445,7 @@ func TestTxConnCommitOrderFailure2(t *testing.T) { } func TestTxConnCommitOrderFailure3(t *testing.T) { - sc, sbc0, sbc1, rss0, rss1, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, rss0, rss1, _ := newTestTxConnEnv(t, "TestTxConn") sc.txConn.mode = vtgatepb.TransactionMode_MULTI queries := []*querypb.BoundQuery{{ @@ -480,7 +480,7 @@ func TestTxConnCommitOrderFailure3(t *testing.T) { } func TestTxConnCommitOrderSuccess(t *testing.T) { - sc, sbc0, sbc1, rss0, rss1, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, rss0, rss1, _ := newTestTxConnEnv(t, "TestTxConn") sc.txConn.mode = vtgatepb.TransactionMode_MULTI queries := []*querypb.BoundQuery{{ @@ -717,7 +717,7 @@ func TestTxConnReservedCommitOrderSuccess(t *testing.T) { } func TestTxConnCommit2PC(t *testing.T) { - sc, sbc0, sbc1, rss0, _, rss01 := newLegacyTestTxConnEnv(t, "TestTxConnCommit2PC") + sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, "TestTxConnCommit2PC") session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) sc.ExecuteMultiShard(ctx, rss0, queries, session, false, false) @@ -733,7 +733,7 @@ func TestTxConnCommit2PC(t *testing.T) { } func TestTxConnCommit2PCOneParticipant(t *testing.T) { - sc, sbc0, _, rss0, _, _ := newLegacyTestTxConnEnv(t, "TestTxConnCommit2PCOneParticipant") + sc, sbc0, _, rss0, _, _ := newTestTxConnEnv(t, "TestTxConnCommit2PCOneParticipant") session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) sc.ExecuteMultiShard(ctx, rss0, queries, session, false, false) session.TransactionMode = vtgatepb.TransactionMode_TWOPC @@ -743,7 +743,7 @@ func TestTxConnCommit2PCOneParticipant(t *testing.T) { } func TestTxConnCommit2PCCreateTransactionFail(t *testing.T) { - sc, sbc0, sbc1, rss0, rss1, _ := newLegacyTestTxConnEnv(t, "TestTxConnCommit2PCCreateTransactionFail") + sc, sbc0, sbc1, rss0, rss1, _ := newTestTxConnEnv(t, "TestTxConnCommit2PCCreateTransactionFail") session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) sc.ExecuteMultiShard(ctx, rss0, queries, session, false, false) @@ -765,7 +765,7 @@ func TestTxConnCommit2PCCreateTransactionFail(t *testing.T) { } func TestTxConnCommit2PCPrepareFail(t *testing.T) { - sc, sbc0, sbc1, rss0, _, rss01 := newLegacyTestTxConnEnv(t, "TestTxConnCommit2PCPrepareFail") + sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, "TestTxConnCommit2PCPrepareFail") session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) sc.ExecuteMultiShard(ctx, rss0, queries, session, false, false) @@ -785,7 +785,7 @@ func TestTxConnCommit2PCPrepareFail(t *testing.T) { } func TestTxConnCommit2PCStartCommitFail(t *testing.T) { - sc, sbc0, sbc1, rss0, _, rss01 := newLegacyTestTxConnEnv(t, "TestTxConnCommit2PCStartCommitFail") + sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, "TestTxConnCommit2PCStartCommitFail") session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) sc.ExecuteMultiShard(ctx, rss0, queries, session, false, false) @@ -805,7 +805,7 @@ func TestTxConnCommit2PCStartCommitFail(t *testing.T) { } func TestTxConnCommit2PCCommitPreparedFail(t *testing.T) { - sc, sbc0, sbc1, rss0, _, rss01 := newLegacyTestTxConnEnv(t, "TestTxConnCommit2PCCommitPreparedFail") + sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, "TestTxConnCommit2PCCommitPreparedFail") session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) sc.ExecuteMultiShard(ctx, rss0, queries, session, false, false) @@ -825,7 +825,7 @@ func TestTxConnCommit2PCCommitPreparedFail(t *testing.T) { } func TestTxConnCommit2PCConcludeTransactionFail(t *testing.T) { - sc, sbc0, sbc1, rss0, _, rss01 := newLegacyTestTxConnEnv(t, "TestTxConnCommit2PCConcludeTransactionFail") + sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, "TestTxConnCommit2PCConcludeTransactionFail") session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) sc.ExecuteMultiShard(ctx, rss0, queries, session, false, false) @@ -845,7 +845,7 @@ func TestTxConnCommit2PCConcludeTransactionFail(t *testing.T) { } func TestTxConnRollback(t *testing.T) { - sc, sbc0, sbc1, rss0, _, rss01 := newLegacyTestTxConnEnv(t, "TxConnRollback") + sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, "TxConnRollback") session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) sc.ExecuteMultiShard(ctx, rss0, queries, session, false, false) @@ -917,7 +917,7 @@ func TestTxConnReservedRollbackFailure(t *testing.T) { } func TestTxConnResolveOnPrepare(t *testing.T) { - sc, sbc0, sbc1, _, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, _, _, _ := newTestTxConnEnv(t, "TestTxConn") dtid := "TestTxConn:0:1234" sbc0.ReadTransactionResults = []*querypb.TransactionMetadata{{ @@ -938,7 +938,7 @@ func TestTxConnResolveOnPrepare(t *testing.T) { } func TestTxConnResolveOnRollback(t *testing.T) { - sc, sbc0, sbc1, _, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, _, _, _ := newTestTxConnEnv(t, "TestTxConn") dtid := "TestTxConn:0:1234" sbc0.ReadTransactionResults = []*querypb.TransactionMetadata{{ @@ -959,7 +959,7 @@ func TestTxConnResolveOnRollback(t *testing.T) { } func TestTxConnResolveOnCommit(t *testing.T) { - sc, sbc0, sbc1, _, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, _, _, _ := newTestTxConnEnv(t, "TestTxConn") dtid := "TestTxConn:0:1234" sbc0.ReadTransactionResults = []*querypb.TransactionMetadata{{ @@ -980,7 +980,7 @@ func TestTxConnResolveOnCommit(t *testing.T) { } func TestTxConnResolveInvalidDTID(t *testing.T) { - sc, _, _, _, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, _, _, _, _, _ := newTestTxConnEnv(t, "TestTxConn") err := sc.txConn.Resolve(ctx, "abcd") want := "invalid parts in dtid: abcd" @@ -988,7 +988,7 @@ func TestTxConnResolveInvalidDTID(t *testing.T) { } func TestTxConnResolveReadTransactionFail(t *testing.T) { - sc, sbc0, _, _, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, _, _, _, _ := newTestTxConnEnv(t, "TestTxConn") dtid := "TestTxConn:0:1234" sbc0.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1 @@ -999,7 +999,7 @@ func TestTxConnResolveReadTransactionFail(t *testing.T) { } func TestTxConnResolveInternalError(t *testing.T) { - sc, sbc0, _, _, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, _, _, _, _ := newTestTxConnEnv(t, "TestTxConn") dtid := "TestTxConn:0:1234" sbc0.ReadTransactionResults = []*querypb.TransactionMetadata{{ @@ -1018,7 +1018,7 @@ func TestTxConnResolveInternalError(t *testing.T) { } func TestTxConnResolveSetRollbackFail(t *testing.T) { - sc, sbc0, sbc1, _, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, _, _, _ := newTestTxConnEnv(t, "TestTxConn") dtid := "TestTxConn:0:1234" sbc0.ReadTransactionResults = []*querypb.TransactionMetadata{{ @@ -1042,7 +1042,7 @@ func TestTxConnResolveSetRollbackFail(t *testing.T) { } func TestTxConnResolveRollbackPreparedFail(t *testing.T) { - sc, sbc0, sbc1, _, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, _, _, _ := newTestTxConnEnv(t, "TestTxConn") dtid := "TestTxConn:0:1234" sbc0.ReadTransactionResults = []*querypb.TransactionMetadata{{ @@ -1066,7 +1066,7 @@ func TestTxConnResolveRollbackPreparedFail(t *testing.T) { } func TestTxConnResolveCommitPreparedFail(t *testing.T) { - sc, sbc0, sbc1, _, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, _, _, _ := newTestTxConnEnv(t, "TestTxConn") dtid := "TestTxConn:0:1234" sbc0.ReadTransactionResults = []*querypb.TransactionMetadata{{ @@ -1090,7 +1090,7 @@ func TestTxConnResolveCommitPreparedFail(t *testing.T) { } func TestTxConnResolveConcludeTransactionFail(t *testing.T) { - sc, sbc0, sbc1, _, _, _ := newLegacyTestTxConnEnv(t, "TestTxConn") + sc, sbc0, sbc1, _, _, _ := newTestTxConnEnv(t, "TestTxConn") dtid := "TestTxConn:0:1234" sbc0.ReadTransactionResults = []*querypb.TransactionMetadata{{ @@ -1180,24 +1180,6 @@ func TestTxConnMultiGoTargets(t *testing.T) { require.NoError(t, err) } -func newLegacyTestTxConnEnv(t *testing.T, name string) (sc *ScatterConn, sbc0, sbc1 *sandboxconn.SandboxConn, rss0, rss1, rss01 []*srvtopo.ResolvedShard) { - t.Helper() - createSandbox(name) - hc := discovery.NewFakeLegacyHealthCheck() - sc = newTestLegacyScatterConn(hc, new(sandboxTopo), "aa") - sbc0 = hc.AddTestTablet("aa", "0", 1, name, "0", topodatapb.TabletType_PRIMARY, true, 1, nil) - sbc1 = hc.AddTestTablet("aa", "1", 1, name, "1", topodatapb.TabletType_PRIMARY, true, 1, nil) - res := srvtopo.NewResolver(&sandboxTopo{}, sc.gateway, "aa") - var err error - rss0, err = res.ResolveDestination(ctx, name, topodatapb.TabletType_PRIMARY, key.DestinationShard("0")) - require.NoError(t, err) - rss1, err = res.ResolveDestination(ctx, name, topodatapb.TabletType_PRIMARY, key.DestinationShard("1")) - require.NoError(t, err) - rss01, err = res.ResolveDestination(ctx, name, topodatapb.TabletType_PRIMARY, key.DestinationShards([]string{"0", "1"})) - require.NoError(t, err) - return sc, sbc0, sbc1, rss0, rss1, rss01 -} - func newTestTxConnEnv(t *testing.T, name string) (sc *ScatterConn, sbc0, sbc1 *sandboxconn.SandboxConn, rss0, rss1, rss01 []*srvtopo.ResolvedShard) { t.Helper() createSandbox(name) diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index d6249796571..084d5ef075a 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -129,10 +129,6 @@ func newVCursorImpl( return nil, err } - // With DiscoveryGateway transactions are only allowed on primary. - if UsingLegacyGateway() && safeSession.InTransaction() && tabletType != topodatapb.TabletType_PRIMARY { - return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "transaction is supported only for primary tablet type, current type: %v", tabletType) - } var ts *topo.Server // We don't have access to the underlying TopoServer if this vtgate is // filtering keyspaces because we don't have an accurate view of the topo. diff --git a/go/vt/vtgate/vstream_manager_test.go b/go/vt/vtgate/vstream_manager_test.go index ecf3305a0ba..7ab0d11fa0d 100644 --- a/go/vt/vtgate/vstream_manager_test.go +++ b/go/vt/vtgate/vstream_manager_test.go @@ -73,8 +73,12 @@ func TestVStreamSkew(t *testing.T) { {numEventsPerShard: 4, shard0idx: 1, shard1idx: 0, expectedDelays: 0}, } previousDelays := int64(0) - vstreamSkewDelayCount = stats.NewCounter("VStreamEventsDelayedBySkewAlignment", - "Number of events that had to wait because the skew across shards was too high") + if vstreamSkewDelayCount == nil { + // HACK: without a mutex we are not guaranteed that this will avoid the panic caused by a race + // between this initialization and the one in vtgate.go + vstreamSkewDelayCount = stats.NewCounter("VStreamEventsDelayedBySkewAlignment", + "Number of events that had to wait because the skew across shards was too high") + } cell := "aa" for idx, tcase := range tcases { diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index a4cc62371d6..63a6b5a0fb4 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -159,7 +159,7 @@ type RegisterVTGate func(vtgateservice.VTGateService) var RegisterVTGates []RegisterVTGate // Init initializes VTGate server. -func Init(ctx context.Context, serv srvtopo.Server, cell string, tabletTypesToWait []topodatapb.TabletType) *VTGate { +func Init(ctx context.Context, hc discovery.HealthCheck, serv srvtopo.Server, cell string, tabletTypesToWait []topodatapb.TabletType) *VTGate { if rpcVTGate != nil { log.Fatalf("VTGate already initialized") } @@ -175,7 +175,7 @@ func Init(ctx context.Context, serv srvtopo.Server, cell string, tabletTypesToWa // Start with the gateway. If we can't reach the topology service, // we can't go on much further, so we log.Fatal out. // TabletGateway can create it's own healthcheck - gw := NewTabletGateway(ctx, nil /*discovery.Healthcheck*/, serv, cell) + gw := NewTabletGateway(ctx, hc, serv, cell) gw.RegisterStats() if err := WaitForTablets(gw, tabletTypesToWait); err != nil { log.Fatalf("gateway.WaitForTablets failed: %v", err) @@ -583,97 +583,3 @@ 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 discovery.FilteringKeyspaces() { - 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 := NewLegacyScatterConn("VttabletCall", tc, gw, hc) - srvResolver := srvtopo.NewResolver(serv, gw, cell) - resolver := NewResolver(srvResolver, serv, cell, sc) - vsm := newVStreamManager(srvResolver, serv, cell) - cacheCfg := &cache.Config{ - MaxEntries: *queryPlanCacheSize, - MaxMemoryUsage: *queryPlanCacheMemory, - LFU: *queryPlanCacheLFU, - } - - rpcVTGate = &VTGate{ - executor: NewExecutor(ctx, serv, cell, resolver, *normalizeQueries, *warnShardedOnly, *streamBufferSize, cacheCfg, nil, *noScatter), - 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"}), - rowsAffected: stats.NewCountersWithMultiLabels( - "VtgateApiRowsAffected", - "Rows affected by a write (DML) operation 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(hc) - - return rpcVTGate -} diff --git a/go/vt/vtgate/vtgate_test.go b/go/vt/vtgate/vtgate_test.go index aee76f7721a..a57a3eadd46 100644 --- a/go/vt/vtgate/vtgate_test.go +++ b/go/vt/vtgate/vtgate_test.go @@ -43,7 +43,7 @@ import ( // This file uses the sandbox_test framework. -var hcVTGateTest *discovery.FakeLegacyHealthCheck +var hcVTGateTest *discovery.FakeHealthCheck var executeOptions = &querypb.ExecuteOptions{ IncludedFields: querypb.ExecuteOptions_TYPE_ONLY, @@ -75,14 +75,9 @@ func init() { } } ` - hcVTGateTest = discovery.NewFakeLegacyHealthCheck() + hcVTGateTest = discovery.NewFakeHealthCheck(nil) *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. - LegacyInit(context.Background(), hcVTGateTest, new(sandboxTopo), "aa", 10, nil) + Init(context.Background(), hcVTGateTest, new(sandboxTopo), "aa", nil) *mysqlServerPort = 0 *mysqlAuthServerImpl = "none" From 6d885cdd653e292ba08ca7d17d37a1c09a786619 Mon Sep 17 00:00:00 2001 From: deepthi Date: Wed, 19 Jan 2022 11:13:13 -0800 Subject: [PATCH 02/10] fix some executor tests Signed-off-by: deepthi --- go/vt/discovery/fake_healthcheck.go | 1 + go/vt/vtgate/executor_dml_test.go | 78 ++++++++++++------------- go/vt/vtgate/executor_framework_test.go | 14 ++++- go/vt/vtgate/vtgate_test.go | 9 ++- 4 files changed, 59 insertions(+), 43 deletions(-) diff --git a/go/vt/discovery/fake_healthcheck.go b/go/vt/discovery/fake_healthcheck.go index aad55923448..d707001f8e9 100644 --- a/go/vt/discovery/fake_healthcheck.go +++ b/go/vt/discovery/fake_healthcheck.go @@ -247,6 +247,7 @@ func (fhc *FakeHealthCheck) Reset() { defer fhc.mu.Unlock() fhc.items = make(map[string]*fhcItem) + fhc.currentTabletUID.Set(0) } // AddFakeTablet inserts a fake entry into FakeHealthCheck. diff --git a/go/vt/vtgate/executor_dml_test.go b/go/vt/vtgate/executor_dml_test.go index 8666776b41f..4358233de2a 100644 --- a/go/vt/vtgate/executor_dml_test.go +++ b/go/vt/vtgate/executor_dml_test.go @@ -133,6 +133,45 @@ func TestUpdateEqual(t *testing.T) { assertQueries(t, sbclookup, wantQueries) } +func TestUpdateEqualWithWriteOnlyLookupUniqueVindex(t *testing.T) { + res := []*sqltypes.Result{sqltypes.MakeTestResult( + sqltypes.MakeTestFields("id|wo_lu_col|lu_col|t2_lu_vdx", "int64|int64|int64|int64"), + "1|2|1|0", + )} + executor, sbc1, sbc2, sbcLookup := createCustomExecutorSetValues(executorVSchema, res) + + _, err := executorExec(executor, "update t2_wo_lookup set lu_col = 5 where wo_lu_col = 2", nil) + require.NoError(t, err) + wantQueries := []*querypb.BoundQuery{ + { + Sql: "select id, wo_lu_col, lu_col, lu_col = 5 from t2_wo_lookup where wo_lu_col = 2 for update", + BindVariables: map[string]*querypb.BindVariable{}, + }, { + Sql: "update t2_wo_lookup set lu_col = 5 where wo_lu_col = 2", + BindVariables: map[string]*querypb.BindVariable{}, + }} + + assertQueries(t, sbc1, wantQueries) + assertQueries(t, sbc2, wantQueries) + + bq1 := &querypb.BoundQuery{ + Sql: "delete from lu_idx where lu_col = :lu_col and keyspace_id = :keyspace_id", + BindVariables: map[string]*querypb.BindVariable{ + "keyspace_id": sqltypes.Uint64BindVariable(1), + "lu_col": sqltypes.Int64BindVariable(1), + }, + } + bq2 := &querypb.BoundQuery{ + Sql: "insert into lu_idx(lu_col, keyspace_id) values (:lu_col_0, :keyspace_id_0)", + BindVariables: map[string]*querypb.BindVariable{ + "keyspace_id_0": sqltypes.Uint64BindVariable(1), + "lu_col_0": sqltypes.Int64BindVariable(5), + }, + } + lookWant := []*querypb.BoundQuery{bq1, bq2, bq1, bq2, bq1, bq2, bq1, bq2, bq1, bq2, bq1, bq2, bq1, bq2, bq1, bq2} + assertQueries(t, sbcLookup, lookWant) +} + func TestUpdateMultiOwned(t *testing.T) { vschema := ` { @@ -434,45 +473,6 @@ func TestDeleteScatter(t *testing.T) { assertQueries(t, sbc2, wantQueries) } -func TestUpdateEqualWithWriteOnlyLookupUniqueVindex(t *testing.T) { - res := []*sqltypes.Result{sqltypes.MakeTestResult( - sqltypes.MakeTestFields("id|wo_lu_col|lu_col|t2_lu_vdx", "int64|int64|int64|int64"), - "1|2|1|0", - )} - executor, sbc1, sbc2, sbcLookup := createCustomExecutorSetValues(executorVSchema, res) - - _, err := executorExec(executor, "update t2_wo_lookup set lu_col = 5 where wo_lu_col = 2", nil) - require.NoError(t, err) - wantQueries := []*querypb.BoundQuery{ - { - Sql: "select id, wo_lu_col, lu_col, lu_col = 5 from t2_wo_lookup where wo_lu_col = 2 for update", - BindVariables: map[string]*querypb.BindVariable{}, - }, { - Sql: "update t2_wo_lookup set lu_col = 5 where wo_lu_col = 2", - BindVariables: map[string]*querypb.BindVariable{}, - }} - - assertQueries(t, sbc1, wantQueries) - assertQueries(t, sbc2, wantQueries) - - bq1 := &querypb.BoundQuery{ - Sql: "delete from lu_idx where lu_col = :lu_col and keyspace_id = :keyspace_id", - BindVariables: map[string]*querypb.BindVariable{ - "keyspace_id": sqltypes.Uint64BindVariable(1), - "lu_col": sqltypes.Int64BindVariable(1), - }, - } - bq2 := &querypb.BoundQuery{ - Sql: "insert into lu_idx(lu_col, keyspace_id) values (:lu_col_0, :keyspace_id_0)", - BindVariables: map[string]*querypb.BindVariable{ - "keyspace_id_0": sqltypes.Uint64BindVariable(1), - "lu_col_0": sqltypes.Int64BindVariable(5), - }, - } - lookWant := []*querypb.BoundQuery{bq1, bq2, bq1, bq2, bq1, bq2, bq1, bq2, bq1, bq2, bq1, bq2, bq1, bq2, bq1, bq2} - assertQueries(t, sbcLookup, lookWant) -} - func TestUpdateEqualWithMultipleLookupVindex(t *testing.T) { executor, sbc1, sbc2, sbcLookup := createCustomExecutorSetValues(executorVSchema, nil) diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go index c0199a898ab..364a7347bcb 100644 --- a/go/vt/vtgate/executor_framework_test.go +++ b/go/vt/vtgate/executor_framework_test.go @@ -458,7 +458,7 @@ func createExecutorEnv() (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn _ = hc.AddTestTablet(cell, "c0-e0", 1, "TestExecutor", "c0-e0", topodatapb.TabletType_PRIMARY, true, 1, nil) _ = hc.AddTestTablet(cell, "e0-", 1, "TestExecutor", "e0-", topodatapb.TabletType_PRIMARY, true, 1, nil) // Below is needed so that SendAnyWherePlan doesn't fail - _ = hc.AddTestTablet(cell, "e0-", 1, "TestXBadVSchema", "-20", topodatapb.TabletType_PRIMARY, true, 1, nil) + _ = hc.AddTestTablet(cell, "random", 1, "TestXBadVSchema", "-20", topodatapb.TabletType_PRIMARY, true, 1, nil) createSandbox(KsTestUnsharded) _ = topo.NewShardInfo(KsTestUnsharded, "0", &topodatapb.Shard{}, nil) @@ -489,6 +489,10 @@ func createExecutorEnv() (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn executor = NewExecutor(context.Background(), serv, cell, resolver, false, false, testBufferSize, cache.DefaultConfig, nil, false) key.AnyShardPicker = DestinationAnyShardPickerFirstShard{} + // create a new session each time so that ShardSessions don't get re-used across tests + primarySession = &vtgatepb.Session{ + TargetString: "@primary", + } return executor, sbc1, sbc2, sbclookup } @@ -507,6 +511,10 @@ func createCustomExecutor(vschema string) (executor *Executor, sbc1, sbc2, sbclo getSandbox(KsTestUnsharded).VSchema = unshardedVSchema executor = NewExecutor(context.Background(), serv, cell, resolver, false, false, testBufferSize, cache.DefaultConfig, nil, false) + // create a new session each time so that ShardSessions don't get re-used across tests + primarySession = &vtgatepb.Session{ + TargetString: "@primary", + } return executor, sbc1, sbc2, sbclookup } @@ -532,6 +540,10 @@ func createCustomExecutorSetValues(vschema string, values []*sqltypes.Result) (e getSandbox(KsTestUnsharded).VSchema = unshardedVSchema executor = NewExecutor(context.Background(), serv, cell, resolver, false, false, testBufferSize, cache.DefaultConfig, nil, false) + // create a new session each time so that ShardSessions don't get re-used across tests + primarySession = &vtgatepb.Session{ + TargetString: "@primary", + } return executor, sbcs[0], sbcs[1], sbclookup } diff --git a/go/vt/vtgate/vtgate_test.go b/go/vt/vtgate/vtgate_test.go index a57a3eadd46..e1b83140acc 100644 --- a/go/vt/vtgate/vtgate_test.go +++ b/go/vt/vtgate/vtgate_test.go @@ -49,9 +49,7 @@ var executeOptions = &querypb.ExecuteOptions{ IncludedFields: querypb.ExecuteOptions_TYPE_ONLY, } -var primarySession = &vtgatepb.Session{ - TargetString: "@primary", -} +var primarySession *vtgatepb.Session func init() { getSandbox(KsTestUnsharded).VSchema = ` @@ -300,6 +298,11 @@ func testErrorPropagation(t *testing.T, sbcs []*sandboxconn.SandboxConn, before func TestErrorPropagation(t *testing.T) { createSandbox(KsTestUnsharded) hcVTGateTest.Reset() + // create a new session each time so that ShardSessions don't get re-used across tests + primarySession = &vtgatepb.Session{ + TargetString: "@primary", + } + sbcm := hcVTGateTest.AddTestTablet("aa", "1.1.1.1", 1001, KsTestUnsharded, "0", topodatapb.TabletType_PRIMARY, true, 1, nil) sbcrdonly := hcVTGateTest.AddTestTablet("aa", "1.1.1.2", 1001, KsTestUnsharded, "0", topodatapb.TabletType_RDONLY, true, 1, nil) sbcs := []*sandboxconn.SandboxConn{ From 266af64d89e9394bb801209497e34c786f789acf Mon Sep 17 00:00:00 2001 From: deepthi Date: Wed, 19 Jan 2022 11:18:00 -0800 Subject: [PATCH 03/10] make retryCount unexported. It is no longer needed outside the vtgate package Signed-off-by: deepthi --- go/vt/vtgate/gateway.go | 4 ++-- go/vt/vtgate/tabletgateway.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go/vt/vtgate/gateway.go b/go/vt/vtgate/gateway.go index e34eaebdfd6..04badfcbf11 100644 --- a/go/vt/vtgate/gateway.go +++ b/go/vt/vtgate/gateway.go @@ -38,9 +38,9 @@ var ( _ = flag.String("gateway_implementation", "tabletgateway", "Deprecated. The only available gateway_implementation is tabletgateway") bufferImplementation = flag.String("buffer_implementation", "keyspace_events", "Allowed values: healthcheck (legacy implementation), keyspace_events (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 + // 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") + retryCount = flag.Int("retry-count", 2, "retry count") ) // A Gateway is the query processing module for each shard, diff --git a/go/vt/vtgate/tabletgateway.go b/go/vt/vtgate/tabletgateway.go index 068f4a97de0..bd2cb32beb6 100644 --- a/go/vt/vtgate/tabletgateway.go +++ b/go/vt/vtgate/tabletgateway.go @@ -91,7 +91,7 @@ func NewTabletGateway(ctx context.Context, hc discovery.HealthCheck, serv srvtop hc: hc, srvTopoServer: serv, localCell: localCell, - retryCount: *RetryCount, + retryCount: *retryCount, statusAggregators: make(map[string]*TabletStatusAggregator), } gw.setupBuffering(ctx) From 6bb18663e899eae06e128218989b7a4c2ce89894 Mon Sep 17 00:00:00 2001 From: Florent Poinsard Date: Mon, 7 Mar 2022 09:52:57 +0100 Subject: [PATCH 04/10] fix: use primarySession after its initialization and fix comments/test names Signed-off-by: Florent Poinsard --- go/vt/vtgate/executor_select_test.go | 4 ++-- go/vt/vtgate/executor_stream_test.go | 1 - go/vt/vtgate/executor_test.go | 2 +- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index 93a967ab22b..005731dab19 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -651,8 +651,8 @@ func TestStreamLimitOffset(t *testing.T) { } func TestSelectLastInsertId(t *testing.T) { - primarySession.LastInsertId = 52 executor, _, _, _ := createExecutorEnv() + primarySession.LastInsertId = 52 executor.normalize = true logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) @@ -672,12 +672,12 @@ func TestSelectLastInsertId(t *testing.T) { } func TestSelectSystemVariables(t *testing.T) { + executor, _, _, _ := createExecutorEnv() primarySession.ReadAfterWrite = &vtgatepb.ReadAfterWrite{ ReadAfterWriteGtid: "a fine gtid", ReadAfterWriteTimeout: 13, SessionTrackGtids: true, } - executor, _, _, _ := createExecutorEnv() executor.normalize = true logChan := QueryLogger.Subscribe("Test") defer QueryLogger.Unsubscribe(logChan) diff --git a/go/vt/vtgate/executor_stream_test.go b/go/vt/vtgate/executor_stream_test.go index e1926cb0250..abce17b242c 100644 --- a/go/vt/vtgate/executor_stream_test.go +++ b/go/vt/vtgate/executor_stream_test.go @@ -48,7 +48,6 @@ func TestStreamSQLUnsharded(t *testing.T) { } func TestStreamSQLSharded(t *testing.T) { - // Special setup: Don't use createExecutorEnv. cell := "aa" hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") diff --git a/go/vt/vtgate/executor_test.go b/go/vt/vtgate/executor_test.go index 6f929b04312..4edcc751f03 100644 --- a/go/vt/vtgate/executor_test.go +++ b/go/vt/vtgate/executor_test.go @@ -115,7 +115,7 @@ func TestExecutorMaxMemoryRowsExceeded(t *testing.T) { } } -func TestLegacyExecutorTransactionsNoAutoCommit(t *testing.T) { +func TestExecutorTransactionsNoAutoCommit(t *testing.T) { executor, _, _, sbclookup := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) From c157eed629a33dc9463110ee5a09a689232c774b Mon Sep 17 00:00:00 2001 From: Florent Poinsard Date: Mon, 7 Mar 2022 10:45:42 +0100 Subject: [PATCH 05/10] fix: use proper tablet alias and hostname in executor test Signed-off-by: Florent Poinsard --- go/vt/vtgate/executor_select_test.go | 2 +- go/vt/vtgate/executor_test.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index 005731dab19..705813d6e09 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -754,7 +754,7 @@ func TestSelectInitializedVitessAwareVariable(t *testing.T) { utils.MustMatch(t, wantResult, result, "Mismatch") } -func TestSelectUserDefindVariable(t *testing.T) { +func TestSelectUserDefinedVariable(t *testing.T) { executor, _, _, _ := createExecutorEnv() executor.normalize = true logChan := QueryLogger.Subscribe("Test") diff --git a/go/vt/vtgate/executor_test.go b/go/vt/vtgate/executor_test.go index 4edcc751f03..a262237d9ec 100644 --- a/go/vt/vtgate/executor_test.go +++ b/go/vt/vtgate/executor_test.go @@ -818,8 +818,8 @@ func TestExecutorShow(t *testing.T) { wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Cell", "Keyspace", "Shard", "TabletType", "State", "Alias", "Hostname", "PrimaryTermStartTime"), Rows: [][]sqltypes.Value{ - buildVarCharRow("FakeCell", "TestExecutor", "-20", "PRIMARY", "SERVING", "aa-0000000000", "-20", "1970-01-01T00:00:01Z"), - buildVarCharRow("FakeCell", "TestXBadVSchema", "-20", "PRIMARY", "SERVING", "aa-0000000000", "e0-", "1970-01-01T00:00:01Z"), + buildVarCharRow("FakeCell", "TestExecutor", "-20", "PRIMARY", "SERVING", "aa-0000000001", "-20", "1970-01-01T00:00:01Z"), + buildVarCharRow("FakeCell", "TestXBadVSchema", "-20", "PRIMARY", "SERVING", "aa-0000000009", "random", "1970-01-01T00:00:01Z"), }, } utils.MustMatch(t, wantqr, qr, query) @@ -839,7 +839,7 @@ func TestExecutorShow(t *testing.T) { wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Cell", "Keyspace", "Shard", "TabletType", "State", "Alias", "Hostname", "PrimaryTermStartTime"), Rows: [][]sqltypes.Value{ - buildVarCharRow("FakeCell", "TestExecutor", "-20", "PRIMARY", "SERVING", "aa-0000000000", "-20", "1970-01-01T00:00:01Z"), + buildVarCharRow("FakeCell", "TestExecutor", "-20", "PRIMARY", "SERVING", "aa-0000000001", "-20", "1970-01-01T00:00:01Z"), }, } utils.MustMatch(t, wantqr, qr, query) From 6c33bb0a229e91b568b930c0d1d03ea52fc47490 Mon Sep 17 00:00:00 2001 From: Florent Poinsard Date: Mon, 7 Mar 2022 15:46:20 +0100 Subject: [PATCH 06/10] fix: updated expected output in vtgate tx and executor tests Signed-off-by: Florent Poinsard --- go/vt/vtgate/executor_test.go | 13 ------------- go/vt/vtgate/tx_conn_test.go | 6 +++--- 2 files changed, 3 insertions(+), 16 deletions(-) diff --git a/go/vt/vtgate/executor_test.go b/go/vt/vtgate/executor_test.go index a262237d9ec..a34a554ecec 100644 --- a/go/vt/vtgate/executor_test.go +++ b/go/vt/vtgate/executor_test.go @@ -178,19 +178,6 @@ func TestExecutorTransactionsNoAutoCommit(t *testing.T) { t.Errorf("logstats: expected no record for no-op rollback, got %v", logStats) } - // Prevent transactions on non-primary. - session = NewSafeSession(&vtgatepb.Session{TargetString: "@replica", InTransaction: true}) - _, err = executor.Execute(ctx, "TestExecute", session, "select id from main1", nil) - require.Error(t, err) - want := "transaction is supported only for primary tablet type, current type: REPLICA" - require.Contains(t, err.Error(), want) - - // Prevent begin on non-primary. - session = NewSafeSession(&vtgatepb.Session{TargetString: "@replica"}) - _, err = executor.Execute(ctx, "TestExecute", session, "begin", nil) - require.Error(t, err) - require.Contains(t, err.Error(), want) - // Prevent use of non-primary if in_transaction is on. session = NewSafeSession(&vtgatepb.Session{TargetString: "@primary", InTransaction: true}) _, err = executor.Execute(ctx, "TestExecute", session, "use @replica", nil) diff --git a/go/vt/vtgate/tx_conn_test.go b/go/vt/vtgate/tx_conn_test.go index ec91b7da5cc..5933450ffaf 100644 --- a/go/vt/vtgate/tx_conn_test.go +++ b/go/vt/vtgate/tx_conn_test.go @@ -100,7 +100,7 @@ func TestTxConnCommitSuccess(t *testing.T) { TabletType: topodatapb.TabletType_PRIMARY, }, TransactionId: 1, - TabletAlias: sbc0.Tablet().Alias, + TabletAlias: sbc1.Tablet().Alias, }}, } utils.MustMatch(t, &wantSession, session.Session, "Session") @@ -469,7 +469,7 @@ func TestTxConnCommitOrderFailure3(t *testing.T) { // The last failed commit must generate a warning. wantSession := vtgatepb.Session{ Warnings: []*querypb.QueryWarning{{ - Message: "post-operation transaction had an error: Code: INVALID_ARGUMENT\nINVALID_ARGUMENT error\n\ntarget: TestTxConn.1.primary", + Message: "post-operation transaction had an error: Code: INVALID_ARGUMENT\nINVALID_ARGUMENT error\n", }}, } utils.MustMatch(t, &wantSession, session.Session, "Session") @@ -558,7 +558,7 @@ func TestTxConnCommitOrderSuccess(t *testing.T) { TabletType: topodatapb.TabletType_PRIMARY, }, TransactionId: 1, - TabletAlias: sbc0.Tablet().Alias, + TabletAlias: sbc1.Tablet().Alias, }}, } utils.MustMatch(t, &wantSession, session.Session, "Session") From 2475fb304a96082d6658d908d1f0838173dff52f Mon Sep 17 00:00:00 2001 From: Florent Poinsard Date: Tue, 8 Mar 2022 19:28:21 +0100 Subject: [PATCH 07/10] refactore: remove gateway discovery test Signed-off-by: Florent Poinsard --- go/vt/vtgate/gateway_test_suite.go | 109 ---------------------------- go/vt/vtgate/grpc_discovery_test.go | 86 ---------------------- 2 files changed, 195 deletions(-) delete mode 100644 go/vt/vtgate/gateway_test_suite.go delete mode 100644 go/vt/vtgate/grpc_discovery_test.go diff --git a/go/vt/vtgate/gateway_test_suite.go b/go/vt/vtgate/gateway_test_suite.go deleted file mode 100644 index e5b0a076620..00000000000 --- a/go/vt/vtgate/gateway_test_suite.go +++ /dev/null @@ -1,109 +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 vtgate - -// This file contains a test suite to run against a Gateway object. -// We re-use the tabletconn test suite, as it tests all queries and parameters -// go through. There are two exceptions: -// - the health check: we just make that one work, so the gateway knows the -// tablet is healthy. -// - the error type returned: it's not a TabletError any more, but a ShardError. -// We still check the error code is correct though which is really all we care -// about. - -import ( - "testing" - - "context" - - "vitess.io/vitess/go/vt/grpcclient" - "vitess.io/vitess/go/vt/topo" - "vitess.io/vitess/go/vt/topo/memorytopo" - "vitess.io/vitess/go/vt/vttablet/queryservice" - "vitess.io/vitess/go/vt/vttablet/tabletconn" - "vitess.io/vitess/go/vt/vttablet/tabletconntest" - - querypb "vitess.io/vitess/go/vt/proto/query" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" -) - -// CreateFakeServers returns the servers to use for these tests -func CreateFakeServers(t *testing.T) (*tabletconntest.FakeQueryService, *topo.Server, string) { - cell := tabletconntest.TestCell - - // the FakeServer is just slightly modified - f := tabletconntest.CreateFakeServer(t) - f.TestingGateway = true - f.StreamHealthResponse = &querypb.StreamHealthResponse{ - Target: tabletconntest.TestTarget, - Serving: true, - TabletExternallyReparentedTimestamp: 1234589, - RealtimeStats: &querypb.RealtimeStats{ - ReplicationLagSeconds: 1, - }, - } - - // The topo server has a single SrvKeyspace - ts := memorytopo.NewServer(cell) - if err := ts.UpdateSrvKeyspace(context.Background(), cell, tabletconntest.TestTarget.Keyspace, &topodatapb.SrvKeyspace{ - Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - { - ServedType: tabletconntest.TestTarget.TabletType, - ShardReferences: []*topodatapb.ShardReference{ - { - Name: tabletconntest.TestTarget.Shard, - }, - }, - }, - }, - }); err != nil { - t.Fatalf("can't add srvKeyspace: %v", err) - } - - return f, ts, cell -} - -// gatewayAdapter implements the TabletConn interface, but sends the -// queries to the Gateway. -type gatewayAdapter struct { - Gateway -} - -// Close should be overridden to make sure we don't close the underlying Gateway. -func (ga *gatewayAdapter) Close(ctx context.Context) error { - return nil -} - -// TestSuite executes a set of tests on the provided gateway. The provided -// gateway needs to be configured with one established connection for -// tabletconntest.TestTarget.{Keyspace, Shard, TabletType} to the -// provided tabletconntest.FakeQueryService. -func TestSuite(t *testing.T, name string, g Gateway, f *tabletconntest.FakeQueryService) { - - protocolName := "gateway-test-" + name - - tabletconn.RegisterDialer(protocolName, func(tablet *topodatapb.Tablet, failFast grpcclient.FailFast) (queryservice.QueryService, error) { - return &gatewayAdapter{Gateway: g}, nil - }) - - tabletconntest.TestSuite(t, protocolName, &topodatapb.Tablet{ - Keyspace: tabletconntest.TestTarget.Keyspace, - Shard: tabletconntest.TestTarget.Shard, - Type: tabletconntest.TestTarget.TabletType, - Alias: tabletconntest.TestAlias, - }, f, nil) -} diff --git a/go/vt/vtgate/grpc_discovery_test.go b/go/vt/vtgate/grpc_discovery_test.go deleted file mode 100644 index 1a18e9fdcbb..00000000000 --- a/go/vt/vtgate/grpc_discovery_test.go +++ /dev/null @@ -1,86 +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 vtgate - -import ( - "flag" - "net" - "testing" - "time" - - "context" - - "google.golang.org/grpc" - - "vitess.io/vitess/go/vt/discovery" - "vitess.io/vitess/go/vt/srvtopo" - "vitess.io/vitess/go/vt/vttablet/grpcqueryservice" - "vitess.io/vitess/go/vt/vttablet/tabletconntest" - - // We will use gRPC to connect, register the dialer - _ "vitess.io/vitess/go/vt/vttablet/grpctabletconn" - - topodatapb "vitess.io/vitess/go/vt/proto/topodata" -) - -// TestGRPCDiscovery tests the discovery gateway with a gRPC -// connection from the gateway to the fake tablet. -func TestGRPCDiscovery(t *testing.T) { - flag.Set("tablet_protocol", "grpc") - - // Fake services for the tablet, topo server. - service, ts, cell := CreateFakeServers(t) - - // Tablet: listen on a random port. - listener, err := net.Listen("tcp", "127.0.0.1:0") - if err != nil { - t.Fatalf("Cannot listen: %v", err) - } - host := listener.Addr().(*net.TCPAddr).IP.String() - port := listener.Addr().(*net.TCPAddr).Port - defer listener.Close() - - // Tablet: create a gRPC server and listen on the port. - server := grpc.NewServer() - grpcqueryservice.Register(server, service) - go server.Serve(listener) - defer server.Stop() - - // VTGate: create the discovery healthcheck, and the gateway. - // Wait for the right tablets to be present. - hc := discovery.NewHealthCheck(context.Background(), 10*time.Second, 2*time.Minute, ts, cell, cell) - rs := srvtopo.NewResilientServer(ts, "TestGRPCDiscovery") - dg := NewTabletGateway(context.Background(), hc, rs, cell) - hc.AddTablet(&topodatapb.Tablet{ - Alias: tabletconntest.TestAlias, - Keyspace: tabletconntest.TestTarget.Keyspace, - Shard: tabletconntest.TestTarget.Shard, - Type: tabletconntest.TestTarget.TabletType, - Hostname: host, - PortMap: map[string]int32{ - "grpc": int32(port), - }, - }) - err = WaitForTablets(dg, []topodatapb.TabletType{tabletconntest.TestTarget.TabletType}) - if err != nil { - t.Fatalf("WaitForTablets failed: %v", err) - } - defer dg.Close(context.Background()) - - // run the test suite. - TestSuite(t, "discovery-grpc", dg, service) -} From 52221273df590d7268c6f6270d2b48f9f0d4af80 Mon Sep 17 00:00:00 2001 From: Florent Poinsard Date: Wed, 9 Mar 2022 10:21:04 +0100 Subject: [PATCH 08/10] fix: reset primarySession before TestSelectScatterPartial Signed-off-by: Florent Poinsard --- go/vt/vtgate/executor_select_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index 705813d6e09..7d410917020 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -1417,6 +1417,9 @@ func TestSelectScatter(t *testing.T) { func TestSelectScatterPartial(t *testing.T) { // Special setup: Don't use createExecutorEnv. + primarySession = &vtgatepb.Session{ + TargetString: "@primary", + } cell := "aa" hc := discovery.NewFakeHealthCheck(nil) s := createSandbox("TestExecutor") From 4b16187ea737cf902e2c28ceb5afcb710bd5237e Mon Sep 17 00:00:00 2001 From: Florent Poinsard Date: Wed, 9 Mar 2022 11:10:09 +0100 Subject: [PATCH 09/10] refactor: use int instead of atomic int for FakeHealthCheck's currentTabletUID Signed-off-by: Florent Poinsard --- go/vt/discovery/fake_healthcheck.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/go/vt/discovery/fake_healthcheck.go b/go/vt/discovery/fake_healthcheck.go index d707001f8e9..aa45c4b9c3d 100644 --- a/go/vt/discovery/fake_healthcheck.go +++ b/go/vt/discovery/fake_healthcheck.go @@ -23,7 +23,6 @@ import ( "google.golang.org/protobuf/proto" - "vitess.io/vitess/go/sync2" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vterrors" @@ -53,7 +52,7 @@ type FakeHealthCheck struct { // mu protects the items map mu sync.RWMutex items map[string]*fhcItem - currentTabletUID sync2.AtomicInt32 + currentTabletUID int // channel to return on subscribe. Pass nil if no subscribe should not return a channel ch chan *TabletHealth } @@ -247,7 +246,7 @@ func (fhc *FakeHealthCheck) Reset() { defer fhc.mu.Unlock() fhc.items = make(map[string]*fhcItem) - fhc.currentTabletUID.Set(0) + fhc.currentTabletUID = 0 } // AddFakeTablet inserts a fake entry into FakeHealthCheck. @@ -255,9 +254,12 @@ func (fhc *FakeHealthCheck) Reset() { // The Listener is called, as if AddTablet had been called. // For flexibility the connection is created via a connFactory callback func (fhc *FakeHealthCheck) 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 { + fhc.mu.Lock() + defer fhc.mu.Unlock() + // tabletUID must be unique - fhc.currentTabletUID.Add(1) - uid := fhc.currentTabletUID.Get() + fhc.currentTabletUID++ + uid := fhc.currentTabletUID t := topo.NewTablet(uint32(uid), cell, host) t.Keyspace = keyspace t.Shard = shard @@ -265,8 +267,6 @@ func (fhc *FakeHealthCheck) AddFakeTablet(cell, host string, port int32, keyspac t.PortMap["vt"] = port key := TabletToMapKey(t) - fhc.mu.Lock() - defer fhc.mu.Unlock() item := fhc.items[key] if item == nil { item = &fhcItem{ From 87fe76d3066383751fa4cd6afedd1e5f8d9079dc Mon Sep 17 00:00:00 2001 From: Florent Poinsard Date: Thu, 10 Mar 2022 08:07:59 +0100 Subject: [PATCH 10/10] refactor: unexported variables in healthcheck.go Signed-off-by: Florent Poinsard --- go/vt/discovery/healthcheck.go | 33 +++++++++---------- go/vt/discovery/legacy_healthcheck.go | 2 +- go/vt/discovery/legacy_topology_watcher.go | 2 +- .../discovery/legacy_topology_watcher_test.go | 8 ++--- go/vt/discovery/topology_watcher_test.go | 6 ++-- 5 files changed, 25 insertions(+), 26 deletions(-) diff --git a/go/vt/discovery/healthcheck.go b/go/vt/discovery/healthcheck.go index bb8bd6499fb..f88b8e554f4 100644 --- a/go/vt/discovery/healthcheck.go +++ b/go/vt/discovery/healthcheck.go @@ -67,29 +67,28 @@ var ( 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 - // AllowedTabletTypes is the list of allowed tablet types. e.g. {PRIMARY, 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") + + // tabletFilters are the keyspace|shard or keyrange filters to apply to the full set of tablets + tabletFilters 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. const ( - DefaultHealthCheckRetryDelay = 5 * time.Second - DefaultHealthCheckTimeout = 1 * time.Minute + defaultHealthCheckRetryDelay = 5 * time.Second + defaultHealthCheckTimeout = 1 * time.Minute - // DefaultTopoReadConcurrency is used as the 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 is used as the default value for // the refresh interval of a topology watcher. @@ -141,7 +140,7 @@ func ParseTabletURLTemplateFromFlag() { func init() { // Flags are not parsed at this point and the default value of the flag (just the hostname) will be used. ParseTabletURLTemplateFromFlag() - flag.Var(&TabletFilters, "tablet_filters", "Specifies a comma-separated list of 'keyspace|shard_name or keyrange' values to filter the tablets to watch") + 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") flag.Var(&KeyspacesToWatch, "keyspaces_to_watch", "Specifies which keyspaces this vtgate should have access to while routing queries or accessing the vschema") } @@ -283,12 +282,12 @@ func NewHealthCheck(ctx context.Context, retryDelay, healthCheckTimeout time.Dur if c == "" { continue } - if len(TabletFilters) > 0 { + if len(tabletFilters) > 0 { if len(KeyspacesToWatch) > 0 { log.Exitf("Only one of -keyspaces_to_watch and -tablet_filters may be specified at a time") } - fbs, err := NewFilterByShard(TabletFilters) + fbs, err := NewFilterByShard(tabletFilters) if err != nil { log.Exitf("Cannot parse tablet_filters parameter: %v", err) } @@ -296,7 +295,7 @@ func NewHealthCheck(ctx context.Context, retryDelay, healthCheckTimeout time.Dur } else if len(KeyspacesToWatch) > 0 { filter = NewFilterByKeyspace(KeyspacesToWatch) } - topoWatchers = append(topoWatchers, NewCellTabletsWatcher(ctx, topoServer, hc, filter, c, *RefreshInterval, *RefreshKnownTablets, *TopoReadConcurrency)) + topoWatchers = append(topoWatchers, NewCellTabletsWatcher(ctx, topoServer, hc, filter, c, *refreshInterval, *refreshKnownTablets, *topoReadConcurrency)) } hc.topoWatchers = topoWatchers diff --git a/go/vt/discovery/legacy_healthcheck.go b/go/vt/discovery/legacy_healthcheck.go index 0f97fa3642a..265c2829057 100644 --- a/go/vt/discovery/legacy_healthcheck.go +++ b/go/vt/discovery/legacy_healthcheck.go @@ -372,7 +372,7 @@ type legacyTabletHealth struct { // NewLegacyDefaultHealthCheck creates a new LegacyHealthCheck object with a default configuration. func NewLegacyDefaultHealthCheck() LegacyHealthCheck { - return NewLegacyHealthCheck(DefaultHealthCheckRetryDelay, DefaultHealthCheckTimeout) + return NewLegacyHealthCheck(defaultHealthCheckRetryDelay, defaultHealthCheckTimeout) } // NewLegacyHealthCheck creates a new LegacyHealthCheck object. diff --git a/go/vt/discovery/legacy_topology_watcher.go b/go/vt/discovery/legacy_topology_watcher.go index cbee6b3b94e..194396df0c1 100644 --- a/go/vt/discovery/legacy_topology_watcher.go +++ b/go/vt/discovery/legacy_topology_watcher.go @@ -55,7 +55,7 @@ func NewLegacyCellTabletsWatcher(ctx context.Context, topoServer *topo.Server, t // 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) { + 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: diff --git a/go/vt/discovery/legacy_topology_watcher_test.go b/go/vt/discovery/legacy_topology_watcher_test.go index b828289b1bd..ae533fda5aa 100644 --- a/go/vt/discovery/legacy_topology_watcher_test.go +++ b/go/vt/discovery/legacy_topology_watcher_test.go @@ -138,7 +138,7 @@ func checkLegacyWatcher(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 = checkLegacyOpCounts(t, tw, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "AddTablet": 1}) @@ -154,7 +154,7 @@ func checkLegacyWatcher(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 { @@ -167,7 +167,7 @@ func checkLegacyWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { // 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) @@ -209,7 +209,7 @@ func checkLegacyWatcher(t *testing.T, cellTablets, refreshKnownTablets bool) { // trigger a ReplaceTablet in loadTablets because the uid does not // match. // - // This case *is* detected even if RefreshKnownTablets is false + // 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 { diff --git a/go/vt/discovery/topology_watcher_test.go b/go/vt/discovery/topology_watcher_test.go index 1ea6bfe2418..7ba29b4eac6 100644 --- a/go/vt/discovery/topology_watcher_test.go +++ b/go/vt/discovery/topology_watcher_test.go @@ -169,7 +169,7 @@ func checkWatcher(t *testing.T, 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, counts, map[string]int64{"ListTablets": 1, "GetTablet": 2, "AddTablet": 1}) @@ -185,7 +185,7 @@ func checkWatcher(t *testing.T, 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 { @@ -198,7 +198,7 @@ func checkWatcher(t *testing.T, refreshKnownTablets bool) { // 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)