From db0dda79189880c2606c62604c91d52322f72205 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Tue, 8 Oct 2024 21:30:49 +0530 Subject: [PATCH 01/24] Implement happy eyeballs --- .github/workflows/testing.yml | 4 +- .../pickfirst/pickfirstleaf/pickfirstleaf.go | 182 +++++++-- .../pickfirstleaf/pickfirstleaf_test.go | 344 ++++++++++++++++++ internal/envconfig/envconfig.go | 5 + 4 files changed, 500 insertions(+), 35 deletions(-) diff --git a/.github/workflows/testing.yml b/.github/workflows/testing.yml index a6576a21fa15..9620c8959efe 100644 --- a/.github/workflows/testing.yml +++ b/.github/workflows/testing.yml @@ -73,7 +73,9 @@ jobs: - type: tests goversion: '1.22' testflags: -race - grpcenv: 'GRPC_EXPERIMENTAL_ENABLE_NEW_PICK_FIRST=true' + grpcenv: | + 'GRPC_EXPERIMENTAL_ENABLE_NEW_PICK_FIRST=true' + 'GRPC_EXPERIMENTAL_ENABLE_PICK_FIRST_HAPPY_EYEBALLS=true' steps: # Setup the environment. diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index 48ce8c50e5c1..0dd04e010782 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -26,10 +26,12 @@ package pickfirstleaf import ( + "context" "encoding/json" "errors" "fmt" "sync" + "time" "google.golang.org/grpc/balancer" "google.golang.org/grpc/balancer/pickfirst/internal" @@ -56,20 +58,29 @@ var ( // It is changed to "pick_first" in init() if this balancer is to be // registered as the default pickfirst. Name = "pick_first_leaf" + // timerFunc allows mocking the timer for testing connection delay related + // functionality. + timerFunc = time.After ) -// TODO: change to pick-first when this becomes the default pick_first policy. -const logPrefix = "[pick-first-leaf-lb %p] " +const ( + // TODO: change to pick-first when this becomes the default pick_first policy. + logPrefix = "[pick-first-leaf-lb %p] " + // connectionDelayInterval is the time to wait for during the happy eyeballs + // pass before starting the next connection attempt. + connectionDelayInterval = 250 * time.Millisecond +) type pickfirstBuilder struct{} func (pickfirstBuilder) Build(cc balancer.ClientConn, _ balancer.BuildOptions) balancer.Balancer { b := &pickfirstBalancer{ - cc: cc, - addressList: addressList{}, - subConns: resolver.NewAddressMap(), - state: connectivity.Connecting, - mu: sync.Mutex{}, + cc: cc, + addressList: addressList{}, + subConns: resolver.NewAddressMap(), + state: connectivity.Connecting, + mu: sync.Mutex{}, + callbackScheduler: callbackScheduler{}, } b.logger = internalgrpclog.NewPrefixLogger(logger, fmt.Sprintf(logPrefix, b)) return b @@ -104,8 +115,9 @@ type scData struct { subConn balancer.SubConn addr resolver.Address - state connectivity.State - lastErr error + state connectivity.State + lastErr error + connectionAttempted bool } func (b *pickfirstBalancer) newSCData(addr resolver.Address) (*scData, error) { @@ -137,10 +149,11 @@ type pickfirstBalancer struct { mu sync.Mutex state connectivity.State // scData for active subonns mapped by address. - subConns *resolver.AddressMap - addressList addressList - firstPass bool - numTF int + subConns *resolver.AddressMap + addressList addressList + firstPass bool + numTF int + callbackScheduler callbackScheduler } // ResolverError is called by the ClientConn when the name resolver produces @@ -232,9 +245,6 @@ func (b *pickfirstBalancer) UpdateClientConnState(state balancer.ClientConnState // SubConn multiple times in the same pass. We don't want this. newAddrs = deDupAddresses(newAddrs) - // Since we have a new set of addresses, we are again at first pass. - b.firstPass = true - // If the previous ready SubConn exists in new address list, // keep this connection and don't create new SubConns. prevAddr := b.addressList.currentAddress() @@ -259,11 +269,11 @@ func (b *pickfirstBalancer) UpdateClientConnState(state balancer.ClientConnState ConnectivityState: connectivity.Connecting, Picker: &picker{err: balancer.ErrNoSubConnAvailable}, }) - b.requestConnectionLocked() + b.startFirstPassLocked() } else if b.state == connectivity.TransientFailure { // If we're in TRANSIENT_FAILURE, we stay in TRANSIENT_FAILURE until // we're READY. See A62. - b.requestConnectionLocked() + b.startFirstPassLocked() } return nil } @@ -278,6 +288,7 @@ func (b *pickfirstBalancer) Close() { b.mu.Lock() defer b.mu.Unlock() b.closeSubConnsLocked() + b.callbackScheduler.close() b.state = connectivity.Shutdown } @@ -288,9 +299,18 @@ func (b *pickfirstBalancer) ExitIdle() { b.mu.Lock() defer b.mu.Unlock() if b.state == connectivity.Idle && b.addressList.currentAddress() == b.addressList.first() { - b.firstPass = true - b.requestConnectionLocked() + b.startFirstPassLocked() + } +} + +func (b *pickfirstBalancer) startFirstPassLocked() { + b.firstPass = true + b.numTF = 0 + // Reset the connection attempt record for existing SubConns. + for _, sd := range b.subConns.Values() { + sd.(*scData).connectionAttempted = false } + b.requestConnectionLocked() } func (b *pickfirstBalancer) closeSubConnsLocked() { @@ -341,6 +361,7 @@ func (b *pickfirstBalancer) reconcileSubConnsLocked(newAddrs []resolver.Address) // shutdownRemainingLocked shuts down remaining subConns. Called when a subConn // becomes ready, which means that all other subConn must be shutdown. func (b *pickfirstBalancer) shutdownRemainingLocked(selected *scData) { + b.callbackScheduler.cancel() for _, v := range b.subConns.Values() { sd := v.(*scData) if sd.subConn != selected.subConn { @@ -384,8 +405,10 @@ func (b *pickfirstBalancer) requestConnectionLocked() { switch scd.state { case connectivity.Idle: scd.subConn.Connect() + b.scheduleNextConnectionLocked() case connectivity.TransientFailure: // Try the next address. + scd.connectionAttempted = true lastErr = scd.lastErr continue case connectivity.Ready: @@ -396,18 +419,44 @@ func (b *pickfirstBalancer) requestConnectionLocked() { b.logger.Errorf("SubConn with state SHUTDOWN present in SubConns map") case connectivity.Connecting: // Wait for the SubConn to report success or failure. + b.scheduleNextConnectionLocked() } return } + // All the remaining addresses in the list are in TRANSIENT_FAILURE, end the - // first pass. - b.endFirstPassLocked(lastErr) + // first pass if possible. + b.endFirstPassIfPossibleLocked(lastErr) +} + +func (b *pickfirstBalancer) scheduleNextConnectionLocked() { + if !envconfig.PickFirstHappyEyeballsEnabled { + return + } + b.callbackScheduler.schedule(func(ctx context.Context) { + b.mu.Lock() + defer b.mu.Unlock() + // If the scheduled task is cancelled while acquiring the mutex, return. + if ctx.Err() != nil { + return + } + if b.logger.V(2) { + b.logger.Infof("Happy Eyeballs timer expired.") + } + if b.addressList.increment() { + b.requestConnectionLocked() + } + }, connectionDelayInterval) } func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.SubConnState) { b.mu.Lock() defer b.mu.Unlock() oldState := sd.state + // Record a connection attempt when existing CONNECTING. + if newState.ConnectivityState == connectivity.TransientFailure { + sd.connectionAttempted = true + } sd.state = newState.ConnectivityState // Previously relevant SubConns can still callback with state updates. // To prevent pickers from returning these obsolete SubConns, this logic @@ -473,17 +522,20 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub sd.lastErr = newState.ConnectionError // Since we're re-using common SubConns while handling resolver // updates, we could receive an out of turn TRANSIENT_FAILURE from - // a pass over the previous address list. We ignore such updates. - - if curAddr := b.addressList.currentAddress(); !equalAddressIgnoringBalAttributes(&curAddr, &sd.addr) { - return - } - if b.addressList.increment() { - b.requestConnectionLocked() - return + // a pass over the previous address list. Happy Eyeballs will also + // cause out of order updates to arrive. + + if curAddr := b.addressList.currentAddress(); equalAddressIgnoringBalAttributes(&curAddr, &sd.addr) { + b.callbackScheduler.cancel() + if b.addressList.increment() { + b.requestConnectionLocked() + return + } } - // End of the first pass. - b.endFirstPassLocked(newState.ConnectionError) + + // End the first pass if we've seen a TRANSIENT_FAILURE from all + // SubConns once. + b.endFirstPassIfPossibleLocked(newState.ConnectionError) } return } @@ -508,9 +560,17 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub } } -func (b *pickfirstBalancer) endFirstPassLocked(lastErr error) { +func (b *pickfirstBalancer) endFirstPassIfPossibleLocked(lastErr error) { + if b.addressList.isValid() || b.subConns.Len() < b.addressList.size() { + return + } + for _, v := range b.subConns.Values() { + sd := v.(*scData) + if !sd.connectionAttempted { + return + } + } b.firstPass = false - b.numTF = 0 b.state = connectivity.TransientFailure b.cc.UpdateState(balancer.State{ @@ -622,3 +682,57 @@ func equalAddressIgnoringBalAttributes(a, b *resolver.Address) bool { a.Attributes.Equal(b.Attributes) && a.Metadata == b.Metadata } + +// callbackScheduleris used to schedule the execution of a callback after a +// a specified delay. It is not safe for concurrent access. +type callbackScheduler struct { + cancelScheduled func() + closed bool + wg sync.WaitGroup +} + +// schedule schedules the execution of a callback. It cancels any previously +// scheduled callbacks. +func (c *callbackScheduler) schedule(f func(context.Context), after time.Duration) { + if c.closed { + return + } + c.cancel() + ctx, cancel := context.WithCancel(context.Background()) + c.cancelScheduled = sync.OnceFunc(cancel) + c.wg.Add(1) + + go func() { + select { + case <-timerFunc(after): + c.wg.Done() + // f() may try to acquire the balancer mutex. Calling wg.Done() + // after f() finishes may cause a dedlock because balancer.Close() + // would be holding the mutex when calling callbackScheduler.close() + // which waits for wg.Done(). + f(ctx) + case <-ctx.Done(): + c.wg.Done() + } + }() +} + +// cancel prevents the execution of the scheduled callback if a callback is +// awaiting execution. If a callback is a callback is already being executed, +// it cancels the context passed to it. +func (c *callbackScheduler) cancel() { + if c.cancelScheduled != nil { + c.cancelScheduled() + } +} + +// close closes the callbackScheduler and waits for all spawned goroutines to +// exit. No callbacks are scheduled after this method returns. +func (c *callbackScheduler) close() { + if c.closed { + return + } + c.cancel() + c.closed = true + c.wg.Wait() +} diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go index 84b3cb65bed4..181c2061e72c 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go @@ -21,12 +21,14 @@ package pickfirstleaf import ( "context" "fmt" + "sync" "testing" "time" "google.golang.org/grpc/attributes" "google.golang.org/grpc/balancer" "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/internal/envconfig" "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/testutils" "google.golang.org/grpc/resolver" @@ -257,3 +259,345 @@ func (s) TestPickFirstLeaf_TFPickerUpdate(t *testing.T) { t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", newTfErr, err) } } + +// TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay verifies that +// pickfirst attempts to connect to the second backend once the happy eyeballs +// timer expires. +func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + originalCfg := envconfig.PickFirstHappyEyeballsEnabled + envconfig.PickFirstHappyEyeballsEnabled = true + defer func() { + envconfig.PickFirstHappyEyeballsEnabled = originalCfg + }() + + timerCh := make(chan struct{}) + originalTimer := timerFunc + timerFunc = func(_ time.Duration) <-chan time.Time { + ret := make(chan time.Time) + go func() { + select { + case <-ctx.Done(): + case <-timerCh: + } + close(ret) + }() + return ret + } + + defer func() { + timerFunc = originalTimer + }() + + cc := testutils.NewBalancerClientConn(t) + bal := pickfirstBuilder{}.Build(cc, balancer.BuildOptions{}) + defer bal.Close() + ccState := balancer.ClientConnState{ + ResolverState: resolver.State{ + Endpoints: []resolver.Endpoint{ + {Addresses: []resolver.Address{{Addr: "1.1.1.1:1"}}}, + {Addresses: []resolver.Address{{Addr: "2.2.2.2:2"}}}, + }, + }, + } + if err := bal.UpdateClientConnState(ccState); err != nil { + t.Fatalf("UpdateClientConnState(%v) returned error: %v", ccState, err) + } + + if err := cc.WaitForPickerWithErr(ctx, balancer.ErrNoSubConnAvailable); err != nil { + t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", balancer.ErrNoSubConnAvailable, err) + } + + if err := cc.WaitForConnectivityState(ctx, connectivity.Connecting); err != nil { + t.Fatalf("cc.WaitForConnectivityState(%v) returned error: %v", connectivity.Connecting, err) + } + + sc0 := <-cc.NewSubConnCh + + // Until the timer fires, no new subchannel should be created. + select { + case <-time.After(defaultTestShortTimeout): + case sc1 := <-cc.NewSubConnCh: + t.Fatalf("Received unexpected subchannel: %v", sc1) + } + + timerCh <- struct{}{} + + sc1 := <-cc.NewSubConnCh + + // Second connection attempt is successful. + sc1.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Connecting}) + sc1.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + + if err := cc.WaitForConnectivityState(ctx, connectivity.Ready); err != nil { + t.Fatalf("cc.WaitForConnectivityState(%v) returned error: %v", connectivity.Ready, err) + } + + if err := cc.WaitForPicker(ctx, func(p balancer.Picker) error { + pr, err := p.Pick(balancer.PickInfo{}) + if err != nil { + return err + } + if pr.SubConn != sc1 { + t.Fatalf("Unexpected SubConn produced by picker, got = %v, want = %v", pr.SubConn, sc1) + } + return nil + }); err != nil { + t.Fatalf("cc.WaitForPicker() returned error: %v", err) + } + + closedSC := <-cc.ShutdownSubConnCh + if closedSC != sc0 { + t.Fatalf("Unexpected closed SubConn, got = %v, want = %v", closedSC, sc0) + } +} + +// TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList verifies that pickfirst +// correctly detects the end of the first happy eyeballs pass when the timer +// causes pickfirst to reach the end of the address list and failures are +// reported out of order. +func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + originalCfg := envconfig.PickFirstHappyEyeballsEnabled + envconfig.PickFirstHappyEyeballsEnabled = true + defer func() { + envconfig.PickFirstHappyEyeballsEnabled = originalCfg + }() + + timerCh := make(chan struct{}) + originalTimer := timerFunc + timerFunc = func(_ time.Duration) <-chan time.Time { + ret := make(chan time.Time) + go func() { + select { + case <-ctx.Done(): + case <-timerCh: + } + close(ret) + }() + return ret + } + + defer func() { + timerFunc = originalTimer + }() + + cc := testutils.NewBalancerClientConn(t) + bal := pickfirstBuilder{}.Build(cc, balancer.BuildOptions{}) + defer bal.Close() + ccState := balancer.ClientConnState{ + ResolverState: resolver.State{ + Endpoints: []resolver.Endpoint{ + {Addresses: []resolver.Address{{Addr: "1.1.1.1:1"}}}, + {Addresses: []resolver.Address{{Addr: "2.2.2.2:2"}}}, + {Addresses: []resolver.Address{{Addr: "3.3.3.3:3"}}}, + }, + }, + } + if err := bal.UpdateClientConnState(ccState); err != nil { + t.Fatalf("UpdateClientConnState(%v) returned error: %v", ccState, err) + } + + if err := cc.WaitForPickerWithErr(ctx, balancer.ErrNoSubConnAvailable); err != nil { + t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", balancer.ErrNoSubConnAvailable, err) + } + + if err := cc.WaitForConnectivityState(ctx, connectivity.Connecting); err != nil { + t.Fatalf("cc.WaitForConnectivityState(%v) returned error: %v", connectivity.Connecting, err) + } + + sc0 := <-cc.NewSubConnCh + // Make the happy eyeballs timer fire twice so that pickfirst reaches the + // last address in the list. + timerCh <- struct{}{} + sc1 := <-cc.NewSubConnCh + timerCh <- struct{}{} + sc2 := <-cc.NewSubConnCh + + // First SubConn Fails. + tfErr := fmt.Errorf("test error") + sc0.UpdateState(balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + ConnectionError: tfErr, + }) + + // No picker should be produced until the first pass is complete. + select { + case <-time.After(defaultTestShortTimeout): + case p := <-cc.NewPickerCh: + sc, err := p.Pick(balancer.PickInfo{}) + t.Fatalf("Unexpected picker update: %v, %v", sc, err) + } + + // Move off the end of the list, pickfirst should still be waiting for TFs + // to be reported. + timerCh <- struct{}{} + + // second SubConn fails. + sc1.UpdateState(balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + ConnectionError: tfErr, + }) + + // second SubConn fails again, we've still not seen the 3rd SubConn fail. + // Second connection attempt is successful. + sc1.UpdateState(balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + ConnectionError: tfErr, + }) + + select { + case <-time.After(defaultTestShortTimeout): + case p := <-cc.NewPickerCh: + sc, err := p.Pick(balancer.PickInfo{}) + t.Fatalf("Unexpected picker update: %v, %v", sc, err) + } + + // Last SubConn fails, this should result in a picker update. + sc2.UpdateState(balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + ConnectionError: tfErr, + }) + + if err := cc.WaitForPickerWithErr(ctx, tfErr); err != nil { + t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", tfErr, err) + } + + // Fail the first SubConn 3 times, causing the next picker update. + tfErr = fmt.Errorf("New test error") + sc0.UpdateState(balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + ConnectionError: tfErr, + }) + sc0.UpdateState(balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + ConnectionError: tfErr, + }) + + select { + case <-time.After(defaultTestShortTimeout): + case p := <-cc.NewPickerCh: + sc, err := p.Pick(balancer.PickInfo{}) + t.Fatalf("Unexpected picker update: %v, %v", sc, err) + } + + sc0.UpdateState(balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + ConnectionError: tfErr, + }) + + if err := cc.WaitForPickerWithErr(ctx, tfErr); err != nil { + t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", tfErr, err) + } +} + +// TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires tests the pickfirst balancer +// by causing a SubConn to fail and then jumping to the 3rd SubConn after the +// happy eyeballs timer expires. +func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + originalCfg := envconfig.PickFirstHappyEyeballsEnabled + envconfig.PickFirstHappyEyeballsEnabled = true + defer func() { + envconfig.PickFirstHappyEyeballsEnabled = originalCfg + }() + + timerMu := sync.Mutex{} + timerCh := make(chan struct{}) + originalTimer := timerFunc + timerFunc = func(_ time.Duration) <-chan time.Time { + ret := make(chan time.Time) + timerMu.Lock() + ch := timerCh + timerMu.Unlock() + go func() { + select { + case <-ctx.Done(): + case <-ch: + } + close(ret) + }() + return ret + } + + defer func() { + timerFunc = originalTimer + }() + + cc := testutils.NewBalancerClientConn(t) + bal := pickfirstBuilder{}.Build(cc, balancer.BuildOptions{}) + defer bal.Close() + ccState := balancer.ClientConnState{ + ResolverState: resolver.State{ + Endpoints: []resolver.Endpoint{ + {Addresses: []resolver.Address{{Addr: "1.1.1.1:1"}}}, + {Addresses: []resolver.Address{{Addr: "2.2.2.2:2"}}}, + {Addresses: []resolver.Address{{Addr: "3.3.3.3:3"}}}, + }, + }, + } + if err := bal.UpdateClientConnState(ccState); err != nil { + t.Fatalf("UpdateClientConnState(%v) returned error: %v", ccState, err) + } + + if err := cc.WaitForPickerWithErr(ctx, balancer.ErrNoSubConnAvailable); err != nil { + t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", balancer.ErrNoSubConnAvailable, err) + } + + if err := cc.WaitForConnectivityState(ctx, connectivity.Connecting); err != nil { + t.Fatalf("cc.WaitForConnectivityState(%v) returned error: %v", connectivity.Connecting, err) + } + + sc0 := <-cc.NewSubConnCh + // Until the timer fires or a TF is reported, no new subchannel should be + // created. + select { + case <-time.After(defaultTestShortTimeout): + case sc1 := <-cc.NewSubConnCh: + t.Fatalf("Received unexpected subchannel: %v", sc1) + } + + // First SubConn Fails. + tfErr := fmt.Errorf("test error") + // Replace the timer channel so that the old timers don't attempt to read + // messages pushed next. + timerMu.Lock() + timerCh = make(chan struct{}) + timerMu.Unlock() + sc0.UpdateState(balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + ConnectionError: tfErr, + }) + + sc1 := <-cc.NewSubConnCh + sc1.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Connecting}) + + // The happy eyeballs timer expires, skipping sc1 and requesting the creation + // of sc2. + timerCh <- struct{}{} + <-cc.NewSubConnCh + + // First SubConn comes out of backoff. + sc0.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Idle}) + // Second SubConn connects. + sc1.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + // Verify that the picker is updated. + if err := cc.WaitForPicker(ctx, func(p balancer.Picker) error { + pr, err := p.Pick(balancer.PickInfo{}) + if err != nil { + return err + } + if pr.SubConn != sc1 { + t.Fatalf("Unexpected SubConn produced by picker, got = %v, want = %v", pr.SubConn, sc1) + } + return nil + }); err != nil { + t.Fatalf("cc.WaitForPicker() returned error: %v", err) + } +} diff --git a/internal/envconfig/envconfig.go b/internal/envconfig/envconfig.go index 6e7dd6b77270..a2ff89a0c855 100644 --- a/internal/envconfig/envconfig.go +++ b/internal/envconfig/envconfig.go @@ -55,6 +55,11 @@ var ( // setting the environment variable "GRPC_EXPERIMENTAL_ENABLE_NEW_PICK_FIRST" // to "true". NewPickFirstEnabled = boolFromEnv("GRPC_EXPERIMENTAL_ENABLE_NEW_PICK_FIRST", false) + // PickFirstHappyEyeballsEnabled is set if the happy eyeballs algorithm is + // used by the new pickfirst LB policy. This option may be overridden + // by setting the environment variable + // "GRPC_EXPERIMENTAL_ENABLE_PICK_FIRST_HAPPY_EYEBALLS" to "true" or "false". + PickFirstHappyEyeballsEnabled = boolFromEnv("GRPC_EXPERIMENTAL_ENABLE_PICK_FIRST_HAPPY_EYEBALLS", false) ) func boolFromEnv(envVar string, def bool) bool { From 826bb03eb0a642e1c3d0b3aea42c02ff57029798 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Fri, 11 Oct 2024 11:25:47 +0530 Subject: [PATCH 02/24] Use timeAfterFunc --- .../pickfirst/pickfirstleaf/pickfirstleaf.go | 98 +++++-------------- .../pickfirstleaf/pickfirstleaf_test.go | 34 ++++--- 2 files changed, 47 insertions(+), 85 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index 0dd04e010782..c1a997d29276 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -58,9 +58,9 @@ var ( // It is changed to "pick_first" in init() if this balancer is to be // registered as the default pickfirst. Name = "pick_first_leaf" - // timerFunc allows mocking the timer for testing connection delay related - // functionality. - timerFunc = time.After + // timerAfterFunc allows mocking the timer for testing connection delay + // related functionality. + timerAfterFunc = time.AfterFunc ) const ( @@ -75,12 +75,12 @@ type pickfirstBuilder struct{} func (pickfirstBuilder) Build(cc balancer.ClientConn, _ balancer.BuildOptions) balancer.Balancer { b := &pickfirstBalancer{ - cc: cc, - addressList: addressList{}, - subConns: resolver.NewAddressMap(), - state: connectivity.Connecting, - mu: sync.Mutex{}, - callbackScheduler: callbackScheduler{}, + cc: cc, + addressList: addressList{}, + subConns: resolver.NewAddressMap(), + state: connectivity.Connecting, + mu: sync.Mutex{}, + cancelScheduled: func() {}, } b.logger = internalgrpclog.NewPrefixLogger(logger, fmt.Sprintf(logPrefix, b)) return b @@ -149,11 +149,11 @@ type pickfirstBalancer struct { mu sync.Mutex state connectivity.State // scData for active subonns mapped by address. - subConns *resolver.AddressMap - addressList addressList - firstPass bool - numTF int - callbackScheduler callbackScheduler + subConns *resolver.AddressMap + addressList addressList + firstPass bool + numTF int + cancelScheduled func() } // ResolverError is called by the ClientConn when the name resolver produces @@ -288,7 +288,7 @@ func (b *pickfirstBalancer) Close() { b.mu.Lock() defer b.mu.Unlock() b.closeSubConnsLocked() - b.callbackScheduler.close() + b.cancelScheduled() b.state = connectivity.Shutdown } @@ -361,7 +361,7 @@ func (b *pickfirstBalancer) reconcileSubConnsLocked(newAddrs []resolver.Address) // shutdownRemainingLocked shuts down remaining subConns. Called when a subConn // becomes ready, which means that all other subConn must be shutdown. func (b *pickfirstBalancer) shutdownRemainingLocked(selected *scData) { - b.callbackScheduler.cancel() + b.cancelScheduled() for _, v := range b.subConns.Values() { sd := v.(*scData) if sd.subConn != selected.subConn { @@ -433,7 +433,9 @@ func (b *pickfirstBalancer) scheduleNextConnectionLocked() { if !envconfig.PickFirstHappyEyeballsEnabled { return } - b.callbackScheduler.schedule(func(ctx context.Context) { + b.cancelScheduled() + ctx, cancel := context.WithCancel(context.Background()) + closeFn := timerAfterFunc(connectionDelayInterval, func() { b.mu.Lock() defer b.mu.Unlock() // If the scheduled task is cancelled while acquiring the mutex, return. @@ -446,7 +448,11 @@ func (b *pickfirstBalancer) scheduleNextConnectionLocked() { if b.addressList.increment() { b.requestConnectionLocked() } - }, connectionDelayInterval) + }).Stop + b.cancelScheduled = sync.OnceFunc(func() { + closeFn() + cancel() + }) } func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.SubConnState) { @@ -526,7 +532,7 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub // cause out of order updates to arrive. if curAddr := b.addressList.currentAddress(); equalAddressIgnoringBalAttributes(&curAddr, &sd.addr) { - b.callbackScheduler.cancel() + b.cancelScheduled() if b.addressList.increment() { b.requestConnectionLocked() return @@ -682,57 +688,3 @@ func equalAddressIgnoringBalAttributes(a, b *resolver.Address) bool { a.Attributes.Equal(b.Attributes) && a.Metadata == b.Metadata } - -// callbackScheduleris used to schedule the execution of a callback after a -// a specified delay. It is not safe for concurrent access. -type callbackScheduler struct { - cancelScheduled func() - closed bool - wg sync.WaitGroup -} - -// schedule schedules the execution of a callback. It cancels any previously -// scheduled callbacks. -func (c *callbackScheduler) schedule(f func(context.Context), after time.Duration) { - if c.closed { - return - } - c.cancel() - ctx, cancel := context.WithCancel(context.Background()) - c.cancelScheduled = sync.OnceFunc(cancel) - c.wg.Add(1) - - go func() { - select { - case <-timerFunc(after): - c.wg.Done() - // f() may try to acquire the balancer mutex. Calling wg.Done() - // after f() finishes may cause a dedlock because balancer.Close() - // would be holding the mutex when calling callbackScheduler.close() - // which waits for wg.Done(). - f(ctx) - case <-ctx.Done(): - c.wg.Done() - } - }() -} - -// cancel prevents the execution of the scheduled callback if a callback is -// awaiting execution. If a callback is a callback is already being executed, -// it cancels the context passed to it. -func (c *callbackScheduler) cancel() { - if c.cancelScheduled != nil { - c.cancelScheduled() - } -} - -// close closes the callbackScheduler and waits for all spawned goroutines to -// exit. No callbacks are scheduled after this method returns. -func (c *callbackScheduler) close() { - if c.closed { - return - } - c.cancel() - c.closed = true - c.wg.Wait() -} diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go index 181c2061e72c..b4ef7216f798 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go @@ -275,14 +275,16 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { timerCh := make(chan struct{}) originalTimer := timerFunc - timerFunc = func(_ time.Duration) <-chan time.Time { - ret := make(chan time.Time) + timerFunc = func(_ time.Duration, f func()) *time.Timer { + // Set a really long expiration to prevent it from triggering + // automatically. + ret := time.AfterFunc(time.Hour, f) go func() { select { case <-ctx.Done(): case <-timerCh: } - close(ret) + ret.Reset(0) }() return ret } @@ -370,14 +372,16 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { timerCh := make(chan struct{}) originalTimer := timerFunc - timerFunc = func(_ time.Duration) <-chan time.Time { - ret := make(chan time.Time) + timerFunc = func(_ time.Duration, f func()) *time.Timer { + // Set a really long expiration to prevent it from triggering + // automatically. + ret := time.AfterFunc(time.Hour, f) go func() { select { case <-ctx.Done(): case <-timerCh: } - close(ret) + ret.Reset(0) }() return ret } @@ -386,6 +390,10 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { timerFunc = originalTimer }() + defer func() { + timerFunc = originalTimer + }() + cc := testutils.NewBalancerClientConn(t) bal := pickfirstBuilder{}.Build(cc, balancer.BuildOptions{}) defer bal.Close() @@ -511,17 +519,19 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { timerMu := sync.Mutex{} timerCh := make(chan struct{}) originalTimer := timerFunc - timerFunc = func(_ time.Duration) <-chan time.Time { - ret := make(chan time.Time) - timerMu.Lock() - ch := timerCh - timerMu.Unlock() + timerFunc = func(_ time.Duration, f func()) *time.Timer { + // Set a really long expiration to prevent it from triggering + // automatically. + ret := time.AfterFunc(time.Hour, f) go func() { + timerMu.Lock() + ch := timerCh + timerMu.Unlock() select { case <-ctx.Done(): case <-ch: } - close(ret) + ret.Reset(0) }() return ret } From 4e68e58a42596f3080cc002abb7a79b1ec450b08 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Fri, 11 Oct 2024 12:24:15 +0530 Subject: [PATCH 03/24] Address review comments --- .../pickfirst/pickfirstleaf/pickfirstleaf.go | 36 +++++++++---------- .../pickfirstleaf/pickfirstleaf_test.go | 33 ++++++----------- internal/envconfig/envconfig.go | 5 ++- 3 files changed, 28 insertions(+), 46 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index c1a997d29276..f7e51b4b7fd8 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -115,9 +115,9 @@ type scData struct { subConn balancer.SubConn addr resolver.Address - state connectivity.State - lastErr error - connectionAttempted bool + state connectivity.State + lastErr error + connectionFailed bool } func (b *pickfirstBalancer) newSCData(addr resolver.Address) (*scData, error) { @@ -298,7 +298,7 @@ func (b *pickfirstBalancer) Close() { func (b *pickfirstBalancer) ExitIdle() { b.mu.Lock() defer b.mu.Unlock() - if b.state == connectivity.Idle && b.addressList.currentAddress() == b.addressList.first() { + if b.state == connectivity.Idle { b.startFirstPassLocked() } } @@ -308,7 +308,7 @@ func (b *pickfirstBalancer) startFirstPassLocked() { b.numTF = 0 // Reset the connection attempt record for existing SubConns. for _, sd := range b.subConns.Values() { - sd.(*scData).connectionAttempted = false + sd.(*scData).connectionFailed = false } b.requestConnectionLocked() } @@ -406,22 +406,27 @@ func (b *pickfirstBalancer) requestConnectionLocked() { case connectivity.Idle: scd.subConn.Connect() b.scheduleNextConnectionLocked() + return case connectivity.TransientFailure: - // Try the next address. - scd.connectionAttempted = true + // The SubConn is being re-used and failed during a previous pass + // over the addressList. It has not completed backoff yet. + // Mark it as having failed and try the next address. + scd.connectionFailed = true lastErr = scd.lastErr continue case connectivity.Ready: // Should never happen. b.logger.Errorf("Requesting a connection even though we have a READY SubConn") + return case connectivity.Shutdown: // Should never happen. b.logger.Errorf("SubConn with state SHUTDOWN present in SubConns map") + return case connectivity.Connecting: // Wait for the SubConn to report success or failure. b.scheduleNextConnectionLocked() + return } - return } // All the remaining addresses in the list are in TRANSIENT_FAILURE, end the @@ -459,9 +464,9 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub b.mu.Lock() defer b.mu.Unlock() oldState := sd.state - // Record a connection attempt when existing CONNECTING. + // Record a connection attempt when exiting CONNECTING. if newState.ConnectivityState == connectivity.TransientFailure { - sd.connectionAttempted = true + sd.connectionFailed = true } sd.state = newState.ConnectivityState // Previously relevant SubConns can still callback with state updates. @@ -572,7 +577,7 @@ func (b *pickfirstBalancer) endFirstPassIfPossibleLocked(lastErr error) { } for _, v := range b.subConns.Values() { sd := v.(*scData) - if !sd.connectionAttempted { + if !sd.connectionFailed { return } } @@ -648,15 +653,6 @@ func (al *addressList) currentAddress() resolver.Address { return al.addresses[al.idx] } -// first returns the first address in the list. If the list is empty, it returns -// an empty address instead. -func (al *addressList) first() resolver.Address { - if len(al.addresses) == 0 { - return resolver.Address{} - } - return al.addresses[0] -} - func (al *addressList) reset() { al.idx = 0 } diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go index b4ef7216f798..4b038da6e4c2 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go @@ -75,21 +75,8 @@ func (s) TestAddressList_Iteration(t *testing.T) { } addressList := addressList{} - emptyAddress := resolver.Address{} - if got, want := addressList.first(), emptyAddress; got != want { - t.Fatalf("addressList.first() = %v, want %v", got, want) - } - addressList.updateAddrs(addrs) - if got, want := addressList.first(), addressList.currentAddress(); got != want { - t.Fatalf("addressList.first() = %v, want %v", got, want) - } - - if got, want := addressList.first(), addrs[0]; got != want { - t.Fatalf("addressList.first() = %v, want %v", got, want) - } - for i := 0; i < len(addrs); i++ { if got, want := addressList.isValid(), true; got != want { t.Fatalf("addressList.isValid() = %t, want %t", got, want) @@ -274,8 +261,8 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { }() timerCh := make(chan struct{}) - originalTimer := timerFunc - timerFunc = func(_ time.Duration, f func()) *time.Timer { + originalTimer := timerAfterFunc + timerAfterFunc = func(_ time.Duration, f func()) *time.Timer { // Set a really long expiration to prevent it from triggering // automatically. ret := time.AfterFunc(time.Hour, f) @@ -290,7 +277,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { } defer func() { - timerFunc = originalTimer + timerAfterFunc = originalTimer }() cc := testutils.NewBalancerClientConn(t) @@ -371,8 +358,8 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { }() timerCh := make(chan struct{}) - originalTimer := timerFunc - timerFunc = func(_ time.Duration, f func()) *time.Timer { + originalTimer := timerAfterFunc + timerAfterFunc = func(_ time.Duration, f func()) *time.Timer { // Set a really long expiration to prevent it from triggering // automatically. ret := time.AfterFunc(time.Hour, f) @@ -387,11 +374,11 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { } defer func() { - timerFunc = originalTimer + timerAfterFunc = originalTimer }() defer func() { - timerFunc = originalTimer + timerAfterFunc = originalTimer }() cc := testutils.NewBalancerClientConn(t) @@ -518,8 +505,8 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { timerMu := sync.Mutex{} timerCh := make(chan struct{}) - originalTimer := timerFunc - timerFunc = func(_ time.Duration, f func()) *time.Timer { + originalTimer := timerAfterFunc + timerAfterFunc = func(_ time.Duration, f func()) *time.Timer { // Set a really long expiration to prevent it from triggering // automatically. ret := time.AfterFunc(time.Hour, f) @@ -537,7 +524,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { } defer func() { - timerFunc = originalTimer + timerAfterFunc = originalTimer }() cc := testutils.NewBalancerClientConn(t) diff --git a/internal/envconfig/envconfig.go b/internal/envconfig/envconfig.go index a2ff89a0c855..725277d15085 100644 --- a/internal/envconfig/envconfig.go +++ b/internal/envconfig/envconfig.go @@ -43,8 +43,7 @@ var ( // EnforceALPNEnabled is set if TLS connections to servers with ALPN disabled // should be rejected. The HTTP/2 protocol requires ALPN to be enabled, this // option is present for backward compatibility. This option may be overridden - // by setting the environment variable "GRPC_ENFORCE_ALPN_ENABLED" to "true" - // or "false". + // by setting the environment variable "GRPC_ENFORCE_ALPN_ENABLED" to "false". EnforceALPNEnabled = boolFromEnv("GRPC_ENFORCE_ALPN_ENABLED", true) // XDSFallbackSupport is the env variable that controls whether support for // xDS fallback is turned on. If this is unset or is false, only the first @@ -58,7 +57,7 @@ var ( // PickFirstHappyEyeballsEnabled is set if the happy eyeballs algorithm is // used by the new pickfirst LB policy. This option may be overridden // by setting the environment variable - // "GRPC_EXPERIMENTAL_ENABLE_PICK_FIRST_HAPPY_EYEBALLS" to "true" or "false". + // "GRPC_EXPERIMENTAL_ENABLE_PICK_FIRST_HAPPY_EYEBALLS" to "true". PickFirstHappyEyeballsEnabled = boolFromEnv("GRPC_EXPERIMENTAL_ENABLE_PICK_FIRST_HAPPY_EYEBALLS", false) ) From fe69816f64cee34ae7d853e761221deb1b541b3a Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Wed, 16 Oct 2024 12:59:46 +0530 Subject: [PATCH 04/24] Move timer func to internal, improve log statement and address review comments --- balancer/pickfirst/internal/internal.go | 14 ++++++++++--- .../pickfirst/pickfirstleaf/pickfirstleaf.go | 10 ++++----- .../pickfirstleaf/pickfirstleaf_test.go | 21 ++++++++++--------- 3 files changed, 26 insertions(+), 19 deletions(-) diff --git a/balancer/pickfirst/internal/internal.go b/balancer/pickfirst/internal/internal.go index c51978945844..da3541b37960 100644 --- a/balancer/pickfirst/internal/internal.go +++ b/balancer/pickfirst/internal/internal.go @@ -18,7 +18,15 @@ // Package internal contains code internal to the pickfirst package. package internal -import "math/rand" +import ( + "math/rand" + "time" +) -// RandShuffle pseudo-randomizes the order of addresses. -var RandShuffle = rand.Shuffle +var ( + // RandShuffle pseudo-randomizes the order of addresses. + RandShuffle = rand.Shuffle + // TimeAfterFunc allows mocking the timer for testing connection delay + // related functionality. + TimeAfterFunc = time.AfterFunc +) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index f7e51b4b7fd8..1084152a9ec6 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -58,9 +58,6 @@ var ( // It is changed to "pick_first" in init() if this balancer is to be // registered as the default pickfirst. Name = "pick_first_leaf" - // timerAfterFunc allows mocking the timer for testing connection delay - // related functionality. - timerAfterFunc = time.AfterFunc ) const ( @@ -438,9 +435,10 @@ func (b *pickfirstBalancer) scheduleNextConnectionLocked() { if !envconfig.PickFirstHappyEyeballsEnabled { return } + curAddr := b.addressList.currentAddress() b.cancelScheduled() ctx, cancel := context.WithCancel(context.Background()) - closeFn := timerAfterFunc(connectionDelayInterval, func() { + closeFn := internal.TimeAfterFunc(connectionDelayInterval, func() { b.mu.Lock() defer b.mu.Unlock() // If the scheduled task is cancelled while acquiring the mutex, return. @@ -448,15 +446,15 @@ func (b *pickfirstBalancer) scheduleNextConnectionLocked() { return } if b.logger.V(2) { - b.logger.Infof("Happy Eyeballs timer expired.") + b.logger.Infof("Happy Eyeballs timer expired while waiting for connection to %q.", curAddr.Addr) } if b.addressList.increment() { b.requestConnectionLocked() } }).Stop b.cancelScheduled = sync.OnceFunc(func() { - closeFn() cancel() + closeFn() }) } diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go index 4b038da6e4c2..1fe61f28fb73 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go @@ -27,6 +27,7 @@ import ( "google.golang.org/grpc/attributes" "google.golang.org/grpc/balancer" + "google.golang.org/grpc/balancer/pickfirst/internal" "google.golang.org/grpc/connectivity" "google.golang.org/grpc/internal/envconfig" "google.golang.org/grpc/internal/grpctest" @@ -261,8 +262,8 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { }() timerCh := make(chan struct{}) - originalTimer := timerAfterFunc - timerAfterFunc = func(_ time.Duration, f func()) *time.Timer { + originalTimer := internal.TimeAfterFunc + internal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { // Set a really long expiration to prevent it from triggering // automatically. ret := time.AfterFunc(time.Hour, f) @@ -277,7 +278,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { } defer func() { - timerAfterFunc = originalTimer + internal.TimeAfterFunc = originalTimer }() cc := testutils.NewBalancerClientConn(t) @@ -358,8 +359,8 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { }() timerCh := make(chan struct{}) - originalTimer := timerAfterFunc - timerAfterFunc = func(_ time.Duration, f func()) *time.Timer { + originalTimer := internal.TimeAfterFunc + internal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { // Set a really long expiration to prevent it from triggering // automatically. ret := time.AfterFunc(time.Hour, f) @@ -374,11 +375,11 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { } defer func() { - timerAfterFunc = originalTimer + internal.TimeAfterFunc = originalTimer }() defer func() { - timerAfterFunc = originalTimer + internal.TimeAfterFunc = originalTimer }() cc := testutils.NewBalancerClientConn(t) @@ -505,8 +506,8 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { timerMu := sync.Mutex{} timerCh := make(chan struct{}) - originalTimer := timerAfterFunc - timerAfterFunc = func(_ time.Duration, f func()) *time.Timer { + originalTimer := internal.TimeAfterFunc + internal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { // Set a really long expiration to prevent it from triggering // automatically. ret := time.AfterFunc(time.Hour, f) @@ -524,7 +525,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { } defer func() { - timerAfterFunc = originalTimer + internal.TimeAfterFunc = originalTimer }() cc := testutils.NewBalancerClientConn(t) From 30223043476ce7ed1e14c67bbd1cf8901e5252b5 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Wed, 16 Oct 2024 13:03:36 +0530 Subject: [PATCH 05/24] Remove env var --- .github/workflows/testing.yml | 4 +--- .../pickfirst/pickfirstleaf/pickfirstleaf.go | 3 --- .../pickfirstleaf/pickfirstleaf_test.go | 19 ------------------- internal/envconfig/envconfig.go | 5 ----- 4 files changed, 1 insertion(+), 30 deletions(-) diff --git a/.github/workflows/testing.yml b/.github/workflows/testing.yml index 9620c8959efe..a6576a21fa15 100644 --- a/.github/workflows/testing.yml +++ b/.github/workflows/testing.yml @@ -73,9 +73,7 @@ jobs: - type: tests goversion: '1.22' testflags: -race - grpcenv: | - 'GRPC_EXPERIMENTAL_ENABLE_NEW_PICK_FIRST=true' - 'GRPC_EXPERIMENTAL_ENABLE_PICK_FIRST_HAPPY_EYEBALLS=true' + grpcenv: 'GRPC_EXPERIMENTAL_ENABLE_NEW_PICK_FIRST=true' steps: # Setup the environment. diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index 1084152a9ec6..c0eaa4cdb820 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -432,9 +432,6 @@ func (b *pickfirstBalancer) requestConnectionLocked() { } func (b *pickfirstBalancer) scheduleNextConnectionLocked() { - if !envconfig.PickFirstHappyEyeballsEnabled { - return - } curAddr := b.addressList.currentAddress() b.cancelScheduled() ctx, cancel := context.WithCancel(context.Background()) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go index 1fe61f28fb73..0b5d09c50a53 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go @@ -29,7 +29,6 @@ import ( "google.golang.org/grpc/balancer" "google.golang.org/grpc/balancer/pickfirst/internal" "google.golang.org/grpc/connectivity" - "google.golang.org/grpc/internal/envconfig" "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/testutils" "google.golang.org/grpc/resolver" @@ -255,12 +254,6 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - originalCfg := envconfig.PickFirstHappyEyeballsEnabled - envconfig.PickFirstHappyEyeballsEnabled = true - defer func() { - envconfig.PickFirstHappyEyeballsEnabled = originalCfg - }() - timerCh := make(chan struct{}) originalTimer := internal.TimeAfterFunc internal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { @@ -352,12 +345,6 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - originalCfg := envconfig.PickFirstHappyEyeballsEnabled - envconfig.PickFirstHappyEyeballsEnabled = true - defer func() { - envconfig.PickFirstHappyEyeballsEnabled = originalCfg - }() - timerCh := make(chan struct{}) originalTimer := internal.TimeAfterFunc internal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { @@ -498,12 +485,6 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - originalCfg := envconfig.PickFirstHappyEyeballsEnabled - envconfig.PickFirstHappyEyeballsEnabled = true - defer func() { - envconfig.PickFirstHappyEyeballsEnabled = originalCfg - }() - timerMu := sync.Mutex{} timerCh := make(chan struct{}) originalTimer := internal.TimeAfterFunc diff --git a/internal/envconfig/envconfig.go b/internal/envconfig/envconfig.go index 725277d15085..f2630c10469f 100644 --- a/internal/envconfig/envconfig.go +++ b/internal/envconfig/envconfig.go @@ -54,11 +54,6 @@ var ( // setting the environment variable "GRPC_EXPERIMENTAL_ENABLE_NEW_PICK_FIRST" // to "true". NewPickFirstEnabled = boolFromEnv("GRPC_EXPERIMENTAL_ENABLE_NEW_PICK_FIRST", false) - // PickFirstHappyEyeballsEnabled is set if the happy eyeballs algorithm is - // used by the new pickfirst LB policy. This option may be overridden - // by setting the environment variable - // "GRPC_EXPERIMENTAL_ENABLE_PICK_FIRST_HAPPY_EYEBALLS" to "true". - PickFirstHappyEyeballsEnabled = boolFromEnv("GRPC_EXPERIMENTAL_ENABLE_PICK_FIRST_HAPPY_EYEBALLS", false) ) func boolFromEnv(envVar string, def bool) bool { From 0a3ffd330d31ae4632bb1efc13cefb1111145495 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Wed, 16 Oct 2024 16:15:09 +0530 Subject: [PATCH 06/24] Change to e2e style test --- .../pickfirstleaf/pickfirstleaf_ext_test.go | 362 ++++++++++++++++++ .../pickfirstleaf/pickfirstleaf_test.go | 336 ---------------- 2 files changed, 362 insertions(+), 336 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 2ab40ef1615a..bb5e74d33d21 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -21,14 +21,17 @@ package pickfirstleaf_test import ( "context" "fmt" + "net" "sync" "testing" "time" "github.com/google/go-cmp/cmp" + "golang.org/x/net/http2" "google.golang.org/grpc" "google.golang.org/grpc/balancer" + pfinternal "google.golang.org/grpc/balancer/pickfirst/internal" "google.golang.org/grpc/balancer/pickfirst/pickfirstleaf" "google.golang.org/grpc/codes" "google.golang.org/grpc/connectivity" @@ -851,6 +854,268 @@ func (s) TestPickFirstLeaf_EmptyAddressList(t *testing.T) { } } +// TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList verifies that pickfirst +// correctly detects the end of the first happy eyeballs pass when the timer +// causes pickfirst to reach the end of the address list and failures are +// reported out of order. +func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + timerCh := make(chan struct{}) + originalTimer := pfinternal.TimeAfterFunc + pfinternal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { + // Set a really long expiration to prevent it from triggering + // automatically. + ret := time.AfterFunc(time.Hour, f) + go func() { + select { + case <-ctx.Done(): + case <-timerCh: + } + ret.Reset(0) + }() + return ret + } + + defer func() { + pfinternal.TimeAfterFunc = originalTimer + }() + + defer func() { + pfinternal.TimeAfterFunc = originalTimer + }() + + servers := newHangingServerGroup(t, 3) + defer servers.close() + rb := manual.NewBuilderWithScheme("whatever") + rb.InitialState(resolver.State{Addresses: servers.addrs}) + cc, err := grpc.NewClient("whatever:///this-gets-overwritten", + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), + grpc.WithResolvers(rb)) + + if err != nil { + t.Fatal(err) + } + defer cc.Close() + cc.Connect() + + testutils.AwaitState(ctx, t, cc, connectivity.Connecting) + + // Verify that only the first server is contacted. + if err := servers.awaitContacted(ctx, 0); err != nil { + t.Fatalf("Server with address %q not contacted: %v", servers.addrs[0], err) + } + + // Ensure no other servers are contacted. + if got, want := servers.isContacted(1), false; got != want { + t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[1], got, want) + } + if got, want := servers.isContacted(2), false; got != want { + t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[2], got, want) + } + + // Make the happy eyeballs timer fire twice so that pickfirst reaches the + // last address in the list. + timerCh <- struct{}{} + + // Verify that the second server is contacted and 3rd isn't. + if err := servers.awaitContacted(ctx, 1); err != nil { + t.Fatalf("Server with address %q not contacted: %v", servers.addrs[1], err) + } + + if got, want := servers.isContacted(2), false; got != want { + t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[2], got, want) + } + timerCh <- struct{}{} + if err := servers.awaitContacted(ctx, 2); err != nil { + t.Fatalf("Server with address %q not contacted: %v", servers.addrs[2], err) + } + + // First SubConn Fails. + servers.closeConn(0) + + // No TF should be reported until the first pass is complete. + shortCtx, shortCancel := context.WithTimeout(ctx, defaultTestShortTimeout) + defer shortCancel() + + testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) + + // Move off the end of the list, pickfirst should still be waiting for TFs + // to be reported. + timerCh <- struct{}{} + + // Third SubConn fails. + servers.closeConn(2) + + testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) + + // Last SubConn fails, this should result in a TF update. + servers.closeConn(1) + testutils.AwaitState(ctx, t, cc, connectivity.TransientFailure) +} + +// TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay verifies that +// pickfirst attempts to connect to the second backend once the happy eyeballs +// timer expires. +func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + timerCh := make(chan struct{}) + originalTimer := pfinternal.TimeAfterFunc + pfinternal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { + // Set a really long expiration to prevent it from triggering + // automatically. + ret := time.AfterFunc(time.Hour, f) + go func() { + select { + case <-ctx.Done(): + case <-timerCh: + } + ret.Reset(0) + }() + return ret + } + + defer func() { + pfinternal.TimeAfterFunc = originalTimer + }() + + defer func() { + pfinternal.TimeAfterFunc = originalTimer + }() + + servers := newHangingServerGroup(t, 2) + defer servers.close() + rb := manual.NewBuilderWithScheme("whatever") + rb.InitialState(resolver.State{Addresses: servers.addrs}) + cc, err := grpc.NewClient("whatever:///this-gets-overwritten", + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), + grpc.WithResolvers(rb)) + + if err != nil { + t.Fatal(err) + } + defer cc.Close() + cc.Connect() + + testutils.AwaitState(ctx, t, cc, connectivity.Connecting) + + // Verify that the first server is contacted. + if err := servers.awaitContacted(ctx, 0); err != nil { + t.Fatalf("Server with address %q not contacted: %v", servers.addrs[0], err) + } + + if got, want := servers.isContacted(1), false; got != want { + t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[1], got, want) + } + + timerCh <- struct{}{} + + // Second connection attempt is successful. + if err := servers.awaitContacted(ctx, 1); err != nil { + t.Fatalf("Server with address %q not contacted: %v", servers.addrs[1], err) + } + servers.enterReady(1) + testutils.AwaitState(ctx, t, cc, connectivity.Ready) +} + +// TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires tests the pickfirst balancer +// by causing a SubConn to fail and then jumping to the 3rd SubConn after the +// happy eyeballs timer expires. +func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + timerMu := sync.Mutex{} + timerCh := make(chan struct{}) + originalTimer := pfinternal.TimeAfterFunc + pfinternal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { + // Set a really long expiration to prevent it from triggering + // automatically. + ret := time.AfterFunc(time.Hour, f) + go func() { + timerMu.Lock() + ch := timerCh + timerMu.Unlock() + select { + case <-ctx.Done(): + case <-ch: + } + ret.Reset(0) + }() + return ret + } + + defer func() { + pfinternal.TimeAfterFunc = originalTimer + }() + + servers := newHangingServerGroup(t, 3) + defer servers.close() + rb := manual.NewBuilderWithScheme("whatever") + rb.InitialState(resolver.State{Addresses: servers.addrs}) + cc, err := grpc.NewClient("whatever:///this-gets-overwritten", + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), + grpc.WithResolvers(rb)) + + if err != nil { + t.Fatal(err) + } + defer cc.Close() + cc.Connect() + + testutils.AwaitState(ctx, t, cc, connectivity.Connecting) + + // Verify that only the first server is contacted. + if err := servers.awaitContacted(ctx, 0); err != nil { + t.Fatalf("Server with address %q not contacted: %v", servers.addrs[0], err) + } + + // Ensure no other servers are contacted. + if got, want := servers.isContacted(1), false; got != want { + t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[1], got, want) + } + if got, want := servers.isContacted(2), false; got != want { + t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[2], got, want) + } + + // First SubConn Fails. + // Replace the timer channel so that the old timers don't attempt to read + // messages pushed next. + timerMu.Lock() + timerCh = make(chan struct{}) + timerMu.Unlock() + servers.closeConn(0) + + // The second server is contacted. + // Verify that only the first server is contacted. + if err := servers.awaitContacted(ctx, 1); err != nil { + t.Fatalf("Server with address %q not contacted: %v", servers.addrs[1], err) + } + + // Ensure no other servers are contacted. + if got, want := servers.isContacted(2), false; got != want { + t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[2], got, want) + } + + // The happy eyeballs timer expires, skipping server[1] and requesting the creation + // of a third SubConn. + timerCh <- struct{}{} + + if err := servers.awaitContacted(ctx, 2); err != nil { + t.Fatalf("Server with address %q not contacted: %v", servers.addrs[2], err) + } + + // Second SubConn connects. + servers.enterReady(1) + testutils.AwaitState(ctx, t, cc, connectivity.Connecting) +} + // stateStoringBalancer stores the state of the subconns being created. type stateStoringBalancer struct { balancer.Balancer @@ -955,3 +1220,100 @@ type ccStateSubscriber struct { func (c *ccStateSubscriber) OnMessage(msg any) { c.transitions = append(c.transitions, msg.(connectivity.State)) } + +// handingServerGroup is a group of servers that accept a TCP connection and +// remain idle until asked to close the connection. They can be used to control +// how long it takes for a subchannel to report a TRANSIENT_FAILURE in tests. +type handingServerGroup struct { + addrs []resolver.Address + listeners []net.Listener + serverConnCloseFuncs []func() + serverContacted []*grpcsync.Event + readyChans []chan struct{} +} + +func newHangingServerGroup(t *testing.T, count int) *handingServerGroup { + listeners := []net.Listener{} + closeFns := []func(){} + addrs := []resolver.Address{} + contactedEvents := []*grpcsync.Event{} + readyChans := []chan struct{}{} + + for i := 0; i < count; i++ { + lis, err := net.Listen("tcp", "localhost:0") + if err != nil { + t.Fatalf("Error while listening. Err: %v", err) + } + listeners = append(listeners, lis) + addrs = append(addrs, resolver.Address{Addr: lis.Addr().String()}) + closeChan := make(chan struct{}) + closeFns = append(closeFns, sync.OnceFunc(func() { close(closeChan) })) + contacted := grpcsync.NewEvent() + contactedEvents = append(contactedEvents, contacted) + readyChan := make(chan struct{}) + readyChans = append(readyChans, readyChan) + + go func() { + conn, err := lis.Accept() + defer conn.Close() + if err != nil { + t.Error(err) + return + } + + contacted.Fire() + + for { + select { + case <-closeChan: + return + case <-readyChan: + framer := http2.NewFramer(conn, conn) + if err := framer.WriteSettings(http2.Setting{}); err != nil { + t.Fatalf("Error while writing settings frame. %v", err) + return + } + + } + } + }() + } + + return &handingServerGroup{ + addrs: addrs, + listeners: listeners, + serverConnCloseFuncs: closeFns, + serverContacted: contactedEvents, + readyChans: readyChans, + } +} + +func (hg *handingServerGroup) close() { + for _, fn := range hg.serverConnCloseFuncs { + fn() + } + for _, l := range hg.listeners { + l.Close() + } +} + +func (hg *handingServerGroup) closeConn(serverIdx int) { + hg.serverConnCloseFuncs[serverIdx]() +} + +func (hg *handingServerGroup) isContacted(serverIdx int) bool { + return hg.serverContacted[serverIdx].HasFired() +} + +func (hg *handingServerGroup) enterReady(serverIdx int) { + hg.readyChans[serverIdx] <- struct{}{} +} + +func (hg *handingServerGroup) awaitContacted(ctx context.Context, serverIdx int) error { + select { + case <-ctx.Done(): + return ctx.Err() + case <-hg.serverContacted[serverIdx].Done(): + } + return nil +} diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go index 0b5d09c50a53..71984a238cd5 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go @@ -21,13 +21,11 @@ package pickfirstleaf import ( "context" "fmt" - "sync" "testing" "time" "google.golang.org/grpc/attributes" "google.golang.org/grpc/balancer" - "google.golang.org/grpc/balancer/pickfirst/internal" "google.golang.org/grpc/connectivity" "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/testutils" @@ -246,337 +244,3 @@ func (s) TestPickFirstLeaf_TFPickerUpdate(t *testing.T) { t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", newTfErr, err) } } - -// TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay verifies that -// pickfirst attempts to connect to the second backend once the happy eyeballs -// timer expires. -func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - - timerCh := make(chan struct{}) - originalTimer := internal.TimeAfterFunc - internal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { - // Set a really long expiration to prevent it from triggering - // automatically. - ret := time.AfterFunc(time.Hour, f) - go func() { - select { - case <-ctx.Done(): - case <-timerCh: - } - ret.Reset(0) - }() - return ret - } - - defer func() { - internal.TimeAfterFunc = originalTimer - }() - - cc := testutils.NewBalancerClientConn(t) - bal := pickfirstBuilder{}.Build(cc, balancer.BuildOptions{}) - defer bal.Close() - ccState := balancer.ClientConnState{ - ResolverState: resolver.State{ - Endpoints: []resolver.Endpoint{ - {Addresses: []resolver.Address{{Addr: "1.1.1.1:1"}}}, - {Addresses: []resolver.Address{{Addr: "2.2.2.2:2"}}}, - }, - }, - } - if err := bal.UpdateClientConnState(ccState); err != nil { - t.Fatalf("UpdateClientConnState(%v) returned error: %v", ccState, err) - } - - if err := cc.WaitForPickerWithErr(ctx, balancer.ErrNoSubConnAvailable); err != nil { - t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", balancer.ErrNoSubConnAvailable, err) - } - - if err := cc.WaitForConnectivityState(ctx, connectivity.Connecting); err != nil { - t.Fatalf("cc.WaitForConnectivityState(%v) returned error: %v", connectivity.Connecting, err) - } - - sc0 := <-cc.NewSubConnCh - - // Until the timer fires, no new subchannel should be created. - select { - case <-time.After(defaultTestShortTimeout): - case sc1 := <-cc.NewSubConnCh: - t.Fatalf("Received unexpected subchannel: %v", sc1) - } - - timerCh <- struct{}{} - - sc1 := <-cc.NewSubConnCh - - // Second connection attempt is successful. - sc1.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Connecting}) - sc1.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) - - if err := cc.WaitForConnectivityState(ctx, connectivity.Ready); err != nil { - t.Fatalf("cc.WaitForConnectivityState(%v) returned error: %v", connectivity.Ready, err) - } - - if err := cc.WaitForPicker(ctx, func(p balancer.Picker) error { - pr, err := p.Pick(balancer.PickInfo{}) - if err != nil { - return err - } - if pr.SubConn != sc1 { - t.Fatalf("Unexpected SubConn produced by picker, got = %v, want = %v", pr.SubConn, sc1) - } - return nil - }); err != nil { - t.Fatalf("cc.WaitForPicker() returned error: %v", err) - } - - closedSC := <-cc.ShutdownSubConnCh - if closedSC != sc0 { - t.Fatalf("Unexpected closed SubConn, got = %v, want = %v", closedSC, sc0) - } -} - -// TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList verifies that pickfirst -// correctly detects the end of the first happy eyeballs pass when the timer -// causes pickfirst to reach the end of the address list and failures are -// reported out of order. -func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - - timerCh := make(chan struct{}) - originalTimer := internal.TimeAfterFunc - internal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { - // Set a really long expiration to prevent it from triggering - // automatically. - ret := time.AfterFunc(time.Hour, f) - go func() { - select { - case <-ctx.Done(): - case <-timerCh: - } - ret.Reset(0) - }() - return ret - } - - defer func() { - internal.TimeAfterFunc = originalTimer - }() - - defer func() { - internal.TimeAfterFunc = originalTimer - }() - - cc := testutils.NewBalancerClientConn(t) - bal := pickfirstBuilder{}.Build(cc, balancer.BuildOptions{}) - defer bal.Close() - ccState := balancer.ClientConnState{ - ResolverState: resolver.State{ - Endpoints: []resolver.Endpoint{ - {Addresses: []resolver.Address{{Addr: "1.1.1.1:1"}}}, - {Addresses: []resolver.Address{{Addr: "2.2.2.2:2"}}}, - {Addresses: []resolver.Address{{Addr: "3.3.3.3:3"}}}, - }, - }, - } - if err := bal.UpdateClientConnState(ccState); err != nil { - t.Fatalf("UpdateClientConnState(%v) returned error: %v", ccState, err) - } - - if err := cc.WaitForPickerWithErr(ctx, balancer.ErrNoSubConnAvailable); err != nil { - t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", balancer.ErrNoSubConnAvailable, err) - } - - if err := cc.WaitForConnectivityState(ctx, connectivity.Connecting); err != nil { - t.Fatalf("cc.WaitForConnectivityState(%v) returned error: %v", connectivity.Connecting, err) - } - - sc0 := <-cc.NewSubConnCh - // Make the happy eyeballs timer fire twice so that pickfirst reaches the - // last address in the list. - timerCh <- struct{}{} - sc1 := <-cc.NewSubConnCh - timerCh <- struct{}{} - sc2 := <-cc.NewSubConnCh - - // First SubConn Fails. - tfErr := fmt.Errorf("test error") - sc0.UpdateState(balancer.SubConnState{ - ConnectivityState: connectivity.TransientFailure, - ConnectionError: tfErr, - }) - - // No picker should be produced until the first pass is complete. - select { - case <-time.After(defaultTestShortTimeout): - case p := <-cc.NewPickerCh: - sc, err := p.Pick(balancer.PickInfo{}) - t.Fatalf("Unexpected picker update: %v, %v", sc, err) - } - - // Move off the end of the list, pickfirst should still be waiting for TFs - // to be reported. - timerCh <- struct{}{} - - // second SubConn fails. - sc1.UpdateState(balancer.SubConnState{ - ConnectivityState: connectivity.TransientFailure, - ConnectionError: tfErr, - }) - - // second SubConn fails again, we've still not seen the 3rd SubConn fail. - // Second connection attempt is successful. - sc1.UpdateState(balancer.SubConnState{ - ConnectivityState: connectivity.TransientFailure, - ConnectionError: tfErr, - }) - - select { - case <-time.After(defaultTestShortTimeout): - case p := <-cc.NewPickerCh: - sc, err := p.Pick(balancer.PickInfo{}) - t.Fatalf("Unexpected picker update: %v, %v", sc, err) - } - - // Last SubConn fails, this should result in a picker update. - sc2.UpdateState(balancer.SubConnState{ - ConnectivityState: connectivity.TransientFailure, - ConnectionError: tfErr, - }) - - if err := cc.WaitForPickerWithErr(ctx, tfErr); err != nil { - t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", tfErr, err) - } - - // Fail the first SubConn 3 times, causing the next picker update. - tfErr = fmt.Errorf("New test error") - sc0.UpdateState(balancer.SubConnState{ - ConnectivityState: connectivity.TransientFailure, - ConnectionError: tfErr, - }) - sc0.UpdateState(balancer.SubConnState{ - ConnectivityState: connectivity.TransientFailure, - ConnectionError: tfErr, - }) - - select { - case <-time.After(defaultTestShortTimeout): - case p := <-cc.NewPickerCh: - sc, err := p.Pick(balancer.PickInfo{}) - t.Fatalf("Unexpected picker update: %v, %v", sc, err) - } - - sc0.UpdateState(balancer.SubConnState{ - ConnectivityState: connectivity.TransientFailure, - ConnectionError: tfErr, - }) - - if err := cc.WaitForPickerWithErr(ctx, tfErr); err != nil { - t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", tfErr, err) - } -} - -// TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires tests the pickfirst balancer -// by causing a SubConn to fail and then jumping to the 3rd SubConn after the -// happy eyeballs timer expires. -func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - - timerMu := sync.Mutex{} - timerCh := make(chan struct{}) - originalTimer := internal.TimeAfterFunc - internal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { - // Set a really long expiration to prevent it from triggering - // automatically. - ret := time.AfterFunc(time.Hour, f) - go func() { - timerMu.Lock() - ch := timerCh - timerMu.Unlock() - select { - case <-ctx.Done(): - case <-ch: - } - ret.Reset(0) - }() - return ret - } - - defer func() { - internal.TimeAfterFunc = originalTimer - }() - - cc := testutils.NewBalancerClientConn(t) - bal := pickfirstBuilder{}.Build(cc, balancer.BuildOptions{}) - defer bal.Close() - ccState := balancer.ClientConnState{ - ResolverState: resolver.State{ - Endpoints: []resolver.Endpoint{ - {Addresses: []resolver.Address{{Addr: "1.1.1.1:1"}}}, - {Addresses: []resolver.Address{{Addr: "2.2.2.2:2"}}}, - {Addresses: []resolver.Address{{Addr: "3.3.3.3:3"}}}, - }, - }, - } - if err := bal.UpdateClientConnState(ccState); err != nil { - t.Fatalf("UpdateClientConnState(%v) returned error: %v", ccState, err) - } - - if err := cc.WaitForPickerWithErr(ctx, balancer.ErrNoSubConnAvailable); err != nil { - t.Fatalf("cc.WaitForPickerWithErr(%v) returned error: %v", balancer.ErrNoSubConnAvailable, err) - } - - if err := cc.WaitForConnectivityState(ctx, connectivity.Connecting); err != nil { - t.Fatalf("cc.WaitForConnectivityState(%v) returned error: %v", connectivity.Connecting, err) - } - - sc0 := <-cc.NewSubConnCh - // Until the timer fires or a TF is reported, no new subchannel should be - // created. - select { - case <-time.After(defaultTestShortTimeout): - case sc1 := <-cc.NewSubConnCh: - t.Fatalf("Received unexpected subchannel: %v", sc1) - } - - // First SubConn Fails. - tfErr := fmt.Errorf("test error") - // Replace the timer channel so that the old timers don't attempt to read - // messages pushed next. - timerMu.Lock() - timerCh = make(chan struct{}) - timerMu.Unlock() - sc0.UpdateState(balancer.SubConnState{ - ConnectivityState: connectivity.TransientFailure, - ConnectionError: tfErr, - }) - - sc1 := <-cc.NewSubConnCh - sc1.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Connecting}) - - // The happy eyeballs timer expires, skipping sc1 and requesting the creation - // of sc2. - timerCh <- struct{}{} - <-cc.NewSubConnCh - - // First SubConn comes out of backoff. - sc0.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Idle}) - // Second SubConn connects. - sc1.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) - // Verify that the picker is updated. - if err := cc.WaitForPicker(ctx, func(p balancer.Picker) error { - pr, err := p.Pick(balancer.PickInfo{}) - if err != nil { - return err - } - if pr.SubConn != sc1 { - t.Fatalf("Unexpected SubConn produced by picker, got = %v, want = %v", pr.SubConn, sc1) - } - return nil - }); err != nil { - t.Fatalf("cc.WaitForPicker() returned error: %v", err) - } -} From 66972677b868e569dd467363daf2e60a49111765 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Wed, 16 Oct 2024 16:30:42 +0530 Subject: [PATCH 07/24] Fix vet --- scripts/vet.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scripts/vet.sh b/scripts/vet.sh index 3b8943b0fa65..aba59a5995bc 100755 --- a/scripts/vet.sh +++ b/scripts/vet.sh @@ -97,13 +97,13 @@ for MOD_FILE in $(find . -name 'go.mod'); do gofmt -s -d -l . 2>&1 | fail_on_output goimports -l . 2>&1 | not grep -vE "\.pb\.go" - go mod tidy -compat=1.21 + go mod tidy -compat=1.22 git status --porcelain 2>&1 | fail_on_output || \ (git status; git --no-pager diff; exit 1) # - Collection of static analysis checks SC_OUT="$(mktemp)" - staticcheck -go 1.21 -checks 'all' ./... >"${SC_OUT}" || true + staticcheck -go 1.22 -checks 'all' ./... >"${SC_OUT}" || true # Error for anything other than checks that need exclusions. noret_grep -v "(ST1000)" "${SC_OUT}" | noret_grep -v "(SA1019)" | noret_grep -v "(ST1003)" | noret_grep -v "(ST1019)\|\(other import of\)" | not grep -v "(SA4000)" From 67f7a1aa73385b687feb0e507dd133625f060c11 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Wed, 16 Oct 2024 16:36:39 +0530 Subject: [PATCH 08/24] Fix vet --- balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index bb5e74d33d21..f4ccbbae9956 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -1255,11 +1255,11 @@ func newHangingServerGroup(t *testing.T, count int) *handingServerGroup { go func() { conn, err := lis.Accept() - defer conn.Close() if err != nil { t.Error(err) return } + defer conn.Close() contacted.Fire() @@ -1270,7 +1270,7 @@ func newHangingServerGroup(t *testing.T, count int) *handingServerGroup { case <-readyChan: framer := http2.NewFramer(conn, conn) if err := framer.WriteSettings(http2.Setting{}); err != nil { - t.Fatalf("Error while writing settings frame. %v", err) + t.Errorf("Error while writing settings frame. %v", err) return } From 9712ec552754db5e2b0127ab936dc59db8de09c5 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Wed, 16 Oct 2024 17:01:27 +0530 Subject: [PATCH 09/24] refactor test --- .../pickfirstleaf/pickfirstleaf_ext_test.go | 168 +++++++++--------- 1 file changed, 81 insertions(+), 87 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index f4ccbbae9956..1d7086516371 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -887,9 +887,9 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { }() servers := newHangingServerGroup(t, 3) - defer servers.close() rb := manual.NewBuilderWithScheme("whatever") - rb.InitialState(resolver.State{Addresses: servers.addrs}) + addrs := resolverAddrsFromHangingServers(servers) + rb.InitialState(resolver.State{Addresses: addrs}) cc, err := grpc.NewClient("whatever:///this-gets-overwritten", grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), @@ -904,16 +904,16 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { testutils.AwaitState(ctx, t, cc, connectivity.Connecting) // Verify that only the first server is contacted. - if err := servers.awaitContacted(ctx, 0); err != nil { - t.Fatalf("Server with address %q not contacted: %v", servers.addrs[0], err) + if err := servers[0].awaitContacted(ctx); err != nil { + t.Fatalf("Server with address %q not contacted: %v", addrs[0], err) } // Ensure no other servers are contacted. - if got, want := servers.isContacted(1), false; got != want { - t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[1], got, want) + if got, want := servers[1].isContacted(), false; got != want { + t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[1], got, want) } - if got, want := servers.isContacted(2), false; got != want { - t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[2], got, want) + if got, want := servers[2].isContacted(), false; got != want { + t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[2], got, want) } // Make the happy eyeballs timer fire twice so that pickfirst reaches the @@ -921,20 +921,20 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { timerCh <- struct{}{} // Verify that the second server is contacted and 3rd isn't. - if err := servers.awaitContacted(ctx, 1); err != nil { - t.Fatalf("Server with address %q not contacted: %v", servers.addrs[1], err) + if err := servers[1].awaitContacted(ctx); err != nil { + t.Fatalf("Server with address %q not contacted: %v", addrs[1], err) } - if got, want := servers.isContacted(2), false; got != want { - t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[2], got, want) + if got, want := servers[2].isContacted(), false; got != want { + t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[2], got, want) } timerCh <- struct{}{} - if err := servers.awaitContacted(ctx, 2); err != nil { - t.Fatalf("Server with address %q not contacted: %v", servers.addrs[2], err) + if err := servers[2].awaitContacted(ctx); err != nil { + t.Fatalf("Server with address %q not contacted: %v", addrs[2], err) } // First SubConn Fails. - servers.closeConn(0) + servers[0].closeConn() // No TF should be reported until the first pass is complete. shortCtx, shortCancel := context.WithTimeout(ctx, defaultTestShortTimeout) @@ -947,12 +947,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { timerCh <- struct{}{} // Third SubConn fails. - servers.closeConn(2) + servers[2].closeConn() testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) // Last SubConn fails, this should result in a TF update. - servers.closeConn(1) + servers[1].closeConn() testutils.AwaitState(ctx, t, cc, connectivity.TransientFailure) } @@ -988,9 +988,9 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { }() servers := newHangingServerGroup(t, 2) - defer servers.close() + addrs := resolverAddrsFromHangingServers(servers) rb := manual.NewBuilderWithScheme("whatever") - rb.InitialState(resolver.State{Addresses: servers.addrs}) + rb.InitialState(resolver.State{Addresses: addrs}) cc, err := grpc.NewClient("whatever:///this-gets-overwritten", grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), @@ -1005,21 +1005,21 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { testutils.AwaitState(ctx, t, cc, connectivity.Connecting) // Verify that the first server is contacted. - if err := servers.awaitContacted(ctx, 0); err != nil { - t.Fatalf("Server with address %q not contacted: %v", servers.addrs[0], err) + if err := servers[0].awaitContacted(ctx); err != nil { + t.Fatalf("Server with address %q not contacted: %v", addrs[0], err) } - if got, want := servers.isContacted(1), false; got != want { - t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[1], got, want) + if got, want := servers[1].isContacted(), false; got != want { + t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[1], got, want) } timerCh <- struct{}{} // Second connection attempt is successful. - if err := servers.awaitContacted(ctx, 1); err != nil { - t.Fatalf("Server with address %q not contacted: %v", servers.addrs[1], err) + if err := servers[1].awaitContacted(ctx); err != nil { + t.Fatalf("Server with address %q not contacted: %v", addrs[1], err) } - servers.enterReady(1) + servers[1].enterReady() testutils.AwaitState(ctx, t, cc, connectivity.Ready) } @@ -1055,9 +1055,9 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { }() servers := newHangingServerGroup(t, 3) - defer servers.close() + addrs := resolverAddrsFromHangingServers(servers) rb := manual.NewBuilderWithScheme("whatever") - rb.InitialState(resolver.State{Addresses: servers.addrs}) + rb.InitialState(resolver.State{Addresses: addrs}) cc, err := grpc.NewClient("whatever:///this-gets-overwritten", grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), @@ -1072,16 +1072,16 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { testutils.AwaitState(ctx, t, cc, connectivity.Connecting) // Verify that only the first server is contacted. - if err := servers.awaitContacted(ctx, 0); err != nil { - t.Fatalf("Server with address %q not contacted: %v", servers.addrs[0], err) + if err := servers[0].awaitContacted(ctx); err != nil { + t.Fatalf("Server with address %q not contacted: %v", addrs[0], err) } // Ensure no other servers are contacted. - if got, want := servers.isContacted(1), false; got != want { - t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[1], got, want) + if got, want := servers[1].isContacted(), false; got != want { + t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[1], got, want) } - if got, want := servers.isContacted(2), false; got != want { - t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[2], got, want) + if got, want := servers[2].isContacted(), false; got != want { + t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[2], got, want) } // First SubConn Fails. @@ -1090,29 +1090,29 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { timerMu.Lock() timerCh = make(chan struct{}) timerMu.Unlock() - servers.closeConn(0) + servers[0].closeConn() // The second server is contacted. // Verify that only the first server is contacted. - if err := servers.awaitContacted(ctx, 1); err != nil { - t.Fatalf("Server with address %q not contacted: %v", servers.addrs[1], err) + if err := servers[1].awaitContacted(ctx); err != nil { + t.Fatalf("Server with address %q not contacted: %v", addrs[1], err) } // Ensure no other servers are contacted. - if got, want := servers.isContacted(2), false; got != want { - t.Fatalf("Servers.isContacted(%q) = %t, want %t", servers.addrs[2], got, want) + if got, want := servers[2].isContacted(), false; got != want { + t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[2], got, want) } // The happy eyeballs timer expires, skipping server[1] and requesting the creation // of a third SubConn. timerCh <- struct{}{} - if err := servers.awaitContacted(ctx, 2); err != nil { - t.Fatalf("Server with address %q not contacted: %v", servers.addrs[2], err) + if err := servers[2].awaitContacted(ctx); err != nil { + t.Fatalf("Server with address %q not contacted: %v", addrs[2], err) } // Second SubConn connects. - servers.enterReady(1) + servers[1].enterReady() testutils.AwaitState(ctx, t, cc, connectivity.Connecting) } @@ -1221,37 +1221,27 @@ func (c *ccStateSubscriber) OnMessage(msg any) { c.transitions = append(c.transitions, msg.(connectivity.State)) } -// handingServerGroup is a group of servers that accept a TCP connection and -// remain idle until asked to close the connection. They can be used to control +// hangingServer is a server that accept a TCP connection and remains idle until +// asked to close or respond to the connection. They can be used to control // how long it takes for a subchannel to report a TRANSIENT_FAILURE in tests. -type handingServerGroup struct { - addrs []resolver.Address - listeners []net.Listener - serverConnCloseFuncs []func() - serverContacted []*grpcsync.Event - readyChans []chan struct{} +type hangingServer struct { + addr resolver.Address + listener net.Listener + closeConn func() + contacted *grpcsync.Event + readyChan chan struct{} } -func newHangingServerGroup(t *testing.T, count int) *handingServerGroup { - listeners := []net.Listener{} - closeFns := []func(){} - addrs := []resolver.Address{} - contactedEvents := []*grpcsync.Event{} - readyChans := []chan struct{}{} - +func newHangingServerGroup(t *testing.T, count int) []*hangingServer { + servers := []*hangingServer{} for i := 0; i < count; i++ { lis, err := net.Listen("tcp", "localhost:0") if err != nil { t.Fatalf("Error while listening. Err: %v", err) } - listeners = append(listeners, lis) - addrs = append(addrs, resolver.Address{Addr: lis.Addr().String()}) closeChan := make(chan struct{}) - closeFns = append(closeFns, sync.OnceFunc(func() { close(closeChan) })) contacted := grpcsync.NewEvent() - contactedEvents = append(contactedEvents, contacted) readyChan := make(chan struct{}) - readyChans = append(readyChans, readyChan) go func() { conn, err := lis.Accept() @@ -1277,43 +1267,47 @@ func newHangingServerGroup(t *testing.T, count int) *handingServerGroup { } } }() - } - - return &handingServerGroup{ - addrs: addrs, - listeners: listeners, - serverConnCloseFuncs: closeFns, - serverContacted: contactedEvents, - readyChans: readyChans, - } -} + server := &hangingServer{ + addr: resolver.Address{Addr: lis.Addr().String()}, + listener: lis, + closeConn: grpcsync.OnceFunc(func() { + close(closeChan) + }), + contacted: contacted, + readyChan: readyChan, + } + servers = append(servers, server) -func (hg *handingServerGroup) close() { - for _, fn := range hg.serverConnCloseFuncs { - fn() - } - for _, l := range hg.listeners { - l.Close() + t.Cleanup(func() { + server.closeConn() + server.listener.Close() + }) } -} -func (hg *handingServerGroup) closeConn(serverIdx int) { - hg.serverConnCloseFuncs[serverIdx]() + return servers } -func (hg *handingServerGroup) isContacted(serverIdx int) bool { - return hg.serverContacted[serverIdx].HasFired() +func (s *hangingServer) isContacted() bool { + return s.contacted.HasFired() } -func (hg *handingServerGroup) enterReady(serverIdx int) { - hg.readyChans[serverIdx] <- struct{}{} +func (s *hangingServer) enterReady() { + s.readyChan <- struct{}{} } -func (hg *handingServerGroup) awaitContacted(ctx context.Context, serverIdx int) error { +func (s *hangingServer) awaitContacted(ctx context.Context) error { select { case <-ctx.Done(): return ctx.Err() - case <-hg.serverContacted[serverIdx].Done(): + case <-s.contacted.Done(): } return nil } + +func resolverAddrsFromHangingServers(servers []*hangingServer) []resolver.Address { + addrs := []resolver.Address{} + for _, srv := range servers { + addrs = append(addrs, srv.addr) + } + return addrs +} From 6c8fb4123c66e8aa2ec9af308bfaef2e494f7316 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Thu, 17 Oct 2024 01:59:53 +0530 Subject: [PATCH 10/24] Avoid creating a context --- balancer/pickfirst/pickfirstleaf/pickfirstleaf.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index c0eaa4cdb820..54e9d926801c 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -26,7 +26,6 @@ package pickfirstleaf import ( - "context" "encoding/json" "errors" "fmt" @@ -434,12 +433,12 @@ func (b *pickfirstBalancer) requestConnectionLocked() { func (b *pickfirstBalancer) scheduleNextConnectionLocked() { curAddr := b.addressList.currentAddress() b.cancelScheduled() - ctx, cancel := context.WithCancel(context.Background()) + cancelled := false // Access to this is protected by the balancer's mutex. closeFn := internal.TimeAfterFunc(connectionDelayInterval, func() { b.mu.Lock() defer b.mu.Unlock() // If the scheduled task is cancelled while acquiring the mutex, return. - if ctx.Err() != nil { + if cancelled { return } if b.logger.V(2) { @@ -450,7 +449,7 @@ func (b *pickfirstBalancer) scheduleNextConnectionLocked() { } }).Stop b.cancelScheduled = sync.OnceFunc(func() { - cancel() + cancelled = true closeFn() }) } From 04a912f521543fd120a6ef8856916004b9bcdd0e Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Thu, 17 Oct 2024 10:55:19 +0530 Subject: [PATCH 11/24] address review comments --- .../pickfirst/pickfirstleaf/pickfirstleaf.go | 50 ++++++++++++------- .../pickfirstleaf/pickfirstleaf_ext_test.go | 25 ++++------ 2 files changed, 43 insertions(+), 32 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index 54e9d926801c..7622b3d2d83a 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -71,12 +71,12 @@ type pickfirstBuilder struct{} func (pickfirstBuilder) Build(cc balancer.ClientConn, _ balancer.BuildOptions) balancer.Balancer { b := &pickfirstBalancer{ - cc: cc, - addressList: addressList{}, - subConns: resolver.NewAddressMap(), - state: connectivity.Connecting, - mu: sync.Mutex{}, - cancelScheduled: func() {}, + cc: cc, + addressList: addressList{}, + subConns: resolver.NewAddressMap(), + state: connectivity.Connecting, + mu: sync.Mutex{}, + cancelConnectionTimer: func() {}, } b.logger = internalgrpclog.NewPrefixLogger(logger, fmt.Sprintf(logPrefix, b)) return b @@ -145,11 +145,11 @@ type pickfirstBalancer struct { mu sync.Mutex state connectivity.State // scData for active subonns mapped by address. - subConns *resolver.AddressMap - addressList addressList - firstPass bool - numTF int - cancelScheduled func() + subConns *resolver.AddressMap + addressList addressList + firstPass bool + numTF int + cancelConnectionTimer func() } // ResolverError is called by the ClientConn when the name resolver produces @@ -284,7 +284,7 @@ func (b *pickfirstBalancer) Close() { b.mu.Lock() defer b.mu.Unlock() b.closeSubConnsLocked() - b.cancelScheduled() + b.cancelConnectionTimer() b.state = connectivity.Shutdown } @@ -357,7 +357,7 @@ func (b *pickfirstBalancer) reconcileSubConnsLocked(newAddrs []resolver.Address) // shutdownRemainingLocked shuts down remaining subConns. Called when a subConn // becomes ready, which means that all other subConn must be shutdown. func (b *pickfirstBalancer) shutdownRemainingLocked(selected *scData) { - b.cancelScheduled() + b.cancelConnectionTimer() for _, v := range b.subConns.Values() { sd := v.(*scData) if sd.subConn != selected.subConn { @@ -419,7 +419,8 @@ func (b *pickfirstBalancer) requestConnectionLocked() { b.logger.Errorf("SubConn with state SHUTDOWN present in SubConns map") return case connectivity.Connecting: - // Wait for the SubConn to report success or failure. + // Wait for the connection attempt to complete or the timer to fire + // before attempting the next address. b.scheduleNextConnectionLocked() return } @@ -431,8 +432,11 @@ func (b *pickfirstBalancer) requestConnectionLocked() { } func (b *pickfirstBalancer) scheduleNextConnectionLocked() { + b.cancelConnectionTimer() + if !b.addressList.hasNext() { + return + } curAddr := b.addressList.currentAddress() - b.cancelScheduled() cancelled := false // Access to this is protected by the balancer's mutex. closeFn := internal.TimeAfterFunc(connectionDelayInterval, func() { b.mu.Lock() @@ -448,7 +452,9 @@ func (b *pickfirstBalancer) scheduleNextConnectionLocked() { b.requestConnectionLocked() } }).Stop - b.cancelScheduled = sync.OnceFunc(func() { + // Access to the cancellation callback held by the balancer is guarded by + // the balancer's mutex, so it's safe to set the boolean from the callback. + b.cancelConnectionTimer = sync.OnceFunc(func() { cancelled = true closeFn() }) @@ -531,7 +537,7 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub // cause out of order updates to arrive. if curAddr := b.addressList.currentAddress(); equalAddressIgnoringBalAttributes(&curAddr, &sd.addr) { - b.cancelScheduled() + b.cancelConnectionTimer() if b.addressList.increment() { b.requestConnectionLocked() return @@ -669,6 +675,16 @@ func (al *addressList) seekTo(needle resolver.Address) bool { return false } +// hasNext returns whether incrementing the addressList will result in moving +// past the end of the list. If the list has already moved past the end, it +// returns false. +func (al *addressList) hasNext() bool { + if !al.isValid() { + return false + } + return al.idx+1 < len(al.addresses) +} + // equalAddressIgnoringBalAttributes returns true is a and b are considered // equal. This is different from the Equal method on the resolver.Address type // which considers all fields to determine equality. Here, we only consider diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 1d7086516371..e99f91468627 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -867,15 +867,15 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { pfinternal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { // Set a really long expiration to prevent it from triggering // automatically. - ret := time.AfterFunc(time.Hour, f) + timer := time.AfterFunc(time.Hour, f) go func() { select { case <-ctx.Done(): case <-timerCh: } - ret.Reset(0) + timer.Reset(0) }() - return ret + return timer } defer func() { @@ -942,10 +942,6 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) - // Move off the end of the list, pickfirst should still be waiting for TFs - // to be reported. - timerCh <- struct{}{} - // Third SubConn fails. servers[2].closeConn() @@ -968,15 +964,15 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { pfinternal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { // Set a really long expiration to prevent it from triggering // automatically. - ret := time.AfterFunc(time.Hour, f) + timer := time.AfterFunc(time.Hour, f) go func() { select { case <-ctx.Done(): case <-timerCh: } - ret.Reset(0) + timer.Reset(0) }() - return ret + return timer } defer func() { @@ -1036,7 +1032,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { pfinternal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { // Set a really long expiration to prevent it from triggering // automatically. - ret := time.AfterFunc(time.Hour, f) + timer := time.AfterFunc(time.Hour, f) go func() { timerMu.Lock() ch := timerCh @@ -1045,9 +1041,9 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { case <-ctx.Done(): case <-ch: } - ret.Reset(0) + timer.Reset(0) }() - return ret + return timer } defer func() { @@ -1092,8 +1088,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { timerMu.Unlock() servers[0].closeConn() - // The second server is contacted. - // Verify that only the first server is contacted. + // Verify that only the second server is contacted. if err := servers[1].awaitContacted(ctx); err != nil { t.Fatalf("Server with address %q not contacted: %v", addrs[1], err) } From 0bb745f029b11c4ed03ae105044391b597c832f9 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Thu, 17 Oct 2024 11:36:00 +0530 Subject: [PATCH 12/24] Use BlockingDialer instead of implementing a hanging server --- .../pickfirstleaf/pickfirstleaf_ext_test.go | 264 ++++++------------ 1 file changed, 83 insertions(+), 181 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index e99f91468627..7e22d2283a3c 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -21,13 +21,11 @@ package pickfirstleaf_test import ( "context" "fmt" - "net" "sync" "testing" "time" "github.com/google/go-cmp/cmp" - "golang.org/x/net/http2" "google.golang.org/grpc" "google.golang.org/grpc/balancer" @@ -70,8 +68,7 @@ func Test(t *testing.T) { } // setupPickFirstLeaf performs steps required for pick_first tests. It starts a -// bunch of backends exporting the TestService, creates a ClientConn to them -// with service config specifying the use of the state_storing LB policy. +// bunch of backends exporting the TestService, and creates a ClientConn to them. func setupPickFirstLeaf(t *testing.T, backendCount int, opts ...grpc.DialOption) (*grpc.ClientConn, *manual.Resolver, *backendManager) { t.Helper() r := manual.NewBuilderWithScheme("whatever") @@ -90,7 +87,6 @@ func setupPickFirstLeaf(t *testing.T, backendCount int, opts ...grpc.DialOption) dopts := []grpc.DialOption{ grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(r), - grpc.WithDefaultServiceConfig(stateStoringServiceConfig), } dopts = append(dopts, opts...) cc, err := grpc.NewClient(r.Scheme()+":///test.server", dopts...) @@ -125,7 +121,7 @@ func (s) TestPickFirstLeaf_SimpleResolverUpdate_FirstServerReady(t *testing.T) { balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 2) + cc, r, bm := setupPickFirstLeaf(t, 2, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -165,7 +161,7 @@ func (s) TestPickFirstLeaf_SimpleResolverUpdate_FirstServerUnReady(t *testing.T) balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 2) + cc, r, bm := setupPickFirstLeaf(t, 2, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -207,7 +203,7 @@ func (s) TestPickFirstLeaf_SimpleResolverUpdate_DuplicateAddrs(t *testing.T) { balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 2) + cc, r, bm := setupPickFirstLeaf(t, 2, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -263,7 +259,7 @@ func (s) TestPickFirstLeaf_ResolverUpdates_DisjointLists(t *testing.T) { balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 4) + cc, r, bm := setupPickFirstLeaf(t, 4, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -326,7 +322,7 @@ func (s) TestPickFirstLeaf_ResolverUpdates_ActiveBackendInUpdatedList(t *testing balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 3) + cc, r, bm := setupPickFirstLeaf(t, 3, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -390,7 +386,7 @@ func (s) TestPickFirstLeaf_ResolverUpdates_InActiveBackendInUpdatedList(t *testi balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 3) + cc, r, bm := setupPickFirstLeaf(t, 3, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -455,7 +451,7 @@ func (s) TestPickFirstLeaf_ResolverUpdates_IdenticalLists(t *testing.T) { balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 2) + cc, r, bm := setupPickFirstLeaf(t, 2, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -528,7 +524,7 @@ func (s) TestPickFirstLeaf_StopConnectedServer_FirstServerRestart(t *testing.T) balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 2) + cc, r, bm := setupPickFirstLeaf(t, 2, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -593,7 +589,7 @@ func (s) TestPickFirstLeaf_StopConnectedServer_SecondServerRestart(t *testing.T) balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 2) + cc, r, bm := setupPickFirstLeaf(t, 2, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -665,7 +661,7 @@ func (s) TestPickFirstLeaf_StopConnectedServer_SecondServerToFirst(t *testing.T) balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 2) + cc, r, bm := setupPickFirstLeaf(t, 2, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -737,7 +733,7 @@ func (s) TestPickFirstLeaf_StopConnectedServer_FirstServerToSecond(t *testing.T) balCh := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balCh}) - cc, r, bm := setupPickFirstLeaf(t, 2) + cc, r, bm := setupPickFirstLeaf(t, 2, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) @@ -811,7 +807,7 @@ func (s) TestPickFirstLeaf_EmptyAddressList(t *testing.T) { defer cancel() balChan := make(chan *stateStoringBalancer, 1) balancer.Register(&stateStoringBalancerBuilder{balancer: balChan}) - cc, r, bm := setupPickFirstLeaf(t, 1) + cc, r, bm := setupPickFirstLeaf(t, 1, grpc.WithDefaultServiceConfig(stateStoringServiceConfig)) addrs := bm.resolverAddrs() stateSubscriber := &ccStateSubscriber{} @@ -886,34 +882,30 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { pfinternal.TimeAfterFunc = originalTimer }() - servers := newHangingServerGroup(t, 3) - rb := manual.NewBuilderWithScheme("whatever") - addrs := resolverAddrsFromHangingServers(servers) - rb.InitialState(resolver.State{Addresses: addrs}) - cc, err := grpc.NewClient("whatever:///this-gets-overwritten", - grpc.WithTransportCredentials(insecure.NewCredentials()), + dialer := testutils.NewBlockingDialer() + cc, rb, bm := setupPickFirstLeaf(t, 3, grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), - grpc.WithResolvers(rb)) - - if err != nil { - t.Fatal(err) - } - defer cc.Close() + grpc.WithContextDialer(dialer.DialContext), + ) + addrs := bm.resolverAddrs() + holds := bm.holds(dialer) + rb.UpdateState(resolver.State{Addresses: addrs}) cc.Connect() testutils.AwaitState(ctx, t, cc, connectivity.Connecting) // Verify that only the first server is contacted. - if err := servers[0].awaitContacted(ctx); err != nil { - t.Fatalf("Server with address %q not contacted: %v", addrs[0], err) + if got, want := holds[0].Wait(ctx), true; got != want { + t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[0], got, want) } // Ensure no other servers are contacted. - if got, want := servers[1].isContacted(), false; got != want { - t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[1], got, want) + if got, want := holds[1].IsStarted(), false; got != want { + t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[1], got, want) } - if got, want := servers[2].isContacted(), false; got != want { - t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[2], got, want) + + if got, want := holds[2].IsStarted(), false; got != want { + t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[2], got, want) } // Make the happy eyeballs timer fire twice so that pickfirst reaches the @@ -921,20 +913,22 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { timerCh <- struct{}{} // Verify that the second server is contacted and 3rd isn't. - if err := servers[1].awaitContacted(ctx); err != nil { - t.Fatalf("Server with address %q not contacted: %v", addrs[1], err) + if got, want := holds[1].Wait(ctx), true; got != want { + t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[1], got, want) } - if got, want := servers[2].isContacted(), false; got != want { - t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[2], got, want) + if got, want := holds[2].IsStarted(), false; got != want { + t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[2], got, want) } + timerCh <- struct{}{} - if err := servers[2].awaitContacted(ctx); err != nil { - t.Fatalf("Server with address %q not contacted: %v", addrs[2], err) + // Verify that the second server is contacted and 3rd isn't. + if got, want := holds[1].Wait(ctx), true; got != want { + t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[1], got, want) } // First SubConn Fails. - servers[0].closeConn() + holds[0].Fail(fmt.Errorf("test error")) // No TF should be reported until the first pass is complete. shortCtx, shortCancel := context.WithTimeout(ctx, defaultTestShortTimeout) @@ -943,12 +937,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) // Third SubConn fails. - servers[2].closeConn() + holds[2].Fail(fmt.Errorf("test error")) testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) // Last SubConn fails, this should result in a TF update. - servers[1].closeConn() + holds[1].Fail(fmt.Errorf("test error")) testutils.AwaitState(ctx, t, cc, connectivity.TransientFailure) } @@ -983,39 +977,34 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { pfinternal.TimeAfterFunc = originalTimer }() - servers := newHangingServerGroup(t, 2) - addrs := resolverAddrsFromHangingServers(servers) - rb := manual.NewBuilderWithScheme("whatever") - rb.InitialState(resolver.State{Addresses: addrs}) - cc, err := grpc.NewClient("whatever:///this-gets-overwritten", - grpc.WithTransportCredentials(insecure.NewCredentials()), + dialer := testutils.NewBlockingDialer() + cc, rb, bm := setupPickFirstLeaf(t, 2, grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), - grpc.WithResolvers(rb)) - - if err != nil { - t.Fatal(err) - } - defer cc.Close() + grpc.WithContextDialer(dialer.DialContext), + ) + addrs := bm.resolverAddrs() + holds := bm.holds(dialer) + rb.UpdateState(resolver.State{Addresses: addrs}) cc.Connect() testutils.AwaitState(ctx, t, cc, connectivity.Connecting) // Verify that the first server is contacted. - if err := servers[0].awaitContacted(ctx); err != nil { - t.Fatalf("Server with address %q not contacted: %v", addrs[0], err) + if got, want := holds[0].Wait(ctx), true; got != want { + t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[0], got, want) } - if got, want := servers[1].isContacted(), false; got != want { - t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[1], got, want) + if got, want := holds[1].IsStarted(), false; got != want { + t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[1], got, want) } timerCh <- struct{}{} // Second connection attempt is successful. - if err := servers[1].awaitContacted(ctx); err != nil { - t.Fatalf("Server with address %q not contacted: %v", addrs[1], err) + if got, want := holds[1].Wait(ctx), true; got != want { + t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[1], got, want) } - servers[1].enterReady() + holds[1].Resume() testutils.AwaitState(ctx, t, cc, connectivity.Ready) } @@ -1050,34 +1039,30 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { pfinternal.TimeAfterFunc = originalTimer }() - servers := newHangingServerGroup(t, 3) - addrs := resolverAddrsFromHangingServers(servers) - rb := manual.NewBuilderWithScheme("whatever") - rb.InitialState(resolver.State{Addresses: addrs}) - cc, err := grpc.NewClient("whatever:///this-gets-overwritten", - grpc.WithTransportCredentials(insecure.NewCredentials()), + dialer := testutils.NewBlockingDialer() + cc, rb, bm := setupPickFirstLeaf(t, 3, grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), - grpc.WithResolvers(rb)) - - if err != nil { - t.Fatal(err) - } - defer cc.Close() + grpc.WithContextDialer(dialer.DialContext), + ) + addrs := bm.resolverAddrs() + holds := bm.holds(dialer) + rb.UpdateState(resolver.State{Addresses: addrs}) cc.Connect() testutils.AwaitState(ctx, t, cc, connectivity.Connecting) // Verify that only the first server is contacted. - if err := servers[0].awaitContacted(ctx); err != nil { - t.Fatalf("Server with address %q not contacted: %v", addrs[0], err) + if got, want := holds[0].Wait(ctx), true; got != want { + t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[0], got, want) } // Ensure no other servers are contacted. - if got, want := servers[1].isContacted(), false; got != want { - t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[1], got, want) + if got, want := holds[1].IsStarted(), false; got != want { + t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[1], got, want) } - if got, want := servers[2].isContacted(), false; got != want { - t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[2], got, want) + + if got, want := holds[2].IsStarted(), false; got != want { + t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[2], got, want) } // First SubConn Fails. @@ -1086,29 +1071,29 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { timerMu.Lock() timerCh = make(chan struct{}) timerMu.Unlock() - servers[0].closeConn() + holds[0].Fail(fmt.Errorf("test error")) // Verify that only the second server is contacted. - if err := servers[1].awaitContacted(ctx); err != nil { - t.Fatalf("Server with address %q not contacted: %v", addrs[1], err) + if got, want := holds[1].Wait(ctx), true; got != want { + t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[1], got, want) } // Ensure no other servers are contacted. - if got, want := servers[2].isContacted(), false; got != want { - t.Fatalf("Server[%q].isContacted() = %t, want %t", addrs[2], got, want) + if got, want := holds[2].IsStarted(), false; got != want { + t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[2], got, want) } // The happy eyeballs timer expires, skipping server[1] and requesting the creation // of a third SubConn. timerCh <- struct{}{} - if err := servers[2].awaitContacted(ctx); err != nil { - t.Fatalf("Server with address %q not contacted: %v", addrs[2], err) + if got, want := holds[2].Wait(ctx), true; got != want { + t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[2], got, want) } // Second SubConn connects. - servers[1].enterReady() - testutils.AwaitState(ctx, t, cc, connectivity.Connecting) + holds[1].Resume() + testutils.AwaitState(ctx, t, cc, connectivity.Ready) } // stateStoringBalancer stores the state of the subconns being created. @@ -1208,6 +1193,14 @@ func (b *backendManager) resolverAddrs() []resolver.Address { return addrs } +func (b *backendManager) holds(dialer *testutils.BlockingDialer) []*testutils.Hold { + holds := []*testutils.Hold{} + for _, addr := range b.resolverAddrs() { + holds = append(holds, dialer.Hold(addr.Addr)) + } + return holds +} + type ccStateSubscriber struct { transitions []connectivity.State } @@ -1215,94 +1208,3 @@ type ccStateSubscriber struct { func (c *ccStateSubscriber) OnMessage(msg any) { c.transitions = append(c.transitions, msg.(connectivity.State)) } - -// hangingServer is a server that accept a TCP connection and remains idle until -// asked to close or respond to the connection. They can be used to control -// how long it takes for a subchannel to report a TRANSIENT_FAILURE in tests. -type hangingServer struct { - addr resolver.Address - listener net.Listener - closeConn func() - contacted *grpcsync.Event - readyChan chan struct{} -} - -func newHangingServerGroup(t *testing.T, count int) []*hangingServer { - servers := []*hangingServer{} - for i := 0; i < count; i++ { - lis, err := net.Listen("tcp", "localhost:0") - if err != nil { - t.Fatalf("Error while listening. Err: %v", err) - } - closeChan := make(chan struct{}) - contacted := grpcsync.NewEvent() - readyChan := make(chan struct{}) - - go func() { - conn, err := lis.Accept() - if err != nil { - t.Error(err) - return - } - defer conn.Close() - - contacted.Fire() - - for { - select { - case <-closeChan: - return - case <-readyChan: - framer := http2.NewFramer(conn, conn) - if err := framer.WriteSettings(http2.Setting{}); err != nil { - t.Errorf("Error while writing settings frame. %v", err) - return - } - - } - } - }() - server := &hangingServer{ - addr: resolver.Address{Addr: lis.Addr().String()}, - listener: lis, - closeConn: grpcsync.OnceFunc(func() { - close(closeChan) - }), - contacted: contacted, - readyChan: readyChan, - } - servers = append(servers, server) - - t.Cleanup(func() { - server.closeConn() - server.listener.Close() - }) - } - - return servers -} - -func (s *hangingServer) isContacted() bool { - return s.contacted.HasFired() -} - -func (s *hangingServer) enterReady() { - s.readyChan <- struct{}{} -} - -func (s *hangingServer) awaitContacted(ctx context.Context) error { - select { - case <-ctx.Done(): - return ctx.Err() - case <-s.contacted.Done(): - } - return nil -} - -func resolverAddrsFromHangingServers(servers []*hangingServer) []resolver.Address { - addrs := []resolver.Address{} - for _, srv := range servers { - addrs = append(addrs, srv.addr) - } - return addrs -} From 99e2e893d140f37ef0a0fae710d0595ca2d7e656 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Thu, 17 Oct 2024 12:05:30 +0530 Subject: [PATCH 13/24] Fix test synchronization --- .../pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 7e22d2283a3c..e85e039b68d1 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -1022,10 +1022,10 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { // Set a really long expiration to prevent it from triggering // automatically. timer := time.AfterFunc(time.Hour, f) + timerMu.Lock() + ch := timerCh + timerMu.Unlock() go func() { - timerMu.Lock() - ch := timerCh - timerMu.Unlock() select { case <-ctx.Done(): case <-ch: @@ -1065,12 +1065,13 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[2], got, want) } - // First SubConn Fails. // Replace the timer channel so that the old timers don't attempt to read // messages pushed next. timerMu.Lock() timerCh = make(chan struct{}) timerMu.Unlock() + + // First SubConn Fails. holds[0].Fail(fmt.Errorf("test error")) // Verify that only the second server is contacted. From 592ba0dbd060733e7198357dea0c23e6e5a830e0 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Fri, 18 Oct 2024 11:48:05 +0530 Subject: [PATCH 14/24] Test refactorings --- .../pickfirstleaf/pickfirstleaf_ext_test.go | 96 +++++++++---------- 1 file changed, 44 insertions(+), 52 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index e85e039b68d1..0a5d2714f376 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -878,15 +878,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { pfinternal.TimeAfterFunc = originalTimer }() - defer func() { - pfinternal.TimeAfterFunc = originalTimer - }() - dialer := testutils.NewBlockingDialer() - cc, rb, bm := setupPickFirstLeaf(t, 3, + opts := []grpc.DialOption{ grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), grpc.WithContextDialer(dialer.DialContext), - ) + } + cc, rb, bm := setupPickFirstLeaf(t, 3, opts...) addrs := bm.resolverAddrs() holds := bm.holds(dialer) rb.UpdateState(resolver.State{Addresses: addrs}) @@ -895,36 +892,35 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { testutils.AwaitState(ctx, t, cc, connectivity.Connecting) // Verify that only the first server is contacted. - if got, want := holds[0].Wait(ctx), true; got != want { - t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[0], got, want) + if holds[0].Wait(ctx) != true { + t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 0, addrs[0]) } // Ensure no other servers are contacted. - if got, want := holds[1].IsStarted(), false; got != want { - t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[1], got, want) + if holds[1].IsStarted() != false { + t.Fatalf("Server %d with address %q contacted unexpectedly", 1, addrs[1]) } - - if got, want := holds[2].IsStarted(), false; got != want { - t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[2], got, want) + if holds[2].IsStarted() != false { + t.Fatalf("Server %d with address %q contacted unexpectedly", 2, addrs[2]) } - // Make the happy eyeballs timer fire twice so that pickfirst reaches the - // last address in the list. + // Make the happy eyeballs timer fire once and verify that the + // second server is contacted, but the third isn't. timerCh <- struct{}{} // Verify that the second server is contacted and 3rd isn't. - if got, want := holds[1].Wait(ctx), true; got != want { - t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[1], got, want) + if holds[1].Wait(ctx) != true { + t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 1, addrs[1]) } - - if got, want := holds[2].IsStarted(), false; got != want { - t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[2], got, want) + if holds[2].IsStarted() != false { + t.Fatalf("Server %d with address %q contacted unexpectedly", 2, addrs[2]) } + // Make the happy eyeballs timer fire once more and verify that the + // third server is contacted. timerCh <- struct{}{} - // Verify that the second server is contacted and 3rd isn't. - if got, want := holds[1].Wait(ctx), true; got != want { - t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[1], got, want) + if holds[2].Wait(ctx) != true { + t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 2, addrs[2]) } // First SubConn Fails. @@ -973,15 +969,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { pfinternal.TimeAfterFunc = originalTimer }() - defer func() { - pfinternal.TimeAfterFunc = originalTimer - }() - dialer := testutils.NewBlockingDialer() - cc, rb, bm := setupPickFirstLeaf(t, 2, + opts := []grpc.DialOption{ grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), grpc.WithContextDialer(dialer.DialContext), - ) + } + cc, rb, bm := setupPickFirstLeaf(t, 2, opts...) addrs := bm.resolverAddrs() holds := bm.holds(dialer) rb.UpdateState(resolver.State{Addresses: addrs}) @@ -989,20 +982,19 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { testutils.AwaitState(ctx, t, cc, connectivity.Connecting) - // Verify that the first server is contacted. - if got, want := holds[0].Wait(ctx), true; got != want { - t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[0], got, want) + // Verify that the first server is contacted and the second is not. + if holds[0].Wait(ctx) != true { + t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 0, addrs[0]) } - - if got, want := holds[1].IsStarted(), false; got != want { - t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[1], got, want) + if holds[1].IsStarted() != false { + t.Fatalf("Server %d with address %q contacted unexpectedly", 1, addrs[1]) } timerCh <- struct{}{} // Second connection attempt is successful. - if got, want := holds[1].Wait(ctx), true; got != want { - t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[1], got, want) + if holds[1].Wait(ctx) != true { + t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 1, addrs[1]) } holds[1].Resume() testutils.AwaitState(ctx, t, cc, connectivity.Ready) @@ -1040,10 +1032,11 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { }() dialer := testutils.NewBlockingDialer() - cc, rb, bm := setupPickFirstLeaf(t, 3, + opts := []grpc.DialOption{ grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), grpc.WithContextDialer(dialer.DialContext), - ) + } + cc, rb, bm := setupPickFirstLeaf(t, 3, opts...) addrs := bm.resolverAddrs() holds := bm.holds(dialer) rb.UpdateState(resolver.State{Addresses: addrs}) @@ -1052,17 +1045,16 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { testutils.AwaitState(ctx, t, cc, connectivity.Connecting) // Verify that only the first server is contacted. - if got, want := holds[0].Wait(ctx), true; got != want { - t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[0], got, want) + if holds[0].Wait(ctx) != true { + t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 0, addrs[0]) } // Ensure no other servers are contacted. - if got, want := holds[1].IsStarted(), false; got != want { - t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[1], got, want) + if holds[1].IsStarted() != false { + t.Fatalf("Server %d with address %q contacted unexpectedly", 1, addrs[1]) } - - if got, want := holds[2].IsStarted(), false; got != want { - t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[2], got, want) + if holds[2].IsStarted() != false { + t.Fatalf("Server %d with address %q contacted unexpectedly", 2, addrs[2]) } // Replace the timer channel so that the old timers don't attempt to read @@ -1075,21 +1067,21 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { holds[0].Fail(fmt.Errorf("test error")) // Verify that only the second server is contacted. - if got, want := holds[1].Wait(ctx), true; got != want { - t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[1], got, want) + if holds[1].Wait(ctx) != true { + t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 1, addrs[1]) } // Ensure no other servers are contacted. - if got, want := holds[2].IsStarted(), false; got != want { - t.Fatalf("holds[%q].IsStarted() = %t, want %t", addrs[2], got, want) + if holds[2].IsStarted() != false { + t.Fatalf("Server %d with address %q contacted unexpectedly", 2, addrs[2]) } // The happy eyeballs timer expires, skipping server[1] and requesting the creation // of a third SubConn. timerCh <- struct{}{} - if got, want := holds[2].Wait(ctx), true; got != want { - t.Fatalf("hold[%q].Wait() = %t, want %t", addrs[2], got, want) + if holds[2].Wait(ctx) != true { + t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 2, addrs[2]) } // Second SubConn connects. From 84d6ed45e853f626c10129b96932500214ba5c1c Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Sun, 20 Oct 2024 03:19:00 +0530 Subject: [PATCH 15/24] Cancel timer when processing new resolver update --- balancer/pickfirst/pickfirstleaf/pickfirstleaf.go | 1 + 1 file changed, 1 insertion(+) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index 7622b3d2d83a..9bd7bfa80c61 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -184,6 +184,7 @@ func (b *pickfirstBalancer) resolverErrorLocked(err error) { func (b *pickfirstBalancer) UpdateClientConnState(state balancer.ClientConnState) error { b.mu.Lock() defer b.mu.Unlock() + b.cancelConnectionTimer() if len(state.ResolverState.Addresses) == 0 && len(state.ResolverState.Endpoints) == 0 { // Cleanup state pertaining to the previous resolver state. // Treat an empty address list like an error by calling b.ResolverError. From 8f63d8e2dc488dacca06ad52eba97ffba87df180 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Wed, 23 Oct 2024 15:30:53 +0530 Subject: [PATCH 16/24] Improve whitespaces and comments --- .../pickfirstleaf/pickfirstleaf_ext_test.go | 48 ++++++++----------- 1 file changed, 21 insertions(+), 27 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 0a5d2714f376..3c33bc261fe5 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -850,11 +850,10 @@ func (s) TestPickFirstLeaf_EmptyAddressList(t *testing.T) { } } -// TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList verifies that pickfirst -// correctly detects the end of the first happy eyeballs pass when the timer -// causes pickfirst to reach the end of the address list and failures are -// reported out of order. -func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { +// Test verifies that pickfirst correctly detects the end of the first happy +// eyeballs pass when the timer causes pickfirst to reach the end of the address +// list and failures are reported out of order. +func (s) TestPickFirstLeaf_HappyEyeballs_TF_AfterEndOfList(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -895,8 +894,6 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { if holds[0].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 0, addrs[0]) } - - // Ensure no other servers are contacted. if holds[1].IsStarted() != false { t.Fatalf("Server %d with address %q contacted unexpectedly", 1, addrs[1]) } @@ -907,8 +904,6 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { // Make the happy eyeballs timer fire once and verify that the // second server is contacted, but the third isn't. timerCh <- struct{}{} - - // Verify that the second server is contacted and 3rd isn't. if holds[1].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 1, addrs[1]) } @@ -934,7 +929,6 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { // Third SubConn fails. holds[2].Fail(fmt.Errorf("test error")) - testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) // Last SubConn fails, this should result in a TF update. @@ -942,9 +936,8 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFAfterEndOfList(t *testing.T) { testutils.AwaitState(ctx, t, cc, connectivity.TransientFailure) } -// TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay verifies that -// pickfirst attempts to connect to the second backend once the happy eyeballs -// timer expires. +// Test verifies that pickfirst attempts to connect to the second backend once +// the happy eyeballs timer expires. func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -982,7 +975,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { testutils.AwaitState(ctx, t, cc, connectivity.Connecting) - // Verify that the first server is contacted and the second is not. + // Verify that only the first server is contacted. if holds[0].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 0, addrs[0]) } @@ -990,20 +983,22 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { t.Fatalf("Server %d with address %q contacted unexpectedly", 1, addrs[1]) } + // Make the happy eyeballs timer fire once and verify that the + // second server is contacted. timerCh <- struct{}{} - - // Second connection attempt is successful. if holds[1].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 1, addrs[1]) } + + // Get the connection attempt to the second server to succeed and verify + // that the channel becomes READY. holds[1].Resume() testutils.AwaitState(ctx, t, cc, connectivity.Ready) } -// TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires tests the pickfirst balancer -// by causing a SubConn to fail and then jumping to the 3rd SubConn after the -// happy eyeballs timer expires. -func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { +// Test tests the pickfirst balancer by causing a SubConn to fail and then +// jumping to the 3rd SubConn after the happy eyeballs timer expires. +func (s) TestPickFirstLeaf_HappyEyeballs_TF_ThenTimerFires(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -1048,8 +1043,6 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { if holds[0].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 0, addrs[0]) } - - // Ensure no other servers are contacted. if holds[1].IsStarted() != false { t.Fatalf("Server %d with address %q contacted unexpectedly", 1, addrs[1]) } @@ -1058,7 +1051,10 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { } // Replace the timer channel so that the old timers don't attempt to read - // messages pushed next. + // messages pushed next. This is required since pickfirst will stop the + // timer, but the fake TimeAfterFunc will still keep waiting on the timer + // channel till the context is cancelled. If there are multiple listeners on + /// the timer channel, they will race to read from the channel. timerMu.Lock() timerCh = make(chan struct{}) timerMu.Unlock() @@ -1070,8 +1066,6 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { if holds[1].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 1, addrs[1]) } - - // Ensure no other servers are contacted. if holds[2].IsStarted() != false { t.Fatalf("Server %d with address %q contacted unexpectedly", 2, addrs[2]) } @@ -1079,12 +1073,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TFThenTimerFires(t *testing.T) { // The happy eyeballs timer expires, skipping server[1] and requesting the creation // of a third SubConn. timerCh <- struct{}{} - if holds[2].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 2, addrs[2]) } - // Second SubConn connects. + // Get the connection attempt to the second server to succeed and verify + // that the channel becomes READY. holds[1].Resume() testutils.AwaitState(ctx, t, cc, connectivity.Ready) } From 66105168d3454eb61e4935c3d3415b83b3ddb951 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Wed, 23 Oct 2024 23:11:38 +0530 Subject: [PATCH 17/24] Refactor fake timer --- balancer/pickfirst/internal/internal.go | 5 +- .../pickfirst/pickfirstleaf/pickfirstleaf.go | 2 +- .../pickfirstleaf/pickfirstleaf_ext_test.go | 98 +++++++------------ 3 files changed, 41 insertions(+), 64 deletions(-) diff --git a/balancer/pickfirst/internal/internal.go b/balancer/pickfirst/internal/internal.go index da3541b37960..73216e5fbca0 100644 --- a/balancer/pickfirst/internal/internal.go +++ b/balancer/pickfirst/internal/internal.go @@ -28,5 +28,8 @@ var ( RandShuffle = rand.Shuffle // TimeAfterFunc allows mocking the timer for testing connection delay // related functionality. - TimeAfterFunc = time.AfterFunc + TimeAfterFunc = func(d time.Duration, f func()) func() { + timer := time.AfterFunc(d, f) + return func() { timer.Stop() } + } ) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index b7c02977c5dc..8af444ad8f29 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -453,7 +453,7 @@ func (b *pickfirstBalancer) scheduleNextConnectionLocked() { if b.addressList.increment() { b.requestConnectionLocked() } - }).Stop + }) // Access to the cancellation callback held by the balancer is guarded by // the balancer's mutex, so it's safe to set the boolean from the callback. b.cancelConnectionTimer = sync.OnceFunc(func() { diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 3c33bc261fe5..0d819d64f6b7 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -857,25 +857,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_AfterEndOfList(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - timerCh := make(chan struct{}) originalTimer := pfinternal.TimeAfterFunc - pfinternal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { - // Set a really long expiration to prevent it from triggering - // automatically. - timer := time.AfterFunc(time.Hour, f) - go func() { - select { - case <-ctx.Done(): - case <-timerCh: - } - timer.Reset(0) - }() - return timer - } - defer func() { pfinternal.TimeAfterFunc = originalTimer }() + triggerTimer, timeAfter := mockTimer() + pfinternal.TimeAfterFunc = timeAfter dialer := testutils.NewBlockingDialer() opts := []grpc.DialOption{ @@ -903,7 +890,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_AfterEndOfList(t *testing.T) { // Make the happy eyeballs timer fire once and verify that the // second server is contacted, but the third isn't. - timerCh <- struct{}{} + triggerTimer() if holds[1].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 1, addrs[1]) } @@ -913,7 +900,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_AfterEndOfList(t *testing.T) { // Make the happy eyeballs timer fire once more and verify that the // third server is contacted. - timerCh <- struct{}{} + triggerTimer() if holds[2].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 2, addrs[2]) } @@ -942,25 +929,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - timerCh := make(chan struct{}) originalTimer := pfinternal.TimeAfterFunc - pfinternal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { - // Set a really long expiration to prevent it from triggering - // automatically. - timer := time.AfterFunc(time.Hour, f) - go func() { - select { - case <-ctx.Done(): - case <-timerCh: - } - timer.Reset(0) - }() - return timer - } - defer func() { pfinternal.TimeAfterFunc = originalTimer }() + triggerTimer, timeAfter := mockTimer() + pfinternal.TimeAfterFunc = timeAfter dialer := testutils.NewBlockingDialer() opts := []grpc.DialOption{ @@ -985,7 +959,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { // Make the happy eyeballs timer fire once and verify that the // second server is contacted. - timerCh <- struct{}{} + triggerTimer() if holds[1].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 1, addrs[1]) } @@ -1002,29 +976,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_ThenTimerFires(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - timerMu := sync.Mutex{} - timerCh := make(chan struct{}) originalTimer := pfinternal.TimeAfterFunc - pfinternal.TimeAfterFunc = func(_ time.Duration, f func()) *time.Timer { - // Set a really long expiration to prevent it from triggering - // automatically. - timer := time.AfterFunc(time.Hour, f) - timerMu.Lock() - ch := timerCh - timerMu.Unlock() - go func() { - select { - case <-ctx.Done(): - case <-ch: - } - timer.Reset(0) - }() - return timer - } - defer func() { pfinternal.TimeAfterFunc = originalTimer }() + triggerTimer, timeAfter := mockTimer() + pfinternal.TimeAfterFunc = timeAfter dialer := testutils.NewBlockingDialer() opts := []grpc.DialOption{ @@ -1050,15 +1007,6 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_ThenTimerFires(t *testing.T) { t.Fatalf("Server %d with address %q contacted unexpectedly", 2, addrs[2]) } - // Replace the timer channel so that the old timers don't attempt to read - // messages pushed next. This is required since pickfirst will stop the - // timer, but the fake TimeAfterFunc will still keep waiting on the timer - // channel till the context is cancelled. If there are multiple listeners on - /// the timer channel, they will race to read from the channel. - timerMu.Lock() - timerCh = make(chan struct{}) - timerMu.Unlock() - // First SubConn Fails. holds[0].Fail(fmt.Errorf("test error")) @@ -1072,7 +1020,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_ThenTimerFires(t *testing.T) { // The happy eyeballs timer expires, skipping server[1] and requesting the creation // of a third SubConn. - timerCh <- struct{}{} + triggerTimer() if holds[2].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 2, addrs[2]) } @@ -1195,3 +1143,29 @@ type ccStateSubscriber struct { func (c *ccStateSubscriber) OnMessage(msg any) { c.transitions = append(c.transitions, msg.(connectivity.State)) } + +// mockTimer returns a fake timeAfterFunc that will not trigger automatically. +// It returns a function that can be called to manually trigger the execution +// of the scheduled callback. +func mockTimer() (triggerFunc func(), timerFunc func(_ time.Duration, f func()) func()) { + timerCh := make(chan struct{}) + triggerFunc = func() { + timerCh <- struct{}{} + } + return triggerFunc, func(_ time.Duration, f func()) func() { + // Set a really long expiration to prevent it from triggering + // automatically. + timer := time.AfterFunc(time.Hour, f) + stopCh := make(chan struct{}) + go func() { + select { + case <-timerCh: + timer.Reset(0) + case <-stopCh: + } + }() + return sync.OnceFunc(func() { + close(stopCh) + }) + } +} From d6bc007a4220955e3c3ea2d7c2706226c17d9383 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Wed, 23 Oct 2024 23:13:47 +0530 Subject: [PATCH 18/24] Don't use expired context --- balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 0d819d64f6b7..3ac6746f2812 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -915,6 +915,9 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_AfterEndOfList(t *testing.T) { testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) // Third SubConn fails. + shortCancel() + shortCtx, shortCancel = context.WithTimeout(ctx, defaultTestShortTimeout) + defer shortCancel() holds[2].Fail(fmt.Errorf("test error")) testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) From 19a31656f8fe6acc9d74957127e845b49bd2f710 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Thu, 24 Oct 2024 01:47:58 +0530 Subject: [PATCH 19/24] Remove unnecessary timer in test --- balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 3ac6746f2812..16436eec162b 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -1158,12 +1158,11 @@ func mockTimer() (triggerFunc func(), timerFunc func(_ time.Duration, f func()) return triggerFunc, func(_ time.Duration, f func()) func() { // Set a really long expiration to prevent it from triggering // automatically. - timer := time.AfterFunc(time.Hour, f) stopCh := make(chan struct{}) go func() { select { case <-timerCh: - timer.Reset(0) + f() case <-stopCh: } }() From 598fdd0a1c41b763ea6b4199ffc8d882dc5a1589 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Thu, 24 Oct 2024 18:52:14 +0530 Subject: [PATCH 20/24] Address review comments --- balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 16436eec162b..d077a26a94f7 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -911,11 +911,9 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_AfterEndOfList(t *testing.T) { // No TF should be reported until the first pass is complete. shortCtx, shortCancel := context.WithTimeout(ctx, defaultTestShortTimeout) defer shortCancel() - testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) // Third SubConn fails. - shortCancel() shortCtx, shortCancel = context.WithTimeout(ctx, defaultTestShortTimeout) defer shortCancel() holds[2].Fail(fmt.Errorf("test error")) @@ -1021,8 +1019,9 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_ThenTimerFires(t *testing.T) { t.Fatalf("Server %d with address %q contacted unexpectedly", 2, addrs[2]) } - // The happy eyeballs timer expires, skipping server[1] and requesting the creation - // of a third SubConn. + // The happy eyeballs timer expires, pickfirst should stop waiting for + // server[1] to report a failure/success and request the creation of a third + // SubConn. triggerTimer() if holds[2].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 2, addrs[2]) From 8b4b28e03ff9656d43f628bd5fc4e27286d16c10 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Wed, 6 Nov 2024 23:42:26 +0530 Subject: [PATCH 21/24] Remove stale comment --- balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index dfc4c9818780..bf957f98b119 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -1313,8 +1313,6 @@ func mockTimer() (triggerFunc func(), timerFunc func(_ time.Duration, f func()) timerCh <- struct{}{} } return triggerFunc, func(_ time.Duration, f func()) func() { - // Set a really long expiration to prevent it from triggering - // automatically. stopCh := make(chan struct{}) go func() { select { From 6c169432496bad0e4ea05cd72bbe37ecc9d1d422 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Thu, 7 Nov 2024 16:00:26 +0530 Subject: [PATCH 22/24] Use rand/v2 --- balancer/pickfirst/internal/internal.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/balancer/pickfirst/internal/internal.go b/balancer/pickfirst/internal/internal.go index 73216e5fbca0..7d66cb491c40 100644 --- a/balancer/pickfirst/internal/internal.go +++ b/balancer/pickfirst/internal/internal.go @@ -19,7 +19,7 @@ package internal import ( - "math/rand" + rand "math/rand/v2" "time" ) From 11fe5158fb53384779d56cbd2cf976b5a9a3b234 Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Fri, 8 Nov 2024 14:44:54 +0530 Subject: [PATCH 23/24] Address review comments --- .../pickfirst/pickfirstleaf/pickfirstleaf.go | 19 ++++++++++--------- internal/envconfig/envconfig.go | 3 ++- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index 40992887b11a..78da3603e573 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -482,19 +482,15 @@ func (b *pickfirstBalancer) requestConnectionLocked() { scd.connectionFailed = true lastErr = scd.lastErr continue - case connectivity.Ready: - // Should never happen. - b.logger.Errorf("Requesting a connection even though we have a READY SubConn") - return - case connectivity.Shutdown: - // Should never happen. - b.logger.Errorf("SubConn with state SHUTDOWN present in SubConns map") - return case connectivity.Connecting: // Wait for the connection attempt to complete or the timer to fire // before attempting the next address. b.scheduleNextConnectionLocked() return + default: + b.logger.Errorf("SubConn with unexpected state %v present in SubConns map.", scd.state) + return + } } @@ -643,10 +639,15 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub } } +// endFirstPassIfPossibleLocked ends the first happy-eyeballs pass if all the +// addresses are tried and their SubConns have reported a failure. func (b *pickfirstBalancer) endFirstPassIfPossibleLocked(lastErr error) { - if b.addressList.isValid() || b.subConns.Len() < b.addressList.size() { + // An optimization to avoid iterating over the entire SubConn map. + if b.addressList.isValid() { return } + // Connect() has been called on all the SubConns. The first pass can be + // ended if all the SubConns have reported a failure. for _, v := range b.subConns.Values() { sd := v.(*scData) if !sd.connectionFailed { diff --git a/internal/envconfig/envconfig.go b/internal/envconfig/envconfig.go index f2630c10469f..6e7dd6b77270 100644 --- a/internal/envconfig/envconfig.go +++ b/internal/envconfig/envconfig.go @@ -43,7 +43,8 @@ var ( // EnforceALPNEnabled is set if TLS connections to servers with ALPN disabled // should be rejected. The HTTP/2 protocol requires ALPN to be enabled, this // option is present for backward compatibility. This option may be overridden - // by setting the environment variable "GRPC_ENFORCE_ALPN_ENABLED" to "false". + // by setting the environment variable "GRPC_ENFORCE_ALPN_ENABLED" to "true" + // or "false". EnforceALPNEnabled = boolFromEnv("GRPC_ENFORCE_ALPN_ENABLED", true) // XDSFallbackSupport is the env variable that controls whether support for // xDS fallback is turned on. If this is unset or is false, only the first From 5c4ff49825e5fda36ea53281da780b2663fb2a4a Mon Sep 17 00:00:00 2001 From: Arjan Bal Date: Tue, 12 Nov 2024 14:26:11 +0530 Subject: [PATCH 24/24] Rename to connectionFailedInFirstPass --- balancer/pickfirst/pickfirstleaf/pickfirstleaf.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index 78da3603e573..aaec87497fd4 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -122,9 +122,9 @@ type scData struct { subConn balancer.SubConn addr resolver.Address - state connectivity.State - lastErr error - connectionFailed bool + state connectivity.State + lastErr error + connectionFailedInFirstPass bool } func (b *pickfirstBalancer) newSCData(addr resolver.Address) (*scData, error) { @@ -314,7 +314,7 @@ func (b *pickfirstBalancer) startFirstPassLocked() { b.numTF = 0 // Reset the connection attempt record for existing SubConns. for _, sd := range b.subConns.Values() { - sd.(*scData).connectionFailed = false + sd.(*scData).connectionFailedInFirstPass = false } b.requestConnectionLocked() } @@ -479,7 +479,7 @@ func (b *pickfirstBalancer) requestConnectionLocked() { // The SubConn is being re-used and failed during a previous pass // over the addressList. It has not completed backoff yet. // Mark it as having failed and try the next address. - scd.connectionFailed = true + scd.connectionFailedInFirstPass = true lastErr = scd.lastErr continue case connectivity.Connecting: @@ -534,7 +534,7 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub oldState := sd.state // Record a connection attempt when exiting CONNECTING. if newState.ConnectivityState == connectivity.TransientFailure { - sd.connectionFailed = true + sd.connectionFailedInFirstPass = true } sd.state = newState.ConnectivityState // Previously relevant SubConns can still callback with state updates. @@ -650,7 +650,7 @@ func (b *pickfirstBalancer) endFirstPassIfPossibleLocked(lastErr error) { // ended if all the SubConns have reported a failure. for _, v := range b.subConns.Values() { sd := v.(*scData) - if !sd.connectionFailed { + if !sd.connectionFailedInFirstPass { return } }