From 48e7708c6d9690dc2499f534b541e109c7df3115 Mon Sep 17 00:00:00 2001 From: Liang Guo Date: Wed, 4 May 2016 15:20:36 -0700 Subject: [PATCH 1/4] Delete shard gateway/conn related code, and also fix unit tests to use discoverygateway. --- go/cmd/vtcombo/main.go | 2 +- go/cmd/vtgate/vtgate.go | 22 +- go/vt/vtgate/balancer.go | 203 ------- go/vt/vtgate/balancer_test.go | 248 -------- go/vt/vtgate/discoverygateway.go | 19 +- go/vt/vtgate/discoverygateway_test.go | 140 +---- go/vt/vtgate/fakehealthcheck_test.go | 126 ++++ go/vt/vtgate/gateway.go | 34 +- go/vt/vtgate/resolver.go | 32 +- go/vt/vtgate/resolver_test.go | 177 +++--- go/vt/vtgate/router_framework_test.go | 16 +- go/vt/vtgate/router_select_test.go | 29 +- go/vt/vtgate/sandbox_test.go | 104 +--- go/vt/vtgate/scatter_conn.go | 46 +- go/vt/vtgate/scatter_conn_test.go | 147 +++-- go/vt/vtgate/shard_conn.go | 495 ---------------- go/vt/vtgate/shard_conn_flaky_test.go | 818 -------------------------- go/vt/vtgate/shardgateway.go | 207 ------- go/vt/vtgate/vertical_split_test.go | 144 ----- go/vt/vtgate/vtgate.go | 22 +- go/vt/vtgate/vtgate_test.go | 233 +++++--- py/vttest/environment.py | 1 + 22 files changed, 568 insertions(+), 2697 deletions(-) delete mode 100644 go/vt/vtgate/balancer.go delete mode 100644 go/vt/vtgate/balancer_test.go create mode 100644 go/vt/vtgate/fakehealthcheck_test.go delete mode 100644 go/vt/vtgate/shard_conn.go delete mode 100644 go/vt/vtgate/shard_conn_flaky_test.go delete mode 100644 go/vt/vtgate/shardgateway.go delete mode 100644 go/vt/vtgate/vertical_split_test.go diff --git a/go/cmd/vtcombo/main.go b/go/cmd/vtcombo/main.go index 010f5f67275..494e0352652 100644 --- a/go/cmd/vtcombo/main.go +++ b/go/cmd/vtcombo/main.go @@ -105,7 +105,7 @@ func main() { topodatapb.TabletType_REPLICA, topodatapb.TabletType_RDONLY, } - vtgate.Init(context.Background(), healthCheck, ts, resilientSrvTopoServer, cell, 1*time.Millisecond /*retryDelay*/, 2 /*retryCount*/, 30*time.Second /*connTimeoutTotal*/, 10*time.Second /*connTimeoutPerConn*/, 365*24*time.Hour /*connLife*/, tabletTypesToWait, 0 /*maxInFlight*/, "" /*testGateway*/) + vtgate.Init(context.Background(), healthCheck, ts, resilientSrvTopoServer, cell, 2 /*retryCount*/, tabletTypesToWait, 0 /*maxInFlight*/) // vtctld configuration and init vtctld.InitVtctld(ts) diff --git a/go/cmd/vtgate/vtgate.go b/go/cmd/vtgate/vtgate.go index 90adb4e6f70..e68111d3e52 100644 --- a/go/cmd/vtgate/vtgate.go +++ b/go/cmd/vtgate/vtgate.go @@ -25,17 +25,13 @@ import ( ) var ( - cell = flag.String("cell", "test_nj", "cell to use") - retryDelay = flag.Duration("retry-delay", 2*time.Millisecond, "retry delay") - retryCount = flag.Int("retry-count", 2, "retry count") - connTimeoutTotal = flag.Duration("conn-timeout-total", 3*time.Second, "vttablet connection timeout (total)") - connTimeoutPerConn = flag.Duration("conn-timeout-per-conn", 1500*time.Millisecond, "vttablet connection timeout (per connection)") - connLife = flag.Duration("conn-life", 365*24*time.Hour, "average life of vttablet connections") - maxInFlight = flag.Int("max-in-flight", 0, "maximum number of calls to allow simultaneously") - healthCheckRetryDelay = flag.Duration("healthcheck_retry_delay", 2*time.Millisecond, "health check retry delay") - healthCheckTimeout = flag.Duration("healthcheck_timeout", time.Minute, "the health check timeout period") - tabletTypesToWait = flag.String("tablet_types_to_wait", "", "wait till connected for specified tablet types during Gateway initialization") - testGateway = flag.String("test_gateway", "", "additional gateway to test health check module") + cell = flag.String("cell", "test_nj", "cell to use") + retryCount = flag.Int("retry-count", 2, "retry count") + maxInFlight = flag.Int("max-in-flight", 0, "maximum number of calls to allow simultaneously") + healthCheckConnTimeout = flag.Duration("healthcheck_conn_timeout", 3*time.Second, "healthcheck connection timeout") + healthCheckRetryDelay = flag.Duration("healthcheck_retry_delay", 2*time.Millisecond, "health check retry delay") + healthCheckTimeout = flag.Duration("healthcheck_timeout", time.Minute, "the health check timeout period") + tabletTypesToWait = flag.String("tablet_types_to_wait", "", "wait till connected for specified tablet types during Gateway initialization") ) var resilientSrvTopoServer *vtgate.ResilientSrvTopoServer @@ -62,7 +58,7 @@ func main() { resilientSrvTopoServer = vtgate.NewResilientSrvTopoServer(ts, "ResilientSrvTopoServer") - healthCheck = discovery.NewHealthCheck(*connTimeoutTotal, *healthCheckRetryDelay, *healthCheckTimeout, "" /* statsSuffix */) + healthCheck = discovery.NewHealthCheck(*healthCheckConnTimeout, *healthCheckRetryDelay, *healthCheckTimeout, "" /* statsSuffix */) tabletTypes := make([]topodatapb.TabletType, 0, 1) if len(*tabletTypesToWait) != 0 { @@ -75,7 +71,7 @@ func main() { tabletTypes = append(tabletTypes, tt) } } - vtg := vtgate.Init(context.Background(), healthCheck, ts, resilientSrvTopoServer, *cell, *retryDelay, *retryCount, *connTimeoutTotal, *connTimeoutPerConn, *connLife, tabletTypes, *maxInFlight, *testGateway) + vtg := vtgate.Init(context.Background(), healthCheck, ts, resilientSrvTopoServer, *cell, *retryCount, tabletTypes, *maxInFlight) servenv.OnRun(func() { addStatusParts(vtg) diff --git a/go/vt/vtgate/balancer.go b/go/vt/vtgate/balancer.go deleted file mode 100644 index 89b120eff2b..00000000000 --- a/go/vt/vtgate/balancer.go +++ /dev/null @@ -1,203 +0,0 @@ -// Copyright 2012, Google Inc. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -package vtgate - -import ( - "flag" - "fmt" - "math/rand" - "sort" - "sync" - "time" - - log "github.com/golang/glog" - - topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" - vtrpcpb "github.com/youtube/vitess/go/vt/proto/vtrpc" - "github.com/youtube/vitess/go/vt/vterrors" -) - -var resetDownConnDelay = flag.Duration("reset-down-conn-delay", 10*time.Minute, "delay to reset a marked down tabletconn") - -// GetEndPointsFunc defines the callback to topo server. -type GetEndPointsFunc func() (*topodatapb.EndPoints, error) - -// Balancer is a simple round-robin load balancer. -// It allows you to temporarily mark down nodes that -// are non-functional. -type Balancer struct { - mu sync.Mutex - addressNodes []*addressStatus - index int - getEndPoints GetEndPointsFunc - retryDelay time.Duration - resetDownConnDelay time.Duration -} - -type addressStatus struct { - endPoint *topodatapb.EndPoint - timeRetry time.Time - balancer *Balancer -} - -// NewBalancer creates a Balancer. getAddresses is the function -// it will use to refresh the list of addresses if one of the -// nodes has been marked down. The list of addresses is shuffled. -// retryDelay specifies the minimum time a node will be marked down -// before it will be cleared for a retry. -func NewBalancer(getEndPoints GetEndPointsFunc, retryDelay time.Duration) *Balancer { - blc := new(Balancer) - blc.getEndPoints = getEndPoints - blc.retryDelay = retryDelay - blc.resetDownConnDelay = *resetDownConnDelay - return blc -} - -// Get returns a single endpoint that was not recently marked down. -// If it finds an address that was down for longer than retryDelay, -// it refreshes the list of addresses and returns the next available -// node. If all addresses are marked down, it waits and retries. -// If a refresh fails, it returns an error. -func (blc *Balancer) Get() (endPoints []*topodatapb.EndPoint, err error) { - blc.mu.Lock() - defer blc.mu.Unlock() - - // Clear timeRetry if it has been a long time - for _, addrNode := range blc.addressNodes { - if time.Now().Sub(addrNode.timeRetry) > blc.resetDownConnDelay { - addrNode.timeRetry = time.Time{} - } - } - - // Get the latest endpoints - err = blc.refresh() - if err != nil { - return []*topodatapb.EndPoint{}, err - } - - // Return all endpoints without markdown and timeRetry < now(), - // so endpoints just marked down (within retryDelay) are ignored. - validEndPoints := make([]*topodatapb.EndPoint, 0, 1) - for _, addrNode := range blc.addressNodes { - if addrNode.timeRetry.IsZero() || addrNode.timeRetry.Before(time.Now()) { - validEndPoints = append(validEndPoints, addrNode.endPoint) - continue - } - break - } - - return validEndPoints, nil -} - -// MarkDown marks the specified address down. Such addresses -// will not be used by Balancer for the duration of retryDelay. -func (blc *Balancer) MarkDown(uid uint32, reason string) { - blc.mu.Lock() - defer blc.mu.Unlock() - if index := findAddrNode(blc.addressNodes, uid); index != -1 { - log.Infof("Marking down %v at %+v (%v)", uid, blc.addressNodes[index].endPoint, reason) - blc.addressNodes[index].timeRetry = time.Now().Add(blc.retryDelay) - } -} - -func (blc *Balancer) refresh() error { - endPoints, err := blc.getEndPoints() - if err != nil { - return err - } - // Add new addressNodes - if endPoints != nil { - for _, endPoint := range endPoints.Entries { - if index := findAddrNode(blc.addressNodes, endPoint.Uid); index == -1 { - addrNode := &addressStatus{ - endPoint: endPoint, - balancer: blc, - } - blc.addressNodes = append(blc.addressNodes, addrNode) - } else { - blc.addressNodes[index].endPoint = endPoint - } - } - } - // Remove those that went away - i := 0 - for i < len(blc.addressNodes) { - if index := findAddress(endPoints, blc.addressNodes[i].endPoint.Uid); index == -1 { - blc.addressNodes = delAddrNode(blc.addressNodes, i) - continue - } - i++ - } - if len(blc.addressNodes) == 0 { - return vterrors.FromError( - vtrpcpb.ErrorCode_INTERNAL_ERROR, - fmt.Errorf("no available addresses"), - ) - } - // Sort endpoints by timeRetry (from ZERO to largest) - sort.Sort(AddressList(blc.addressNodes)) - // Randomize endpoints with ZERO timeRetry - shuffle(blc.addressNodes, findFirstAddrNodeNonZeroTimeRetry(blc.addressNodes)) - return nil -} - -// AddressList is the slice of addressStatus. -type AddressList []*addressStatus - -func (al AddressList) Len() int { - return len(al) -} - -func (al AddressList) Swap(i, j int) { - al[i], al[j] = al[j], al[i] -} - -func (al AddressList) Less(i, j int) bool { - return al[i].timeRetry.Before(al[j].timeRetry) -} - -func findFirstAddrNodeNonZeroTimeRetry(addressNodes []*addressStatus) (index int) { - for i, addrNode := range addressNodes { - if !addrNode.timeRetry.IsZero() { - return i - } - } - return len(addressNodes) -} - -func findAddrNode(addressNodes []*addressStatus, uid uint32) (index int) { - for i, addrNode := range addressNodes { - if uid == addrNode.endPoint.Uid { - return i - } - } - return -1 -} - -func findAddress(endPoints *topodatapb.EndPoints, uid uint32) (index int) { - if endPoints == nil { - return -1 - } - for i, endPoint := range endPoints.Entries { - if uid == endPoint.Uid { - return i - } - } - return -1 -} - -func delAddrNode(addressNodes []*addressStatus, index int) []*addressStatus { - copy(addressNodes[index:len(addressNodes)-1], addressNodes[index+1:]) - return addressNodes[:len(addressNodes)-1] -} - -// shuffle uses the Fisher-Yates algorithm. -func shuffle(addressNodes []*addressStatus, length int) { - index := 0 - for i := length - 1; i > 0; i-- { - index = rand.Intn(i + 1) - addressNodes[i], addressNodes[index] = addressNodes[index], addressNodes[i] - } -} diff --git a/go/vt/vtgate/balancer_test.go b/go/vt/vtgate/balancer_test.go deleted file mode 100644 index 97ca2c6adcf..00000000000 --- a/go/vt/vtgate/balancer_test.go +++ /dev/null @@ -1,248 +0,0 @@ -// Copyright 2012, Google Inc. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -package vtgate - -import ( - "fmt" - "testing" - "time" - - topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" -) - -var ( - RetryDelay = time.Duration(1 * time.Second) -) - -var counter = 0 - -func endPoints3() (*topodatapb.EndPoints, error) { - counter++ - return &topodatapb.EndPoints{ - Entries: []*topodatapb.EndPoint{ - { - Uid: 0, - Host: "0", - PortMap: map[string]int32{ - "vt": 1, - }, - }, - { - Uid: 1, - Host: "1", - PortMap: map[string]int32{ - "vt": 2, - }, - }, - { - Uid: 2, - Host: "2", - PortMap: map[string]int32{ - "vt": 3, - }, - }, - }, - }, nil -} - -func TestRandomness(t *testing.T) { - for i := 0; i < 100; i++ { - b := NewBalancer(endPoints3, RetryDelay) - endPoints, _ := b.Get() - // Ensure that you don't always get the first element at front - // in the balancer. - if endPoints[0].Uid == 0 { - continue - } - return - } - t.Errorf("end points are not shuffled") -} - -func TestFindAddress(t *testing.T) { - addrs, _ := endPoints3() - goti := findAddress(addrs, 1) - if goti != 1 { - t.Errorf("want 1, got %d", goti) - } -} - -func TestFindDeleteAddrNode(t *testing.T) { - addrNodes := []*addressStatus{ - {endPoint: &topodatapb.EndPoint{Uid: 0}}, - {endPoint: &topodatapb.EndPoint{Uid: 1}}, - {endPoint: &topodatapb.EndPoint{Uid: 2}}, - } - goti := findAddrNode(addrNodes, 1) - if goti != 1 { - t.Errorf("want 1, got %d", goti) - } - addrNodes = delAddrNode(addrNodes, 1) - // The middle node "1" was deleted. - if len(addrNodes) != 2 { - t.Errorf("want 2, got %d", len(addrNodes)) - } - if addrNodes[1].endPoint.Uid != 2 { - t.Errorf("want 2, got %v", addrNodes[1].endPoint.Uid) - } -} - -func endPointsError() (*topodatapb.EndPoints, error) { - return nil, fmt.Errorf("expected error") -} - -func endPointsNone() (*topodatapb.EndPoints, error) { - return nil, nil -} - -func TestGetAddressesFail(t *testing.T) { - b := NewBalancer(endPointsError, RetryDelay) - _, err := b.Get() - // Ensure that end point errors are returned correctly. - want := "expected error" - if err == nil || err.Error() != want { - t.Errorf("want %s, got %v", want, err) - } - - b.getEndPoints = endPointsNone - _, err = b.Get() - // Ensure no available addresses is treated as error - want = "no available addresses" - if err == nil || err.Error() != want { - t.Errorf("want nil, got %v", err) - } - - b.getEndPoints = endPoints3 - _, err = b.Get() - // Ensure no error is returned if end point doesn't fail. - if err != nil { - t.Errorf("want nil, got %v", err) - } -} - -func TestGetSimple(t *testing.T) { - b := NewBalancer(endPoints3, RetryDelay) - firstEndPoints, _ := b.Get() - for i := 0; i < 100; i++ { - endPoints, _ := b.Get() - if endPoints[0].Uid == firstEndPoints[0].Uid { - continue - } - return - } - // Ensure that the same address is not always returned. - t.Errorf("ids are equal: %v", firstEndPoints[0]) -} - -func TestMarkDown(t *testing.T) { - start := counter - retryDelay := 100 * time.Millisecond - b := NewBalancer(endPoints3, retryDelay) - addrs, _ := b.Get() - b.MarkDown(addrs[0].Uid, "") - addrs, _ = b.Get() - b.MarkDown(addrs[0].Uid, "") - addrs1, _ := b.Get() - addrs2, _ := b.Get() - // Two addresses are marked down. Only one address is avaiilable. - if addrs1[0].Uid != addrs2[0].Uid { - t.Errorf("ids are not equal: %v, %v", addrs1[0], addrs2[0]) - } - addrs, _ = b.Get() - b.MarkDown(addrs[0].Uid, "") - // All were marked down. Get should return immediately with empty endpoints. - done := make(chan struct{}) - go func() { - addrs, _ = b.Get() - if len(addrs) != 0 { - t.Errorf("Get() returned endpoints, want 0, got %v", len(addrs)) - } - time.Sleep(retryDelay) - addrs, _ = b.Get() - if len(addrs) == 0 { - t.Errorf("Get() returned no endpoints, want >0, got %v", len(addrs)) - } - close(done) - }() - select { - case <-done: - case <-time.After(10 * time.Second): - t.Errorf("Get() is stuck in Sleep()") - } - if addrs[0].Host == "" { - t.Errorf("want non-empty") - } - // Ensure end points were refreshed, counter should have gone up. - if start == counter { - t.Errorf("want %v < %v", start, counter) - } -} - -var addrNum uint32 = 10 - -func endPointsMorph() (*topodatapb.EndPoints, error) { - addrNum++ - return &topodatapb.EndPoints{ - Entries: []*topodatapb.EndPoint{ - { - Uid: addrNum, - Host: fmt.Sprintf("%d", addrNum), - PortMap: map[string]int32{ - "vt": 1, - }, - }, - { - Uid: 1, - Host: "1", - PortMap: map[string]int32{ - "vt": int32(addrNum), - }, - }, - { - Uid: 2, - Host: "2", - PortMap: map[string]int32{ - "vt": 3, - }, - }, - }, - }, nil -} - -func TestRefresh(t *testing.T) { - b := NewBalancer(endPointsMorph, RetryDelay) - b.refresh() - index := findAddrNode(b.addressNodes, 11) - // "11" should be found in the list. - if index == -1 { - t.Errorf("want other than -1: %v", index) - } - // "1" should be in the list with port 11 - portStart := b.addressNodes[findAddrNode(b.addressNodes, 1)].endPoint.PortMap["vt"] - if portStart != 11 { - t.Errorf("want 11, got %v", portStart) - } - b.MarkDown(1, "") - b.refresh() - // "11" should not be found. It should be "12" now. - index = findAddrNode(b.addressNodes, 11) - if index != -1 { - t.Errorf("want -1, got %v", index) - } - index = findAddrNode(b.addressNodes, 12) - if index == -1 { - t.Errorf("got other than -1: %v", index) - } - index = findAddrNode(b.addressNodes, 1) - // "1" should be marked down (non-zero timeRetry) - if b.addressNodes[index].timeRetry.IsZero() { - t.Errorf("want non-zero, got 0") - } - // "1" should have the updated port 12 - portNew := b.addressNodes[index].endPoint.PortMap["vt"] - if portNew != 12 { - t.Errorf("want 12, got %v", portNew) - } -} diff --git a/go/vt/vtgate/discoverygateway.go b/go/vt/vtgate/discoverygateway.go index 75e3ec6cda9..effcdf4b43e 100644 --- a/go/vt/vtgate/discoverygateway.go +++ b/go/vt/vtgate/discoverygateway.go @@ -15,7 +15,6 @@ import ( "golang.org/x/net/context" "github.com/youtube/vitess/go/sqltypes" - "github.com/youtube/vitess/go/stats" "github.com/youtube/vitess/go/vt/discovery" "github.com/youtube/vitess/go/vt/tabletserver/querytypes" "github.com/youtube/vitess/go/vt/tabletserver/tabletconn" @@ -42,7 +41,7 @@ func init() { RegisterGatewayCreator(gatewayImplementationDiscovery, createDiscoveryGateway) } -func createDiscoveryGateway(hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, cell string, _ time.Duration, retryCount int, _, _, _ time.Duration, _ *stats.MultiTimings, tabletTypesToWait []topodatapb.TabletType) Gateway { +func createDiscoveryGateway(hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, cell string, retryCount int, tabletTypesToWait []topodatapb.TabletType) Gateway { dg := &discoveryGateway{ hc: hc, topoServer: topoServer, @@ -95,11 +94,6 @@ func (dg *discoveryGateway) waitForEndPoints() error { return err } -// InitializeConnections creates connections to VTTablets. -func (dg *discoveryGateway) InitializeConnections(ctx context.Context) error { - return nil -} - // Execute executes the non-streaming query for the specified keyspace, shard, and tablet type. func (dg *discoveryGateway) Execute(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, query string, bindVars map[string]interface{}, transactionID int64) (qr *sqltypes.Result, err error) { err = dg.withRetry(ctx, keyspace, shard, tabletType, func(conn tabletconn.TabletConn) error { @@ -389,7 +383,7 @@ func (dg *discoveryGateway) getEndPoints(keyspace, shard string, tabletType topo return epList } -// WrapError returns ShardConnError which preserves the original error code if possible, +// WrapError returns ShardError which preserves the original error code if possible, // adds the connection context // and adds a bit to determine whether the keyspace/shard needs to be // re-resolved for a potential sharding event. @@ -398,18 +392,11 @@ func WrapError(in error, keyspace, shard string, tabletType topodatapb.TabletTyp return nil } shardIdentifier := fmt.Sprintf("%s.%s.%s, %+v", keyspace, shard, strings.ToLower(tabletType.String()), endPoint) - code := tabletconn.ERR_NORMAL - serverError, ok := in.(*tabletconn.ServerError) - if ok { - code = serverError.Code - } - shardConnErr := &ShardConnError{ - Code: code, + return &ShardError{ ShardIdentifier: shardIdentifier, InTransaction: inTransaction, Err: in, EndPointCode: vterrors.RecoverVtErrorCode(in), } - return shardConnErr } diff --git a/go/vt/vtgate/discoverygateway_test.go b/go/vt/vtgate/discoverygateway_test.go index 6ba6ed6648d..d19226c9607 100644 --- a/go/vt/vtgate/discoverygateway_test.go +++ b/go/vt/vtgate/discoverygateway_test.go @@ -2,17 +2,15 @@ package vtgate import ( "fmt" + "reflect" "testing" - "time" "golang.org/x/net/context" - "github.com/youtube/vitess/go/vt/discovery" "github.com/youtube/vitess/go/vt/tabletserver/querytypes" - "github.com/youtube/vitess/go/vt/tabletserver/tabletconn" "github.com/youtube/vitess/go/vt/topo" + "github.com/youtube/vitess/go/vt/vterrors" - querypb "github.com/youtube/vitess/go/vt/proto/query" topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" vtrpcpb "github.com/youtube/vitess/go/vt/proto/vtrpc" ) @@ -71,7 +69,7 @@ func TestDiscoveryGatewayGetEndPoints(t *testing.T) { keyspace := "ks" shard := "0" hc := newFakeHealthCheck() - dg := createDiscoveryGateway(hc, topo.Server{}, nil, "local", time.Millisecond, 2, time.Second, time.Second, time.Second, nil, nil).(*discoveryGateway) + dg := createDiscoveryGateway(hc, topo.Server{}, nil, "local", 2, nil).(*discoveryGateway) // replica should only use local ones hc.Reset() @@ -97,13 +95,13 @@ func testDiscoveryGatewayGeneric(t *testing.T, streaming bool, f func(dg Gateway shard := "0" tabletType := topodatapb.TabletType_REPLICA hc := newFakeHealthCheck() - dg := createDiscoveryGateway(hc, topo.Server{}, nil, "cell", time.Millisecond, 2, time.Second, time.Second, time.Second, nil, nil) + dg := createDiscoveryGateway(hc, topo.Server{}, nil, "cell", 2, nil) // no endpoint hc.Reset() want := "shard, host: ks.0.replica, , no valid endpoint" err := f(dg, keyspace, shard, tabletType) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_INTERNAL_ERROR) + verifyShardError(t, err, want, vtrpcpb.ErrorCode_INTERNAL_ERROR) if hc.GetStatsFromTargetCounter != 1 { t.Errorf("hc.GetStatsFromTargetCounter = %v; want 1", hc.GetStatsFromTargetCounter) } @@ -113,7 +111,7 @@ func testDiscoveryGatewayGeneric(t *testing.T, streaming bool, f func(dg Gateway hc.addTestEndPoint("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, false, 10, fmt.Errorf("no connection"), nil) want = "shard, host: ks.0.replica, , no valid endpoint" err = f(dg, keyspace, shard, tabletType) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_INTERNAL_ERROR) + verifyShardError(t, err, want, vtrpcpb.ErrorCode_INTERNAL_ERROR) if hc.GetStatsFromTargetCounter != 1 { t.Errorf("hc.GetStatsFromTargetCounter = %v; want 1", hc.GetStatsFromTargetCounter) } @@ -123,7 +121,7 @@ func testDiscoveryGatewayGeneric(t *testing.T, streaming bool, f func(dg Gateway ep1 := hc.addTestEndPoint("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, false, 10, nil, nil) want = fmt.Sprintf(`shard, host: ks.0.replica, , no valid endpoint`) err = f(dg, keyspace, shard, tabletType) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_INTERNAL_ERROR) + verifyShardError(t, err, want, vtrpcpb.ErrorCode_INTERNAL_ERROR) if hc.GetStatsFromTargetCounter != 1 { t.Errorf("hc.GetStatsFromTargetCounter = %v; want 1", hc.GetStatsFromTargetCounter) } @@ -170,7 +168,7 @@ func testDiscoveryGatewayGeneric(t *testing.T, streaming bool, f func(dg Gateway ep1 = hc.addTestEndPoint("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, true, 10, nil, &sandboxConn{mustFailServer: 1}) want = fmt.Sprintf(`shard, host: ks.0.replica, %+v, error: err`, ep1) err = f(dg, keyspace, shard, tabletType) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_BAD_INPUT) + verifyShardError(t, err, want, vtrpcpb.ErrorCode_BAD_INPUT) if hc.GetStatsFromTargetCounter != 1 { t.Errorf("hc.GetStatsFromTargetCounter = %v; want 1", hc.GetStatsFromTargetCounter) } @@ -180,7 +178,7 @@ func testDiscoveryGatewayGeneric(t *testing.T, streaming bool, f func(dg Gateway ep1 = hc.addTestEndPoint("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, true, 10, nil, &sandboxConn{mustFailConn: 1}) want = fmt.Sprintf(`shard, host: ks.0.replica, %+v, error: conn`, ep1) err = f(dg, keyspace, shard, tabletType) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_UNKNOWN_ERROR) + verifyShardError(t, err, want, vtrpcpb.ErrorCode_UNKNOWN_ERROR) if hc.GetStatsFromTargetCounter != 1 { t.Errorf("hc.GetStatsFromTargetCounter = %v; want 1", hc.GetStatsFromTargetCounter) } @@ -202,7 +200,7 @@ func testDiscoveryGatewayTransact(t *testing.T, streaming bool, f func(dg Gatewa shard := "0" tabletType := topodatapb.TabletType_REPLICA hc := newFakeHealthCheck() - dg := createDiscoveryGateway(hc, topo.Server{}, nil, "cell", time.Millisecond, 2, time.Second, time.Second, time.Second, nil, nil) + dg := createDiscoveryGateway(hc, topo.Server{}, nil, "cell", 2, nil) // retry error - no retry hc.Reset() @@ -225,121 +223,21 @@ func testDiscoveryGatewayTransact(t *testing.T, streaming bool, f func(dg Gatewa ep1 = hc.addTestEndPoint("cell", "1.1.1.1", 1001, keyspace, shard, tabletType, true, 10, nil, &sandboxConn{mustFailConn: 1}) want := fmt.Sprintf(`shard, host: ks.0.replica, %+v, error: conn`, ep1) err = f(dg, keyspace, shard, tabletType) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_UNKNOWN_ERROR) + verifyShardError(t, err, want, vtrpcpb.ErrorCode_UNKNOWN_ERROR) if hc.GetStatsFromTargetCounter != 1 { t.Errorf("hc.GetStatsFromTargetCounter = %v; want 1", hc.GetStatsFromTargetCounter) } } -func newFakeHealthCheck() *fakeHealthCheck { - return &fakeHealthCheck{items: make(map[string]*fhcItem)} -} - -type fhcItem struct { - eps *discovery.EndPointStats - conn tabletconn.TabletConn -} - -type fakeHealthCheck struct { - items map[string]*fhcItem - - // stats - GetStatsFromTargetCounter int - GetStatsFromKeyspaceShardCounter int -} - -func (fhc *fakeHealthCheck) Reset() { - fhc.GetStatsFromTargetCounter = 0 - fhc.GetStatsFromKeyspaceShardCounter = 0 - fhc.items = make(map[string]*fhcItem) -} - -// SetListener sets the listener for healthcheck updates. -func (fhc *fakeHealthCheck) SetListener(listener discovery.HealthCheckStatsListener) { -} - -// AddEndPoint adds the endpoint, and starts health check. -func (fhc *fakeHealthCheck) AddEndPoint(cell, name string, endPoint *topodatapb.EndPoint) { - key := discovery.EndPointToMapKey(endPoint) - item := &fhcItem{ - eps: &discovery.EndPointStats{ - EndPoint: endPoint, - Cell: cell, - Name: name, - }, +func verifyShardError(t *testing.T, err error, wantErr string, wantCode vtrpcpb.ErrorCode) { + if err == nil || err.Error() != wantErr { + t.Errorf("wanted error: %s, got error: %v", wantErr, err) } - fhc.items[key] = item -} - -// RemoveEndPoint removes the endpoint, and stops the health check. -func (fhc *fakeHealthCheck) RemoveEndPoint(endPoint *topodatapb.EndPoint) { - key := discovery.EndPointToMapKey(endPoint) - delete(fhc.items, key) -} - -// GetEndPointStatsFromKeyspaceShard returns all EndPointStats for the given keyspace/shard. -func (fhc *fakeHealthCheck) GetEndPointStatsFromKeyspaceShard(keyspace, shard string) []*discovery.EndPointStats { - fhc.GetStatsFromKeyspaceShardCounter++ - var res []*discovery.EndPointStats - for _, item := range fhc.items { - if item.eps.Target == nil { - continue - } - if item.eps.Target.Keyspace == keyspace && item.eps.Target.Shard == shard { - res = append(res, item.eps) - } - } - return res -} - -// GetEndPointStatsFromTarget returns all EndPointStats for the given target. -func (fhc *fakeHealthCheck) GetEndPointStatsFromTarget(keyspace, shard string, tabletType topodatapb.TabletType) []*discovery.EndPointStats { - fhc.GetStatsFromTargetCounter++ - var res []*discovery.EndPointStats - for _, item := range fhc.items { - if item.eps.Target == nil { - continue - } - if item.eps.Target.Keyspace == keyspace && item.eps.Target.Shard == shard && item.eps.Target.TabletType == tabletType { - res = append(res, item.eps) - } - } - return res -} - -// GetConnection returns the TabletConn of the given endpoint. -func (fhc *fakeHealthCheck) GetConnection(endPoint *topodatapb.EndPoint) tabletconn.TabletConn { - key := discovery.EndPointToMapKey(endPoint) - if item := fhc.items[key]; item != nil { - return item.conn + if _, ok := err.(*ShardError); !ok { + t.Errorf("wanted error type *ShardConnError, got error type: %v", reflect.TypeOf(err)) } - return nil -} - -// CacheStatus returns a displayable version of the cache. -func (fhc *fakeHealthCheck) CacheStatus() discovery.EndPointsCacheStatusList { - return nil -} - -// Close stops the healthcheck. -func (fhc *fakeHealthCheck) Close() error { - return nil -} - -func (fhc *fakeHealthCheck) addTestEndPoint(cell, host string, port int32, keyspace, shard string, tabletType topodatapb.TabletType, serving bool, reparentTS int64, err error, conn tabletconn.TabletConn) *topodatapb.EndPoint { - ep := topo.NewEndPoint(0, host) - ep.PortMap["vt"] = port - key := discovery.EndPointToMapKey(ep) - item := fhc.items[key] - if item == nil { - fhc.AddEndPoint(cell, "", ep) - item = fhc.items[key] + code := vterrors.RecoverVtErrorCode(err) + if code != wantCode { + t.Errorf("wanted error code: %s, got: %v", wantCode, code) } - item.eps.Target = &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: tabletType} - item.eps.Serving = serving - item.eps.TabletExternallyReparentedTimestamp = reparentTS - item.eps.Stats = &querypb.RealtimeStats{} - item.eps.LastError = err - item.conn = conn - return ep } diff --git a/go/vt/vtgate/fakehealthcheck_test.go b/go/vt/vtgate/fakehealthcheck_test.go new file mode 100644 index 00000000000..174d9a668d6 --- /dev/null +++ b/go/vt/vtgate/fakehealthcheck_test.go @@ -0,0 +1,126 @@ +package vtgate + +import ( + "github.com/youtube/vitess/go/vt/discovery" + "github.com/youtube/vitess/go/vt/tabletserver/tabletconn" + "github.com/youtube/vitess/go/vt/topo" + + querypb "github.com/youtube/vitess/go/vt/proto/query" + topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" +) + +func newFakeHealthCheck() *fakeHealthCheck { + return &fakeHealthCheck{items: make(map[string]*fhcItem)} +} + +type fhcItem struct { + eps *discovery.EndPointStats + conn tabletconn.TabletConn +} + +type fakeHealthCheck struct { + items map[string]*fhcItem + + // stats + GetStatsFromTargetCounter int + GetStatsFromKeyspaceShardCounter int +} + +func (fhc *fakeHealthCheck) Reset() { + fhc.GetStatsFromTargetCounter = 0 + fhc.GetStatsFromKeyspaceShardCounter = 0 + fhc.items = make(map[string]*fhcItem) +} + +// SetListener sets the listener for healthcheck updates. +func (fhc *fakeHealthCheck) SetListener(listener discovery.HealthCheckStatsListener) { +} + +// AddEndPoint adds the endpoint, and starts health check. +func (fhc *fakeHealthCheck) AddEndPoint(cell, name string, endPoint *topodatapb.EndPoint) { + key := discovery.EndPointToMapKey(endPoint) + item := &fhcItem{ + eps: &discovery.EndPointStats{ + EndPoint: endPoint, + Cell: cell, + Name: name, + }, + } + fhc.items[key] = item +} + +// RemoveEndPoint removes the endpoint, and stops the health check. +func (fhc *fakeHealthCheck) RemoveEndPoint(endPoint *topodatapb.EndPoint) { + key := discovery.EndPointToMapKey(endPoint) + delete(fhc.items, key) +} + +// GetEndPointStatsFromKeyspaceShard returns all EndPointStats for the given keyspace/shard. +func (fhc *fakeHealthCheck) GetEndPointStatsFromKeyspaceShard(keyspace, shard string) []*discovery.EndPointStats { + fhc.GetStatsFromKeyspaceShardCounter++ + var res []*discovery.EndPointStats + for _, item := range fhc.items { + if item.eps.Target == nil { + continue + } + if item.eps.Target.Keyspace == keyspace && item.eps.Target.Shard == shard { + res = append(res, item.eps) + } + } + return res +} + +// GetEndPointStatsFromTarget returns all EndPointStats for the given target. +func (fhc *fakeHealthCheck) GetEndPointStatsFromTarget(keyspace, shard string, tabletType topodatapb.TabletType) []*discovery.EndPointStats { + fhc.GetStatsFromTargetCounter++ + var res []*discovery.EndPointStats + for _, item := range fhc.items { + if item.eps.Target == nil { + continue + } + if item.eps.Target.Keyspace == keyspace && item.eps.Target.Shard == shard && item.eps.Target.TabletType == tabletType { + res = append(res, item.eps) + } + } + return res +} + +// GetConnection returns the TabletConn of the given endpoint. +func (fhc *fakeHealthCheck) GetConnection(endPoint *topodatapb.EndPoint) tabletconn.TabletConn { + key := discovery.EndPointToMapKey(endPoint) + if item := fhc.items[key]; item != nil { + return item.conn + } + return nil +} + +// CacheStatus returns a displayable version of the cache. +func (fhc *fakeHealthCheck) CacheStatus() discovery.EndPointsCacheStatusList { + return nil +} + +// Close stops the healthcheck. +func (fhc *fakeHealthCheck) Close() error { + return nil +} + +func (fhc *fakeHealthCheck) addTestEndPoint(cell, host string, port int32, keyspace, shard string, tabletType topodatapb.TabletType, serving bool, reparentTS int64, err error, conn tabletconn.TabletConn) *topodatapb.EndPoint { + if conn != nil { + conn.SetTarget(keyspace, shard, tabletType) + } + ep := topo.NewEndPoint(0, host) + ep.PortMap["vt"] = port + key := discovery.EndPointToMapKey(ep) + item := fhc.items[key] + if item == nil { + fhc.AddEndPoint(cell, "", ep) + item = fhc.items[key] + } + item.eps.Target = &querypb.Target{Keyspace: keyspace, Shard: shard, TabletType: tabletType} + item.eps.Serving = serving + item.eps.TabletExternallyReparentedTimestamp = reparentTS + item.eps.Stats = &querypb.RealtimeStats{} + item.eps.LastError = err + item.conn = conn + return ep +} diff --git a/go/vt/vtgate/gateway.go b/go/vt/vtgate/gateway.go index 93868b5f0f2..4682cee3a5b 100644 --- a/go/vt/vtgate/gateway.go +++ b/go/vt/vtgate/gateway.go @@ -6,6 +6,7 @@ package vtgate import ( "flag" + "fmt" "strings" "sync" "time" @@ -21,19 +22,17 @@ import ( querypb "github.com/youtube/vitess/go/vt/proto/query" topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" + vtrpcpb "github.com/youtube/vitess/go/vt/proto/vtrpc" ) var ( // GatewayImplementation controls the implementation of Gateway. - GatewayImplementation = flag.String("gateway_implementation", "shardgateway", "The implementation of gateway") + GatewayImplementation = flag.String("gateway_implementation", "discoverygateway", "The implementation of gateway") ) // A Gateway is the query processing module for each shard, // which is used by ScatterConn. type Gateway interface { - // InitializeConnections creates connections to VTTablets. - InitializeConnections(ctx context.Context) error - // Execute executes the non-streaming query for the specified keyspace, shard, and tablet type. Execute(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, query string, bindVars map[string]interface{}, transactionID int64) (*sqltypes.Result, error) @@ -86,7 +85,7 @@ type Gateway interface { } // GatewayCreator is the func which can create the actual gateway object. -type GatewayCreator func(hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, cell string, retryDelay time.Duration, retryCount int, connTimeoutTotal, connTimeoutPerConn, connLife time.Duration, connTimings *stats.MultiTimings, tabletTypesToWait []topodatapb.TabletType) Gateway +type GatewayCreator func(hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, cell string, retryCount int, tabletTypesToWait []topodatapb.TabletType) Gateway var gatewayCreators = make(map[string]GatewayCreator) @@ -117,6 +116,31 @@ func GetGatewayCreatorByName(name string) GatewayCreator { return gc } +// ShardError is the error about a specific shard. +// It implements vterrors.VtError. +type ShardError struct { + ShardIdentifier string + InTransaction bool + // Preserve the original error, so that we don't need to parse the error string. + Err error + // EndPointCode is the error code to use for all the endpoint errors in aggregate + EndPointCode vtrpcpb.ErrorCode +} + +// Error returns the error string. +func (e *ShardError) Error() string { + if e.ShardIdentifier == "" { + return fmt.Sprintf("%v", e.Err) + } + return fmt.Sprintf("shard, host: %s, %v", e.ShardIdentifier, e.Err) +} + +// VtErrorCode returns the underlying Vitess error code. +// This is part of vterrors.VtError interface. +func (e *ShardError) VtErrorCode() vtrpcpb.ErrorCode { + return e.EndPointCode +} + // GatewayEndPointCacheStatusList is a slice of GatewayEndPointCacheStatus. type GatewayEndPointCacheStatusList []*GatewayEndPointCacheStatus diff --git a/go/vt/vtgate/resolver.go b/go/vt/vtgate/resolver.go index 4c80f87c507..76303557e5f 100644 --- a/go/vt/vtgate/resolver.go +++ b/go/vt/vtgate/resolver.go @@ -12,11 +12,9 @@ import ( "reflect" "sort" "strings" - "time" "github.com/youtube/vitess/go/sqltypes" "github.com/youtube/vitess/go/vt/discovery" - "github.com/youtube/vitess/go/vt/tabletserver/tabletconn" "github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/vterrors" "golang.org/x/net/context" @@ -50,31 +48,23 @@ type Resolver struct { // NewResolver creates a new Resolver. All input parameters are passed through // for creating ScatterConn. -func NewResolver(hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, statsName, cell string, retryDelay time.Duration, retryCount int, connTimeoutTotal, connTimeoutPerConn, connLife time.Duration, tabletTypesToWait []topodatapb.TabletType, testGateway string) *Resolver { +func NewResolver(hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, statsName, cell string, retryCount int, tabletTypesToWait []topodatapb.TabletType) *Resolver { return &Resolver{ - scatterConn: NewScatterConn(hc, topoServer, serv, statsName, cell, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, tabletTypesToWait, testGateway), + scatterConn: NewScatterConn(hc, topoServer, serv, statsName, cell, retryCount, tabletTypesToWait), toposerv: serv, cell: cell, } } -// InitializeConnections pre-initializes VTGate by connecting to vttablets of all keyspace/shard/type. -// It is not necessary to call this function before serving queries, -// but it would reduce connection overhead when serving. -func (res *Resolver) InitializeConnections(ctx context.Context) error { - return res.scatterConn.InitializeConnections(ctx) -} - -// isConnError will be true if the error comes from the connection layer (ShardConn or -// ScatterConn). The error code from the conn error is also returned. -func isConnError(err error) (int, bool) { +// isRetryableError will be true if the error should be retried. +func isRetryableError(err error) bool { switch e := err.(type) { case *ScatterConnError: - return e.Code, true - case *ShardConnError: - return e.Code, true + return e.Retryable + case *ShardError: + return e.EndPointCode == vtrpcpb.ErrorCode_QUERY_NOT_SERVED default: - return 0, false + return false } } @@ -142,7 +132,7 @@ func (res *Resolver) Execute( tabletType, NewSafeSession(session), notInTransaction) - if connErrorCode, ok := isConnError(err); ok && connErrorCode == tabletconn.ERR_RETRY { + if isRetryableError(err) { resharding := false newKeyspace, newShards, err := mapToShards(keyspace) if err != nil { @@ -205,7 +195,7 @@ func (res *Resolver) ExecuteEntityIds( tabletType, NewSafeSession(session), notInTransaction) - if connErrorCode, ok := isConnError(err); ok && connErrorCode == tabletconn.ERR_RETRY { + if isRetryableError(err) { resharding := false newKeyspace, newShardIDMap, err := mapEntityIdsToShards( ctx, @@ -281,7 +271,7 @@ func (res *Resolver) ExecuteBatch( } // If lower level retries failed, check if there was a resharding event // and retry again if needed. - if connErrorCode, ok := isConnError(err); ok && connErrorCode == tabletconn.ERR_RETRY { + if isRetryableError(err) { newBatchRequest, buildErr := buildBatchRequest() if buildErr != nil { return nil, buildErr diff --git a/go/vt/vtgate/resolver_test.go b/go/vt/vtgate/resolver_test.go index 7403ff187fa..741aab68bd6 100644 --- a/go/vt/vtgate/resolver_test.go +++ b/go/vt/vtgate/resolver_test.go @@ -14,6 +14,7 @@ import ( "testing" "github.com/youtube/vitess/go/sqltypes" + "github.com/youtube/vitess/go/vt/discovery" "github.com/youtube/vitess/go/vt/tabletserver/tabletconn" "github.com/youtube/vitess/go/vt/topo" "golang.org/x/net/context" @@ -21,13 +22,14 @@ import ( querypb "github.com/youtube/vitess/go/vt/proto/query" topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" vtgatepb "github.com/youtube/vitess/go/vt/proto/vtgate" + vtrpcpb "github.com/youtube/vitess/go/vt/proto/vtrpc" ) // This file uses the sandbox_test framework. func TestResolverExecuteKeyspaceIds(t *testing.T) { - testResolverGeneric(t, "TestResolverExecuteKeyspaceIds", func() (*sqltypes.Result, error) { - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testResolverGeneric(t, "TestResolverExecuteKeyspaceIds", func(hc discovery.HealthCheck) (*sqltypes.Result, error) { + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) return res.ExecuteKeyspaceIds(context.Background(), "query", nil, @@ -40,8 +42,8 @@ func TestResolverExecuteKeyspaceIds(t *testing.T) { } func TestResolverExecuteKeyRanges(t *testing.T) { - testResolverGeneric(t, "TestResolverExecuteKeyRanges", func() (*sqltypes.Result, error) { - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testResolverGeneric(t, "TestResolverExecuteKeyRanges", func(hc discovery.HealthCheck) (*sqltypes.Result, error) { + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) return res.ExecuteKeyRanges(context.Background(), "query", nil, @@ -54,8 +56,8 @@ func TestResolverExecuteKeyRanges(t *testing.T) { } func TestResolverExecuteEntityIds(t *testing.T) { - testResolverGeneric(t, "TestResolverExecuteEntityIds", func() (*sqltypes.Result, error) { - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testResolverGeneric(t, "TestResolverExecuteEntityIds", func(hc discovery.HealthCheck) (*sqltypes.Result, error) { + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) return res.ExecuteEntityIds(context.Background(), "query", nil, @@ -80,8 +82,8 @@ func TestResolverExecuteEntityIds(t *testing.T) { } func TestResolverExecuteBatchKeyspaceIds(t *testing.T) { - testResolverGeneric(t, "TestResolverExecuteBatchKeyspaceIds", func() (*sqltypes.Result, error) { - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testResolverGeneric(t, "TestResolverExecuteBatchKeyspaceIds", func(hc discovery.HealthCheck) (*sqltypes.Result, error) { + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) qrs, err := res.ExecuteBatchKeyspaceIds(context.Background(), []*vtgatepb.BoundKeyspaceIdQuery{{ Query: &querypb.BoundQuery{ @@ -105,14 +107,14 @@ func TestResolverExecuteBatchKeyspaceIds(t *testing.T) { } func TestResolverStreamExecuteKeyspaceIds(t *testing.T) { - createSandbox("TestResolverStreamExecuteKeyspaceIds") - testResolverStreamGeneric(t, "TestResolverStreamExecuteKeyspaceIds", func() (*sqltypes.Result, error) { - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + keyspace := "TestResolverStreamExecuteKeyspaceIds" + testResolverStreamGeneric(t, keyspace, func(hc discovery.HealthCheck) (*sqltypes.Result, error) { + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) qr := new(sqltypes.Result) err := res.StreamExecuteKeyspaceIds(context.Background(), "query", nil, - "TestResolverStreamExecuteKeyspaceIds", + keyspace, [][]byte{{0x10}, {0x15}}, topodatapb.TabletType_MASTER, func(r *sqltypes.Result) error { @@ -121,13 +123,13 @@ func TestResolverStreamExecuteKeyspaceIds(t *testing.T) { }) return qr, err }) - testResolverStreamGeneric(t, "TestResolverStreamExecuteKeyspaceIds", func() (*sqltypes.Result, error) { - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testResolverStreamGeneric(t, keyspace, func(hc discovery.HealthCheck) (*sqltypes.Result, error) { + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) qr := new(sqltypes.Result) err := res.StreamExecuteKeyspaceIds(context.Background(), "query", nil, - "TestResolverStreamExecuteKeyspaceIds", + keyspace, [][]byte{{0x10}, {0x15}, {0x25}}, topodatapb.TabletType_MASTER, func(r *sqltypes.Result) error { @@ -139,15 +141,15 @@ func TestResolverStreamExecuteKeyspaceIds(t *testing.T) { } func TestResolverStreamExecuteKeyRanges(t *testing.T) { - createSandbox("TestResolverStreamExecuteKeyRanges") + keyspace := "TestResolverStreamExecuteKeyRanges" // streaming a single shard - testResolverStreamGeneric(t, "TestResolverStreamExecuteKeyRanges", func() (*sqltypes.Result, error) { - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testResolverStreamGeneric(t, keyspace, func(hc discovery.HealthCheck) (*sqltypes.Result, error) { + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) qr := new(sqltypes.Result) err := res.StreamExecuteKeyRanges(context.Background(), "query", nil, - "TestResolverStreamExecuteKeyRanges", + keyspace, []*topodatapb.KeyRange{{Start: []byte{0x10}, End: []byte{0x15}}}, topodatapb.TabletType_MASTER, func(r *sqltypes.Result) error { @@ -157,13 +159,13 @@ func TestResolverStreamExecuteKeyRanges(t *testing.T) { return qr, err }) // streaming multiple shards - testResolverStreamGeneric(t, "TestResolverStreamExecuteKeyRanges", func() (*sqltypes.Result, error) { - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testResolverStreamGeneric(t, keyspace, func(hc discovery.HealthCheck) (*sqltypes.Result, error) { + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) qr := new(sqltypes.Result) err := res.StreamExecuteKeyRanges(context.Background(), "query", nil, - "TestResolverStreamExecuteKeyRanges", + keyspace, []*topodatapb.KeyRange{{Start: []byte{0x10}, End: []byte{0x25}}}, topodatapb.TabletType_MASTER, func(r *sqltypes.Result) error { @@ -174,14 +176,16 @@ func TestResolverStreamExecuteKeyRanges(t *testing.T) { }) } -func testResolverGeneric(t *testing.T, name string, action func() (*sqltypes.Result, error)) { +func testResolverGeneric(t *testing.T, name string, action func(hc discovery.HealthCheck) (*sqltypes.Result, error)) { // successful execute s := createSandbox(name) sbc0 := &sandboxConn{} - s.MapTestConn("-20", sbc0) sbc1 := &sandboxConn{} - s.MapTestConn("20-40", sbc1) - _, err := action() + hc := newFakeHealthCheck() + hc.addTestEndPoint("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1.1.1.1", 1002, name, "20-40", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + + _, err := action(hc) if err != nil { t.Errorf("want nil, got %v", err) } @@ -195,10 +199,11 @@ func testResolverGeneric(t *testing.T, name string, action func() (*sqltypes.Res // non-retryable failure s.Reset() sbc0 = &sandboxConn{mustFailServer: 1} - s.MapTestConn("-20", sbc0) sbc1 = &sandboxConn{mustFailRetry: 1} - s.MapTestConn("20-40", sbc1) - _, err = action() + hc.Reset() + hc.addTestEndPoint("aa", "-20", 1, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "20-40", 1, name, "20-40", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + _, err = action(hc) want1 := fmt.Sprintf("shard, host: %s.-20.master, host:\"-20\" port_map: , error: err", name) want2 := fmt.Sprintf("shard, host: %s.20-40.master, host:\"20-40\" port_map: , retry: err", name) want := []string{want1, want2} @@ -227,10 +232,11 @@ func testResolverGeneric(t *testing.T, name string, action func() (*sqltypes.Res // retryable failure, no sharding event s.Reset() sbc0 = &sandboxConn{mustFailRetry: 1} - s.MapTestConn("-20", sbc0) sbc1 = &sandboxConn{mustFailFatal: 1} - s.MapTestConn("20-40", sbc1) - _, err = action() + hc.Reset() + hc.addTestEndPoint("aa", "-20", 1, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "20-40", 1, name, "20-40", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + _, err = action(hc) want1 = fmt.Sprintf("shard, host: %s.-20.master, host:\"-20\" port_map: , retry: err", name) want2 = fmt.Sprintf("shard, host: %s.20-40.master, host:\"20-40\" port_map: , fatal: err", name) want = []string{want1, want2} @@ -260,14 +266,15 @@ func testResolverGeneric(t *testing.T, name string, action func() (*sqltypes.Res s.Reset() addSandboxServedFrom(name, name+"ServedFrom0") sbc0 = &sandboxConn{} - s.MapTestConn("-20", sbc0) sbc1 = &sandboxConn{} - s.MapTestConn("20-40", sbc1) + hc.Reset() + hc.addTestEndPoint("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1.1.1.1", 1002, name, "20-40", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) s0 := createSandbox(name + "ServedFrom0") // make sure we have a fresh copy s0.ShardSpec = "-80-" sbc2 := &sandboxConn{} - s0.MapTestConn("-80", sbc2) - _, err = action() + hc.addTestEndPoint("aa", "1.1.1.1", 1003, name+"ServedFrom0", "-80", topodatapb.TabletType_MASTER, true, 1, nil, sbc2) + _, err = action(hc) if err != nil { t.Errorf("want nil, got %v", err) } @@ -294,17 +301,21 @@ func testResolverGeneric(t *testing.T, name string, action func() (*sqltypes.Res // retryable failure, vertical resharding s.Reset() sbc0 = &sandboxConn{} - s.MapTestConn("-20", sbc0) sbc1 = &sandboxConn{mustFailFatal: 1} - s.MapTestConn("20-40", sbc1) + hc.Reset() + hc.addTestEndPoint("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1.1.1.1", 1002, name, "20-40", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) i := 0 s.SrvKeyspaceCallback = func() { if i == 1 { addSandboxServedFrom(name, name+"ServedFrom") + hc.Reset() + hc.addTestEndPoint("aa", "1.1.1.1", 1001, name+"ServedFrom", "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1.1.1.1", 1002, name+"ServedFrom", "20-40", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) } i++ } - _, err = action() + _, err = action(hc) if err != nil { t.Errorf("want nil, got %v", err) } @@ -323,19 +334,21 @@ func testResolverGeneric(t *testing.T, name string, action func() (*sqltypes.Res // retryable failure, horizontal resharding s.Reset() sbc0 = &sandboxConn{} - s.MapTestConn("-20", sbc0) sbc1 = &sandboxConn{mustFailRetry: 1} - s.MapTestConn("20-40", sbc1) + hc.Reset() + hc.addTestEndPoint("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1.1.1.1", 1002, name, "20-40", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) i = 0 s.SrvKeyspaceCallback = func() { if i == 1 { s.ShardSpec = "-20-30-40-60-80-a0-c0-e0-" - s.MapTestConn("-20", sbc0) - s.MapTestConn("20-30", sbc1) + hc.Reset() + hc.addTestEndPoint("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1.1.1.1", 1002, name, "20-30", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) } i++ } - _, err = action() + _, err = action(hc) if err != nil { t.Errorf("want nil, got %v", err) } @@ -352,14 +365,15 @@ func testResolverGeneric(t *testing.T, name string, action func() (*sqltypes.Res } } -func testResolverStreamGeneric(t *testing.T, name string, action func() (*sqltypes.Result, error)) { +func testResolverStreamGeneric(t *testing.T, name string, action func(hc discovery.HealthCheck) (*sqltypes.Result, error)) { // successful execute s := createSandbox(name) sbc0 := &sandboxConn{} - s.MapTestConn("-20", sbc0) sbc1 := &sandboxConn{} - s.MapTestConn("20-40", sbc1) - _, err := action() + hc := newFakeHealthCheck() + hc.addTestEndPoint("aa", "1.1.1.1", 1001, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1.1.1.1", 1002, name, "20-40", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + _, err := action(hc) if err != nil { t.Errorf("want nil, got %v", err) } @@ -370,10 +384,11 @@ func testResolverStreamGeneric(t *testing.T, name string, action func() (*sqltyp // failure s.Reset() sbc0 = &sandboxConn{mustFailRetry: 1} - s.MapTestConn("-20", sbc0) sbc1 = &sandboxConn{} - s.MapTestConn("20-40", sbc1) - _, err = action() + hc.Reset() + hc.addTestEndPoint("aa", "-20", 1, name, "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "20-40", 1, name, "20-40", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + _, err = action(hc) want := fmt.Sprintf("shard, host: %s.-20.master, host:\"-20\" port_map: , retry: err", name) if err == nil || err.Error() != want { t.Errorf("want\n%s\ngot\n%v", want, err) @@ -447,19 +462,20 @@ func TestResolverBuildEntityIds(t *testing.T) { } func TestResolverDmlOnMultipleKeyspaceIds(t *testing.T) { - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") - - s := createSandbox("TestResolverDmlOnMultipleKeyspaceIds") + keyspace := "TestResolverDmlOnMultipleKeyspaceIds" + createSandbox(keyspace) sbc0 := &sandboxConn{} - s.MapTestConn("-20", sbc0) sbc1 := &sandboxConn{} - s.MapTestConn("20-40", sbc1) + hc := newFakeHealthCheck() + hc.addTestEndPoint("aa", "1.1.1.1", 1001, keyspace, "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1.1.1.1", 1002, keyspace, "20-40", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) errStr := "DML should not span multiple keyspace_ids" _, err := res.ExecuteKeyspaceIds(context.Background(), "update table set a = b", nil, - "TestResolverExecuteKeyspaceIds", + keyspace, [][]byte{{0x10}, {0x25}}, topodatapb.TabletType_MASTER, nil, @@ -470,11 +486,13 @@ func TestResolverDmlOnMultipleKeyspaceIds(t *testing.T) { } func TestResolverExecBatchReresolve(t *testing.T) { - s := createSandbox("TestResolverExecBatchReresolve") + keyspace := "TestResolverExecBatchReresolve" + createSandbox(keyspace) sbc := &sandboxConn{mustFailRetry: 20} - s.MapTestConn("0", sbc) + hc := newFakeHealthCheck() + hc.addTestEndPoint("aa", "0", 1, keyspace, "0", topodatapb.TabletType_MASTER, true, 1, nil, sbc) - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) callcount := 0 buildBatchRequest := func() (*scatterBatchRequest, error) { @@ -484,7 +502,7 @@ func TestResolverExecBatchReresolve(t *testing.T) { Sql: "query", BindVariables: nil, }, - Keyspace: "TestResolverExecBatchReresolve", + Keyspace: keyspace, Shards: []string{"0"}, }} return boundShardQueriesToScatterBatchRequest(queries) @@ -505,11 +523,13 @@ func TestResolverExecBatchReresolve(t *testing.T) { } func TestResolverExecBatchAsTransaction(t *testing.T) { - s := createSandbox("TestResolverExecBatchAsTransaction") + keyspace := "TestResolverExecBatchAsTransaction" + createSandbox(keyspace) sbc := &sandboxConn{mustFailRetry: 20} - s.MapTestConn("0", sbc) + hc := newFakeHealthCheck() + hc.addTestEndPoint("aa", "0", 1, keyspace, "0", topodatapb.TabletType_MASTER, true, 1, nil, sbc) - res := NewResolver(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, 0, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + res := NewResolver(hc, topo.Server{}, new(sandboxTopo), "", "aa", 0, nil) callcount := 0 buildBatchRequest := func() (*scatterBatchRequest, error) { @@ -519,7 +539,7 @@ func TestResolverExecBatchAsTransaction(t *testing.T) { Sql: "query", BindVariables: nil, }, - Keyspace: "TestResolverExecBatchAsTransaction", + Keyspace: keyspace, Shards: []string{"0"}, }} return boundShardQueriesToScatterBatchRequest(queries) @@ -528,7 +548,7 @@ func TestResolverExecBatchAsTransaction(t *testing.T) { _, err := res.ExecuteBatch(context.Background(), topodatapb.TabletType_MASTER, true, nil, buildBatchRequest) want := "shard, host: TestResolverExecBatchAsTransaction.0.master, host:\"0\" port_map: , retry: err" if err == nil || err.Error() != want { - t.Errorf("want got, got none") + t.Errorf("want %v, got %v", want, err) } // Ensure scatter did not re-resolve if callcount != 1 { @@ -540,23 +560,28 @@ func TestResolverExecBatchAsTransaction(t *testing.T) { } } -func TestIsConnError(t *testing.T) { +func TestIsRetryableError(t *testing.T) { var connErrorTests = []struct { in error - outCode int outBool bool }{ - {fmt.Errorf("generic error"), 0, false}, - {&ScatterConnError{Code: 9}, 9, true}, - {&ShardConnError{Code: 9}, 9, true}, - {&tabletconn.ServerError{Code: 9}, 0, false}, + {fmt.Errorf("generic error"), false}, + {&ScatterConnError{Retryable: true}, true}, + {&ScatterConnError{Retryable: false}, false}, + {&ShardError{EndPointCode: vtrpcpb.ErrorCode_QUERY_NOT_SERVED}, true}, + {&ShardError{EndPointCode: vtrpcpb.ErrorCode_INTERNAL_ERROR}, false}, + // tabletconn.ServerError will not come directly here, + // they'll be wrapped in ScatterConnError or ShardConnError. + // So they can't be retried as is. + {&tabletconn.ServerError{ServerCode: vtrpcpb.ErrorCode_QUERY_NOT_SERVED}, false}, + {&tabletconn.ServerError{ServerCode: vtrpcpb.ErrorCode_PERMISSION_DENIED}, false}, } for _, tt := range connErrorTests { - gotCode, gotBool := isConnError(tt.in) - if (gotCode != tt.outCode) || (gotBool != tt.outBool) { - t.Errorf("isConnError(%v) => (%v, %v), want (%v, %v)", - tt.in, gotCode, gotBool, tt.outCode, tt.outBool) + gotBool := isRetryableError(tt.in) + if gotBool != tt.outBool { + t.Errorf("isConnError(%v) => %v, want %v", + tt.in, gotBool, tt.outBool) } } } diff --git a/go/vt/vtgate/router_framework_test.go b/go/vt/vtgate/router_framework_test.go index 4c344050618..ecf27a3237b 100644 --- a/go/vt/vtgate/router_framework_test.go +++ b/go/vt/vtgate/router_framework_test.go @@ -5,8 +5,6 @@ package vtgate import ( - "time" - "github.com/youtube/vitess/go/sqltypes" "github.com/youtube/vitess/go/vt/topo" "golang.org/x/net/context" @@ -157,16 +155,18 @@ var unshardedVSchema = ` ` func createRouterEnv() (router *Router, sbc1, sbc2, sbclookup *sandboxConn) { + cell := "aa" + hc := newFakeHealthCheck() s := createSandbox("TestRouter") s.VSchema = routerVSchema sbc1 = &sandboxConn{} sbc2 = &sandboxConn{} - s.MapTestConn("-20", sbc1) - s.MapTestConn("40-60", sbc2) + hc.addTestEndPoint(cell, "-20", 1, "TestRouter", "-20", topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + hc.addTestEndPoint(cell, "40-60", 1, "TestRouter", "40-60", topodatapb.TabletType_MASTER, true, 1, nil, sbc2) - l := createSandbox(KsTestUnsharded) + createSandbox(KsTestUnsharded) sbclookup = &sandboxConn{} - l.MapTestConn("0", sbclookup) + hc.addTestEndPoint(cell, "0", 1, KsTestUnsharded, "0", topodatapb.TabletType_MASTER, true, 1, nil, sbclookup) bad := createSandbox("TestBadSharding") bad.VSchema = badVSchema @@ -174,8 +174,8 @@ func createRouterEnv() (router *Router, sbc1, sbc2, sbclookup *sandboxConn) { getSandbox(KsTestUnsharded).VSchema = unshardedVSchema serv := new(sandboxTopo) - scatterConn := NewScatterConn(nil, topo.Server{}, serv, "", "aa", 1*time.Second, 10, 20*time.Millisecond, 10*time.Millisecond, 24*time.Hour, nil, "") - router = NewRouter(context.Background(), serv, "aa", "", scatterConn) + scatterConn := NewScatterConn(hc, topo.Server{}, serv, "", cell, 10, nil) + router = NewRouter(context.Background(), serv, cell, "", scatterConn) return router, sbc1, sbc2, sbclookup } diff --git a/go/vt/vtgate/router_select_test.go b/go/vt/vtgate/router_select_test.go index e654a72702d..632ee0b6cfe 100644 --- a/go/vt/vtgate/router_select_test.go +++ b/go/vt/vtgate/router_select_test.go @@ -9,15 +9,16 @@ import ( "reflect" "strings" "testing" - "time" "golang.org/x/net/context" "github.com/youtube/vitess/go/sqltypes" - querypb "github.com/youtube/vitess/go/vt/proto/query" "github.com/youtube/vitess/go/vt/tabletserver/querytypes" "github.com/youtube/vitess/go/vt/topo" _ "github.com/youtube/vitess/go/vt/vtgate/vindexes" + + querypb "github.com/youtube/vitess/go/vt/proto/query" + topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" ) func TestUnsharded(t *testing.T) { @@ -497,6 +498,8 @@ func TestSelectINFail(t *testing.T) { func TestSelectScatter(t *testing.T) { // Special setup: Don't use createRouterEnv. + cell := "aa" + hc := newFakeHealthCheck() s := createSandbox("TestRouter") s.VSchema = routerVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -505,11 +508,11 @@ func TestSelectScatter(t *testing.T) { for _, shard := range shards { sbc := &sandboxConn{} conns = append(conns, sbc) - s.MapTestConn(shard, sbc) + hc.addTestEndPoint(cell, shard, 1, "TestRouter", shard, topodatapb.TabletType_MASTER, true, 1, nil, sbc) } serv := new(sandboxTopo) - scatterConn := NewScatterConn(nil, topo.Server{}, serv, "", "aa", 1*time.Second, 10, 2*time.Millisecond, 1*time.Millisecond, 24*time.Hour, nil, "") - router := NewRouter(context.Background(), serv, "aa", "", scatterConn) + scatterConn := NewScatterConn(hc, topo.Server{}, serv, "", cell, 10, nil) + router := NewRouter(context.Background(), serv, cell, "", scatterConn) _, err := routerExec(router, "select id from user", nil) if err != nil { @@ -528,6 +531,8 @@ func TestSelectScatter(t *testing.T) { func TestStreamSelectScatter(t *testing.T) { // Special setup: Don't use createRouterEnv. + cell := "aa" + hc := newFakeHealthCheck() s := createSandbox("TestRouter") s.VSchema = routerVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -536,11 +541,11 @@ func TestStreamSelectScatter(t *testing.T) { for _, shard := range shards { sbc := &sandboxConn{} conns = append(conns, sbc) - s.MapTestConn(shard, sbc) + hc.addTestEndPoint(cell, shard, 1, "TestRouter", shard, topodatapb.TabletType_MASTER, true, 1, nil, sbc) } serv := new(sandboxTopo) - scatterConn := NewScatterConn(nil, topo.Server{}, serv, "", "aa", 1*time.Second, 10, 2*time.Millisecond, 1*time.Millisecond, 24*time.Hour, nil, "") - router := NewRouter(context.Background(), serv, "aa", "", scatterConn) + scatterConn := NewScatterConn(hc, topo.Server{}, serv, "", cell, 10, nil) + router := NewRouter(context.Background(), serv, cell, "", scatterConn) sql := "select id from user" result, err := routerStream(router, sql) @@ -568,6 +573,8 @@ func TestStreamSelectScatter(t *testing.T) { func TestSelectScatterFail(t *testing.T) { // Special setup: Don't use createRouterEnv. + cell := "aa" + hc := newFakeHealthCheck() s := createSandbox("TestRouter") s.VSchema = routerVSchema getSandbox(KsTestUnsharded).VSchema = unshardedVSchema @@ -577,11 +584,11 @@ func TestSelectScatterFail(t *testing.T) { for _, shard := range shards { sbc := &sandboxConn{} conns = append(conns, sbc) - s.MapTestConn(shard, sbc) + hc.addTestEndPoint(cell, shard, 1, "TestRouter", shard, topodatapb.TabletType_MASTER, true, 1, nil, sbc) } serv := new(sandboxTopo) - scatterConn := NewScatterConn(nil, topo.Server{}, serv, "", "aa", 1*time.Second, 10, 2*time.Millisecond, 1*time.Millisecond, 24*time.Hour, nil, "") - router := NewRouter(context.Background(), serv, "aa", "", scatterConn) + scatterConn := NewScatterConn(hc, topo.Server{}, serv, "", cell, 10, nil) + router := NewRouter(context.Background(), serv, cell, "", scatterConn) _, err := routerExec(router, "select id from user", nil) want := "paramsSelectScatter: keyspace TestRouter fetch error: topo error GetSrvKeyspace" diff --git a/go/vt/vtgate/sandbox_test.go b/go/vt/vtgate/sandbox_test.go index 3d8d557d7ae..6370c2370dd 100644 --- a/go/vt/vtgate/sandbox_test.go +++ b/go/vt/vtgate/sandbox_test.go @@ -74,12 +74,6 @@ type sandbox struct { // SrvKeyspaceMustFail specifies how often GetSrvKeyspace must fail before succeeding SrvKeyspaceMustFail int - // EndPointCounter tracks how often GetEndPoints was called - EndPointCounter sync2.AtomicInt64 - - // EndPointMustFail specifies how often GetEndPoints must fail before succeeding - EndPointMustFail int - // DialCounter tracks how often sandboxDialer was called DialCounter int @@ -98,18 +92,14 @@ type sandbox struct { // SrvKeyspaceCallback specifies the callback function in GetSrvKeyspace SrvKeyspaceCallback func() - TestConns map[string]map[uint32]tabletconn.TabletConn - VSchema string + VSchema string } func (s *sandbox) Reset() { s.sandmu.Lock() defer s.sandmu.Unlock() - s.TestConns = make(map[string]map[uint32]tabletconn.TabletConn) s.SrvKeyspaceCounter = 0 s.SrvKeyspaceMustFail = 0 - s.EndPointCounter.Set(0) - s.EndPointMustFail = 0 s.DialCounter = 0 s.DialMustFail = 0 s.DialMustTimeout = 0 @@ -118,43 +108,6 @@ func (s *sandbox) Reset() { s.SrvKeyspaceCallback = nil } -// a sandboxableConn is a tablet.TabletConn that allows you -// to set the endPoint. MapTestConn uses it to set some good -// defaults. This way, you have the option of calling MapTestConn -// with variables other than sandboxConn. -type sandboxableConn interface { - tabletconn.TabletConn - setEndPoint(*topodatapb.EndPoint) -} - -func (s *sandbox) MapTestConn(shard string, conn sandboxableConn) { - s.sandmu.Lock() - defer s.sandmu.Unlock() - conns, ok := s.TestConns[shard] - if !ok { - conns = make(map[uint32]tabletconn.TabletConn) - } - uid := uint32(len(conns)) - conn.setEndPoint(&topodatapb.EndPoint{ - Uid: uid, - Host: shard, - PortMap: map[string]int32{"vt": 1}, - }) - conns[uid] = conn - s.TestConns[shard] = conns -} - -func (s *sandbox) DeleteTestConn(shard string, conn tabletconn.TabletConn) { - s.sandmu.Lock() - defer s.sandmu.Unlock() - conns, ok := s.TestConns[shard] - if !ok { - panic(fmt.Sprintf("unknown shard: %v", shard)) - } - delete(conns, conn.EndPoint().Uid) - s.TestConns[shard] = conns -} - var DefaultShardSpec = "-20-40-60-80-a0-c0-e0-" func getAllShards(shardSpec string) ([]*topodatapb.KeyRange, error) { @@ -237,7 +190,6 @@ func createUnshardedKeyspace() (*topodatapb.SrvKeyspace, error) { // sandboxTopo satisfies the SrvTopoServer interface type sandboxTopo struct { - callbackGetEndPoints func(st *sandboxTopo) } func (sct *sandboxTopo) GetSrvKeyspaceNames(ctx context.Context, cell string) ([]string, error) { @@ -296,22 +248,7 @@ func (sct *sandboxTopo) GetSrvShard(ctx context.Context, cell, keyspace, shard s } func (sct *sandboxTopo) GetEndPoints(ctx context.Context, cell, keyspace, shard string, tabletType topodatapb.TabletType) (*topodatapb.EndPoints, int64, error) { - sand := getSandbox(keyspace) - sand.EndPointCounter.Add(1) - if sct.callbackGetEndPoints != nil { - sct.callbackGetEndPoints(sct) - } - if sand.EndPointMustFail > 0 { - sand.EndPointMustFail-- - return nil, -1, fmt.Errorf("topo error") - } - - conns := sand.TestConns[shard] - ep := &topodatapb.EndPoints{} - for _, conn := range conns { - ep.Entries = append(ep.Entries, conn.EndPoint()) - } - return ep, -1, nil + return nil, -1, fmt.Errorf("Unsupported") } func sandboxDialer(ctx context.Context, endPoint *topodatapb.EndPoint, keyspace, shard string, tabletType topodatapb.TabletType, timeout time.Duration) (tabletconn.TabletConn, error) { @@ -328,17 +265,19 @@ func sandboxDialer(ctx context.Context, endPoint *topodatapb.EndPoint, keyspace, sand.DialMustTimeout-- return nil, tabletconn.OperationalError(fmt.Sprintf("conn unreachable")) } - conns := sand.TestConns[shard] - if conns == nil { - panic(fmt.Sprintf("can't find shard %v", shard)) - } - tconn := conns[endPoint.Uid] - return tconn, nil + sbc := &sandboxConn{} + sbc.endPoint = endPoint + sbc.SetTarget(keyspace, shard, tabletType) + return sbc, nil } // sandboxConn satisfies the TabletConn interface type sandboxConn struct { - endPoint *topodatapb.EndPoint + keyspace string + shard string + tabletType topodatapb.TabletType + endPoint *topodatapb.EndPoint + mustFailRetry int mustFailFatal int mustFailServer int @@ -346,16 +285,12 @@ type sandboxConn struct { mustFailTxPool int mustFailNotTx int - // A callback to tweak the behavior on each conn call - onConnUse func(*sandboxConn) - // These Count vars report how often the corresponding // functions were called. ExecCount sync2.AtomicInt64 BeginCount sync2.AtomicInt64 CommitCount sync2.AtomicInt64 RollbackCount sync2.AtomicInt64 - CloseCount sync2.AtomicInt64 AsTransactionCount sync2.AtomicInt64 // Queries stores the non-batch requests received. @@ -375,9 +310,6 @@ type sandboxConn struct { } func (sbc *sandboxConn) getError() error { - if sbc.onConnUse != nil { - sbc.onConnUse(sbc) - } if sbc.mustFailRetry > 0 { sbc.mustFailRetry-- return &tabletconn.ServerError{ @@ -555,14 +487,12 @@ func (sbc *sandboxConn) SplitQueryV2( splitCount int64, numRowsPerQueryPart int64, algorithm querypb.SplitQueryRequest_Algorithm) ([]querytypes.QuerySplit, error) { - // The sandbox stores the shard name in the endPoint Host field. - shard := sbc.endPoint.Host splits := []querytypes.QuerySplit{ { Sql: fmt.Sprintf( "query:%v, splitColumns:%v, splitCount:%v,"+ " numRowsPerQueryPart:%v, algorithm:%v, shard:%v", - query, splitColumns, splitCount, numRowsPerQueryPart, algorithm, shard), + query, splitColumns, splitCount, numRowsPerQueryPart, algorithm, sbc.shard), }, } return splits, nil @@ -575,21 +505,19 @@ func (sbc *sandboxConn) StreamHealth(ctx context.Context) (tabletconn.StreamHeal // Close does not change ExecCount func (sbc *sandboxConn) Close() { - sbc.CloseCount.Add(1) } func (sbc *sandboxConn) SetTarget(keyspace, shard string, tabletType topodatapb.TabletType) error { - return fmt.Errorf("not implemented, vtgate doesn't use target yet") + sbc.keyspace = keyspace + sbc.shard = shard + sbc.tabletType = tabletType + return nil } func (sbc *sandboxConn) EndPoint() *topodatapb.EndPoint { return sbc.endPoint } -func (sbc *sandboxConn) setEndPoint(ep *topodatapb.EndPoint) { - sbc.endPoint = ep -} - func (sbc *sandboxConn) getNextResult() *sqltypes.Result { if len(sbc.results) != 0 { r := sbc.results[0] diff --git a/go/vt/vtgate/scatter_conn.go b/go/vt/vtgate/scatter_conn.go index 8c1694a73ba..94ef0eb7d08 100644 --- a/go/vt/vtgate/scatter_conn.go +++ b/go/vt/vtgate/scatter_conn.go @@ -19,7 +19,6 @@ import ( "github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/discovery" "github.com/youtube/vitess/go/vt/tabletserver/querytypes" - "github.com/youtube/vitess/go/vt/tabletserver/tabletconn" "github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/vterrors" @@ -57,42 +56,18 @@ type shardActionTransactionFunc func(shard string, shouldBegin bool, transaction // NewScatterConn creates a new ScatterConn. All input parameters are passed through // for creating the appropriate connections. -func NewScatterConn(hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, statsName, cell string, retryDelay time.Duration, retryCount int, connTimeoutTotal, connTimeoutPerConn, connLife time.Duration, tabletTypesToWait []topodatapb.TabletType, testGateway string) *ScatterConn { +func NewScatterConn(hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, statsName, cell string, retryCount int, tabletTypesToWait []topodatapb.TabletType) *ScatterConn { tabletCallErrorCountStatsName := "" - tabletConnectStatsName := "" if statsName != "" { tabletCallErrorCountStatsName = statsName + "ErrorCount" - tabletConnectStatsName = statsName + "TabletConnect" } - connTimings := stats.NewMultiTimings(tabletConnectStatsName, []string{"Keyspace", "ShardName", "DbType"}) - gateway := GetGatewayCreator()(hc, topoServer, serv, cell, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connTimings, tabletTypesToWait) + gateway := GetGatewayCreator()(hc, topoServer, serv, cell, retryCount, tabletTypesToWait) - sc := &ScatterConn{ + return &ScatterConn{ timings: stats.NewMultiTimings(statsName, []string{"Operation", "Keyspace", "ShardName", "DbType"}), tabletCallErrorCount: stats.NewMultiCounters(tabletCallErrorCountStatsName, []string{"Operation", "Keyspace", "ShardName", "DbType"}), gateway: gateway, } - - // this is to test health checking module when using existing gateway - if testGateway != "" { - if gc := GetGatewayCreatorByName(testGateway); gc != nil { - sc.testGateway = gc(hc, topoServer, serv, cell, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connTimings, nil) - } - } - - return sc -} - -// InitializeConnections pre-initializes connections for all shards. -// It also populates topology cache by accessing it. -// It is not necessary to call this function before serving queries, -// but it would reduce connection overhead when serving. -func (stc *ScatterConn) InitializeConnections(ctx context.Context) error { - // temporarily start healthchecking regardless of gateway used - if stc.testGateway != nil { - stc.testGateway.InitializeConnections(ctx) - } - return stc.gateway.InitializeConnections(ctx) } func (stc *ScatterConn) startAction(name, keyspace, shard string, tabletType topodatapb.TabletType) (time.Time, []string) { @@ -762,7 +737,7 @@ func (stc *ScatterConn) GetGatewayCacheStatus() GatewayEndPointCacheStatusList { // ScatterConnError is the ScatterConn specific error. type ScatterConnError struct { - Code int + Retryable bool // Preserve the original errors, so that we don't need to parse the error string. Errs []error // serverCode is the error code to use for all the server errors in aggregate @@ -782,21 +757,14 @@ func (stc *ScatterConn) aggregateErrors(errors []error) error { } allRetryableError := true for _, e := range errors { - connError, ok := e.(*ShardConnError) - if !ok || (connError.Code != tabletconn.ERR_RETRY && connError.Code != tabletconn.ERR_FATAL) || connError.InTransaction { + connError, ok := e.(*ShardError) + if !ok || (connError.EndPointCode != vtrpcpb.ErrorCode_QUERY_NOT_SERVED && connError.EndPointCode != vtrpcpb.ErrorCode_INTERNAL_ERROR) || connError.InTransaction { allRetryableError = false break } } - var code int - if allRetryableError { - code = tabletconn.ERR_RETRY - } else { - code = tabletconn.ERR_NORMAL - } - return &ScatterConnError{ - Code: code, + Retryable: allRetryableError, Errs: errors, serverCode: aggregateVtGateErrorCodes(errors), } diff --git a/go/vt/vtgate/scatter_conn_test.go b/go/vt/vtgate/scatter_conn_test.go index 94f03324788..9369ed0bdd9 100644 --- a/go/vt/vtgate/scatter_conn_test.go +++ b/go/vt/vtgate/scatter_conn_test.go @@ -8,9 +8,9 @@ import ( "fmt" "reflect" "testing" - "time" "github.com/youtube/vitess/go/sqltypes" + "github.com/youtube/vitess/go/vt/discovery" "github.com/youtube/vitess/go/vt/tabletserver/tabletconn" "github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/vterrors" @@ -24,16 +24,20 @@ import ( // This file uses the sandbox_test framework. +var ( + retryCount = 3 +) + func TestScatterConnExecute(t *testing.T) { - testScatterConnGeneric(t, "TestScatterConnExecute", func(shards []string) (*sqltypes.Result, error) { - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testScatterConnGeneric(t, "TestScatterConnExecute", func(hc discovery.HealthCheck, shards []string) (*sqltypes.Result, error) { + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) return stc.Execute(context.Background(), "query", nil, "TestScatterConnExecute", shards, topodatapb.TabletType_REPLICA, nil, false) }) } func TestScatterConnExecuteMulti(t *testing.T) { - testScatterConnGeneric(t, "TestScatterConnExecuteMulti", func(shards []string) (*sqltypes.Result, error) { - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testScatterConnGeneric(t, "TestScatterConnExecuteMulti", func(hc discovery.HealthCheck, shards []string) (*sqltypes.Result, error) { + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) shardVars := make(map[string]map[string]interface{}) for _, shard := range shards { shardVars[shard] = nil @@ -43,8 +47,8 @@ func TestScatterConnExecuteMulti(t *testing.T) { } func TestScatterConnExecuteBatch(t *testing.T) { - testScatterConnGeneric(t, "TestScatterConnExecuteBatch", func(shards []string) (*sqltypes.Result, error) { - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testScatterConnGeneric(t, "TestScatterConnExecuteBatch", func(hc discovery.HealthCheck, shards []string) (*sqltypes.Result, error) { + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) queries := []*vtgatepb.BoundShardQuery{{ Query: &querypb.BoundQuery{ Sql: "query", @@ -66,8 +70,8 @@ func TestScatterConnExecuteBatch(t *testing.T) { } func TestScatterConnStreamExecute(t *testing.T) { - testScatterConnGeneric(t, "TestScatterConnStreamExecute", func(shards []string) (*sqltypes.Result, error) { - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testScatterConnGeneric(t, "TestScatterConnStreamExecute", func(hc discovery.HealthCheck, shards []string) (*sqltypes.Result, error) { + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) qr := new(sqltypes.Result) err := stc.StreamExecute(context.Background(), "query", nil, "TestScatterConnStreamExecute", shards, topodatapb.TabletType_REPLICA, func(r *sqltypes.Result) error { appendResult(qr, r) @@ -78,8 +82,8 @@ func TestScatterConnStreamExecute(t *testing.T) { } func TestScatterConnStreamExecuteMulti(t *testing.T) { - testScatterConnGeneric(t, "TestScatterConnStreamExecuteMulti", func(shards []string) (*sqltypes.Result, error) { - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + testScatterConnGeneric(t, "TestScatterConnStreamExecuteMulti", func(hc discovery.HealthCheck, shards []string) (*sqltypes.Result, error) { + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) qr := new(sqltypes.Result) shardVars := make(map[string]map[string]interface{}) for _, shard := range shards { @@ -116,10 +120,11 @@ func verifyErrorCode(t *testing.T, err error, wantCode vtrpcpb.ErrorCode) { } } -func testScatterConnGeneric(t *testing.T, name string, f func(shards []string) (*sqltypes.Result, error)) { +func testScatterConnGeneric(t *testing.T, name string, f func(hc discovery.HealthCheck, shards []string) (*sqltypes.Result, error)) { + hc := newFakeHealthCheck() // no shard s := createSandbox(name) - qr, err := f(nil) + qr, err := f(hc, nil) if qr.RowsAffected != 0 { t.Errorf("want 0, got %v", qr.RowsAffected) } @@ -130,8 +135,8 @@ func testScatterConnGeneric(t *testing.T, name string, f func(shards []string) ( // single shard s.Reset() sbc := &sandboxConn{mustFailServer: 1} - s.MapTestConn("0", sbc) - qr, err = f([]string{"0"}) + hc.addTestEndPoint("aa", "0", 1, name, "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc) + qr, err = f(hc, []string{"0"}) want := fmt.Sprintf("shard, host: %v.0.replica, host:\"0\" port_map: , error: err", name) // Verify server error string. if err == nil || err.Error() != want { @@ -145,10 +150,11 @@ func testScatterConnGeneric(t *testing.T, name string, f func(shards []string) ( // two shards s.Reset() sbc0 := &sandboxConn{mustFailServer: 1} - s.MapTestConn("0", sbc0) sbc1 := &sandboxConn{mustFailServer: 1} - s.MapTestConn("1", sbc1) - _, err = f([]string{"0", "1"}) + hc.Reset() + hc.addTestEndPoint("aa", "0", 1, name, "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1", 1, name, "1", topodatapb.TabletType_REPLICA, true, 1, nil, sbc1) + _, err = f(hc, []string{"0", "1"}) // Verify server errors are consolidated. want = fmt.Sprintf("shard, host: %v.0.replica, host:\"0\" port_map: , error: err\nshard, host: %v.1.replica, host:\"1\" port_map: , error: err", name, name) verifyScatterConnError(t, err, want, vtrpcpb.ErrorCode_BAD_INPUT) @@ -163,10 +169,11 @@ func testScatterConnGeneric(t *testing.T, name string, f func(shards []string) ( // two shards with different errors s.Reset() sbc0 = &sandboxConn{mustFailServer: 1} - s.MapTestConn("0", sbc0) sbc1 = &sandboxConn{mustFailTxPool: 1} - s.MapTestConn("1", sbc1) - _, err = f([]string{"0", "1"}) + hc.Reset() + hc.addTestEndPoint("aa", "0", 1, name, "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1", 1, name, "1", topodatapb.TabletType_REPLICA, true, 1, nil, sbc1) + _, err = f(hc, []string{"0", "1"}) // Verify server errors are consolidated. want = fmt.Sprintf("shard, host: %v.0.replica, host:\"0\" port_map: , error: err\nshard, host: %v.1.replica, host:\"1\" port_map: , tx_pool_full: err", name, name) // We should only surface the higher priority error code @@ -182,8 +189,9 @@ func testScatterConnGeneric(t *testing.T, name string, f func(shards []string) ( // duplicate shards s.Reset() sbc = &sandboxConn{} - s.MapTestConn("0", sbc) - qr, err = f([]string{"0", "0"}) + hc.Reset() + hc.addTestEndPoint("aa", "0", 1, name, "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc) + qr, err = f(hc, []string{"0", "0"}) // Ensure that we executed only once. if execCount := sbc.ExecCount.Get(); execCount != 1 { t.Errorf("want 1, got %v", execCount) @@ -192,10 +200,11 @@ func testScatterConnGeneric(t *testing.T, name string, f func(shards []string) ( // no errors s.Reset() sbc0 = &sandboxConn{} - s.MapTestConn("0", sbc0) sbc1 = &sandboxConn{} - s.MapTestConn("1", sbc1) - qr, err = f([]string{"0", "1"}) + hc.Reset() + hc.addTestEndPoint("aa", "0", 1, name, "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1", 1, name, "1", topodatapb.TabletType_REPLICA, true, 1, nil, sbc1) + qr, err = f(hc, []string{"0", "1"}) if err != nil { t.Errorf("want nil, got %v", err) } @@ -214,12 +223,13 @@ func testScatterConnGeneric(t *testing.T, name string, f func(shards []string) ( } func TestMultiExecs(t *testing.T) { - s := createSandbox("TestMultiExecs") + createSandbox("TestMultiExecs") sbc0 := &sandboxConn{} - s.MapTestConn("0", sbc0) sbc1 := &sandboxConn{} - s.MapTestConn("1", sbc1) - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + hc := newFakeHealthCheck() + hc.addTestEndPoint("aa", "0", 1, "TestMultiExecs", "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1", 1, "TestMultiExecs", "1", topodatapb.TabletType_REPLICA, true, 1, nil, sbc1) + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) shardVars := map[string]map[string]interface{}{ "0": { "bv0": 0, @@ -249,10 +259,11 @@ func TestMultiExecs(t *testing.T) { } func TestScatterConnStreamExecuteSendError(t *testing.T) { - s := createSandbox("TestScatterConnStreamExecuteSendError") + createSandbox("TestScatterConnStreamExecuteSendError") sbc := &sandboxConn{} - s.MapTestConn("0", sbc) - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + hc := newFakeHealthCheck() + hc.addTestEndPoint("aa", "0", 1, "TestScatterConnStreamExecuteSendError", "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc) + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) err := stc.StreamExecute(context.Background(), "query", nil, "TestScatterConnStreamExecuteSendError", []string{"0"}, topodatapb.TabletType_REPLICA, func(*sqltypes.Result) error { return fmt.Errorf("send error") }) @@ -264,10 +275,11 @@ func TestScatterConnStreamExecuteSendError(t *testing.T) { } func TestScatterCommitRollbackIncorrectSession(t *testing.T) { - s := createSandbox("TestScatterCommitRollbackIncorrectSession") + createSandbox("TestScatterCommitRollbackIncorrectSession") sbc0 := &sandboxConn{} - s.MapTestConn("0", sbc0) - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + hc := newFakeHealthCheck() + hc.addTestEndPoint("aa", "0", 1, "TestScatterCommitRollbackIncorrectSession", "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc0) + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) // nil session err := stc.Commit(context.Background(), nil) @@ -285,12 +297,13 @@ func TestScatterCommitRollbackIncorrectSession(t *testing.T) { } func TestScatterConnCommitSuccess(t *testing.T) { - s := createSandbox("TestScatterConnCommitSuccess") + createSandbox("TestScatterConnCommitSuccess") sbc0 := &sandboxConn{} - s.MapTestConn("0", sbc0) sbc1 := &sandboxConn{} - s.MapTestConn("1", sbc1) - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + hc := newFakeHealthCheck() + hc.addTestEndPoint("aa", "0", 1, "TestScatterConnCommitSuccess", "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1", 1, "TestScatterConnCommitSuccess", "1", topodatapb.TabletType_REPLICA, true, 1, nil, sbc1) + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) // Sequence the executes to ensure commit order session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) @@ -349,12 +362,13 @@ func TestScatterConnCommitSuccess(t *testing.T) { } func TestScatterConnRollback(t *testing.T) { - s := createSandbox("TestScatterConnRollback") + createSandbox("TestScatterConnRollback") sbc0 := &sandboxConn{} - s.MapTestConn("0", sbc0) sbc1 := &sandboxConn{} - s.MapTestConn("1", sbc1) - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + hc := newFakeHealthCheck() + hc.addTestEndPoint("aa", "0", 1, "TestScatterConnRollback", "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1", 1, "TestScatterConnRollback", "1", topodatapb.TabletType_REPLICA, true, 1, nil, sbc1) + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) // Sequence the executes to ensure commit order session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) @@ -376,30 +390,11 @@ func TestScatterConnRollback(t *testing.T) { } } -func TestScatterConnClose(t *testing.T) { - s := createSandbox("TestScatterConnClose") - sbc := &sandboxConn{} - s.MapTestConn("0", sbc) - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", 1*time.Millisecond, 3, 2*time.Millisecond, 1*time.Millisecond, 24*time.Hour, nil, "") - stc.Execute(context.Background(), "query1", nil, "TestScatterConnClose", []string{"0"}, topodatapb.TabletType_REPLICA, nil, false) - stc.Close() - // retry for 10s as Close() is async. - for i := 0; i < 10; i++ { - if closeCount := sbc.CloseCount.Get(); closeCount == 1 { - return - } - time.Sleep(1 * time.Second) - } - if closeCount := sbc.CloseCount.Get(); closeCount != 1 { - t.Errorf("want 1, got %d", closeCount) - } -} - func TestScatterConnError(t *testing.T) { err := &ScatterConnError{ - Code: 12, + Retryable: false, Errs: []error{ - &ShardConnError{Code: 10, Err: &tabletconn.ServerError{Err: "tabletconn error"}}, + &ShardError{EndPointCode: vtrpcpb.ErrorCode_PERMISSION_DENIED, Err: &tabletconn.ServerError{Err: "tabletconn error"}}, fmt.Errorf("generic error"), tabletconn.ConnClosed, }, @@ -415,13 +410,15 @@ func TestScatterConnError(t *testing.T) { func TestScatterConnQueryNotInTransaction(t *testing.T) { s := createSandbox("TestScatterConnQueryNotInTransaction") + hc := newFakeHealthCheck() // case 1: read query (not in transaction) followed by write query, not in the same shard. sbc0 := &sandboxConn{} - s.MapTestConn("0", sbc0) sbc1 := &sandboxConn{} - s.MapTestConn("1", sbc1) - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + hc.Reset() + hc.addTestEndPoint("aa", "0", 1, "TestScatterConnQueryNotInTransaction", "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1", 1, "TestScatterConnQueryNotInTransaction", "1", topodatapb.TabletType_REPLICA, true, 1, nil, sbc1) + stc := NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) session := NewSafeSession(&vtgatepb.Session{InTransaction: true}) stc.Execute(context.Background(), "query1", nil, "TestScatterConnQueryNotInTransaction", []string{"0"}, topodatapb.TabletType_REPLICA, session, true) stc.Execute(context.Background(), "query1", nil, "TestScatterConnQueryNotInTransaction", []string{"1"}, topodatapb.TabletType_REPLICA, session, false) @@ -458,10 +455,11 @@ func TestScatterConnQueryNotInTransaction(t *testing.T) { // case 2: write query followed by read query (not in transaction), not in the same shard. s.Reset() sbc0 = &sandboxConn{} - s.MapTestConn("0", sbc0) sbc1 = &sandboxConn{} - s.MapTestConn("1", sbc1) - stc = NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + hc.Reset() + hc.addTestEndPoint("aa", "0", 1, "TestScatterConnQueryNotInTransaction", "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1", 1, "TestScatterConnQueryNotInTransaction", "1", topodatapb.TabletType_REPLICA, true, 1, nil, sbc1) + stc = NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) session = NewSafeSession(&vtgatepb.Session{InTransaction: true}) stc.Execute(context.Background(), "query1", nil, "TestScatterConnQueryNotInTransaction", []string{"0"}, topodatapb.TabletType_REPLICA, session, false) stc.Execute(context.Background(), "query1", nil, "TestScatterConnQueryNotInTransaction", []string{"1"}, topodatapb.TabletType_REPLICA, session, true) @@ -498,10 +496,11 @@ func TestScatterConnQueryNotInTransaction(t *testing.T) { // case 3: write query followed by read query, in the same shard. s.Reset() sbc0 = &sandboxConn{} - s.MapTestConn("0", sbc0) sbc1 = &sandboxConn{} - s.MapTestConn("1", sbc1) - stc = NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, nil, "") + hc.Reset() + hc.addTestEndPoint("aa", "0", 1, "TestScatterConnQueryNotInTransaction", "0", topodatapb.TabletType_REPLICA, true, 1, nil, sbc0) + hc.addTestEndPoint("aa", "1", 1, "TestScatterConnQueryNotInTransaction", "1", topodatapb.TabletType_REPLICA, true, 1, nil, sbc1) + stc = NewScatterConn(hc, topo.Server{}, new(sandboxTopo), "", "aa", retryCount, nil) session = NewSafeSession(&vtgatepb.Session{InTransaction: true}) stc.Execute(context.Background(), "query1", nil, "TestScatterConnQueryNotInTransaction", []string{"0"}, topodatapb.TabletType_REPLICA, session, false) stc.Execute(context.Background(), "query1", nil, "TestScatterConnQueryNotInTransaction", []string{"0", "1"}, topodatapb.TabletType_REPLICA, session, true) diff --git a/go/vt/vtgate/shard_conn.go b/go/vt/vtgate/shard_conn.go deleted file mode 100644 index ff1182442a9..00000000000 --- a/go/vt/vtgate/shard_conn.go +++ /dev/null @@ -1,495 +0,0 @@ -// Copyright 2012, Google Inc. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -package vtgate - -import ( - "fmt" - "strings" - "sync" - "time" - - "github.com/youtube/vitess/go/sqltypes" - "github.com/youtube/vitess/go/stats" - "github.com/youtube/vitess/go/sync2" - "github.com/youtube/vitess/go/timer" - "github.com/youtube/vitess/go/vt/concurrency" - "github.com/youtube/vitess/go/vt/tabletserver/querytypes" - "github.com/youtube/vitess/go/vt/tabletserver/tabletconn" - "github.com/youtube/vitess/go/vt/topo" - "github.com/youtube/vitess/go/vt/vterrors" - "github.com/youtube/vitess/go/vt/vtgate/masterbuffer" - "golang.org/x/net/context" - - querypb "github.com/youtube/vitess/go/vt/proto/query" - topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" - vtrpcpb "github.com/youtube/vitess/go/vt/proto/vtrpc" -) - -var danglingTabletConn = stats.NewInt("DanglingTabletConn") - -// ShardConn represents a load balanced connection to a group -// of vttablets that belong to the same shard. ShardConn can -// be concurrently used across goroutines. Such requests are -// interleaved on the same underlying connection. -type ShardConn struct { - keyspace string - shard string - tabletType topodatapb.TabletType - retryDelay time.Duration - retryCount int - connTimeoutTotal time.Duration - connTimeoutPerConn time.Duration - connLife time.Duration - balancer *Balancer - consolidator *sync2.Consolidator - ticker *timer.RandTicker - - connectTimings *stats.MultiTimings - - // conn needs a mutex because it can change during the lifetime of ShardConn. - mu sync.Mutex - conn tabletconn.TabletConn -} - -// NewShardConn creates a new ShardConn. It creates a Balancer using -// serv, cell, keyspace, tabletType and retryDelay. retryCount is the max -// number of retries before a ShardConn returns an error on an operation. -func NewShardConn(ctx context.Context, serv topo.SrvTopoServer, cell, keyspace, shard string, tabletType topodatapb.TabletType, retryDelay time.Duration, retryCount int, connTimeoutTotal, connTimeoutPerConn, connLife time.Duration, tabletConnectTimings *stats.MultiTimings) *ShardConn { - getAddresses := func() (*topodatapb.EndPoints, error) { - endpoints, _, err := serv.GetEndPoints(ctx, cell, keyspace, shard, tabletType) - if err != nil { - return nil, vterrors.NewVitessError( - vtrpcpb.ErrorCode_INTERNAL_ERROR, err, - "endpoints fetch error: %v", err, - ) - } - return endpoints, nil - } - blc := NewBalancer(getAddresses, retryDelay) - var ticker *timer.RandTicker - if tabletType != topodatapb.TabletType_MASTER { - ticker = timer.NewRandTicker(connLife, connLife/2) - } - sdc := &ShardConn{ - keyspace: keyspace, - shard: shard, - tabletType: tabletType, - retryDelay: retryDelay, - retryCount: retryCount, - connTimeoutTotal: connTimeoutTotal, - connTimeoutPerConn: connTimeoutPerConn, - connLife: connLife, - balancer: blc, - ticker: ticker, - consolidator: sync2.NewConsolidator(), - connectTimings: tabletConnectTimings, - } - if ticker != nil { - go func() { - for range ticker.C { - sdc.closeCurrent() - } - }() - } - return sdc -} - -// ShardConnError is the shard conn specific error. -type ShardConnError struct { - Code int - ShardIdentifier string - InTransaction bool - // Preserve the original error, so that we don't need to parse the error string. - Err error - // EndPointCode is the error code to use for all the endpoint errors in aggregate - EndPointCode vtrpcpb.ErrorCode -} - -func (e *ShardConnError) Error() string { - if e.ShardIdentifier == "" { - return fmt.Sprintf("%v", e.Err) - } - return fmt.Sprintf("shard, host: %s, %v", e.ShardIdentifier, e.Err) -} - -// VtErrorCode returns the underlying Vitess error code -func (e *ShardConnError) VtErrorCode() vtrpcpb.ErrorCode { return e.EndPointCode } - -// Dial creates tablet connection and connects to the vttablet. -// It is not necessary to call this function before serving queries, -// but it would reduce connection overhead when serving the first query. -func (sdc *ShardConn) Dial(ctx context.Context) error { - return sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - return nil - }, 0, false) -} - -// Execute executes a non-streaming query on vttablet. If there are connection errors, -// it retries retryCount times before failing. It does not retry if the connection is in -// the middle of a transaction. -func (sdc *ShardConn) Execute(ctx context.Context, query string, bindVars map[string]interface{}, transactionID int64) (qr *sqltypes.Result, err error) { - err = sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - var innerErr error - qr, innerErr = conn.Execute(ctx, query, bindVars, transactionID) - return innerErr - }, transactionID, false) - return qr, err -} - -// ExecuteBatch executes a group of queries. The retry rules are the same as Execute. -func (sdc *ShardConn) ExecuteBatch(ctx context.Context, queries []querytypes.BoundQuery, asTransaction bool, transactionID int64) (qrs []sqltypes.Result, err error) { - err = sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - var innerErr error - qrs, innerErr = conn.ExecuteBatch(ctx, queries, asTransaction, transactionID) - return innerErr - }, transactionID, false) - return qrs, err -} - -// StreamExecute executes a streaming query on vttablet. The retry rules are the same as Execute. -func (sdc *ShardConn) StreamExecute(ctx context.Context, query string, bindVars map[string]interface{}) (sqltypes.ResultStream, error) { - var usedConn tabletconn.TabletConn - var stream sqltypes.ResultStream - err := sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - var err error - stream, err = conn.StreamExecute(ctx, query, bindVars) - usedConn = conn - return err - }, 0, true) - if err != nil { - return nil, err - } - return stream, nil -} - -// Begin begins a transaction. The retry rules are the same as Execute. -func (sdc *ShardConn) Begin(ctx context.Context) (transactionID int64, err error) { - err = sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - var innerErr error - transactionID, innerErr = conn.Begin(ctx) - return innerErr - }, 0, false) - return transactionID, err -} - -// Commit commits the current transaction. The retry rules are the same as Execute. -func (sdc *ShardConn) Commit(ctx context.Context, transactionID int64) (err error) { - return sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - return conn.Commit(ctx, transactionID) - }, transactionID, false) -} - -// Rollback rolls back the current transaction. The retry rules are the same as Execute. -func (sdc *ShardConn) Rollback(ctx context.Context, transactionID int64) (err error) { - return sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - return conn.Rollback(ctx, transactionID) - }, transactionID, false) -} - -// BeginExecute executes a begin and non-streaming query on -// vttablet. If there are connection errors, it retries retryCount -// times before failing. It does not retry if the connection is in the -// middle of a transaction. -func (sdc *ShardConn) BeginExecute(ctx context.Context, query string, bindVars map[string]interface{}) (qr *sqltypes.Result, transactionID int64, err error) { - err = sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - var innerErr error - qr, transactionID, innerErr = conn.BeginExecute(ctx, query, bindVars) - return innerErr - }, 0, false) - return qr, transactionID, err -} - -// BeginExecuteBatch executes a begin and a group of queries. The -// retry rules are the same as BeginExecute. -func (sdc *ShardConn) BeginExecuteBatch(ctx context.Context, queries []querytypes.BoundQuery, asTransaction bool) (qrs []sqltypes.Result, transactionID int64, err error) { - err = sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - var innerErr error - qrs, transactionID, innerErr = conn.BeginExecuteBatch(ctx, queries, asTransaction) - return innerErr - }, 0, false) - return qrs, transactionID, err -} - -// SplitQuery splits a query into sub queries. The retry rules are the same as Execute. -func (sdc *ShardConn) SplitQuery(ctx context.Context, sql string, bindVariables map[string]interface{}, splitColumn string, splitCount int64) (queries []querytypes.QuerySplit, err error) { - err = sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - var innerErr error - queries, innerErr = conn.SplitQuery(ctx, querytypes.BoundQuery{ - Sql: sql, - BindVariables: bindVariables, - }, splitColumn, splitCount) - return innerErr - }, 0, false) - return -} - -// SplitQueryV2 splits a query into sub queries. The retry rules are the same as Execute. -// TODO(erez): Rename to SplitQuery after the migration to SplitQuery V2 is done. -func (sdc *ShardConn) SplitQueryV2( - ctx context.Context, - sql string, - bindVariables map[string]interface{}, - splitColumns []string, - splitCount int64, - numRowsPerQueryPart int64, - algorithm querypb.SplitQueryRequest_Algorithm) (queries []querytypes.QuerySplit, err error) { - - err = sdc.withRetry(ctx, func(conn tabletconn.TabletConn) error { - var innerErr error - queries, innerErr = conn.SplitQueryV2(ctx, querytypes.BoundQuery{ - Sql: sql, - BindVariables: bindVariables, - }, splitColumns, splitCount, numRowsPerQueryPart, algorithm) - return innerErr - }, 0, false) - return -} - -// Close closes the underlying TabletConn. -func (sdc *ShardConn) Close() { - if sdc.ticker != nil { - sdc.ticker.Stop() - } - sdc.closeCurrent() -} - -func (sdc *ShardConn) closeCurrent() { - sdc.mu.Lock() - defer sdc.mu.Unlock() - if sdc.conn == nil { - return - } - go func(conn tabletconn.TabletConn) { - danglingTabletConn.Add(1) - conn.Close() - danglingTabletConn.Add(-1) - }(sdc.conn) - sdc.conn = nil -} - -// withRetry sets up the connection and executes the action. If there are connection 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 (sdc *ShardConn) withRetry(ctx context.Context, action func(conn tabletconn.TabletConn) error, transactionID int64, isStreaming bool) error { - var conn tabletconn.TabletConn - var endPoint *topodatapb.EndPoint - var err error - var isTimeout bool - inTransaction := (transactionID != 0) - // execute the action at least once even without retrying - for i := 0; i < sdc.retryCount+1; i++ { - conn, endPoint, isTimeout, err = sdc.getConn(ctx) - if err != nil { - if isTimeout || i == sdc.retryCount { - break - } - time.Sleep(sdc.retryDelay) - continue - } - - // Potentially buffer this request. - if bufferErr := masterbuffer.FakeBuffer(sdc.keyspace, sdc.shard, sdc.tabletType, inTransaction, i); bufferErr != nil { - return bufferErr - } - - err = action(conn) - if sdc.canRetry(ctx, err, transactionID, conn, isStreaming) { - continue - } - break - } - return sdc.WrapError(err, endPoint, inTransaction) -} - -type connectResult struct { - Conn tabletconn.TabletConn - EndPoint *topodatapb.EndPoint - IsTimeout bool -} - -// getConn reuses an existing connection if possible. -// If no connection is available, -// it creates a new connection if no connection is being created. -// Otherwise it waits for the connection to be created. -func (sdc *ShardConn) getConn(ctx context.Context) (conn tabletconn.TabletConn, endPoint *topodatapb.EndPoint, isTimeout bool, err error) { - sdc.mu.Lock() - if sdc.conn != nil { - conn = sdc.conn - endPoint = conn.EndPoint() - sdc.mu.Unlock() - return conn, endPoint, false, nil - } - - key := fmt.Sprintf("%s.%s.%s", sdc.keyspace, sdc.shard, strings.ToLower(sdc.tabletType.String())) - q, ok := sdc.consolidator.Create(key) - sdc.mu.Unlock() - if ok { - defer q.Broadcast() - conn, endPoint, isTimeout, err := sdc.getNewConn(ctx) - q.Result = &connectResult{Conn: conn, EndPoint: endPoint, IsTimeout: isTimeout} - q.Err = err - } else { - q.Wait() - } - - connResult := q.Result.(*connectResult) - return connResult.Conn, connResult.EndPoint, connResult.IsTimeout, q.Err -} - -// getNewConn creates a new tablet connection with a separate per conn timeout. -// It limits the overall timeout to connTimeoutTotal by checking elapsed time after each blocking call. -func (sdc *ShardConn) getNewConn(ctx context.Context) (conn tabletconn.TabletConn, endPoint *topodatapb.EndPoint, isTimeout bool, err error) { - startTime := time.Now() - - endPoints, err := sdc.balancer.Get() - if err != nil { - // Error when getting endpoint - return nil, nil, false, err - } - if len(endPoints) == 0 { - // No valid endpoint - return nil, nil, false, vterrors.FromError( - vtrpcpb.ErrorCode_INTERNAL_ERROR, - fmt.Errorf("no valid endpoint"), - ) - } - if time.Now().Sub(startTime) >= sdc.connTimeoutTotal { - return nil, nil, true, vterrors.FromError( - vtrpcpb.ErrorCode_DEADLINE_EXCEEDED, - fmt.Errorf("timeout when getting endpoints"), - ) - } - - // Iterate through all endpoints to create a connection - perConnTimeout := sdc.getConnTimeoutPerConn(len(endPoints)) - allErrors := new(concurrency.AllErrorRecorder) - for _, endPoint := range endPoints { - perConnStartTime := time.Now() - conn, err = tabletconn.GetDialer()(ctx, endPoint, sdc.keyspace, sdc.shard, sdc.tabletType, perConnTimeout) - if err == nil { - sdc.connectTimings.Record([]string{sdc.keyspace, sdc.shard, strings.ToLower(sdc.tabletType.String())}, perConnStartTime) - sdc.mu.Lock() - defer sdc.mu.Unlock() - sdc.conn = conn - return conn, endPoint, false, nil - } - // Markdown the endpoint if it failed to connect - sdc.balancer.MarkDown(endPoint.Uid, err.Error()) - vtErr := vterrors.NewVitessError( - // TODO(aaijazi): what about OperationalErrors here? - vterrors.RecoverVtErrorCode(err), err, - "%v %+v", err, endPoint, - ) - allErrors.RecordError(vtErr) - if time.Now().Sub(startTime) >= sdc.connTimeoutTotal { - err = vterrors.FromError( - vtrpcpb.ErrorCode_DEADLINE_EXCEEDED, - fmt.Errorf("timeout when connecting to %+v", endPoint), - ) - allErrors.RecordError(err) - return nil, nil, true, allErrors.AggrError(AggregateVtGateErrors) - } - } - return nil, nil, false, allErrors.Error() -} - -// getConnTimeoutPerConn determines the appropriate timeout per connection. -func (sdc *ShardConn) getConnTimeoutPerConn(endPointCount int) time.Duration { - if endPointCount <= 1 { - return sdc.connTimeoutTotal - } - if sdc.connTimeoutPerConn > sdc.connTimeoutTotal { - return sdc.connTimeoutTotal - } - return sdc.connTimeoutPerConn -} - -// canRetry determines whether a query can be retried or not. -// OperationalErrors like retry/fatal cause a reconnect and retry if query is not in a txn. -// TxPoolFull causes a retry and all other errors are non-retry. -func (sdc *ShardConn) canRetry(ctx context.Context, err error, transactionID int64, conn tabletconn.TabletConn, isStreaming bool) bool { - if err == nil { - return false - } - // Do not retry if ctx.Done() is closed. - select { - case <-ctx.Done(): - return false - default: - } - if serverError, ok := err.(*tabletconn.ServerError); ok { - switch serverError.Code { - case tabletconn.ERR_FATAL: - // Do not retry on fatal error for streaming query. - // For streaming query, vttablet sends: - // - RETRY, if streaming is not started yet; - // - FATAL, if streaming is broken halfway. - // For non-streaming query, handle as ERR_RETRY. - if isStreaming { - return false - } - fallthrough - case tabletconn.ERR_RETRY: - // Retry on RETRY and FATAL if not in a transaction. - inTransaction := (transactionID != 0) - sdc.markDown(conn, err.Error()) - return !inTransaction - default: - // Not retry for TX_POOL_FULL and normal server errors. - return false - } - } - // Do not retry on operational error. - // TODO(liang): handle the case when VTGate is idle - // while vttablet is gracefully shutdown. - // We want to retry in that case. - sdc.markDown(conn, err.Error()) - return false -} - -// markDown closes conn and temporarily marks the associated -// end point as unusable. -func (sdc *ShardConn) markDown(conn tabletconn.TabletConn, reason string) { - sdc.mu.Lock() - defer sdc.mu.Unlock() - if conn != sdc.conn { - return - } - sdc.balancer.MarkDown(conn.EndPoint().Uid, reason) - - go func(conn tabletconn.TabletConn) { - danglingTabletConn.Add(1) - conn.Close() - danglingTabletConn.Add(-1) - }(sdc.conn) - sdc.conn = nil -} - -// WrapError returns ShardConnError which preserves the original error code if possible, -// adds the connection context -// and adds a bit to determine whether the keyspace/shard needs to be -// re-resolved for a potential sharding event. -func (sdc *ShardConn) WrapError(in error, endPoint *topodatapb.EndPoint, inTransaction bool) (wrapped error) { - if in == nil { - return nil - } - shardIdentifier := fmt.Sprintf("%s.%s.%s, %+v", sdc.keyspace, sdc.shard, strings.ToLower(sdc.tabletType.String()), endPoint) - code := tabletconn.ERR_NORMAL - serverError, ok := in.(*tabletconn.ServerError) - if ok { - code = serverError.Code - } - - shardConnErr := &ShardConnError{ - Code: code, - ShardIdentifier: shardIdentifier, - InTransaction: inTransaction, - Err: in, - EndPointCode: vterrors.RecoverVtErrorCode(in), - } - return shardConnErr -} diff --git a/go/vt/vtgate/shard_conn_flaky_test.go b/go/vt/vtgate/shard_conn_flaky_test.go deleted file mode 100644 index d3cfb2f6d7c..00000000000 --- a/go/vt/vtgate/shard_conn_flaky_test.go +++ /dev/null @@ -1,818 +0,0 @@ -// Copyright 2012, Google Inc. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -package vtgate - -import ( - "fmt" - "reflect" - "strings" - "testing" - "time" - - "github.com/youtube/vitess/go/stats" - "golang.org/x/net/context" - - "github.com/youtube/vitess/go/vt/tabletserver/querytypes" - "github.com/youtube/vitess/go/vt/vterrors" - - topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" - vtrpcpb "github.com/youtube/vitess/go/vt/proto/vtrpc" -) - -// This file uses the sandbox_test framework. - -var ( - retryCount = 3 - retryDelay = 1 * time.Millisecond - connTimeoutTotal = 20 * time.Millisecond - connTimeoutPerConn = 10 * time.Millisecond - connLife = 24 * time.Hour - connectTimings = stats.NewMultiTimings("", []string{"Keyspace", "ShardName", "DbType"}) -) - -func TestShardConnExecute(t *testing.T) { - testShardConnGeneric(t, "TestShardConnExecute", false, func() error { - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnExecute", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - _, err := sdc.Execute(context.Background(), "query", nil, 0) - return err - }) - testShardConnTransact(t, "TestShardConnExecute", func() error { - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnExecute", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - _, err := sdc.Execute(context.Background(), "query", nil, 1) - return err - }) -} - -func TestShardConnExecuteBatch(t *testing.T) { - testShardConnGeneric(t, "TestShardConnExecuteBatch", false, func() error { - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnExecuteBatch", "0", topodatapb.TabletType_REPLICA, 1*time.Millisecond, 3, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - queries := []querytypes.BoundQuery{{"query", nil}} - _, err := sdc.ExecuteBatch(context.Background(), queries, false, 0) - return err - }) - testShardConnTransact(t, "TestShardConnExecuteBatch", func() error { - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnExecuteBatch", "0", topodatapb.TabletType_REPLICA, 1*time.Millisecond, 3, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - queries := []querytypes.BoundQuery{{"query", nil}} - _, err := sdc.ExecuteBatch(context.Background(), queries, false, 1) - return err - }) -} - -func TestShardConnExecuteStream(t *testing.T) { - testShardConnGeneric(t, "TestShardConnExecuteStream", true, func() error { - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnExecuteStream", "0", topodatapb.TabletType_REPLICA, 1*time.Millisecond, 3, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - _, err := sdc.StreamExecute(context.Background(), "query", nil) - return err - }) -} - -func TestShardConnBegin(t *testing.T) { - testShardConnGeneric(t, "TestShardConnBegin", false, func() error { - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnBegin", "0", topodatapb.TabletType_REPLICA, 1*time.Millisecond, 3, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - _, err := sdc.Begin(context.Background()) - return err - }) -} - -func TestShardConnCommit(t *testing.T) { - testShardConnTransact(t, "TestShardConnCommit", func() error { - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnCommit", "0", topodatapb.TabletType_REPLICA, 1*time.Millisecond, 3, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - return sdc.Commit(context.Background(), 1) - }) -} - -func TestShardConnRollback(t *testing.T) { - testShardConnTransact(t, "TestShardConnRollback", func() error { - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnRollback", "0", topodatapb.TabletType_REPLICA, 1*time.Millisecond, 3, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - return sdc.Rollback(context.Background(), 1) - }) -} - -func verifyShardConnError(t *testing.T, err error, wantErr string, wantCode vtrpcpb.ErrorCode) { - if err == nil || err.Error() != wantErr { - t.Errorf("wanted error: %s, got error: %v", wantErr, err) - } - if _, ok := err.(*ShardConnError); !ok { - t.Errorf("wanted error type *ShardConnError, got error type: %v", reflect.TypeOf(err)) - } - code := vterrors.RecoverVtErrorCode(err) - if code != wantCode { - t.Errorf("wanted error code: %s, got: %v", wantCode, code) - } -} - -func testShardConnGeneric(t *testing.T, name string, streaming bool, f func() error) { - // Topo failure - s := createSandbox(name) - s.EndPointMustFail = retryCount + 1 - - want := fmt.Sprintf("shard, host: %v.0.replica, , endpoints fetch error: topo error", name) - err := f() - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_INTERNAL_ERROR) - - if epCount := s.EndPointCounter.Get(); epCount != int64(retryCount+1) { - t.Errorf("want %v, got %v", (retryCount + 1), epCount) - } - - // Connect failure - s.Reset() - sbc := &sandboxConn{} - s.MapTestConn("0", sbc) - s.DialMustFail = 4 - err = f() - want = fmt.Sprintf("shard, host: %v.0.replica, %+v, conn error %+v", name, nil, sbc.EndPoint()) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_UNKNOWN_ERROR) - // Ensure we dialed 4 times before failing. - if s.DialCounter != 4 { - t.Errorf("want 4, got %v", s.DialCounter) - } - - // no valid endpoints as the only connection is marked down - // **** It tests the behavior when retryCount is odd. - // When the retryCount is even, the error should be "retry: err". **** - s.Reset() - sbc = &sandboxConn{mustFailRetry: retryCount + 1} - s.MapTestConn("0", sbc) - err = f() - want = fmt.Sprintf("shard, host: %v.0.replica, %+v, no valid endpoint", name, nil) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_INTERNAL_ERROR) - // Ensure we dialed 2 times before failing. - if s.DialCounter != 2 { - t.Errorf("want 2, got %v", s.DialCounter) - } - // Ensure we executed 2 times before failing. - if count := sbc.ExecCount.Get() + sbc.BeginCount.Get(); count != 2 { - t.Errorf("want 2, got %v", count) - } - - // retry error (one failure) - s.Reset() - sbc = &sandboxConn{mustFailRetry: 1} - s.MapTestConn("0", sbc) - err = f() - if err != nil { - t.Errorf("want nil, got %v", err) - } - // Ensure we dialed twice (second one succeeded) - if s.DialCounter != 2 { - t.Errorf("want 2, got %v", s.DialCounter) - } - // Ensure we executed twice (second one succeeded) - if count := sbc.ExecCount.Get() + sbc.BeginCount.Get(); count != 2 { - t.Errorf("want 2, got %v", count) - } - - // fatal error (one failure) - s.Reset() - sbc = &sandboxConn{mustFailFatal: 1} - s.MapTestConn("0", sbc) - err = f() - // streaming queries don't retry on fatal - if streaming { - want = fmt.Sprintf("shard, host: %v.0.replica, host:\"0\" port_map: , fatal: err", name) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_INTERNAL_ERROR) - } else { - if err != nil { - t.Errorf("want nil, got %v", err) - } - } - wantCounter := 2 - if streaming { - wantCounter = 1 - } - // Ensure we dialed twice (second one succeeded) - if s.DialCounter != wantCounter { - t.Errorf("want %v, got %v", wantCounter, s.DialCounter) - } - // Ensure we executed twice (second one succeeded) - if count := sbc.ExecCount.Get() + sbc.BeginCount.Get(); count != int64(wantCounter) { - t.Errorf("want %v, got %v", wantCounter, count) - } - - // server error - s.Reset() - sbc = &sandboxConn{mustFailServer: 1} - s.MapTestConn("0", sbc) - err = f() - want = fmt.Sprintf("shard, host: %v.0.replica, host:\"0\" port_map: , error: err", name) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_BAD_INPUT) - // Ensure we did not redial. - if s.DialCounter != 1 { - t.Errorf("want 1, got %v", s.DialCounter) - } - // Ensure we did not re-execute. - if count := sbc.ExecCount.Get() + sbc.BeginCount.Get(); count != 1 { - t.Errorf("want 1, got %v", count) - } - - // conn error (one failure) - // no retry on OperationalError - s.Reset() - sbc = &sandboxConn{mustFailConn: 1} - s.MapTestConn("0", sbc) - err = f() - want = fmt.Sprintf("shard, host: %v.0.replica, host:\"0\" port_map: , error: conn", name) - verifyShardConnError(t, err, want, vtrpcpb.ErrorCode_UNKNOWN_ERROR) - if err == nil || err.Error() != want { - t.Errorf("want %v, got %v", want, err) - } - // Ensure we did not redail. - if s.DialCounter != 1 { - t.Errorf("want 1, got %v", s.DialCounter) - } - // Ensure we did not re-execute. - if count := sbc.ExecCount.Get() + sbc.BeginCount.Get(); count != 1 { - t.Errorf("want 1, got %v", count) - } - - // no failures - s.Reset() - sbc = &sandboxConn{} - s.MapTestConn("0", sbc) - err = f() - if err != nil { - t.Errorf("want nil, got %v", err) - } - if s.DialCounter != 1 { - t.Errorf("want 1, got %v", s.DialCounter) - } - if count := sbc.ExecCount.Get() + sbc.BeginCount.Get(); count != 1 { - t.Errorf("want 1, got %v", count) - } -} - -func testShardConnTransact(t *testing.T, name string, f func() error) { - // retry error - s := createSandbox(name) - sbc := &sandboxConn{mustFailRetry: 3} - s.MapTestConn("0", sbc) - err := f() - want := fmt.Sprintf("shard, host: %v.0.replica, host:\"0\" port_map: , retry: err", name) - if err == nil || err.Error() != want { - t.Errorf("want %s, got %v", want, err) - } - // Should not retry if we're in transaction - if count := sbc.BeginCount.Get() + sbc.ExecCount.Get() + sbc.CommitCount.Get() + sbc.RollbackCount.Get(); count != 1 { - t.Errorf("want 1, got %v", count) - } - - // conn error - s.Reset() - sbc = &sandboxConn{mustFailConn: 3} - s.MapTestConn("0", sbc) - err = f() - want = fmt.Sprintf("shard, host: %v.0.replica, host:\"0\" port_map: , error: conn", name) - if err == nil || err.Error() != want { - t.Errorf("want %s, got %v", want, err) - } - // Should not retry if we're in transaction - if count := sbc.ExecCount.Get() + sbc.CommitCount.Get() + sbc.RollbackCount.Get(); count != 1 { - t.Errorf("want 1, got %v", count) - } -} - -func TestShardConnBeginOther(t *testing.T) { - // tx_pool_full - s := createSandbox("TestShardConnBeginOther") - sbc := &sandboxConn{mustFailTxPool: 1} - s.MapTestConn("0", sbc) - want := fmt.Sprintf("shard, host: TestShardConnBeginOther.0.replica, host:\"0\" port_map: , tx_pool_full: err") - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnBeginOther", "0", topodatapb.TabletType_REPLICA, 10*time.Millisecond, 3, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - _, err := sdc.Begin(context.Background()) - if err == nil || err.Error() != want { - t.Errorf("want %v, got %v", want, err) - } - // There should have been no redial. - if s.DialCounter != 1 { - t.Errorf("want 1, got %v", s.DialCounter) - } - // Account for 1 call to Begin. - if count := sbc.BeginCount.Get(); count != 1 { - t.Errorf("want 1, got %v", count) - } -} - -func TestShardConnStreamingRetry(t *testing.T) { - // ERR_RETRY - s := createSandbox("TestShardConnStreamingRetry") - sbc := &sandboxConn{mustFailRetry: 1} - s.MapTestConn("0", sbc) - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnStreamingRetry", "0", topodatapb.TabletType_REPLICA, 10*time.Millisecond, 3, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - _, err := sdc.StreamExecute(context.Background(), "query", nil) - if err != nil { - t.Errorf("want nil, got %v", err) - } - if s.DialCounter != 2 { - t.Errorf("want 2, got %v", s.DialCounter) - } - if execCount := sbc.ExecCount.Get(); execCount != 2 { - t.Errorf("want 2, got %v", execCount) - } - - // ERR_FATAL - s.Reset() - sbc = &sandboxConn{mustFailFatal: 1} - s.MapTestConn("0", sbc) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnStreamingRetry", "0", topodatapb.TabletType_REPLICA, 10*time.Millisecond, 3, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - _, err = sdc.StreamExecute(context.Background(), "query", nil) - want := "shard, host: TestShardConnStreamingRetry.0.replica, host:\"0\" port_map: , fatal: err" - if err == nil || err.Error() != want { - t.Errorf("want %v, got %v", want, err) - } - if s.DialCounter != 1 { - t.Errorf("want 1, got %v", s.DialCounter) - } - if execCount := sbc.ExecCount.Get(); execCount != 1 { - t.Errorf("want 1, got %v", execCount) - } -} - -func TestShardConnTimeout(t *testing.T) { - s := createSandbox("TestShardConnTimeout") - // case 1: one endpoint, per conn timeout becomes total timeout - s.DialMustTimeout = 1 - sbc := &sandboxConn{} - s.MapTestConn("0", sbc) - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnTimeout", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - startTime := time.Now() - _, err := sdc.Execute(context.Background(), "query", nil, 0) - execDuration := time.Now().Sub(startTime) - if execDuration < connTimeoutTotal { - t.Errorf("timeout too fast, want > %v, got %v", connTimeoutTotal, execDuration) - } - if execDuration > connTimeoutTotal*2 { - t.Errorf("timeout too slow, want < %v, got %v", connTimeoutTotal*2, execDuration) - } - if err == nil || strings.Contains(err.Error(), "timeout") == false { - t.Errorf("want timeout in error, got %v", err) - } - - // case 2: per conn timeout is longer than total timeout (by error) - s.Reset() - s.DialMustTimeout = 1 - sbc = &sandboxConn{} - s.MapTestConn("0", sbc) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnTimeout", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutTotal*3, connLife, connectTimings) - startTime = time.Now() - _, err = sdc.Execute(context.Background(), "query", nil, 0) - execDuration = time.Now().Sub(startTime) - if execDuration < connTimeoutTotal { - t.Errorf("timeout too fast, want > %v, got %v", connTimeoutTotal, execDuration) - } - if execDuration > connTimeoutTotal*2 { - t.Errorf("timeout want < %v, got %v", connTimeoutTotal*2, execDuration) - } - if err == nil || strings.Contains(err.Error(), "timeout") == false { - t.Errorf("want timeout in error, got %v", err) - } - - // case 3: per conn timeout is honored if there are more than one endpoint - // with one endpoint times out - s.Reset() - s.DialMustTimeout = 1 - sbc = &sandboxConn{} - sbc2 := &sandboxConn{} - s.MapTestConn("0", sbc) - s.MapTestConn("0", sbc2) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnTimeout", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - startTime = time.Now() - _, err = sdc.Execute(context.Background(), "query", nil, 0) - execDuration = time.Now().Sub(startTime) - if execDuration < connTimeoutPerConn { - t.Errorf("timeout too fast, want > %v, got %v", connTimeoutPerConn, execDuration) - } - if execDuration > connTimeoutTotal { - t.Errorf("timeout too slow, want < %v, got %v", connTimeoutTotal, execDuration) - } - if err != nil { - t.Errorf("want nil, got %v", err) - } - - // case 4: per conn timeout is honored if there are more than one endpoint - // with two endpoints time out - s.Reset() - s.DialMustTimeout = 2 - sbc = &sandboxConn{} - sbc2 = &sandboxConn{} - s.MapTestConn("0", sbc) - s.MapTestConn("0", sbc2) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnTimeout", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - startTime = time.Now() - _, err = sdc.Execute(context.Background(), "query", nil, 0) - execDuration = time.Now().Sub(startTime) - if execDuration < connTimeoutTotal { - t.Errorf("timeout too fast, want > %v, got %v", connTimeoutTotal, execDuration) - } - if execDuration > connTimeoutTotal*2 { - t.Errorf("timeout too slow, want < %v, got %v", connTimeoutTotal*2, execDuration) - } - if err == nil || strings.Contains(err.Error(), "timeout") == false { - t.Errorf("want timeout in error, got %v", err) - } -} - -func TestShardConnReconnect(t *testing.T) { - retryDelay := 50 * time.Millisecond - retryCount := 5 - s := createSandbox("TestShardConnReconnect") - // case 1: resolved 0 endpoint, return error - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - startTime := time.Now() - _, err := sdc.Execute(context.Background(), "query", nil, 0) - execDuration := time.Now().Sub(startTime) - if execDuration < (retryDelay * time.Duration(retryCount)) { - t.Errorf("retry too fast, want %v, got %v", retryDelay*time.Duration(retryCount), execDuration) - } - if execDuration > retryDelay*time.Duration(retryCount+1) { - t.Errorf("retry too slow, want %v, got %v", retryDelay*time.Duration(retryCount+1), execDuration) - } - if err == nil { - t.Errorf("want error, got nil") - } - if epCount := s.EndPointCounter.Get(); epCount != int64(retryCount+1) { - t.Errorf("want %v, got %v", retryCount+1, epCount) - } - - // case 2.1: resolve 1 endpoint and connect failed -> resolve and retry without spamming - s.Reset() - s.DialMustFail = 1 - sbc := &sandboxConn{} - s.MapTestConn("0", sbc) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - timeStart := time.Now() - sdc.Execute(context.Background(), "query", nil, 0) - timeDuration := time.Now().Sub(timeStart) - if timeDuration < retryDelay { - t.Errorf("want no spam delay %v, got %v", retryDelay, timeDuration) - } - if timeDuration > retryDelay*2 { - t.Errorf("want instant resolve %v, got %v", retryDelay, timeDuration) - } - if epCount := s.EndPointCounter.Get(); epCount != 2 { - t.Errorf("want 2, got %v", epCount) - } - - // case 2.2: resolve 1 endpoint and execute failed with retryable error -> resolve and retry without spamming - s.Reset() - sbc = &sandboxConn{mustFailRetry: 1} - s.MapTestConn("0", sbc) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - timeStart = time.Now() - sdc.Execute(context.Background(), "query", nil, 0) - timeDuration = time.Now().Sub(timeStart) - if timeDuration < retryDelay { - t.Errorf("want no spam delay %v, got %v", retryDelay, timeDuration) - } - if timeDuration > retryDelay*2 { - t.Errorf("want instant resolve %v, got %v", retryDelay, timeDuration) - } - if epCount := s.EndPointCounter.Get(); epCount != 3 { - t.Errorf("want 3, got %v", epCount) - } - - // case 2.3: resolve 1 endpoint and execute failed with OperationalError -> no retry - s.Reset() - sbc = &sandboxConn{mustFailConn: 1} - s.MapTestConn("0", sbc) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - timeStart = time.Now() - sdc.Execute(context.Background(), "query", nil, 0) - timeDuration = time.Now().Sub(timeStart) - if timeDuration > retryDelay { - t.Errorf("want instant fail %v, got %v", retryDelay, timeDuration) - } - if epCount := s.EndPointCounter.Get(); epCount != 1 { - t.Errorf("want 1, got %v", epCount) - } - - // case 3.1: resolve 3 endpoints, failed connection to 1st one -> resolve and connect to 2nd one - s.Reset() - s.DialMustFail = 1 - sbc0 := &sandboxConn{} - sbc1 := &sandboxConn{} - sbc2 := &sandboxConn{} - s.MapTestConn("0", sbc0) - s.MapTestConn("0", sbc1) - s.MapTestConn("0", sbc2) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - timeStart = time.Now() - sdc.Execute(context.Background(), "query", nil, 0) - timeDuration = time.Now().Sub(timeStart) - if timeDuration >= retryDelay { - t.Errorf("want no delay, got %v", timeDuration) - } - if execCount := sbc0.ExecCount.Get() + sbc1.ExecCount.Get() + sbc2.ExecCount.Get(); execCount != 1 { - t.Errorf("want 1, got %v", execCount) - } - if epCount := s.EndPointCounter.Get(); epCount != 1 { - t.Errorf("want 1, got %v", epCount) - } - - // case 3.2: resolve 3 endpoints, failed execution on 1st one -> resolve and execute on 2nd one - s.Reset() - countConnUse := 0 - onConnUse := func(conn *sandboxConn) { - if countConnUse == 0 { - conn.mustFailRetry = 1 - } - countConnUse++ - } - sbc0 = &sandboxConn{onConnUse: onConnUse} - sbc1 = &sandboxConn{onConnUse: onConnUse} - sbc2 = &sandboxConn{onConnUse: onConnUse} - s.MapTestConn("0", sbc0) - s.MapTestConn("0", sbc1) - s.MapTestConn("0", sbc2) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - timeStart = time.Now() - sdc.Execute(context.Background(), "query", nil, 0) - timeDuration = time.Now().Sub(timeStart) - if timeDuration >= retryDelay { - t.Errorf("want no delay, got %v", timeDuration) - } - if execCount := sbc0.ExecCount.Get() + sbc1.ExecCount.Get() + sbc2.ExecCount.Get(); execCount != 2 { - t.Errorf("want 2, got %v", execCount) - } - { - execCount0 := sbc0.ExecCount.Get() - execCount1 := sbc1.ExecCount.Get() - execCount2 := sbc2.ExecCount.Get() - if execCount0 > 1 || execCount1 > 1 || execCount2 > 1 { - t.Errorf("want no more than 1, got %v,%v,%v", execCount0, execCount1, execCount2) - } - } - if epCount := s.EndPointCounter.Get(); epCount != 2 { - t.Errorf("want 2, got %v", epCount) - } - - // case 4: resolve 3 endpoints, failed connection to 1st, failed execution on 2nd -> resolve and execute on 3rd one - s.Reset() - s.DialMustFail = 1 - countConnUse = 0 - onConnUse = func(conn *sandboxConn) { - if countConnUse == 0 { - conn.mustFailRetry = 1 - } - countConnUse++ - } - sbc0 = &sandboxConn{onConnUse: onConnUse} - sbc1 = &sandboxConn{onConnUse: onConnUse} - sbc2 = &sandboxConn{onConnUse: onConnUse} - s.MapTestConn("0", sbc0) - s.MapTestConn("0", sbc1) - s.MapTestConn("0", sbc2) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - timeStart = time.Now() - sdc.Execute(context.Background(), "query", nil, 0) - timeDuration = time.Now().Sub(timeStart) - if timeDuration >= retryDelay { - t.Errorf("want no delay, got %v", timeDuration) - } - { - execCount0 := sbc0.ExecCount.Get() - execCount1 := sbc1.ExecCount.Get() - execCount2 := sbc2.ExecCount.Get() - if sum := execCount0 + execCount1 + execCount2; sum != 2 { - t.Errorf("want 2, got %v", sum) - } - if execCount0 > 1 || execCount1 > 1 || execCount2 > 1 { - t.Errorf("want no more than 1, got %v,%v,%v", execCount0, execCount1, execCount2) - } - } - if epCount := s.EndPointCounter.Get(); epCount != 2 { - t.Errorf("want 2, got %v", epCount) - } - - // case 5: always resolve the same 3 endpoints, all 3 execution failed -> resolve and use the first one - s.Reset() - var firstConn *sandboxConn - countConnUse = 0 - onConnUse = func(conn *sandboxConn) { - if countConnUse == 0 { - firstConn = conn - } - countConnUse++ - } - sbc0 = &sandboxConn{mustFailRetry: 1, onConnUse: onConnUse} - sbc1 = &sandboxConn{mustFailRetry: 1, onConnUse: onConnUse} - sbc2 = &sandboxConn{mustFailRetry: 1, onConnUse: onConnUse} - s.MapTestConn("0", sbc0) - s.MapTestConn("0", sbc1) - s.MapTestConn("0", sbc2) - sdc = NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - timeStart = time.Now() - sdc.Execute(context.Background(), "query", nil, 0) - timeDuration = time.Now().Sub(timeStart) - if timeDuration < retryDelay { - t.Errorf("want no spam delay %v, got %v", retryDelay, timeDuration) - } - if timeDuration > retryDelay*2 { - t.Errorf("want instant resolve %v, got %v", retryDelay, timeDuration) - } - for _, conn := range []*sandboxConn{sbc0, sbc1, sbc2} { - var wantExecCount int64 = 1 - if conn == firstConn { - wantExecCount = 2 - } - if execCount := conn.ExecCount.Get(); execCount != wantExecCount { - t.Errorf("want %v, got %v", wantExecCount, execCount) - } - } - if epCount := s.EndPointCounter.Get(); epCount != 5 { - t.Errorf("want 5, got %v", epCount) - } - - // case 6: resolve 3 endpoints with 1st execution failed, resolve to a new set without the failed one -> try a random one - s.Reset() - firstConn = nil - onConnUse = func(conn *sandboxConn) { - if firstConn == nil { - firstConn = conn - conn.mustFailRetry = 1 - } - } - sbc0 = &sandboxConn{onConnUse: onConnUse} - sbc1 = &sandboxConn{onConnUse: onConnUse} - sbc2 = &sandboxConn{onConnUse: onConnUse} - sbc3 := &sandboxConn{} - s.MapTestConn("0", sbc0) - s.MapTestConn("0", sbc1) - s.MapTestConn("0", sbc2) - countGetEndPoints := 0 - onGetEndPoints := func(st *sandboxTopo) { - if countGetEndPoints == 1 { - s.MapTestConn("0", sbc3) - s.DeleteTestConn("0", firstConn) - } - countGetEndPoints++ - } - sdc = NewShardConn(context.Background(), &sandboxTopo{callbackGetEndPoints: onGetEndPoints}, "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - timeStart = time.Now() - sdc.Execute(context.Background(), "query", nil, 0) - timeDuration = time.Now().Sub(timeStart) - if timeDuration >= retryDelay { - t.Errorf("want no delay, got %v", timeDuration) - } - if execCount := firstConn.ExecCount.Get(); execCount != 1 { - t.Errorf("want 1, got %v", execCount) - } - var totalExecCount int64 - for _, conn := range s.TestConns["0"] { - totalExecCount += conn.(*sandboxConn).ExecCount.Get() - } - if totalExecCount != 1 { - t.Errorf("want 1, got %v", totalExecCount) - } - if epCount := s.EndPointCounter.Get(); epCount != 2 { - t.Errorf("want 2, got %v", epCount) - } - - // case 7: resolve 3 bad endpoints with execution failed - // upon resolve, 2nd bad endpoint changed address (once only) but still fails on execution - // -> should only use the 1st endpoint after all other endpoints are tried out - s.Reset() - var secondConn *sandboxConn - countConnUse = 0 - onConnUse = func(conn *sandboxConn) { - if countConnUse == 0 { - firstConn = conn - } else if countConnUse == 1 { - secondConn = conn - } - countConnUse++ - } - sbc0 = &sandboxConn{mustFailRetry: 1, onConnUse: onConnUse} - sbc1 = &sandboxConn{mustFailRetry: 1, onConnUse: onConnUse} - sbc2 = &sandboxConn{mustFailRetry: 1, onConnUse: onConnUse} - sbc3 = &sandboxConn{mustFailRetry: 1} - s.MapTestConn("0", sbc0) - s.MapTestConn("0", sbc1) - s.MapTestConn("0", sbc2) - countGetEndPoints = 0 - onGetEndPoints = func(st *sandboxTopo) { - if countGetEndPoints == 2 { - s.MapTestConn("0", sbc3) - s.DeleteTestConn("0", secondConn) - } - countGetEndPoints++ - } - sdc = NewShardConn(context.Background(), &sandboxTopo{callbackGetEndPoints: onGetEndPoints}, "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - timeStart = time.Now() - sdc.Execute(context.Background(), "query", nil, 0) - timeDuration = time.Now().Sub(timeStart) - if timeDuration < retryDelay { - t.Errorf("want no spam delay %v, got %v", retryDelay, timeDuration) - } - if timeDuration > retryDelay*2 { - t.Errorf("want instant resolve %v, got %v", retryDelay, timeDuration) - } - if execCount := secondConn.ExecCount.Get(); execCount != 1 { - t.Errorf("want 1, got %v", execCount) - } - if execCount := firstConn.ExecCount.Get(); execCount != 2 { - t.Errorf("want 2, got %v", execCount) - } - for _, conn := range s.TestConns["0"] { - if execCount := conn.(*sandboxConn).ExecCount.Get(); conn != firstConn && execCount != 1 { - t.Errorf("want 1, got %v", execCount) - } - } - if epCount := s.EndPointCounter.Get(); epCount != 6 { - t.Errorf("want 6, got %v", epCount) - } - - // case 8: resolve 3 bad endpoints with execution failed, - // after resolve, all endpoints are valid on new addresses - // -> random use an endpoint without delay - s.Reset() - firstConn = nil - onConnUse = func(conn *sandboxConn) { - if firstConn == nil { - firstConn = conn - } - } - sbc0 = &sandboxConn{mustFailRetry: 1, onConnUse: onConnUse} - sbc1 = &sandboxConn{mustFailRetry: 1, onConnUse: onConnUse} - sbc2 = &sandboxConn{mustFailRetry: 1, onConnUse: onConnUse} - sbc3 = &sandboxConn{} - sbc4 := &sandboxConn{} - sbc5 := &sandboxConn{} - s.MapTestConn("0", sbc0) - s.MapTestConn("0", sbc1) - s.MapTestConn("0", sbc2) - countGetEndPoints = 0 - onGetEndPoints = func(st *sandboxTopo) { - if countGetEndPoints == 1 { - s.MapTestConn("0", sbc3) - s.MapTestConn("0", sbc4) - s.MapTestConn("0", sbc5) - s.DeleteTestConn("0", sbc0) - s.DeleteTestConn("0", sbc1) - s.DeleteTestConn("0", sbc2) - } - countGetEndPoints++ - } - sdc = NewShardConn(context.Background(), &sandboxTopo{callbackGetEndPoints: onGetEndPoints}, "aa", "TestShardConnReconnect", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, connectTimings) - timeStart = time.Now() - sdc.Execute(context.Background(), "query", nil, 0) - timeDuration = time.Now().Sub(timeStart) - if timeDuration >= retryDelay { - t.Errorf("want no delay, got %v", timeDuration) - } - if execCount := firstConn.ExecCount.Get(); execCount != 1 { - t.Errorf("want 1, got %v", execCount) - } - for _, conn := range []*sandboxConn{sbc0, sbc1, sbc2} { - if execCount := conn.ExecCount.Get(); conn != firstConn && execCount != 0 { - t.Errorf("want 0, got %v", execCount) - } - } - if sum := sbc3.ExecCount.Get() + sbc4.ExecCount.Get() + sbc5.ExecCount.Get(); sum != 1 { - t.Errorf("want 1, got %v", sum) - } - if epCount := s.EndPointCounter.Get(); epCount != 2 { - t.Errorf("want 2, got %v", epCount) - } -} - -func TestReplicaShardConnLife(t *testing.T) { - // auto-reconnect for non-master - retryDelay := 10 * time.Millisecond - retryCount := 5 - s := createSandbox("TestReplicaShardConnLife") - sbc := &sandboxConn{} - s.MapTestConn("0", sbc) - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestReplicaShardConnLife", "0", topodatapb.TabletType_REPLICA, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, 10*time.Millisecond, connectTimings) - sdc.Execute(context.Background(), "query", nil, 0) - if s.DialCounter != 1 { - t.Errorf("DialCounter: %d, want 1", s.DialCounter) - } - time.Sleep(20 * time.Millisecond) - sdc.Execute(context.Background(), "query", nil, 0) - if s.DialCounter != 2 { - t.Errorf("DialCounter: %d, want 2", s.DialCounter) - } - sdc.Close() -} - -func TestMasterShardConnLife(t *testing.T) { - // Do not auto-reconnect for master - retryDelay := 10 * time.Millisecond - retryCount := 5 - s := createSandbox("TestMasterShardConnLife") - sbc := &sandboxConn{} - s.MapTestConn("0", sbc) - sdc := NewShardConn(context.Background(), new(sandboxTopo), "aa", "TestMasterShardConnLife", "0", topodatapb.TabletType_MASTER, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, 10*time.Millisecond, connectTimings) - sdc.Execute(context.Background(), "query", nil, 0) - if s.DialCounter != 1 { - t.Errorf("DialCounter: %d, want 1", s.DialCounter) - } - time.Sleep(20 * time.Millisecond) - sdc.Execute(context.Background(), "query", nil, 0) - if s.DialCounter != 1 { - t.Errorf("DialCounter: %d, want 1", s.DialCounter) - } - sdc.Close() -} diff --git a/go/vt/vtgate/shardgateway.go b/go/vt/vtgate/shardgateway.go deleted file mode 100644 index 6175e33e0a1..00000000000 --- a/go/vt/vtgate/shardgateway.go +++ /dev/null @@ -1,207 +0,0 @@ -// Copyright 2015, Google Inc. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -package vtgate - -import ( - "fmt" - "strings" - "sync" - "time" - - "golang.org/x/net/context" - - "github.com/youtube/vitess/go/sqltypes" - "github.com/youtube/vitess/go/stats" - "github.com/youtube/vitess/go/vt/concurrency" - "github.com/youtube/vitess/go/vt/discovery" - "github.com/youtube/vitess/go/vt/tabletserver/querytypes" - "github.com/youtube/vitess/go/vt/topo" - - querypb "github.com/youtube/vitess/go/vt/proto/query" - topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" -) - -const ( - gatewayImplementationShard = "shardgateway" -) - -func init() { - RegisterGatewayCreator(gatewayImplementationShard, createShardGateway) -} - -func createShardGateway(hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, cell string, retryDelay time.Duration, retryCount int, connTimeoutTotal, connTimeoutPerConn, connLife time.Duration, connTimings *stats.MultiTimings, _ []topodatapb.TabletType) Gateway { - return &shardGateway{ - toposerv: serv, - cell: cell, - retryDelay: retryDelay, - retryCount: retryCount, - connTimeoutTotal: connTimeoutTotal, - connTimeoutPerConn: connTimeoutPerConn, - connLife: connLife, - connTimings: connTimings, - shardConns: make(map[string]*ShardConn), - } -} - -// A Gateway is the query processing module for each shard. -type shardGateway struct { - toposerv topo.SrvTopoServer - cell string - retryDelay time.Duration - retryCount int - connTimeoutTotal time.Duration - connTimeoutPerConn time.Duration - connLife time.Duration - connTimings *stats.MultiTimings - - mu sync.RWMutex - shardConns map[string]*ShardConn -} - -// InitializeConnections pre-initializes connections for all shards. -// It also populates topology cache by accessing it. -// It is not necessary to call this function before serving queries, -// but it would reduce connection overhead when serving. -func (sg *shardGateway) InitializeConnections(ctx context.Context) error { - ksNames, err := sg.toposerv.GetSrvKeyspaceNames(ctx, sg.cell) - if err != nil { - return err - } - var wg sync.WaitGroup - var errRecorder concurrency.AllErrorRecorder - for _, ksName := range ksNames { - wg.Add(1) - go func(keyspace string) { - defer wg.Done() - // get SrvKeyspace for cell/keyspace - ks, err := sg.toposerv.GetSrvKeyspace(ctx, sg.cell, keyspace) - if err != nil { - errRecorder.RecordError(err) - return - } - // work on all shards of all serving tablet types - for _, ksPartition := range ks.Partitions { - tt := ksPartition.ServedType - for _, shard := range ksPartition.ShardReferences { - wg.Add(1) - go func(shardName string, tabletType topodatapb.TabletType) { - defer wg.Done() - err = sg.getConnection(ctx, keyspace, shardName, tabletType).Dial(ctx) - if err != nil { - errRecorder.RecordError(err) - return - } - }(shard.Name, tt) - } - } - }(ksName) - } - wg.Wait() - if errRecorder.HasErrors() { - return errRecorder.AggrError(AggregateVtGateErrors) - } - return nil -} - -// Execute executes the non-streaming query for the specified keyspace, shard, and tablet type. -func (sg *shardGateway) Execute(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType, query string, bindVars map[string]interface{}, transactionID int64) (*sqltypes.Result, error) { - return sg.getConnection(ctx, keyspace, shard, tabletType).Execute(ctx, query, bindVars, transactionID) -} - -// ExecuteBatch executes a group of queries for the specified keyspace, shard, and tablet type. -func (sg *shardGateway) ExecuteBatch(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType, queries []querytypes.BoundQuery, asTransaction bool, transactionID int64) ([]sqltypes.Result, error) { - return sg.getConnection(ctx, keyspace, shard, tabletType).ExecuteBatch(ctx, queries, asTransaction, transactionID) -} - -// StreamExecute executes a streaming query for the specified keyspace, shard, and tablet type. -func (sg *shardGateway) StreamExecute(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType, query string, bindVars map[string]interface{}) (sqltypes.ResultStream, error) { - return sg.getConnection(ctx, keyspace, shard, tabletType).StreamExecute(ctx, query, bindVars) -} - -// Begin starts a transaction for the specified keyspace, shard, and tablet type. -// It returns the transaction ID. -func (sg *shardGateway) Begin(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType) (int64, error) { - return sg.getConnection(ctx, keyspace, shard, tabletType).Begin(ctx) -} - -// Commit commits the current transaction for the specified keyspace, shard, and tablet type. -func (sg *shardGateway) Commit(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType, transactionID int64) error { - return sg.getConnection(ctx, keyspace, shard, tabletType).Commit(ctx, transactionID) -} - -// Rollback rolls back the current transaction for the specified keyspace, shard, and tablet type. -func (sg *shardGateway) Rollback(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType, transactionID int64) error { - return sg.getConnection(ctx, keyspace, shard, tabletType).Rollback(ctx, transactionID) -} - -// BeginExecute executes a begin and the non-streaming query for the -// specified keyspace, shard, and tablet type. -func (sg *shardGateway) BeginExecute(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType, query string, bindVars map[string]interface{}) (*sqltypes.Result, int64, error) { - return sg.getConnection(ctx, keyspace, shard, tabletType).BeginExecute(ctx, query, bindVars) -} - -// BeginExecuteBatch executes a begin and a group of queries for the -// specified keyspace, shard, and tablet type. -func (sg *shardGateway) BeginExecuteBatch(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType, queries []querytypes.BoundQuery, asTransaction bool) ([]sqltypes.Result, int64, error) { - return sg.getConnection(ctx, keyspace, shard, tabletType).BeginExecuteBatch(ctx, queries, asTransaction) -} - -// SplitQuery splits a query into sub-queries for the specified keyspace, shard, and tablet type. -func (sg *shardGateway) SplitQuery(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType, sql string, bindVars map[string]interface{}, splitColumn string, splitCount int64) ([]querytypes.QuerySplit, error) { - return sg.getConnection(ctx, keyspace, shard, tabletType).SplitQuery(ctx, sql, bindVars, splitColumn, splitCount) -} - -// SplitQuery splits a query into sub-queries for the specified keyspace, shard, and tablet type. -// TODO(erez): Rename to SplitQuery after the migration to SplitQuery V2 is done. -func (sg *shardGateway) SplitQueryV2( - ctx context.Context, - keyspace string, - shard string, - tabletType topodatapb.TabletType, - sql string, - bindVars map[string]interface{}, - splitColumns []string, - splitCount int64, - numRowsPerQueryPart int64, - algorithm querypb.SplitQueryRequest_Algorithm) ([]querytypes.QuerySplit, error) { - return sg.getConnection(ctx, keyspace, shard, tabletType).SplitQueryV2( - ctx, sql, bindVars, splitColumns, splitCount, numRowsPerQueryPart, algorithm) -} - -// Close shuts down the underlying connections. -func (sg *shardGateway) Close(ctx context.Context) error { - sg.mu.Lock() - defer sg.mu.Unlock() - for _, v := range sg.shardConns { - v.Close() - } - sg.shardConns = make(map[string]*ShardConn) - return nil -} - -// CacheStatus returns a list of GatewayEndPointCacheStatus per endpoint. -func (sg *shardGateway) CacheStatus() GatewayEndPointCacheStatusList { - return nil -} - -func (sg *shardGateway) getConnection(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType) *ShardConn { - key := fmt.Sprintf("%s.%s.%s", keyspace, shard, strings.ToLower(tabletType.String())) - sg.mu.RLock() - sdc, ok := sg.shardConns[key] - sg.mu.RUnlock() - if ok { - return sdc - } - - sg.mu.Lock() - defer sg.mu.Unlock() - sdc, ok = sg.shardConns[key] - if ok { - return sdc - } - sdc = NewShardConn(ctx, sg.toposerv, sg.cell, keyspace, shard, tabletType, sg.retryDelay, sg.retryCount, sg.connTimeoutTotal, sg.connTimeoutPerConn, sg.connLife, sg.connTimings) - sg.shardConns[key] = sdc - return sdc -} diff --git a/go/vt/vtgate/vertical_split_test.go b/go/vt/vtgate/vertical_split_test.go deleted file mode 100644 index 53cff3cbdba..00000000000 --- a/go/vt/vtgate/vertical_split_test.go +++ /dev/null @@ -1,144 +0,0 @@ -// Copyright 2012, Google Inc. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -package vtgate - -import ( - "testing" - "time" - - "github.com/youtube/vitess/go/sqltypes" - "github.com/youtube/vitess/go/vt/topo" - "golang.org/x/net/context" - - querypb "github.com/youtube/vitess/go/vt/proto/query" - topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" - vtgatepb "github.com/youtube/vitess/go/vt/proto/vtgate" -) - -// This file uses the sandbox_test framework. - -func TestExecuteKeyspaceAlias(t *testing.T) { - testVerticalSplitGeneric(t, false, func(shards []string) (*sqltypes.Result, error) { - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", 1*time.Millisecond, 3, 20*time.Millisecond, 10*time.Millisecond, 24*time.Hour, nil, "") - return stc.Execute(context.Background(), "query", nil, KsTestUnshardedServedFrom, shards, topodatapb.TabletType_RDONLY, nil, false) - }) -} - -func TestBatchExecuteKeyspaceAlias(t *testing.T) { - testVerticalSplitGeneric(t, false, func(shards []string) (*sqltypes.Result, error) { - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", 1*time.Millisecond, 3, 20*time.Millisecond, 10*time.Millisecond, 24*time.Hour, nil, "") - queries := []*vtgatepb.BoundShardQuery{{ - Query: &querypb.BoundQuery{ - Sql: "query", - BindVariables: nil, - }, - Keyspace: KsTestUnshardedServedFrom, - Shards: shards, - }} - scatterRequest, err := boundShardQueriesToScatterBatchRequest(queries) - if err != nil { - return nil, err - } - qrs, err := stc.ExecuteBatch(context.Background(), scatterRequest, topodatapb.TabletType_RDONLY, false, nil) - if err != nil { - return nil, err - } - return &qrs[0], err - }) -} - -func TestStreamExecuteKeyspaceAlias(t *testing.T) { - testVerticalSplitGeneric(t, true, func(shards []string) (*sqltypes.Result, error) { - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", 1*time.Millisecond, 3, 20*time.Millisecond, 10*time.Millisecond, 24*time.Hour, nil, "") - qr := new(sqltypes.Result) - err := stc.StreamExecute(context.Background(), "query", nil, KsTestUnshardedServedFrom, shards, topodatapb.TabletType_RDONLY, func(r *sqltypes.Result) error { - appendResult(qr, r) - return nil - }) - return qr, err - }) -} - -func TestInTransactionKeyspaceAlias(t *testing.T) { - s := createSandbox(KsTestUnshardedServedFrom) - sbc := &sandboxConn{mustFailRetry: 3} - s.MapTestConn("0", sbc) - - stc := NewScatterConn(nil, topo.Server{}, new(sandboxTopo), "", "aa", 1*time.Millisecond, 3, 20*time.Millisecond, 10*time.Millisecond, 24*time.Hour, nil, "") - session := NewSafeSession(&vtgatepb.Session{ - InTransaction: true, - ShardSessions: []*vtgatepb.Session_ShardSession{{ - Target: &querypb.Target{ - Keyspace: KsTestUnshardedServedFrom, - Shard: "0", - TabletType: topodatapb.TabletType_MASTER, - }, - TransactionId: 1, - }}, - }) - _, err := stc.Execute(context.Background(), "query", nil, KsTestUnshardedServedFrom, []string{"0"}, topodatapb.TabletType_MASTER, session, false) - want := "shard, host: TestUnshardedServedFrom.0.master, host:\"0\" port_map: , retry: err" - if err == nil || err.Error() != want { - t.Errorf("want '%v', got '%v'", want, err) - } - // Ensure that we tried once, no retry here - // since we are in a transaction. - if execCount := sbc.ExecCount.Get(); execCount != 1 { - t.Errorf("want 1, got %v", execCount) - } -} - -func testVerticalSplitGeneric(t *testing.T, isStreaming bool, f func(shards []string) (*sqltypes.Result, error)) { - // Retry Error, for keyspace that is redirected should succeed. - s := createSandbox(KsTestUnshardedServedFrom) - sbc := &sandboxConn{mustFailRetry: 1} - s.MapTestConn("0", sbc) - _, err := f([]string{"0"}) - if err != nil { - t.Errorf("want nil, got %v", err) - } - // Ensure that we tried 2 times, 1 for retry and 1 for redirect. - if execCount := sbc.ExecCount.Get(); execCount != 2 { - t.Errorf("want 2, got %v", execCount) - } - - // Fatal Error, for keyspace that is redirected should succeed. - s.Reset() - sbc = &sandboxConn{mustFailFatal: 1} - s.MapTestConn("0", sbc) - _, err = f([]string{"0"}) - if isStreaming { - want := "shard, host: TestUnshardedServedFrom.0.rdonly, host:\"0\" port_map: , fatal: err" - if err == nil || err.Error() != want { - t.Errorf("want '%v', got '%v'", want, err) - } - // Ensure that we tried only once. - if execCount := sbc.ExecCount.Get(); execCount != 1 { - t.Errorf("want 1, got %v", execCount) - } - } else { - if err != nil { - t.Errorf("want nil, got %v", err) - } - // Ensure that we tried 2 times, 1 for retry and 1 for redirect. - if execCount := sbc.ExecCount.Get(); execCount != 2 { - t.Errorf("want 2, got %v", execCount) - } - } - - // Error, for keyspace that is redirected should succeed. - s.Reset() - sbc = &sandboxConn{mustFailServer: 3} - s.MapTestConn("0", sbc) - _, err = f([]string{"0"}) - want := "shard, host: TestUnshardedServedFrom.0.rdonly, host:\"0\" port_map: , error: err" - if err == nil || err.Error() != want { - t.Errorf("want '%v', got '%v'", want, err) - } - // Ensure that we tried once, no retry here. - if execCount := sbc.ExecCount.Get(); execCount != 1 { - t.Errorf("want 1, got %v", execCount) - } -} diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index 6846a9648c7..f5eef81f8de 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -95,12 +95,12 @@ type RegisterVTGate func(vtgateservice.VTGateService) var RegisterVTGates []RegisterVTGate // Init initializes VTGate server. -func Init(ctx context.Context, hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, cell string, retryDelay time.Duration, retryCount int, connTimeoutTotal, connTimeoutPerConn, connLife time.Duration, tabletTypesToWait []topodatapb.TabletType, maxInFlight int, testGateway string) *VTGate { +func Init(ctx context.Context, hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, cell string, retryCount int, tabletTypesToWait []topodatapb.TabletType, maxInFlight int) *VTGate { if rpcVTGate != nil { log.Fatalf("VTGate already initialized") } rpcVTGate = &VTGate{ - resolver: NewResolver(hc, topoServer, serv, "VttabletCall", cell, retryDelay, retryCount, connTimeoutTotal, connTimeoutPerConn, connLife, tabletTypesToWait, testGateway), + resolver: NewResolver(hc, topoServer, serv, "VttabletCall", cell, retryCount, tabletTypesToWait), timings: stats.NewMultiTimings("VtgateApi", []string{"Operation", "Keyspace", "DbType"}), rowsReturned: stats.NewMultiCounters("VtgateApiRowsReturned", []string{"Operation", "Keyspace", "DbType"}), @@ -141,22 +141,6 @@ func Init(ctx context.Context, hc discovery.HealthCheck, topoServer topo.Server, return rpcVTGate } -// InitializeConnections pre-initializes VTGate by connecting to vttablets of all keyspace/shard/type. -// It is not necessary to call this function before serving queries, -// but it would reduce connection overhead when serving. -func (vtg *VTGate) InitializeConnections(ctx context.Context) (err error) { - defer vtg.HandlePanic(&err) - - log.Infof("Initialize VTTablet connections") - err = vtg.resolver.InitializeConnections(ctx) - if err != nil { - log.Errorf("failed to initialize connections: %v", err) - return err - } - log.Infof("Initialize VTTablet connections completed") - return nil -} - // Execute executes a non-streaming query by routing based on the values in the query. func (vtg *VTGate) Execute(ctx context.Context, sql string, bindVariables map[string]interface{}, keyspace string, tabletType topodatapb.TabletType, session *vtgatepb.Session, notInTransaction bool) (*sqltypes.Result, error) { startTime := time.Now() @@ -800,7 +784,7 @@ func isErrorCausedByVTGate(err error) bool { switch e := e.(type) { case *ScatterConnError: errQueue = append(errQueue, e.Errs...) - case *ShardConnError: + case *ShardError: errQueue = append(errQueue, e.Err) case tabletconn.OperationalError: // tabletconn.Cancelled errors are due to client behavior, not VTGate errors. diff --git a/go/vt/vtgate/vtgate_test.go b/go/vt/vtgate/vtgate_test.go index 3e1c9a4ae29..88a8a0d60d7 100644 --- a/go/vt/vtgate/vtgate_test.go +++ b/go/vt/vtgate/vtgate_test.go @@ -12,7 +12,6 @@ import ( "sort" "strings" "testing" - "time" "github.com/youtube/vitess/go/sqltypes" "github.com/youtube/vitess/go/vt/key" @@ -28,6 +27,8 @@ import ( // This file uses the sandbox_test framework. +var hcVTGateTest *fakeHealthCheck + func init() { getSandbox(KsTestUnsharded).VSchema = ` { @@ -37,13 +38,15 @@ func init() { } } ` - Init(context.Background(), nil, topo.Server{}, new(sandboxTopo), "aa", 1*time.Second, 10, 2*time.Millisecond, 1*time.Millisecond, 24*time.Hour, nil, 0, "") + hcVTGateTest = newFakeHealthCheck() + Init(context.Background(), hcVTGateTest, topo.Server{}, new(sandboxTopo), "aa", 10, nil, 0) } func TestVTGateExecute(t *testing.T) { - sandbox := createSandbox(KsTestUnsharded) + createSandbox(KsTestUnsharded) sbc := &sandboxConn{} - sandbox.MapTestConn("0", sbc) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, KsTestUnsharded, "0", topodatapb.TabletType_MASTER, true, 1, nil, sbc) qr, err := rpcVTGate.Execute(context.Background(), "select id from t1", nil, @@ -101,9 +104,10 @@ func TestVTGateExecute(t *testing.T) { } func TestVTGateExecuteWithKeyspace(t *testing.T) { - sandbox := createSandbox(KsTestUnsharded) + createSandbox(KsTestUnsharded) sbc := &sandboxConn{} - sandbox.MapTestConn("0", sbc) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, KsTestUnsharded, "0", topodatapb.TabletType_MASTER, true, 1, nil, sbc) qr, err := rpcVTGate.Execute(context.Background(), "select id from none", nil, @@ -131,14 +135,17 @@ func TestVTGateExecuteWithKeyspace(t *testing.T) { } func TestVTGateExecuteShards(t *testing.T) { - sandbox := createSandbox("TestVTGateExecuteShards") + ks := "TestVTGateExecuteShards" + shard := "0" + createSandbox(ks) sbc := &sandboxConn{} - sandbox.MapTestConn("0", sbc) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, ks, shard, topodatapb.TabletType_REPLICA, true, 1, nil, sbc) qr, err := rpcVTGate.ExecuteShards(context.Background(), "query", nil, - "TestVTGateExecuteShards", - []string{"0"}, + ks, + []string{shard}, topodatapb.TabletType_REPLICA, nil, false) @@ -156,8 +163,8 @@ func TestVTGateExecuteShards(t *testing.T) { rpcVTGate.ExecuteShards(context.Background(), "query", nil, - "TestVTGateExecuteShards", - []string{"0"}, + ks, + []string{shard}, topodatapb.TabletType_REPLICA, session, false) @@ -165,8 +172,8 @@ func TestVTGateExecuteShards(t *testing.T) { InTransaction: true, ShardSessions: []*vtgatepb.Session_ShardSession{{ Target: &querypb.Target{ - Keyspace: "TestVTGateExecuteShards", - Shard: "0", + Keyspace: ks, + Shard: shard, TabletType: topodatapb.TabletType_REPLICA, }, TransactionId: 1, @@ -185,8 +192,8 @@ func TestVTGateExecuteShards(t *testing.T) { rpcVTGate.ExecuteShards(context.Background(), "query", nil, - "TestVTGateExecuteShards", - []string{"0"}, + ks, + []string{shard}, topodatapb.TabletType_REPLICA, session, false) @@ -201,16 +208,20 @@ func TestVTGateExecuteShards(t *testing.T) { } func TestVTGateExecuteKeyspaceIds(t *testing.T) { - s := createSandbox("TestVTGateExecuteKeyspaceIds") + ks := "TestVTGateExecuteKeyspaceIds" + shard1 := "-20" + shard2 := "20-40" + createSandbox(ks) sbc1 := &sandboxConn{} sbc2 := &sandboxConn{} - s.MapTestConn("-20", sbc1) - s.MapTestConn("20-40", sbc2) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, ks, shard1, topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1002, ks, shard2, topodatapb.TabletType_MASTER, true, 1, nil, sbc2) // Test for successful execution qr, err := rpcVTGate.ExecuteKeyspaceIds(context.Background(), "query", nil, - "TestVTGateExecuteKeyspaceIds", + ks, [][]byte{{0x10}}, topodatapb.TabletType_MASTER, nil, @@ -232,7 +243,7 @@ func TestVTGateExecuteKeyspaceIds(t *testing.T) { rpcVTGate.ExecuteKeyspaceIds(context.Background(), "query", nil, - "TestVTGateExecuteKeyspaceIds", + ks, [][]byte{{0x10}}, topodatapb.TabletType_MASTER, session, @@ -241,8 +252,8 @@ func TestVTGateExecuteKeyspaceIds(t *testing.T) { InTransaction: true, ShardSessions: []*vtgatepb.Session_ShardSession{{ Target: &querypb.Target{ - Keyspace: "TestVTGateExecuteKeyspaceIds", - Shard: "-20", + Keyspace: ks, + Shard: shard1, TabletType: topodatapb.TabletType_MASTER, }, TransactionId: 1, @@ -259,7 +270,7 @@ func TestVTGateExecuteKeyspaceIds(t *testing.T) { qr, err = rpcVTGate.ExecuteKeyspaceIds(context.Background(), "query", nil, - "TestVTGateExecuteKeyspaceIds", + ks, [][]byte{{0x10}, {0x30}}, topodatapb.TabletType_MASTER, session, @@ -270,16 +281,20 @@ func TestVTGateExecuteKeyspaceIds(t *testing.T) { } func TestVTGateExecuteKeyRanges(t *testing.T) { - s := createSandbox("TestVTGateExecuteKeyRanges") + ks := "TestVTGateExecuteKeyRanges" + shard1 := "-20" + shard2 := "20-40" + createSandbox(ks) sbc1 := &sandboxConn{} sbc2 := &sandboxConn{} - s.MapTestConn("-20", sbc1) - s.MapTestConn("20-40", sbc2) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, ks, shard1, topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1002, ks, shard2, topodatapb.TabletType_MASTER, true, 1, nil, sbc2) // Test for successful execution qr, err := rpcVTGate.ExecuteKeyRanges(context.Background(), "query", nil, - "TestVTGateExecuteKeyRanges", + ks, []*topodatapb.KeyRange{{End: []byte{0x20}}}, topodatapb.TabletType_MASTER, nil, @@ -301,7 +316,7 @@ func TestVTGateExecuteKeyRanges(t *testing.T) { qr, err = rpcVTGate.ExecuteKeyRanges(context.Background(), "query", nil, - "TestVTGateExecuteKeyRanges", + ks, []*topodatapb.KeyRange{{End: []byte{0x20}}}, topodatapb.TabletType_MASTER, session, @@ -313,8 +328,8 @@ func TestVTGateExecuteKeyRanges(t *testing.T) { InTransaction: true, ShardSessions: []*vtgatepb.Session_ShardSession{{ Target: &querypb.Target{ - Keyspace: "TestVTGateExecuteKeyRanges", - Shard: "-20", + Keyspace: ks, + Shard: shard1, TabletType: topodatapb.TabletType_MASTER, }, TransactionId: 1, @@ -330,7 +345,7 @@ func TestVTGateExecuteKeyRanges(t *testing.T) { // Test for multiple shards qr, err = rpcVTGate.ExecuteKeyRanges(context.Background(), "query", nil, - "TestVTGateExecuteKeyRanges", + ks, []*topodatapb.KeyRange{{Start: []byte{0x10}, End: []byte{0x30}}}, topodatapb.TabletType_MASTER, nil, @@ -341,16 +356,20 @@ func TestVTGateExecuteKeyRanges(t *testing.T) { } func TestVTGateExecuteEntityIds(t *testing.T) { - s := createSandbox("TestVTGateExecuteEntityIds") + ks := "TestVTGateExecuteEntityIds" + shard1 := "-20" + shard2 := "20-40" + createSandbox(ks) sbc1 := &sandboxConn{} sbc2 := &sandboxConn{} - s.MapTestConn("-20", sbc1) - s.MapTestConn("20-40", sbc2) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, ks, shard1, topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1002, ks, shard2, topodatapb.TabletType_MASTER, true, 1, nil, sbc2) // Test for successful execution qr, err := rpcVTGate.ExecuteEntityIds(context.Background(), "query", nil, - "TestVTGateExecuteEntityIds", + ks, "kid", []*vtgatepb.ExecuteEntityIdsRequest_EntityId{ { @@ -379,7 +398,7 @@ func TestVTGateExecuteEntityIds(t *testing.T) { rpcVTGate.ExecuteEntityIds(context.Background(), "query", nil, - "TestVTGateExecuteEntityIds", + ks, "kid", []*vtgatepb.ExecuteEntityIdsRequest_EntityId{ { @@ -395,8 +414,8 @@ func TestVTGateExecuteEntityIds(t *testing.T) { InTransaction: true, ShardSessions: []*vtgatepb.Session_ShardSession{{ Target: &querypb.Target{ - Keyspace: "TestVTGateExecuteEntityIds", - Shard: "-20", + Keyspace: ks, + Shard: shard1, TabletType: topodatapb.TabletType_MASTER, }, TransactionId: 1, @@ -413,7 +432,7 @@ func TestVTGateExecuteEntityIds(t *testing.T) { // Test for multiple shards qr, err = rpcVTGate.ExecuteEntityIds(context.Background(), "query", nil, - "TestVTGateExecuteEntityIds", + ks, "kid", []*vtgatepb.ExecuteEntityIdsRequest_EntityId{ { @@ -436,24 +455,30 @@ func TestVTGateExecuteEntityIds(t *testing.T) { } func TestVTGateExecuteBatchShards(t *testing.T) { - s := createSandbox("TestVTGateExecuteBatchShards") - s.MapTestConn("-20", &sandboxConn{}) - s.MapTestConn("20-40", &sandboxConn{}) + ks := "TestVTGateExecuteBatchShards" + createSandbox(ks) + shard1 := "-20" + shard2 := "20-40" + sbc1 := &sandboxConn{} + sbc2 := &sandboxConn{} + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, ks, shard1, topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1002, ks, shard2, topodatapb.TabletType_MASTER, true, 1, nil, sbc2) qrl, err := rpcVTGate.ExecuteBatchShards(context.Background(), []*vtgatepb.BoundShardQuery{{ Query: &querypb.BoundQuery{ Sql: "query", BindVariables: nil, }, - Keyspace: "TestVTGateExecuteBatchShards", - Shards: []string{"-20", "20-40"}, + Keyspace: ks, + Shards: []string{shard1, shard2}, }, { Query: &querypb.BoundQuery{ Sql: "query", BindVariables: nil, }, - Keyspace: "TestVTGateExecuteBatchShards", - Shards: []string{"-20", "20-40"}, + Keyspace: ks, + Shards: []string{shard1, shard2}, }}, topodatapb.TabletType_MASTER, false, @@ -475,15 +500,15 @@ func TestVTGateExecuteBatchShards(t *testing.T) { Sql: "query", BindVariables: nil, }, - Keyspace: "TestVTGateExecuteBatchShards", - Shards: []string{"-20", "20-40"}, + Keyspace: ks, + Shards: []string{shard1, shard2}, }, { Query: &querypb.BoundQuery{ Sql: "query", BindVariables: nil, }, - Keyspace: "TestVTGateExecuteBatchShards", - Shards: []string{"-20", "20-40"}, + Keyspace: ks, + Shards: []string{shard1, shard2}, }}, topodatapb.TabletType_MASTER, false, @@ -494,9 +519,15 @@ func TestVTGateExecuteBatchShards(t *testing.T) { } func TestVTGateExecuteBatchKeyspaceIds(t *testing.T) { - s := createSandbox("TestVTGateExecuteBatchKeyspaceIds") - s.MapTestConn("-20", &sandboxConn{}) - s.MapTestConn("20-40", &sandboxConn{}) + ks := "TestVTGateExecuteBatchKeyspaceIds" + shard1 := "-20" + shard2 := "20-40" + createSandbox(ks) + sbc1 := &sandboxConn{} + sbc2 := &sandboxConn{} + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, ks, shard1, topodatapb.TabletType_MASTER, true, 1, nil, sbc1) + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1002, ks, shard2, topodatapb.TabletType_MASTER, true, 1, nil, sbc2) kid10 := []byte{0x10} kid30 := []byte{0x30} qrl, err := rpcVTGate.ExecuteBatchKeyspaceIds(context.Background(), @@ -505,14 +536,14 @@ func TestVTGateExecuteBatchKeyspaceIds(t *testing.T) { Sql: "query", BindVariables: nil, }, - Keyspace: "TestVTGateExecuteBatchKeyspaceIds", + Keyspace: ks, KeyspaceIds: [][]byte{kid10, kid30}, }, { Query: &querypb.BoundQuery{ Sql: "query", BindVariables: nil, }, - Keyspace: "TestVTGateExecuteBatchKeyspaceIds", + Keyspace: ks, KeyspaceIds: [][]byte{kid10, kid30}, }}, topodatapb.TabletType_MASTER, @@ -535,14 +566,14 @@ func TestVTGateExecuteBatchKeyspaceIds(t *testing.T) { Sql: "query", BindVariables: nil, }, - Keyspace: "TestVTGateExecuteBatchKeyspaceIds", + Keyspace: ks, KeyspaceIds: [][]byte{kid10, kid30}, }, { Query: &querypb.BoundQuery{ Sql: "query", BindVariables: nil, }, - Keyspace: "TestVTGateExecuteBatchKeyspaceIds", + Keyspace: ks, KeyspaceIds: [][]byte{kid10, kid30}, }}, topodatapb.TabletType_MASTER, @@ -554,9 +585,12 @@ func TestVTGateExecuteBatchKeyspaceIds(t *testing.T) { } func TestVTGateStreamExecute(t *testing.T) { - sandbox := createSandbox(KsTestUnsharded) + ks := KsTestUnsharded + shard := "0" + createSandbox(ks) sbc := &sandboxConn{} - sandbox.MapTestConn("0", sbc) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, ks, shard, topodatapb.TabletType_MASTER, true, 1, nil, sbc) var qrs []*sqltypes.Result err := rpcVTGate.StreamExecute(context.Background(), "select id from t1", @@ -577,17 +611,21 @@ func TestVTGateStreamExecute(t *testing.T) { } func TestVTGateStreamExecuteKeyspaceIds(t *testing.T) { - s := createSandbox("TestVTGateStreamExecuteKeyspaceIds") + ks := "TestVTGateStreamExecuteKeyspaceIds" + shard1 := "-20" + shard2 := "20-40" + createSandbox(ks) sbc := &sandboxConn{} - s.MapTestConn("-20", sbc) sbc1 := &sandboxConn{} - s.MapTestConn("20-40", sbc1) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, ks, shard1, topodatapb.TabletType_MASTER, true, 1, nil, sbc) + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1002, ks, shard2, topodatapb.TabletType_MASTER, true, 1, nil, sbc1) // Test for successful execution var qrs []*sqltypes.Result err := rpcVTGate.StreamExecuteKeyspaceIds(context.Background(), "query", nil, - "TestVTGateStreamExecuteKeyspaceIds", + ks, [][]byte{{0x10}}, topodatapb.TabletType_MASTER, func(r *sqltypes.Result) error { @@ -607,7 +645,7 @@ func TestVTGateStreamExecuteKeyspaceIds(t *testing.T) { err = rpcVTGate.StreamExecuteKeyspaceIds(context.Background(), "query", nil, - "TestVTGateStreamExecuteKeyspaceIds", + ks, [][]byte{{0x10}, {0x15}}, topodatapb.TabletType_MASTER, func(r *sqltypes.Result) error { @@ -625,7 +663,7 @@ func TestVTGateStreamExecuteKeyspaceIds(t *testing.T) { err = rpcVTGate.StreamExecuteKeyspaceIds(context.Background(), "query", nil, - "TestVTGateStreamExecuteKeyspaceIds", + ks, [][]byte{{0x10}, {0x30}}, topodatapb.TabletType_MASTER, func(r *sqltypes.Result) error { @@ -638,17 +676,21 @@ func TestVTGateStreamExecuteKeyspaceIds(t *testing.T) { } func TestVTGateStreamExecuteKeyRanges(t *testing.T) { - s := createSandbox("TestVTGateStreamExecuteKeyRanges") + ks := "TestVTGateStreamExecuteKeyRanges" + shard1 := "-20" + shard2 := "20-40" + createSandbox(ks) sbc := &sandboxConn{} - s.MapTestConn("-20", sbc) sbc1 := &sandboxConn{} - s.MapTestConn("20-40", sbc1) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, ks, shard1, topodatapb.TabletType_MASTER, true, 1, nil, sbc) + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1002, ks, shard2, topodatapb.TabletType_MASTER, true, 1, nil, sbc1) // Test for successful execution var qrs []*sqltypes.Result err := rpcVTGate.StreamExecuteKeyRanges(context.Background(), "query", nil, - "TestVTGateStreamExecuteKeyRanges", + ks, []*topodatapb.KeyRange{{End: []byte{0x20}}}, topodatapb.TabletType_MASTER, func(r *sqltypes.Result) error { @@ -667,7 +709,7 @@ func TestVTGateStreamExecuteKeyRanges(t *testing.T) { err = rpcVTGate.StreamExecuteKeyRanges(context.Background(), "query", nil, - "TestVTGateStreamExecuteKeyRanges", + ks, []*topodatapb.KeyRange{{Start: []byte{0x10}, End: []byte{0x40}}}, topodatapb.TabletType_MASTER, func(r *sqltypes.Result) error { @@ -680,16 +722,19 @@ func TestVTGateStreamExecuteKeyRanges(t *testing.T) { } func TestVTGateStreamExecuteShards(t *testing.T) { - s := createSandbox("TestVTGateStreamExecuteShards") + ks := "TestVTGateStreamExecuteShards" + shard := "0" + createSandbox(ks) sbc := &sandboxConn{} - s.MapTestConn("0", sbc) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, ks, shard, topodatapb.TabletType_MASTER, true, 1, nil, sbc) // Test for successful execution var qrs []*sqltypes.Result err := rpcVTGate.StreamExecuteShards(context.Background(), "query", nil, - "TestVTGateStreamExecuteShards", - []string{"0"}, + ks, + []string{shard}, topodatapb.TabletType_MASTER, func(r *sqltypes.Result) error { qrs = append(qrs, r) @@ -707,9 +752,13 @@ func TestVTGateStreamExecuteShards(t *testing.T) { func TestVTGateSplitQuery(t *testing.T) { keyspace := "TestVTGateSplitQuery" keyranges, _ := key.ParseShardingSpec(DefaultShardSpec) - s := createSandbox(keyspace) + createSandbox(keyspace) + hcVTGateTest.Reset() + port := int32(1001) for _, kr := range keyranges { - s.MapTestConn(key.KeyRangeString(kr), &sandboxConn{}) + sbc := &sandboxConn{} + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", port, keyspace, key.KeyRangeString(kr), topodatapb.TabletType_RDONLY, true, 1, nil, sbc) + port++ } sql := "select col1, col2 from table" splitCount := 24 @@ -766,9 +815,13 @@ func TestVTGateSplitQueryV2Sharded(t *testing.T) { if err != nil { t.Fatalf("got: %v, want: nil", err) } - s := createSandbox(keyspace) + createSandbox(keyspace) + hcVTGateTest.Reset() + port := int32(1001) for _, kr := range keyranges { - s.MapTestConn(key.KeyRangeString(kr), &sandboxConn{}) + sbc := &sandboxConn{} + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", port, keyspace, key.KeyRangeString(kr), topodatapb.TabletType_RDONLY, true, 1, nil, sbc) + port++ } sql := "select col1, col2 from table" bindVars := map[string]interface{}{"bv1": nil} @@ -843,11 +896,10 @@ func TestVTGateSplitQueryV2Sharded(t *testing.T) { func TestVTGateSplitQueryV2Unsharded(t *testing.T) { keyspace := KsTestUnsharded - s := getSandbox(keyspace) - if s == nil { - t.Fatalf("Can't find unsharded sandbox.") - } - s.MapTestConn("0", &sandboxConn{}) + createSandbox(keyspace) + sbc := &sandboxConn{} + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "1.1.1.1", 1001, keyspace, "0", topodatapb.TabletType_RDONLY, true, 1, nil, sbc) sql := "select col1, col2 from table" bindVars := map[string]interface{}{"bv1": nil} splitColumns := []string{"sc1", "sc2"} @@ -918,9 +970,9 @@ func TestIsErrorCausedByVTGate(t *testing.T) { Code: tabletconn.ERR_RETRY, Err: "vttablet: retry: error message", } - shardConnUnknownErr := &ShardConnError{Err: unknownError} - shardConnServerErr := &ShardConnError{Err: serverError} - shardConnCancelledErr := &ShardConnError{Err: tabletconn.Cancelled} + shardConnUnknownErr := &ShardError{Err: unknownError} + shardConnServerErr := &ShardError{Err: serverError} + shardConnCancelledErr := &ShardError{Err: tabletconn.Cancelled} scatterConnErrAllUnknownErrs := &ScatterConnError{ Errs: []error{unknownError, unknownError, unknownError}, @@ -1213,9 +1265,10 @@ func TestAnnotatingExecuteBatchShards(t *testing.T) { // a previous call to this method. func setUpSandboxWithTwoShards(keyspace string) (string, []*sandboxConn) { shards := []*sandboxConn{{}, {}} - aSandbox := createSandbox(keyspace) - aSandbox.MapTestConn("-20", shards[0]) - aSandbox.MapTestConn("20-40", shards[1]) + createSandbox(keyspace) + hcVTGateTest.Reset() + hcVTGateTest.addTestEndPoint("aa", "-20", 1, keyspace, "-20", topodatapb.TabletType_MASTER, true, 1, nil, shards[0]) + hcVTGateTest.addTestEndPoint("aa", "20-40", 1, keyspace, "20-40", topodatapb.TabletType_MASTER, true, 1, nil, shards[1]) return keyspace, shards } diff --git a/py/vttest/environment.py b/py/vttest/environment.py index aeaf0d5b740..082391af2c6 100644 --- a/py/vttest/environment.py +++ b/py/vttest/environment.py @@ -60,6 +60,7 @@ def extra_vtcombo_parameters(): 'grpc-vtgateservice', 'grpc-vtctl', ]), + '-cells_to_watch', 'test', ] From b5196b6a96c2f95f3450b5aef5ad72b547f41181 Mon Sep 17 00:00:00 2001 From: Liang Guo Date: Thu, 5 May 2016 14:19:16 -0700 Subject: [PATCH 2/4] Fix end2end vttest. --- go/cmd/vtcombo/tablet_map.go | 10 ++++++++++ go/vt/tabletmanager/healthcheck.go | 2 +- go/vt/vtgate/discoverygateway.go | 10 +++++++++- 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/go/cmd/vtcombo/tablet_map.go b/go/cmd/vtcombo/tablet_map.go index fb5bac8f39f..cd665c3b045 100644 --- a/go/cmd/vtcombo/tablet_map.go +++ b/go/cmd/vtcombo/tablet_map.go @@ -187,6 +187,16 @@ func initTabletMap(ts topo.Server, topology string, mysqld mysqlctl.MysqlDaemon, return &internalTabletManagerClient{} }) *tmclient.TabletManagerProtocol = "internal" + + // run healthcheck on all vttablets + tmc := tmclient.NewTabletManagerClient() + for _, tablet := range tabletMap { + tabletInfo, err := ts.GetTablet(ctx, tablet.agent.TabletAlias) + if err != nil { + log.Fatalf("cannot find tablet: %+v", tablet.agent.TabletAlias) + } + tmc.RunHealthCheck(ctx, tabletInfo, tablet.tabletType) + } } // diff --git a/go/vt/tabletmanager/healthcheck.go b/go/vt/tabletmanager/healthcheck.go index 249cde46361..990f96ce13c 100644 --- a/go/vt/tabletmanager/healthcheck.go +++ b/go/vt/tabletmanager/healthcheck.go @@ -179,7 +179,7 @@ func (agent *ActionAgent) runHealthCheck(targetTabletType topodatapb.TabletType) // figure out if we should be running the query service shouldBeServing := false - if topo.IsRunningQueryService(targetTabletType) && !agent.BinlogPlayerMap.isRunningFilteredReplication() { + if topo.IsRunningQueryService(targetTabletType) && (agent.BinlogPlayerMap == nil || !agent.BinlogPlayerMap.isRunningFilteredReplication()) { shouldBeServing = true if tabletControl != nil { if tabletControl.DisableQueryService { diff --git a/go/vt/vtgate/discoverygateway.go b/go/vt/vtgate/discoverygateway.go index 88e95400bbe..23aea674f32 100644 --- a/go/vt/vtgate/discoverygateway.go +++ b/go/vt/vtgate/discoverygateway.go @@ -52,6 +52,7 @@ func createDiscoveryGateway(hc discovery.HealthCheck, topoServer topo.Server, se tabletsWatchers: make([]*discovery.TopologyWatcher, 0, 1), } dg.hc.SetListener(dg) + log.Infof("loading tablets for cells: %v", *cellsToWatch) for _, c := range strings.Split(*cellsToWatch, ",") { if c == "" { continue @@ -83,15 +84,22 @@ func (dg *discoveryGateway) waitForEndPoints() error { return nil } + log.Infof("Waiting for endpoints") ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() err := discovery.WaitForAllEndPoints(ctx, dg.hc, dg.srvTopoServer, dg.localCell, dg.tabletTypesToWait) if err == discovery.ErrWaitForEndPointsTimeout { // ignore this error, we will still start up, and may not serve // all endpoints. + log.Warningf("Timeout when waiting for endpoints") err = nil } - return err + if err != nil { + log.Errorf("Error when waiting for endpoints: %v", err) + return err + } + log.Infof("Waiting for endpoints completed") + return nil } // Execute executes the non-streaming query for the specified keyspace, shard, and tablet type. From cd17688ed0b3d869f99740b391be030e0bdaf3dd Mon Sep 17 00:00:00 2001 From: Liang Guo Date: Wed, 11 May 2016 14:02:05 -0700 Subject: [PATCH 3/4] Fix VTGate argv for integration tests. --- test/utils.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/utils.py b/test/utils.py index 045001a3f83..471c8ed8ffb 100644 --- a/test/utils.py +++ b/test/utils.py @@ -531,7 +531,7 @@ def __init__(self, port=None): def start(self, cell='test_nj', retry_count=2, topo_impl=None, cache_ttl='1s', - timeout_total='2s', + healthcheck_conn_timeout='2s', extra_args=None, tablets=None, tablet_types_to_wait='MASTER,REPLICA'): """Start vtgate. Saves it into the global vtgate variable if not set yet.""" @@ -542,7 +542,7 @@ def start(self, cell='test_nj', retry_count=2, '-retry-count', str(retry_count), '-log_dir', environment.vtlogroot, '-srv_topo_cache_ttl', cache_ttl, - '-conn-timeout-total', timeout_total, + '-healthcheck_conn_timeout', healthcheck_conn_timeout, '-tablet_protocol', protocols_flavor().tabletconn_protocol(), '-gateway_implementation', vtgate_gateway_flavor().flavor(), '-tablet_grpc_combine_begin_execute', From badc4d061b3f4eca4cba1867333bb12ab621fa13 Mon Sep 17 00:00:00 2001 From: Liang Guo Date: Wed, 11 May 2016 14:43:35 -0700 Subject: [PATCH 4/4] Add some comments. --- go/vt/vtgate/fakehealthcheck_test.go | 12 ++++++++---- go/vt/vtgate/gateway.go | 6 ++++-- go/vt/vtgate/sandbox_test.go | 8 ++++++++ 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/go/vt/vtgate/fakehealthcheck_test.go b/go/vt/vtgate/fakehealthcheck_test.go index 174d9a668d6..fffc23f583b 100644 --- a/go/vt/vtgate/fakehealthcheck_test.go +++ b/go/vt/vtgate/fakehealthcheck_test.go @@ -18,14 +18,17 @@ type fhcItem struct { conn tabletconn.TabletConn } +// fakeHealthCheck implements discovery.HealthCheck. type fakeHealthCheck struct { items map[string]*fhcItem - // stats - GetStatsFromTargetCounter int + // GetStatsFromTargetCounter counts GetEndpointStatsFromTarget() being called. + GetStatsFromTargetCounter int + // GetStatsFromKeyspaceShardCounter counts GetEndPointStatsFromKeyspaceShard() being called. GetStatsFromKeyspaceShardCounter int } +// Reset cleans up the internal state. func (fhc *fakeHealthCheck) Reset() { fhc.GetStatsFromTargetCounter = 0 fhc.GetStatsFromKeyspaceShardCounter = 0 @@ -36,7 +39,7 @@ func (fhc *fakeHealthCheck) Reset() { func (fhc *fakeHealthCheck) SetListener(listener discovery.HealthCheckStatsListener) { } -// AddEndPoint adds the endpoint, and starts health check. +// AddEndPoint adds the endpoint. func (fhc *fakeHealthCheck) AddEndPoint(cell, name string, endPoint *topodatapb.EndPoint) { key := discovery.EndPointToMapKey(endPoint) item := &fhcItem{ @@ -49,7 +52,7 @@ func (fhc *fakeHealthCheck) AddEndPoint(cell, name string, endPoint *topodatapb. fhc.items[key] = item } -// RemoveEndPoint removes the endpoint, and stops the health check. +// RemoveEndPoint removes the endpoint. func (fhc *fakeHealthCheck) RemoveEndPoint(endPoint *topodatapb.EndPoint) { key := discovery.EndPointToMapKey(endPoint) delete(fhc.items, key) @@ -104,6 +107,7 @@ func (fhc *fakeHealthCheck) Close() error { return nil } +// addTestEndPoint inserts a fake entry into fakeHealthCheck. func (fhc *fakeHealthCheck) addTestEndPoint(cell, host string, port int32, keyspace, shard string, tabletType topodatapb.TabletType, serving bool, reparentTS int64, err error, conn tabletconn.TabletConn) *topodatapb.EndPoint { if conn != nil { conn.SetTarget(keyspace, shard, tabletType) diff --git a/go/vt/vtgate/gateway.go b/go/vt/vtgate/gateway.go index 4682cee3a5b..91fb1551753 100644 --- a/go/vt/vtgate/gateway.go +++ b/go/vt/vtgate/gateway.go @@ -119,9 +119,11 @@ func GetGatewayCreatorByName(name string) GatewayCreator { // ShardError is the error about a specific shard. // It implements vterrors.VtError. type ShardError struct { + // ShardIdentifier is the keyspace+shard. ShardIdentifier string - InTransaction bool - // Preserve the original error, so that we don't need to parse the error string. + // InTransaction indicates if it is inside a transaction. + InTransaction bool + // Err preserves the original error, so that we don't need to parse the error string. Err error // EndPointCode is the error code to use for all the endpoint errors in aggregate EndPointCode vtrpcpb.ErrorCode diff --git a/go/vt/vtgate/sandbox_test.go b/go/vt/vtgate/sandbox_test.go index 20a61ae944a..95d8d2fa762 100644 --- a/go/vt/vtgate/sandbox_test.go +++ b/go/vt/vtgate/sandbox_test.go @@ -92,9 +92,11 @@ type sandbox struct { // SrvKeyspaceCallback specifies the callback function in GetSrvKeyspace SrvKeyspaceCallback func() + // VSchema specifies the vschema in JSON format. VSchema string } +// Reset cleans up sandbox internal state. func (s *sandbox) Reset() { s.sandmu.Lock() defer s.sandmu.Unlock() @@ -108,6 +110,7 @@ func (s *sandbox) Reset() { s.SrvKeyspaceCallback = nil } +// DefaultShardSpec is the default sharding scheme for testing. var DefaultShardSpec = "-20-40-60-80-a0-c0-e0-" func getAllShards(shardSpec string) ([]*topodatapb.KeyRange, error) { @@ -192,6 +195,7 @@ func createUnshardedKeyspace() (*topodatapb.SrvKeyspace, error) { type sandboxTopo struct { } +// GetSrvKeyspaceNames is part of SrvTopoServer. func (sct *sandboxTopo) GetSrvKeyspaceNames(ctx context.Context, cell string) ([]string, error) { sandboxMu.Lock() defer sandboxMu.Unlock() @@ -202,6 +206,7 @@ func (sct *sandboxTopo) GetSrvKeyspaceNames(ctx context.Context, cell string) ([ return keyspaces, nil } +// GetSrvKeyspace is part of SrvTopoServer. func (sct *sandboxTopo) GetSrvKeyspace(ctx context.Context, cell, keyspace string) (*topodatapb.SrvKeyspace, error) { sand := getSandbox(keyspace) if sand.SrvKeyspaceCallback != nil { @@ -236,6 +241,7 @@ func (sct *sandboxTopo) GetSrvKeyspace(ctx context.Context, cell, keyspace strin return createShardedSrvKeyspace(sand.ShardSpec, sand.KeyspaceServedFrom) } +// WatchVSchema is part of SrvTopoServer. func (sct *sandboxTopo) WatchVSchema(ctx context.Context, keyspace string) (notifications <-chan string, err error) { result := make(chan string, 1) value := getSandbox(keyspace).VSchema @@ -243,10 +249,12 @@ func (sct *sandboxTopo) WatchVSchema(ctx context.Context, keyspace string) (noti return result, nil } +// GetSrvShard is part of SrvTopoServer. func (sct *sandboxTopo) GetSrvShard(ctx context.Context, cell, keyspace, shard string) (*topodatapb.SrvShard, error) { return nil, fmt.Errorf("Unsupported") } +// GetEndPoints is part of SrvTopoServer. func (sct *sandboxTopo) GetEndPoints(ctx context.Context, cell, keyspace, shard string, tabletType topodatapb.TabletType) (*topodatapb.EndPoints, int64, error) { return nil, -1, fmt.Errorf("Unsupported") }