From 1dd14ceb963cb4be6eff9e57ccf475c64d8049ba Mon Sep 17 00:00:00 2001 From: Zach Reyes <39203661+zasweq@users.noreply.github.com> Date: Mon, 23 May 2022 15:14:50 -0400 Subject: [PATCH 01/22] Added Outlier Detection Balancer --- internal/grpcrand/grpcrand.go | 7 + test/xds/xds_client_outlier_detection_test.go | 76 + .../balancer/outlierdetection/balancer.go | 854 +++++++- .../outlierdetection/balancer_test.go | 1794 +++++++++++++++++ .../balancer/outlierdetection/callcounter.go | 78 + .../outlierdetection/callcounter_test.go | 99 + .../outlierdetection/subconn_wrapper.go | 62 + 7 files changed, 2969 insertions(+), 1 deletion(-) create mode 100644 test/xds/xds_client_outlier_detection_test.go create mode 100644 xds/internal/balancer/outlierdetection/callcounter.go create mode 100644 xds/internal/balancer/outlierdetection/callcounter_test.go create mode 100644 xds/internal/balancer/outlierdetection/subconn_wrapper.go diff --git a/internal/grpcrand/grpcrand.go b/internal/grpcrand/grpcrand.go index 740f83c2b76..517ea70642a 100644 --- a/internal/grpcrand/grpcrand.go +++ b/internal/grpcrand/grpcrand.go @@ -52,6 +52,13 @@ func Intn(n int) int { return r.Intn(n) } +// Int31n implements rand.Int31n on the grpcrand global source. +func Int31n(n int32) int32 { + mu.Lock() + defer mu.Unlock() + return r.Int31n(n) +} + // Float64 implements rand.Float64 on the grpcrand global source. func Float64() float64 { mu.Lock() diff --git a/test/xds/xds_client_outlier_detection_test.go b/test/xds/xds_client_outlier_detection_test.go new file mode 100644 index 00000000000..d500de03c80 --- /dev/null +++ b/test/xds/xds_client_outlier_detection_test.go @@ -0,0 +1,76 @@ +/* + * + * Copyright 2022 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package xds_test + +import ( + "context" + "fmt" + "testing" + + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/envconfig" + "google.golang.org/grpc/internal/testutils/xds/e2e" + + testgrpc "google.golang.org/grpc/test/grpc_testing" + testpb "google.golang.org/grpc/test/grpc_testing" +) + +func (s) TestOutlierDetection(t *testing.T) { + oldOD := envconfig.XDSOutlierDetection + envconfig.XDSOutlierDetection = true + internal.RegisterOutlierDetectionBalancerForTesting() + defer func() { + envconfig.XDSOutlierDetection = oldOD + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + managementServer, nodeID, _, resolver, cleanup1 := e2e.SetupManagementServer(t) + defer cleanup1() + + port, cleanup2 := startTestService(t, nil) + defer cleanup2() + + const serviceName = "my-service-client-side-xds" + resources := e2e.DefaultClientResources(e2e.ResourceParams{ + DialTarget: serviceName, + NodeID: nodeID, + Host: "localhost", + Port: port, + SecLevel: e2e.SecurityLevelNone, + }) + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + if err := managementServer.Update(ctx, resources); err != nil { + t.Fatal(err) + } + + // Create a ClientConn and make a successful RPC. + cc, err := grpc.Dial(fmt.Sprintf("xds:///%s", serviceName), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(resolver)) + if err != nil { + t.Fatalf("failed to dial local test server: %v", err) + } + defer cc.Close() + + client := testgrpc.NewTestServiceClient(cc) + if _, err := client.EmptyCall(ctx, &testpb.Empty{}, grpc.WaitForReady(true)); err != nil { + t.Fatalf("rpc EmptyCall() failed: %v", err) + } +} diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 8729461383e..6c99f94e049 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -25,13 +25,29 @@ import ( "encoding/json" "errors" "fmt" + "math" + "sync" + "sync/atomic" + "time" + "unsafe" "google.golang.org/grpc/balancer" + "google.golang.org/grpc/connectivity" "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/buffer" "google.golang.org/grpc/internal/envconfig" + "google.golang.org/grpc/internal/grpcrand" + "google.golang.org/grpc/internal/grpcsync" + "google.golang.org/grpc/resolver" "google.golang.org/grpc/serviceconfig" ) +// Globals to stub out in tests. +var ( + afterFunc = time.AfterFunc + now = time.Now +) + // Name is the name of the outlier detection balancer. const Name = "outlier_detection_experimental" @@ -51,7 +67,18 @@ func init() { type bb struct{} func (bb) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Balancer { - return nil + am := resolver.NewAddressMap() + b := &outlierDetectionBalancer{ + cc: cc, + bOpts: bOpts, + closed: grpcsync.NewEvent(), + odAddrs: am, + scWrappers: make(map[balancer.SubConn]*subConnWrapper), + scUpdateCh: buffer.NewUnbounded(), + pickerUpdateCh: buffer.NewUnbounded(), + } + go b.run() + return b } func (bb) ParseConfig(s json.RawMessage) (serviceconfig.LoadBalancingConfig, error) { @@ -82,6 +109,7 @@ func (bb) ParseConfig(s json.RawMessage) (serviceconfig.LoadBalancingConfig, err return nil, fmt.Errorf("OutlierDetectionLoadBalancingConfig.base_ejection_time = %s; must be >= 0", lbCfg.BaseEjectionTime) case lbCfg.MaxEjectionTime < 0: return nil, fmt.Errorf("OutlierDetectionLoadBalancingConfig.max_ejection_time = %s; must be >= 0", lbCfg.MaxEjectionTime) + // "The fields max_ejection_percent, // success_rate_ejection.enforcement_percentage, // failure_percentage_ejection.threshold, and @@ -105,3 +133,827 @@ func (bb) ParseConfig(s json.RawMessage) (serviceconfig.LoadBalancingConfig, err func (bb) Name() string { return Name } + +// scUpdate wraps a subConn update to be sent to the child balancer. +type scUpdate struct { + scw *subConnWrapper + state balancer.SubConnState +} + +type ejectedUpdate struct { + scw *subConnWrapper + ejected bool // true for ejected, false for unejected +} + +type outlierDetectionBalancer struct { + numAddrsEjected int // For fast calculations of percentage of addrs ejected + + childState balancer.State + recentPickerNoop bool + + closed *grpcsync.Event + cc balancer.ClientConn + bOpts balancer.BuildOptions + + // childMu protects child and also updates to the child (to uphold the + // balancer.Balancer API guarantee of synchronous calls). It also protects + // against run() reading that the child is not nil for SubConn updates, and + // then UpdateClientConnState or Close writing to the the child. + childMu sync.Mutex + child balancer.Balancer + + // mu guards access to a lot of the core LB Policy State. It also prevents + // intersplicing certain operations. + // + // ex 1: interval timer goes off, outlier detection algorithm starts running + // based on knobs in odCfg. in the middle of running the algorithm, a + // ClientConn update comes in and writes to odCfg. This causes undefined + // behavior for the interval timer algorithm. + // + // ex 2: Updating the odAddrs map from UpdateAddresses in the middle of + // running the interval timer algorithm which uses odAddrs heavily. This + // will cause undefined behavior for the interval timer algorithm. + mu sync.Mutex + odAddrs *resolver.AddressMap + odCfg *LBConfig + scWrappers map[balancer.SubConn]*subConnWrapper + timerStartTime time.Time + intervalTimer *time.Timer + + scUpdateCh *buffer.Unbounded + pickerUpdateCh *buffer.Unbounded +} + +// noopConfig returns whether this balancer is configured with a logical no-op +// configuration or not. +func (b *outlierDetectionBalancer) noopConfig() bool { + return b.odCfg.SuccessRateEjection == nil && b.odCfg.FailurePercentageEjection == nil +} + +func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnState) error { + lbCfg, ok := s.BalancerConfig.(*LBConfig) + if !ok { + return balancer.ErrBadResolverState + } + + // Reject whole config if any errors, don't persist it for later + bb := balancer.Get(lbCfg.ChildPolicy.Name) + if bb == nil { + return fmt.Errorf("balancer %q not registered", lbCfg.ChildPolicy.Name) + } + + if b.child == nil || b.odCfg.ChildPolicy.Name != lbCfg.ChildPolicy.Name { + b.childMu.Lock() + if b.child != nil { + b.child.Close() + } + // What if this is nil? Seems fine + b.child = bb.Build(b, b.bOpts) + b.childMu.Unlock() + } + + b.mu.Lock() + b.odCfg = lbCfg + + // When the outlier_detection LB policy receives an address update, it will + // create a map entry for each subchannel address in the list, and remove + // each map entry for a subchannel address not in the list. + addrs := make(map[resolver.Address]bool) + for _, addr := range s.ResolverState.Addresses { + addrs[addr] = true + b.odAddrs.Set(addr, newObject()) + } + for _, addr := range b.odAddrs.Keys() { + if !addrs[addr] { + b.odAddrs.Delete(addr) + } + } + + // When a new config is provided, if the timer start timestamp is unset, set + // it to the current time and start the timer for the configured interval, + // then for each address, reset the call counters. + var interval time.Duration + if b.timerStartTime.IsZero() { + b.timerStartTime = time.Now() + for _, obj := range b.objects() { + obj.callCounter.clear() + } + interval = b.odCfg.Interval + } else { + // If the timer start timestamp is set, instead cancel the existing + // timer and start the timer for the configured interval minus the + // difference between the current time and the previous start timestamp, + // or 0 if that would be negative. + interval = b.odCfg.Interval - (now().Sub(b.timerStartTime)) + if interval < 0 { + interval = 0 + } + } + + if !b.noopConfig() { + if b.intervalTimer != nil { + b.intervalTimer.Stop() + } + b.intervalTimer = afterFunc(interval, func() { + b.intervalTimerAlgorithm() + }) + } else { + // "If a config is provided with both the `success_rate_ejection` and + // `failure_percentage_ejection` fields unset, skip starting the timer and + // unset the timer start timestamp." + b.timerStartTime = time.Time{} + // Should we stop the timer here as well? Not defined in gRFC but I feel + // like it might make sense as you don't want to eject addresses. Also + // how will addresses eventually get unejected in this case if only one + // more pass of the interval timer after no-op configuration comes in? + } + b.mu.Unlock() + b.pickerUpdateCh.Put(lbCfg) + + // then pass the address list along to the child policy. + b.childMu.Lock() + defer b.childMu.Unlock() + return b.child.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: s.ResolverState, + BalancerConfig: b.odCfg.ChildPolicy.Config, + }) +} + +func (b *outlierDetectionBalancer) ResolverError(err error) { + if b.child != nil { + b.childMu.Lock() + defer b.childMu.Unlock() + b.child.ResolverError(err) + } +} + +func (b *outlierDetectionBalancer) UpdateSubConnState(sc balancer.SubConn, state balancer.SubConnState) { + b.mu.Lock() + defer b.mu.Unlock() + scw, ok := b.scWrappers[sc] + if !ok { + // Return, shouldn't happen if passed up scw + return + } + if state.ConnectivityState == connectivity.Shutdown { + delete(b.scWrappers, scw.SubConn) + } + b.scUpdateCh.Put(&scUpdate{ + scw: scw, + state: state, + }) + +} + +func (b *outlierDetectionBalancer) Close() { + b.closed.Fire() + if b.child != nil { + b.childMu.Lock() + b.child.Close() + b.child = nil + b.childMu.Unlock() + } + + // Any other cleanup needs to happen (subconns, other resources?) + b.mu.Lock() + defer b.mu.Unlock() + if b.intervalTimer != nil { + b.intervalTimer.Stop() + } +} + +func (b *outlierDetectionBalancer) ExitIdle() { + if b.child == nil { + return + } + if ei, ok := b.child.(balancer.ExitIdler); ok { + b.childMu.Lock() + defer b.childMu.Unlock() + ei.ExitIdle() + return + } + + // Fallback for children handled in clusterimpl balancer Removing SubConns + // is defined in API and also in graceful switch balancer, but already done + // in ClusterImpl. I guess we should do that here? +} + +// "The outlier_detection LB policy will provide a picker that delegates to the +// child policy's picker, and when the request finishes, increment the +// corresponding counter in the map entry referenced by the subchannel wrapper +// that was picked." - A50 +type wrappedPicker struct { + childPicker balancer.Picker + noopPicker bool +} + +func (wp *wrappedPicker) Pick(info balancer.PickInfo) (balancer.PickResult, error) { + pr, err := wp.childPicker.Pick(info) + if err != nil { + return balancer.PickResult{}, err + } + + done := func(di balancer.DoneInfo) { + if !wp.noopPicker { + incrementCounter(pr.SubConn, di) + } + if pr.Done != nil { + pr.Done(di) + } + } + // Shouldn't happen, defensive programming. + scw, ok := pr.SubConn.(*subConnWrapper) + if !ok { + return balancer.PickResult{ + SubConn: pr.SubConn, + Done: done, + }, nil + } + return balancer.PickResult{ + SubConn: scw.SubConn, + Done: done, + }, nil +} + +func incrementCounter(sc balancer.SubConn, info balancer.DoneInfo) { + scw, ok := sc.(*subConnWrapper) + if !ok { + // Shouldn't happen, as comes from child + return + } + + // scw.obj and callCounter.activeBucket can be written to concurrently (the + // pointers themselves). Thus, protect the reads here with atomics to + // prevent data corruption. There exists a race in which you read the object + // or active bucket pointer and then that pointer points to deprecated + // memory. If this goroutine yields the processor, in between reading the + // object pointer and writing to the active bucket, UpdateAddresses can + // switch the obj the scw points to. Writing to an outdated addresses is a + // very small race and tolerable. After reading callCounter.activeBucket in + // this picker a swap call can concurrently change what activeBucket points + // to. A50 says to swap the pointer, but I decided to make create new memory + // for both active and inactive bucket, and have this race instead write to + // deprecated memory. If you swap the pointers, this write would write to + // the inactive buckets memory, which is read throughout in the interval + // timers algorithm. + obj := (*object)(atomic.LoadPointer(&scw.obj)) + if obj == nil { + return + } + ab := (*bucket)(atomic.LoadPointer(&obj.callCounter.activeBucket)) + + if info.Err == nil { + atomic.AddInt64(&ab.numSuccesses, 1) + } else { + atomic.AddInt64(&ab.numFailures, 1) + } + atomic.AddInt64(&ab.requestVolume, 1) +} + +func (b *outlierDetectionBalancer) UpdateState(s balancer.State) { + b.pickerUpdateCh.Put(s) + +} + +func (b *outlierDetectionBalancer) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) { + // "When the child policy asks for a subchannel, the outlier_detection will + // wrap the subchannel with a wrapper." - A50 + sc, err := b.cc.NewSubConn(addrs, opts) + if err != nil { + return nil, err + } + scw := &subConnWrapper{ + SubConn: sc, + addresses: addrs, + scUpdateCh: b.scUpdateCh, + } + b.mu.Lock() + defer b.mu.Unlock() + b.scWrappers[sc] = scw + if len(addrs) != 1 { + return scw, nil + } + + val, ok := b.odAddrs.Get(addrs[0]) + if !ok { + return scw, nil + } + + obj, ok := val.(*object) + if !ok { + return scw, nil + } + obj.sws = append(obj.sws, scw) + atomic.StorePointer(&scw.obj, unsafe.Pointer(obj)) + + // "If that address is currently ejected, that subchannel wrapper's eject + // method will be called." - A50 + if !obj.latestEjectionTimestamp.IsZero() { + scw.eject() + } + return scw, nil +} + +func (b *outlierDetectionBalancer) RemoveSubConn(sc balancer.SubConn) { + scw, ok := sc.(*subConnWrapper) + if !ok { // Shouldn't happen + return + } + // Remove the wrapped SubConn from the parent Client Conn. We don't remove + // from map entry until we get a Shutdown state for the SubConn, as we need + // that data to forward that state down. + b.cc.RemoveSubConn(scw.SubConn) +} + +// appendIfPresent appends the scw to the address, if the address is present in +// the Outlier Detection balancers address map. Returns nil if not present, and +// the map entry if present. +func (b *outlierDetectionBalancer) appendIfPresent(addr resolver.Address, scw *subConnWrapper) *object { + val, ok := b.odAddrs.Get(addr) + if !ok { + return nil + } + obj, ok := val.(*object) + if !ok { + // shouldn't happen, logical no-op + return nil + } + obj.sws = append(obj.sws, scw) + atomic.StorePointer(&scw.obj, unsafe.Pointer(obj)) + return obj +} + +// removeSubConnFromAddressesMapEntry removes the scw from it's map entry if +// present. +func (b *outlierDetectionBalancer) removeSubConnFromAddressesMapEntry(scw *subConnWrapper) { + obj := (*object)(atomic.LoadPointer(&scw.obj)) + if obj == nil { + return + } + for i, sw := range obj.sws { + if scw == sw { + obj.sws = append(obj.sws[:i], obj.sws[i+1:]...) + break + } + } +} + +// sameAddrForMap returns if two addresses are the same in regards to subchannel +// uniqueness/identity (i.e. what the addresses map is keyed on - address +// string, Server Name, and Attributes). +func sameAddrForMap(oldAddr resolver.Address, newAddr resolver.Address) bool { + if oldAddr.Addr != newAddr.Addr { + return false + } + if oldAddr.ServerName != newAddr.ServerName { + return false + } + return oldAddr.Attributes.Equal(newAddr.Attributes) +} + +func (b *outlierDetectionBalancer) UpdateAddresses(sc balancer.SubConn, addrs []resolver.Address) { + scw, ok := sc.(*subConnWrapper) + if !ok { + // Return, shouldn't happen if passed up scw + return + } + + b.cc.UpdateAddresses(scw.SubConn, addrs) + b.mu.Lock() + defer b.mu.Unlock() + + // Note that 0 addresses is a valid update/state for a SubConn to be in. + // This is correctly handled by this algorithm (handled as part of a non singular + // old address/new address). + if len(scw.addresses) == 1 { + if len(addrs) == 1 { // single address to single address + // If everything we care for in regards to address specificity for a + // list of SubConn's (Addr, ServerName, Attributes) is the same, + // then there is nothing to do past this point. + if sameAddrForMap(scw.addresses[0], addrs[0]) { + return + } + // 1. Remove Subchannel from Addresses map entry if present in Addresses map. + b.removeSubConnFromAddressesMapEntry(scw) + // 2. Add Subchannel to Addresses map entry if new address present in map. + obj := b.appendIfPresent(addrs[0], scw) + // 3. Relay state with eject() recalculated (using the corresponding + // map entry to see if it's currently ejected). + if obj == nil { // uneject unconditionally because could have come from an ejected address + scw.eject() + } else { + if obj.latestEjectionTimestamp.IsZero() { // relay new updated subconn state + scw.uneject() + } else { + scw.eject() + } + } + } else { // single address to multiple addresses + // 1. Remove Subchannel from Addresses map entry if present in Addresses map. + b.removeSubConnFromAddressesMapEntry(scw) + // 2. Clear the Subchannel wrapper's Call Counter entry. + obj := (*object)(atomic.LoadPointer(&scw.obj)) + if obj != nil { + obj.callCounter.clear() + } + // 3. Uneject the Subchannel in case it was previously ejected. + scw.uneject() + } + } else { + if len(addrs) == 1 { // multiple addresses to single address + // 1. Add Subchannel to Addresses map entry if new address present in map. + obj := b.appendIfPresent(addrs[0], scw) + if obj != nil && !obj.latestEjectionTimestamp.IsZero() { + scw.eject() + } + } // else is multiple to multiple - no op, continued to be ignored by outlier detection. + } + + scw.addresses = addrs +} + +func (b *outlierDetectionBalancer) ResolveNow(opts resolver.ResolveNowOptions) { + b.cc.ResolveNow(opts) +} + +func (b *outlierDetectionBalancer) Target() string { + return b.cc.Target() +} + +// objects returns a list of objects corresponding to every address in the address map. +func (b *outlierDetectionBalancer) objects() []*object { + var objs []*object + for _, addr := range b.odAddrs.Keys() { + val, ok := b.odAddrs.Get(addr) + if !ok { // Shouldn't happen + continue + } + obj, ok := val.(*object) + if !ok { + continue + } + objs = append(objs, obj) + } + return objs +} + +func max(x, y int64) int64 { + if x < y { + return y + } + return x +} + +func min(x, y int64) int64 { + if x < y { + return x + } + return y +} + +func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { + b.mu.Lock() + defer b.mu.Unlock() + b.timerStartTime = time.Now() + + // 2. For each address, swap the call counter's buckets in that address's + // map entry. + for _, obj := range b.objects() { + obj.callCounter.swap() + } + + // 3. If the success_rate_ejection configuration field is set, run the + // success rate algorithm. + if b.odCfg.SuccessRateEjection != nil { + b.successRateAlgorithm() + } + + // 4. If the failure_percentage_ejection configuration field is set, run the + // failure percentage algorithm. + if b.odCfg.FailurePercentageEjection != nil { + b.failurePercentageAlgorithm() + } + + // 5. For each address in the map: + for _, addr := range b.odAddrs.Keys() { + val, ok := b.odAddrs.Get(addr) + if !ok { + continue + } + obj, ok := val.(*object) + if !ok { + continue + } + // If the address is not ejected and the multiplier is greater than 0, + // decrease the multiplier by 1. + if obj.latestEjectionTimestamp.IsZero() && obj.ejectionTimeMultiplier > 0 { + obj.ejectionTimeMultiplier-- + continue + } + // If the address is ejected, and the current time is after + // ejection_timestamp + min(base_ejection_time (type: time.Time) * + // multiplier (type: int), max(base_ejection_time (type: time.Time), + // max_ejection_time (type: time.Time))), un-eject the address. + if !obj.latestEjectionTimestamp.IsZero() && now().After(obj.latestEjectionTimestamp.Add(time.Duration(min(b.odCfg.BaseEjectionTime.Nanoseconds()*obj.ejectionTimeMultiplier, max(b.odCfg.BaseEjectionTime.Nanoseconds(), b.odCfg.MaxEjectionTime.Nanoseconds()))))) { // need to way to inject a desired bool here at a certain point in tests, mock time.Now to return a late time, mock time.After to always return true... + b.unejectAddress(addr) + } + } + // This conditional only for testing (since the interval timer algorithm is + // called manually), will never hit in production. + if b.intervalTimer != nil { + b.intervalTimer.Stop() + } + b.intervalTimer = afterFunc(b.odCfg.Interval, func() { + b.intervalTimerAlgorithm() + }) +} + +func (b *outlierDetectionBalancer) run() { + for { + select { + case update := <-b.scUpdateCh.Get(): + b.scUpdateCh.Load() + switch u := update.(type) { + case *scUpdate: + scw := u.scw + scw.latestState = u.state + b.childMu.Lock() + if !scw.ejected && b.child != nil { + b.child.UpdateSubConnState(scw, u.state) + } + b.childMu.Unlock() + case *ejectedUpdate: + scw := u.scw + scw.ejected = u.ejected + var stateToUpdate balancer.SubConnState + if u.ejected { + // "The wrapper will report a state update with the + // TRANSIENT_FAILURE state, and will stop passing along + // updates from the underlying subchannel." + stateToUpdate = balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + } + } else { + // "The wrapper will report a state update with the latest + // update from the underlying subchannel, and resume passing + // along updates from the underlying subchannel." + stateToUpdate = scw.latestState // If this has never been written to will send connectivity IDLE which seems fine to me + } + b.childMu.Lock() + if b.child != nil { + b.child.UpdateSubConnState(scw, stateToUpdate) + } + b.childMu.Unlock() + } + case update := <-b.pickerUpdateCh.Get(): + b.pickerUpdateCh.Load() + if b.closed.HasFired() { // don't send picker updates to grpc after the balancer has been closed + return + } + switch u := update.(type) { + case balancer.State: + b.childState = u + b.mu.Lock() + noopCfg := b.noopConfig() + b.mu.Unlock() + b.recentPickerNoop = noopCfg + b.cc.UpdateState(balancer.State{ + ConnectivityState: b.childState.ConnectivityState, + // The outlier_detection LB policy will provide a picker that delegates to + // the child policy's picker, and when the request finishes, increment the + // corresponding counter in the map entry referenced by the subchannel + // wrapper that was picked. + Picker: &wrappedPicker{ + childPicker: b.childState.Picker, + // If both the `success_rate_ejection` and + // `failure_percentage_ejection` fields are unset in the + // configuration, the picker should not do that counting. + noopPicker: noopCfg, + }, + }) + case *LBConfig: + noopCfg := u.SuccessRateEjection == nil && u.FailurePercentageEjection == nil + if b.childState.Picker != nil && noopCfg != b.recentPickerNoop { + b.recentPickerNoop = noopCfg + b.cc.UpdateState(balancer.State{ + ConnectivityState: b.childState.ConnectivityState, + // The outlier_detection LB policy will provide a picker that delegates to + // the child policy's picker, and when the request finishes, increment the + // corresponding counter in the map entry referenced by the subchannel + // wrapper that was picked. + Picker: &wrappedPicker{ + childPicker: b.childState.Picker, + // If both the `success_rate_ejection` and + // `failure_percentage_ejection` fields are unset in the + // configuration, the picker should not do that counting. + noopPicker: noopCfg, + }, + }) + } + } + case <-b.closed.Done(): + return + } + } +} + +// numAddrsWithAtLeastRequestVolume returns the number of addresses present in +// the map that have request volume of at least requestVolume. +func (b *outlierDetectionBalancer) numAddrsWithAtLeastRequestVolume() uint32 { + var numAddrs uint32 + for _, obj := range b.objects() { + if uint32(obj.callCounter.inactiveBucket.requestVolume) >= b.odCfg.SuccessRateEjection.RequestVolume { + numAddrs++ + } + } + return numAddrs +} + +// meanAndStdDevOfSucceseesAtLeastRequestVolume returns the mean and std dev of +// the number of requests of addresses that have at least requestVolume. +func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume() (float64, float64) { + // 2. Calculate the mean and standard deviation of the fractions of + // successful requests among addresses with total request volume of at least + // success_rate_ejection.request_volume. + var totalFractionOfSuccessfulRequests float64 + var mean float64 + + for _, obj := range b.objects() { + // "of at least success_rate_ejection.request_volume" + if uint32(obj.callCounter.inactiveBucket.requestVolume) >= b.odCfg.SuccessRateEjection.RequestVolume { + totalFractionOfSuccessfulRequests += float64(obj.callCounter.inactiveBucket.numSuccesses) / float64(obj.callCounter.inactiveBucket.requestVolume) + } + } + mean = totalFractionOfSuccessfulRequests / float64(b.odAddrs.Len()) + var sumOfSquares float64 + for _, obj := range b.objects() { + devFromMean := (float64(obj.callCounter.inactiveBucket.numSuccesses) / float64(obj.callCounter.inactiveBucket.requestVolume)) - mean + sumOfSquares += devFromMean * devFromMean + } + + variance := sumOfSquares / float64(b.odAddrs.Len()) + return mean, math.Sqrt(variance) + +} + +func (b *outlierDetectionBalancer) successRateAlgorithm() { + // 1. If the number of addresses with request volume of at least + // success_rate_ejection.request_volume is less than + // success_rate_ejection.minimum_hosts, stop. + if b.numAddrsWithAtLeastRequestVolume() < b.odCfg.SuccessRateEjection.MinimumHosts { // TODO: O(n) search, is there a way to optimize this? + return + } + + // 2. Calculate the mean and standard deviation of the fractions of + // successful requests among addresses with total request volume of at least + // success_rate_ejection.request_volume. + mean, stddev := b.meanAndStdDevOfSuccessesAtLeastRequestVolume() + + // 3. For each address: + for _, addr := range b.odAddrs.Keys() { + val, ok := b.odAddrs.Get(addr) + if !ok { + continue + } + obj, ok := val.(*object) + if !ok { + continue + } + ccb := obj.callCounter.inactiveBucket + sre := b.odCfg.SuccessRateEjection + // i. If the percentage of ejected addresses is greater than + // max_ejection_percent, stop. + if float64(b.numAddrsEjected)/float64(b.odAddrs.Len())*100 > float64(b.odCfg.MaxEjectionPercent) { + return + } + + // ii. If the address's total request volume is less than + // success_rate_ejection.request_volume, continue to the next address. + if ccb.requestVolume < int64(sre.RequestVolume) { + continue + } + + // iii. If the address's success rate is less than (mean - stdev * + // (success_rate_ejection.stdev_factor / 1000)) + successRate := float64(ccb.numSuccesses) / float64(ccb.requestVolume) + if successRate < (mean - stddev*(float64(sre.StdevFactor)/1000)) { + // then choose a random integer in [0, 100). If that number is less + // than success_rate_ejection.enforcement_percentage, eject that + // address. + if uint32(grpcrand.Int31n(100)) < sre.EnforcementPercentage { + b.ejectAddress(addr) + } + } + } +} + +func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { + // 1. If the number of addresses is less than + // failure_percentage_ejection.minimum_hosts, stop. + if uint32(b.odAddrs.Len()) < b.odCfg.FailurePercentageEjection.MinimumHosts { + return + } + + // 2. For each address: + for _, addr := range b.odAddrs.Keys() { + val, ok := b.odAddrs.Get(addr) + if !ok { + continue + } + obj, ok := val.(*object) + if !ok { + continue + } + ccb := obj.callCounter.inactiveBucket + fpe := b.odCfg.FailurePercentageEjection + // i. If the percentage of ejected addresses is greater than + // max_ejection_percent, stop. + if float64(b.numAddrsEjected)/float64(b.odAddrs.Len())*100 > float64(b.odCfg.MaxEjectionPercent) { + return + } + // ii. If the address's total request volume is less than + // failure_percentage_ejection.request_volume, continue to the next + // address. + if uint32(ccb.requestVolume) < fpe.RequestVolume { + continue + } + // 2c. If the address's failure percentage is greater than + // failure_percentage_ejection.threshold + failurePercentage := (float64(ccb.numFailures) / float64(ccb.requestVolume)) * 100 + if failurePercentage > float64(b.odCfg.FailurePercentageEjection.Threshold) { + // then choose a random integer in [0, 100). If that number is less + // than failiure_percentage_ejection.enforcement_percentage, eject + // that address. + if uint32(grpcrand.Int31n(100)) < b.odCfg.FailurePercentageEjection.EnforcementPercentage { + b.ejectAddress(addr) + } + } + } +} + +func (b *outlierDetectionBalancer) ejectAddress(addr resolver.Address) { + val, ok := b.odAddrs.Get(addr) + if !ok { // Shouldn't happen + return + } + obj, ok := val.(*object) + if !ok { // Shouldn't happen + return + } + + b.numAddrsEjected++ + + // To eject an address, set the current ejection timestamp to the timestamp + // that was recorded when the timer fired, increase the ejection time + // multiplier by 1, and call eject() on each subchannel wrapper in that + // address's subchannel wrapper list. + obj.latestEjectionTimestamp = b.timerStartTime + obj.ejectionTimeMultiplier++ + for _, sbw := range obj.sws { + sbw.eject() + } +} + +func (b *outlierDetectionBalancer) unejectAddress(addr resolver.Address) { + val, ok := b.odAddrs.Get(addr) + if !ok { // Shouldn't happen + return + } + obj, ok := val.(*object) + if !ok { // Shouldn't happen + return + } + b.numAddrsEjected-- + + // To un-eject an address, set the current ejection timestamp to null + // (doesn't he mean latest ejection timestamp?, in Golang null for time is + // logically equivalent in practice to the time zero value) and call + // uneject() on each subchannel wrapper in that address's subchannel wrapper + // list. + obj.latestEjectionTimestamp = time.Time{} + for _, sbw := range obj.sws { + sbw.uneject() + } +} + +type object struct { + // The call result counter object + callCounter *callCounter + + // The latest ejection timestamp, or null if the address is currently not + // ejected + latestEjectionTimestamp time.Time // We represent the branching logic on the null with a time.Zero() value + + // The current ejection time multiplier, starting at 0 + ejectionTimeMultiplier int64 + + // A list of subchannel wrapper objects that correspond to this address + sws []*subConnWrapper +} + +func newObject() *object { + return &object{ + callCounter: newCallCounter(), + sws: make([]*subConnWrapper, 0), + } +} diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index 106e2b64dbc..16ae32cdc83 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -19,20 +19,34 @@ package outlierdetection import ( + "context" "encoding/json" "errors" + "fmt" "strings" + "sync" "testing" "time" "github.com/google/go-cmp/cmp" + "github.com/google/go-cmp/cmp/cmpopts" "google.golang.org/grpc/balancer" + "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/grpcsync" "google.golang.org/grpc/internal/grpctest" internalserviceconfig "google.golang.org/grpc/internal/serviceconfig" + "google.golang.org/grpc/internal/testutils" + "google.golang.org/grpc/resolver" "google.golang.org/grpc/serviceconfig" "google.golang.org/grpc/xds/internal/balancer/clusterimpl" ) +var ( + defaultTestTimeout = 5 * time.Second + defaultTestShortTimeout = 10 * time.Millisecond +) + type s struct { grpctest.Tester } @@ -272,6 +286,9 @@ func (lbc *LBConfig) Equal(lbc2 *LBConfig) bool { func init() { balancer.Register(errParseConfigBuilder{}) + balancer.Register(errParseConfigBuilder{}) + balancer.Register(tcibBuilder{}) + balancer.Register(verifyBalancerBuilder{}) } type errParseConfigBuilder struct{} @@ -287,3 +304,1780 @@ func (errParseConfigBuilder) Name() string { func (errParseConfigBuilder) ParseConfig(c json.RawMessage) (serviceconfig.LoadBalancingConfig, error) { return nil, errors.New("some error") } + +func setup(t *testing.T) (*outlierDetectionBalancer, *testutils.TestClientConn) { + t.Helper() + builder := balancer.Get(Name) + if builder == nil { + t.Fatalf("balancer.Get(%q) returned nil", Name) + } + tcc := testutils.NewTestClientConn(t) + odB := builder.Build(tcc, balancer.BuildOptions{}) + return odB.(*outlierDetectionBalancer), tcc +} + +const tcibname = "testClusterImplBalancer" +const verifyBalancerName = "verifyBalancer" + +type tcibBuilder struct{} + +func (tcibBuilder) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Balancer { + return &testClusterImplBalancer{ + ccsCh: testutils.NewChannel(), + scStateCh: testutils.NewChannel(), + resolverErrCh: testutils.NewChannel(), + closeCh: testutils.NewChannel(), + exitIdleCh: testutils.NewChannel(), + cc: cc, + } +} + +func (tcibBuilder) Name() string { + return tcibname +} + +type testClusterImplBalancerConfig struct { + serviceconfig.LoadBalancingConfig +} + +type testClusterImplBalancer struct { + // ccsCh is a channel used to signal the receipt of a ClientConn update. + ccsCh *testutils.Channel + // scStateCh is a channel used to signal the receipt of a SubConn update. + scStateCh *testutils.Channel + // resolverErrCh is a channel used to signal a resolver error. + resolverErrCh *testutils.Channel + // closeCh is a channel used to signal the closing of this balancer. + closeCh *testutils.Channel + exitIdleCh *testutils.Channel + // cc is the balancer.ClientConn passed to this test balancer as part of the + // Build() call. + cc balancer.ClientConn +} + +type subConnWithState struct { + sc balancer.SubConn + state balancer.SubConnState +} + +func (tb *testClusterImplBalancer) UpdateClientConnState(ccs balancer.ClientConnState) error { + tb.ccsCh.Send(ccs) + return nil +} + +func (tb *testClusterImplBalancer) ResolverError(err error) { + tb.resolverErrCh.Send(err) +} + +func (tb *testClusterImplBalancer) UpdateSubConnState(sc balancer.SubConn, state balancer.SubConnState) { + tb.scStateCh.Send(subConnWithState{sc: sc, state: state}) +} + +func (tb *testClusterImplBalancer) Close() { + tb.closeCh.Send(struct{}{}) +} + +// waitForClientConnUpdate verifies if the testClusterImplBalancer receives the +// provided ClientConnState within a reasonable amount of time. +func (tb *testClusterImplBalancer) waitForClientConnUpdate(ctx context.Context, wantCCS balancer.ClientConnState) error { + ccs, err := tb.ccsCh.Receive(ctx) + if err != nil { + return err + } + gotCCS := ccs.(balancer.ClientConnState) + if diff := cmp.Diff(gotCCS, wantCCS, cmpopts.IgnoreFields(resolver.State{}, "Attributes")); diff != "" { + return fmt.Errorf("received unexpected ClientConnState, diff (-got +want): %v", diff) + } + return nil +} + +// waitForSubConnUpdate verifies if the testClusterImplBalancer receives the +// provided SubConn and SubConnState within a reasonable amount of time. +func (tb *testClusterImplBalancer) waitForSubConnUpdate(ctx context.Context, wantSCS subConnWithState) error { + scs, err := tb.scStateCh.Receive(ctx) + if err != nil { + return err + } + gotSCS := scs.(subConnWithState) + if !cmp.Equal(gotSCS, wantSCS, cmp.AllowUnexported(subConnWithState{}, testutils.TestSubConn{}, subConnWrapper{}, object{}), cmpopts.IgnoreFields(subConnWrapper{}, "scUpdateCh")) { + return fmt.Errorf("received SubConnState: %+v, want %+v", gotSCS, wantSCS) + } + return nil +} + +// waitForClose verifies if the testClusterImplBalancer receives a Close call +// within a reasonable amount of time. +func (tb *testClusterImplBalancer) waitForClose(ctx context.Context) error { + if _, err := tb.closeCh.Receive(ctx); err != nil { + return err + } + return nil +} + +// TestUpdateClientConnState invokes the UpdateClientConnState method on the +// odBalancer with different inputs and verifies that the child balancer is built +// and updated properly. +func (s) TestUpdateClientConnState(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + od, _ := setup(t) + defer func() { + od.Close() // this will leak a goroutine otherwise + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + od.UpdateClientConnState(balancer.ClientConnState{ + BalancerConfig: &LBConfig{ + Interval: 10 * time.Second, + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 1900, + EnforcementPercentage: 100, + MinimumHosts: 5, + RequestVolume: 100, + }, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 85, + EnforcementPercentage: 5, + MinimumHosts: 5, + RequestVolume: 50, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + // The child balancer should be created and forwarded the ClientConn update + // from the first successful UpdateClientConnState call. + if err := od.child.(*testClusterImplBalancer).waitForClientConnUpdate(ctx, balancer.ClientConnState{ + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } +} + +// TestUpdateClientConnStateDifferentType invokes the UpdateClientConnState +// method on the odBalancer with two different types and verifies that the child +// balancer is built and updated properly on the first, and the second update +// closes the child and builds a new one. +func (s) TestUpdateClientConnStateDifferentType(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + od, _ := setup(t) + defer func() { + od.Close() // this will leak a goroutine otherwise + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + od.UpdateClientConnState(balancer.ClientConnState{ + BalancerConfig: &LBConfig{ + Interval: 10 * time.Second, + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 1900, + EnforcementPercentage: 100, + MinimumHosts: 5, + RequestVolume: 100, + }, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 85, + EnforcementPercentage: 5, + MinimumHosts: 5, + RequestVolume: 50, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + ciChild := od.child.(*testClusterImplBalancer) + // The child balancer should be created and forwarded the ClientConn update + // from the first successful UpdateClientConnState call. + if err := ciChild.waitForClientConnUpdate(ctx, balancer.ClientConnState{ + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + + od.UpdateClientConnState(balancer.ClientConnState{ + BalancerConfig: &LBConfig{ + Interval: 10 * time.Second, + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 1900, + EnforcementPercentage: 100, + MinimumHosts: 5, + RequestVolume: 100, + }, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 85, + EnforcementPercentage: 5, + MinimumHosts: 5, + RequestVolume: 50, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: verifyBalancerName, + Config: verifyBalancerConfig{}, + }, + }, + }) + + // Verify previous child balancer closed. + if err := ciChild.waitForClose(ctx); err != nil { + t.Fatalf("Error waiting for Close() call on child balancer %v", err) + } +} + +// TestUpdateState tests that an UpdateState call gets forwarded to the +// ClientConn. +func (s) TestUpdateState(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + od, tcc := setup(t) + defer func() { + od.Close() + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + od.UpdateClientConnState(balancer.ClientConnState{ + BalancerConfig: &LBConfig{ + Interval: 10 * time.Second, + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 1900, + EnforcementPercentage: 100, + MinimumHosts: 5, + RequestVolume: 100, + }, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 85, + EnforcementPercentage: 5, + MinimumHosts: 5, + RequestVolume: 50, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &testutils.TestConstPicker{}, + }) + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + // Should forward the connectivity State to Client Conn. + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case state := <-tcc.NewStateCh: + if state != connectivity.Ready { + t.Fatalf("ClientConn received connectivity state %v, want %v", state, connectivity.Ready) + } + } + + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case picker := <-tcc.NewPickerCh: + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + } +} + +// TestClose tests the Close operation on the Outlier Detection Balancer. The +// Close operation should close the child. +func (s) TestClose(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + defer func() { + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + od, _ := setup(t) + + od.UpdateClientConnState(balancer.ClientConnState{ // could pull this out to the setup() call as well, maybe a wrapper on what is currently there... + BalancerConfig: &LBConfig{ + Interval: 10 * time.Second, + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 1900, + EnforcementPercentage: 100, + MinimumHosts: 5, + RequestVolume: 100, + }, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 85, + EnforcementPercentage: 5, + MinimumHosts: 5, + RequestVolume: 50, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + ciChild := od.child.(*testClusterImplBalancer) + if err := ciChild.waitForClientConnUpdate(ctx, balancer.ClientConnState{ + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + + od.Close() + + // Verify child balancer closed. + if err := ciChild.waitForClose(ctx); err != nil { + t.Fatalf("Error waiting for Close() call on child balancer %v", err) + } +} + +// TestUpdateAddresses tests the functionality of UpdateAddresses and any +// changes in the addresses/plurality of those addresses for a SubConn. +func (s) TestUpdateAddresses(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + od, tcc := setup(t) + defer func() { + od.Close() + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 10 * time.Second, + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + FailurePercentageEjection: &FailurePercentageEjection{ // have this eject one but not the other + Threshold: 50, + EnforcementPercentage: 100, + MinimumHosts: 2, + RequestVolume: 3, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + child := od.child.(*testClusterImplBalancer) + if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + }, + }, + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + + scw1, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address1", + }, + }, balancer.NewSubConnOptions{}) + + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + select { + case <-ctx.Done(): + t.Fatal("timeout waiting for NewSubConn to be called on test Client Conn") + case <-tcc.NewSubConnCh: + } + _, ok := scw1.(*subConnWrapper) + if !ok { + t.Fatalf("SubConn passed downward should have been a subConnWrapper") + } + + scw2, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address2", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &rrPicker{ + scs: []balancer.SubConn{scw1, scw2}, + }, + }) + + // Setup the system to where one address is ejected and one address + // isn't. + select { + case <-ctx.Done(): + t.Fatal("timeout while waiting for a UpdateState call on the ClientConn") + case picker := <-tcc.NewPickerCh: + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + // Eject the second address. + pi, err = picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + od.intervalTimerAlgorithm() + // verify UpdateSubConnState() got called with TRANSIENT_FAILURE for + // child with address that was ejected. + if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + sc: scw2, + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected + }); err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + } + + // Update scw1 to another address that is currently ejected. This should + // cause scw1 to get ejected. + od.UpdateAddresses(scw1, []resolver.Address{ + { + Addr: "address2", + }, + }) + + // verify that update addresses gets forwarded to ClientConn. + select { + case <-ctx.Done(): + t.Fatal("timeout while waiting for a UpdateState call on the ClientConn") + case <-tcc.UpdateAddressesAddrsCh: + } + // verify scw1 got ejected (UpdateSubConnState called with TRANSIENT + // FAILURE). + if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + sc: scw1, + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected + }); err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + + // Update scw1 to multiple addresses. This should cause scw1 to get + // unejected, as is it no longer being tracked for Outlier Detection. + od.UpdateAddresses(scw1, []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + }) + // verify scw2 got unejected (UpdateSubConnState called with recent state). + if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + sc: scw1, + state: balancer.SubConnState{ConnectivityState: connectivity.Idle}, // If you uneject a SubConn that hasn't received a UpdateSubConnState, IDLE is recent state. This seems fine or is this wrong? + }); err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + + // Update scw1 to a different multiple addresses list. A change of addresses + // in which the plurality goes from multiple to multiple should be a no-op, + // as the address continues to be ignored by outlier detection. + od.UpdateAddresses(scw1, []resolver.Address{ + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }) + // Verify no downstream effects. + sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + if err := od.child.(*testClusterImplBalancer).waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { + t.Fatalf("no SubConn update should have been sent (no SubConn got ejected/unejected)") + } + + // Update scw1 back to a single address, which is ejected. This should cause + // the SubConn to be re-ejected. + + od.UpdateAddresses(scw1, []resolver.Address{ + { + Addr: "address2", + }, + }) + // verify scw1 got ejected (UpdateSubConnState called with TRANSIENT FAILURE). + if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + sc: scw1, + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected + }); err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } +} + +// Three important pieces of functionality run() synchronizes in regards to UpdateState calls towards grpc: + +// 1. On a config update, checking if the picker was actually created or not. +// 2. Keeping track of the most recent connectivity state sent from the child (determined by UpdateState()). +// * This will always forward up with most recent noopCfg bit +// 3. Keeping track of the most recent no-op config bit (determined by UpdateClientConnState()) +// * Will only forward up if no-op config bit changed and picker was already created + +// TestPicker tests the Picker updates sent upward to grpc from the Outlier +// Detection Balancer. Two things can trigger a picker update, an +// UpdateClientConnState call (can flip the no-op config bit that affects +// Picker) and an UpdateState call (determines the connectivity state sent +// upward). +func (s) TestPicker(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + od, tcc := setup(t) + defer func() { + od.Close() + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + }, + }, + BalancerConfig: &LBConfig{ // TODO: S/ variable + Interval: 10 * time.Second, + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 1900, + EnforcementPercentage: 100, + MinimumHosts: 5, + RequestVolume: 100, + }, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 85, + EnforcementPercentage: 5, + MinimumHosts: 5, + RequestVolume: 50, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + child := od.child.(*testClusterImplBalancer) + if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + }, + }, + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + + scw, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address1", + }, + }, balancer.NewSubConnOptions{}) + + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &testutils.TestConstPicker{ + SC: scw, + }, + }) + + // Should forward the connectivity State to Client Conn. + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case state := <-tcc.NewStateCh: + if state != connectivity.Ready { + t.Fatalf("ClientConn received connectivity state %v, want %v", state, connectivity.Ready) + } + } + + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case picker := <-tcc.NewPickerCh: + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + od.mu.Lock() + val, ok := od.odAddrs.Get(resolver.Address{ + Addr: "address1", + }) + if !ok { + t.Fatal("map entry for address: address1 not present in map") + } + obj, ok := val.(*object) + if !ok { + t.Fatal("map value isn't obj type") + } + bucketWant := &bucket{ + numSuccesses: 1, + numFailures: 1, + requestVolume: 2, + } + if diff := cmp.Diff((*bucket)(obj.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call from picker + t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) + } + od.mu.Unlock() + } + + // UpdateClientConnState with a noop config + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 1<<63 - 1, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + }, + }, + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + + // The connectivity state sent to the Client Conn should be the persisted + // recent state received from the last UpdateState() call, which in this + // case is connecting. + select { + case <-ctx.Done(): + t.Fatal("timeout waiting for picker update on ClientConn, should have updated because no-op config changed on UpdateClientConnState") + case state := <-tcc.NewStateCh: + if state != connectivity.Ready { + t.Fatalf("ClientConn received connectivity state %v, want %v", state, connectivity.Ready) + } + } + + select { + case <-ctx.Done(): + t.Fatal("timeout waiting for picker update on ClientConn, should have updated because no-op config changed on UpdateClientConnState") + case picker := <-tcc.NewPickerCh: + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + od.mu.Lock() + val, ok := od.odAddrs.Get(resolver.Address{ + Addr: "address1", + }) + if !ok { + t.Fatal("map entry for address: address1 not present in map") + } + obj, ok := val.(*object) + if !ok { + t.Fatal("map value isn't obj type") + } + + // The active bucket should be cleared because the interval timer + // algorithm didn't run in between ClientConn updates and the picker + // should not count, as the outlier detection balancer is configured + // with a no-op configuration. + bucketWant := &bucket{} + if diff := cmp.Diff((*bucket)(obj.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call + t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) + } + od.mu.Unlock() + } + + // UpdateState with a connecting state. This new most recent connectivity + // state should be forwarded to the Client Conn, alongside the most recent + // noop config bit which is true. + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Connecting, + Picker: &testutils.TestConstPicker{ + SC: scw, + }, + }) + + // Should forward the most recent connectivity State to Client Conn. + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case state := <-tcc.NewStateCh: + if state != connectivity.Connecting { + t.Fatalf("ClientConn received connectivity state %v, want %v", state, connectivity.Connecting) + } + } + + // Should forward the picker containing the most recent no-op config bit. + select { + case <-ctx.Done(): + t.Fatal("timeout while waiting for a UpdateState call on the ClientConn") + case picker := <-tcc.NewPickerCh: + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + od.mu.Lock() + val, ok := od.odAddrs.Get(resolver.Address{ + Addr: "address1", + }) + if !ok { + t.Fatal("map entry for address: address1 not present in map") + } + obj, ok := val.(*object) + if !ok { + t.Fatal("map value isn't obj type") + } + + // The active bucket should be cleared because the interval timer + // algorithm didn't run in between ClientConn updates and the picker + // should not count, as the outlier detection balancer is configured + // with a no-op configuration. + bucketWant := &bucket{} + if diff := cmp.Diff((*bucket)(obj.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call + t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) + } + od.mu.Unlock() + } +} + +type rrPicker struct { + scs []balancer.SubConn + next int +} + +func (rrp *rrPicker) Pick(balancer.PickInfo) (balancer.PickResult, error) { + sc := rrp.scs[rrp.next] + rrp.next = (rrp.next + 1) % len(rrp.scs) + return balancer.PickResult{SubConn: sc}, nil +} + +// TestEjectUnejectSuccessRate tests the functionality of the interval timer +// algorithm of ejecting/unejecting SubConns when configured with +// SuccessRateEjection. It also tests a desired invariant of a SubConnWrapper +// being ejected or unejected, which is to either forward or not forward SubConn +// updates from grpc. +func (s) TestEjectUnejectSuccessRate(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + // Setup the outlier detection balancer to a point where it will be in a + // situation to potentially eject addresses. + od, tcc := setup(t) + defer func() { + od.Close() + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 500, + EnforcementPercentage: 100, + MinimumHosts: 3, + RequestVolume: 3, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + child := od.child.(*testClusterImplBalancer) + if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + + scw1, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address1", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + scw2, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address2", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + scw3, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address3", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &rrPicker{ + scs: []balancer.SubConn{scw1, scw2, scw3}, + }, + }) + + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case picker := <-tcc.NewPickerCh: + // Set each of the three upstream addresses to have five successes each. + // This should cause none of the addresses to be ejected as none of them + // are outliers according to the success rate algorithm. + for i := 0; i < 3; i++ { + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + } + + od.intervalTimerAlgorithm() + + // verify no UpdateSubConnState() call on the child, as no addresses got + // ejected (ejected address will cause an UpdateSubConnState call). + sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { + t.Fatalf("no SubConn update should have been sent (no SubConn got ejected)") + } + + // Since no addresses are ejected, a SubConn update should forward down + // to the child. + od.UpdateSubConnState(scw1.(*subConnWrapper).SubConn, balancer.SubConnState{ + ConnectivityState: connectivity.Connecting, + }) + + if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + sc: scw1, + state: balancer.SubConnState{ConnectivityState: connectivity.Connecting}, + }); err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + + // Set two of the upstream addresses to have five successes each, and + // one of the upstream addresses to have five failures. This should + // cause the address which has five failures to be ejected according the + // SuccessRateAlgorithm. + for i := 0; i < 2; i++ { + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + } + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + + // should eject address that always errored. + od.intervalTimerAlgorithm() + // Due to the address being ejected, the SubConn with that address + // should be ejected, meaning a TRANSIENT_FAILURE connectivity state + // gets reported to the child. + if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + sc: scw3, + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected + }); err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + // Only one address should be ejected. + sCtx, cancel = context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { + t.Fatalf("Only one SubConn update should have been sent (only one SubConn got ejected)") + } + + // Now that an address is ejected, SubConn updates for SubConns using + // that address should not be forwarded downward. These SubConn updates + // will be cached to update the child sometime in the future when the + // address gets unejected. + od.UpdateSubConnState(pi.SubConn, balancer.SubConnState{ + ConnectivityState: connectivity.Connecting, + }) + if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { + t.Fatalf("SubConn update should not have been forwarded (the SubConn is ejected)") + } + + // Override now to cause the interval timer algorithm to always uneject a SubConn. + defer func(n func() time.Time) { + now = n + }(now) + + now = func() time.Time { + return time.Now().Add(time.Second * 1000) // will cause to always uneject addresses which are ejected + } + od.intervalTimerAlgorithm() + + // unejected SubConn should report latest persisted state - which is + // connecting from earlier. + if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + sc: scw3, + state: balancer.SubConnState{ConnectivityState: connectivity.Connecting}, + }); err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + + } + +} + +// TestEjectFailureRate tests the functionality of the interval timer +// algorithm of ejecting SubConns when configured with +// FailurePercentageEjection. +func (s) TestEjectFailureRate(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + od, tcc := setup(t) + defer func() { + internal.UnregisterOutlierDetectionBalancerForTesting() + defer od.Close() + }() + + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 50, + EnforcementPercentage: 100, + MinimumHosts: 3, + RequestVolume: 3, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + child := od.child.(*testClusterImplBalancer) + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + + scw1, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address1", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + scw2, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address2", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + scw3, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address3", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &rrPicker{ + scs: []balancer.SubConn{scw1, scw2, scw3}, + }, + }) + + select { + case <-ctx.Done(): + t.Fatal("timeout while waiting for a UpdateState call on the ClientConn") + case picker := <-tcc.NewPickerCh: + // Set each upstream address to have five successes each. This should + // cause none of the addresses to be ejected as none of them are below + // the failure percentage threshold. + for i := 0; i < 3; i++ { + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + } + + od.intervalTimerAlgorithm() + + sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { + t.Fatalf("no SubConn update should have been sent (no SubConn got ejected)") + } + + // Set two upstream addresses to have five successes each, and one + // upstream address to have five failures. This should cause the address + // with five failures to be ejected according to the Failure Percentage + // Algorithm. + for i := 0; i < 2; i++ { + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + } + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + + // should eject address that always errored. + od.intervalTimerAlgorithm() + + // verify UpdateSubConnState() got called with TRANSIENT_FAILURE for + // child in address that was ejected. + if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + sc: scw3, + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected + }); err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + + // verify only one address got ejected + sCtx, cancel = context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { + t.Fatalf("Only one SubConn update should have been sent (only one SubConn got ejected)") + } + } +} + +// TestDurationOfInterval tests the configured interval timer. On the first +// config received, the Outlier Detection balancer should configure the timer +// with whatever is directly specified on the config. On subsequent configs +// received, the Outlier Detection balancer should configure the timer with +// whatever interval is configured minus the difference between the current time +// and the previous start timestamp. For a no-op configuration, the timer should +// not be configured at all. +func (s) TestDurationOfInterval(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + od, _ := setup(t) + defer func(af func(d time.Duration, f func()) *time.Timer) { + od.Close() + afterFunc = af + internal.UnregisterOutlierDetectionBalancerForTesting() + }(afterFunc) + + durationChan := testutils.NewChannel() + afterFunc = func(dur time.Duration, _ func()) *time.Timer { + durationChan.Send(dur) + return time.NewTimer(1<<63 - 1) + } + + od.UpdateClientConnState(balancer.ClientConnState{ + BalancerConfig: &LBConfig{ + Interval: 8 * time.Second, + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 1900, + EnforcementPercentage: 100, + MinimumHosts: 5, + RequestVolume: 100, + }, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 85, + EnforcementPercentage: 5, + MinimumHosts: 5, + RequestVolume: 50, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + if err := od.child.(*testClusterImplBalancer).waitForClientConnUpdate(ctx, balancer.ClientConnState{ + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + + d, err := durationChan.Receive(ctx) + if err != nil { + t.Fatalf("Error receiving duration from afterFunc() call: %v", err) + } + dur := d.(time.Duration) + // The configured duration should be 8 seconds - what the balancer was + // configured with. + if dur.Seconds() != 8 { + t.Fatalf("configured duration should have been 8 seconds to start timer") + } + + defer func(n func() time.Time) { + now = n + }(now) + now = func() time.Time { + return time.Now().Add(time.Second * 5) + } + + // UpdateClientConnState with an interval of 9 seconds. Due to 5 seconds + // already passing (from overridden time.Now function), this should start an + // interval timer of ~4 seconds. + od.UpdateClientConnState(balancer.ClientConnState{ + BalancerConfig: &LBConfig{ + Interval: 9 * time.Second, + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 1900, + EnforcementPercentage: 100, + MinimumHosts: 5, + RequestVolume: 100, + }, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 85, + EnforcementPercentage: 5, + MinimumHosts: 5, + RequestVolume: 50, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + ctx, cancel = context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + if err := od.child.(*testClusterImplBalancer).waitForClientConnUpdate(ctx, balancer.ClientConnState{ + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + d, err = durationChan.Receive(ctx) + if err != nil { + t.Fatalf("Error receiving duration from afterFunc() call: %v", err) + } + dur = d.(time.Duration) + if dur.Seconds() < 3.5 || 4.5 < dur.Seconds() { + t.Fatalf("configured duration should have been around 4 seconds to start timer") + } + + // UpdateClientConnState with a no-op config. This shouldn't configure the + // interval timer at all due to it being a no-op. + od.UpdateClientConnState(balancer.ClientConnState{ + BalancerConfig: &LBConfig{ + Interval: 10 * time.Second, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + ctx, cancel = context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + if err := od.child.(*testClusterImplBalancer).waitForClientConnUpdate(ctx, balancer.ClientConnState{ + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + // No timer should have been started. + sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + _, err = durationChan.Receive(sCtx) + if err == nil { + t.Fatal("No timer should have started.") + } +} + +// TestConcurrentPickerCountsWithIntervalTimer tests concurrent picker updates +// (writing to the callCounter) and the interval timer algorithm, which reads +// the callCounter. +func (s) TestConcurrentPickerCountsWithIntervalTimer(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + od, tcc := setup(t) + defer func() { + defer od.Close() + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ // Have both Success Rate and Failure Percentage to step through all the interval timer code + StdevFactor: 500, + EnforcementPercentage: 100, + MinimumHosts: 3, + RequestVolume: 3, + }, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 50, + EnforcementPercentage: 100, + MinimumHosts: 3, + RequestVolume: 3, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + if err := od.child.(*testClusterImplBalancer).waitForClientConnUpdate(ctx, balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + + scw1, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address1", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + scw2, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address2", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + scw3, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address3", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &rrPicker{ + scs: []balancer.SubConn{scw1, scw2, scw3}, + }, + }) + + var picker balancer.Picker + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case picker = <-tcc.NewPickerCh: + } + + // Spawn a goroutine that constantly picks and invokes the Done callback + // counting for successful and failing RPC's. + finished := make(chan struct{}) + go func() { + // constantly update the picker to test for no race conditions causing + // corrupted memory (have concurrent pointer reads/writes). + for { + select { + case <-finished: + return + default: + } + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + continue + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + time.Sleep(1 * time.Nanosecond) + } + }() + + od.intervalTimerAlgorithm() // causes two swaps on the callCounter + od.intervalTimerAlgorithm() + close(finished) +} + +// TestConcurrentOperations calls different operations on the balancer in +// separate goroutines to test for any race conditions and deadlocks. It also +// uses a child balancer which verifies that no operations on the child get +// called after the child balancer is closed. +func (s) TestConcurrentOperations(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + od, tcc := setup(t) + defer func() { + od.Close() + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ // Have both Success Rate and Failure Percentage to step through all the interval timer code + StdevFactor: 500, + EnforcementPercentage: 100, + MinimumHosts: 3, + RequestVolume: 3, + }, + FailurePercentageEjection: &FailurePercentageEjection{ + Threshold: 50, + EnforcementPercentage: 100, + MinimumHosts: 3, + RequestVolume: 3, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: verifyBalancerName, + Config: verifyBalancerConfig{}, + }, + }, + }) + + od.child.(*verifyBalancer).t = t + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + scw1, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address1", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + scw2, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address2", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + scw3, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address3", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &rrPicker{ + scs: []balancer.SubConn{scw2, scw3}, + }, + }) + + var picker balancer.Picker + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case picker = <-tcc.NewPickerCh: + } + + finished := make(chan struct{}) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + for { + select { + case <-finished: + return + default: + } + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + continue + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + time.Sleep(1 * time.Nanosecond) + } + }() + + wg.Add(1) + go func() { + defer wg.Done() + for { + select { + case <-finished: + return + default: + } + od.intervalTimerAlgorithm() + } + }() + + // call Outlier Detection's balancer.ClientConn operations asynchrously. + // balancer.ClientConn operations have no guarantee from the API to be + // called synchronously. + wg.Add(1) + go func() { + defer wg.Done() + for { + select { + case <-finished: + return + default: + } + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &rrPicker{ + scs: []balancer.SubConn{scw2, scw3}, + }, + }) + time.Sleep(1 * time.Nanosecond) + } + }() + + wg.Add(1) + go func() { + defer wg.Done() + od.NewSubConn([]resolver.Address{{Addr: "address4"}}, balancer.NewSubConnOptions{}) + }() + + wg.Add(1) + go func() { + defer wg.Done() + od.RemoveSubConn(scw1) + }() + + wg.Add(1) + go func() { + defer wg.Done() + od.UpdateAddresses(scw2, []resolver.Address{ + { + Addr: "address3", + }, + }) + }() + + // Call balancer.Balancers synchronously in this goroutine, upholding the + // balancer.Balancer API guarantee. + od.UpdateClientConnState(balancer.ClientConnState{ // This will delete addresses and flip to no op + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 1<<63 - 1, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: verifyBalancerName, + Config: verifyBalancerConfig{}, + }, + }, + }) + + // Call balancer.Balancers synchronously in this goroutine, upholding the + // balancer.Balancer API guarantee. + od.UpdateSubConnState(scw1, balancer.SubConnState{ + ConnectivityState: connectivity.Connecting, + }) + od.ResolverError(errors.New("some error")) + od.ExitIdle() + od.Close() + close(finished) + wg.Wait() +} + +type verifyBalancerConfig struct { + serviceconfig.LoadBalancingConfig +} + +type verifyBalancerBuilder struct{} + +func (verifyBalancerBuilder) Build(cc balancer.ClientConn, opts balancer.BuildOptions) balancer.Balancer { + return &verifyBalancer{ + closed: grpcsync.NewEvent(), + } +} + +func (verifyBalancerBuilder) Name() string { + return verifyBalancerName +} + +// verifyBalancer is a balancer that verifies after a Close() call, +// no other balancer.Balancer methods are called afterward. +type verifyBalancer struct { + closed *grpcsync.Event + // To fail the test if any balancer.Balancer operation gets called after + // Close(). + t *testing.T +} + +func (vb *verifyBalancer) UpdateClientConnState(ccs balancer.ClientConnState) error { + if vb.closed.HasFired() { + vb.t.Fatal("UpdateClientConnState was called after Close(), which breaks the balancer API") + } + return nil +} + +func (vb *verifyBalancer) ResolverError(err error) { + if vb.closed.HasFired() { + vb.t.Fatal("ResolverError was called after Close(), which breaks the balancer API") + } +} + +func (vb *verifyBalancer) UpdateSubConnState(sc balancer.SubConn, state balancer.SubConnState) { + if vb.closed.HasFired() { + vb.t.Fatal("UpdateSubConnState was called after Close(), which breaks the balancer API") + } +} + +func (vb *verifyBalancer) Close() { + vb.closed.Fire() +} + +func (vb *verifyBalancer) ExitIdle() { + if vb.closed.HasFired() { + vb.t.Fatal("ExitIdle was called after Close(), which breaks the balancer API") + } +} diff --git a/xds/internal/balancer/outlierdetection/callcounter.go b/xds/internal/balancer/outlierdetection/callcounter.go new file mode 100644 index 00000000000..2c4667d7d1c --- /dev/null +++ b/xds/internal/balancer/outlierdetection/callcounter.go @@ -0,0 +1,78 @@ +/* + * + * Copyright 2022 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package outlierdetection + +import ( + "sync/atomic" + "unsafe" +) + +type bucket struct { + numSuccesses int64 + numFailures int64 + requestVolume int64 // numSuccesses + numFailures, needed because this number will be used in interval timer algorithm +} + +func newCallCounter() *callCounter { + return &callCounter{ + activeBucket: unsafe.Pointer(&bucket{}), + inactiveBucket: &bucket{}, + } +} + +type callCounter struct { + // "The object contains two buckets, and each bucket has a number counting + // successes, and another counting failures." - A50 + + // activeBucket updates every time a call finishes (from picker passed to + // Client Conn), so protect pointer read with atomic load of unsafe.Pointer + // so picker does not have to grab a mutex per RPC, the critical path. + activeBucket unsafe.Pointer + inactiveBucket *bucket +} + +func (cc *callCounter) clear() { + atomic.StorePointer(&cc.activeBucket, unsafe.Pointer(&bucket{})) + cc.inactiveBucket = &bucket{} +} + +// "When the timer triggers, the inactive bucket is zeroed and swapped with the +// active bucket. Then the inactive bucket contains the number of successes and +// failures since the last time the timer triggered. Those numbers are used to +// evaluate the ejection criteria." - A50 +func (cc *callCounter) swap() { + ab := (*bucket)(atomic.LoadPointer(&cc.activeBucket)) + // Don't do it exactly like defined but the same logically, as picker reads + // ref to active bucket so instead of swapping the pointers (inducing race + // conditions where picker writes to inactive bucket which is being used for + // outlier detection algorithm, copy active bucket to new memory on heap, + // picker updates which race simply write to deprecated heap memory + // activeBucket used to point to). The other options is to do this as + // defined, swap the pointers and have atomic reads of the Inactive Bucket + // in interval timer algorithm, but I think this is cleaner in regards to + // dealing with picker race condition. See the wrappedPicker explanation for + // the write to activeBucket for a more in depth explanation. + cc.inactiveBucket = &bucket{ + numSuccesses: atomic.LoadInt64(&ab.numSuccesses), + numFailures: atomic.LoadInt64(&ab.numFailures), + requestVolume: atomic.LoadInt64(&ab.requestVolume), + } + atomic.StorePointer(&cc.activeBucket, unsafe.Pointer(&bucket{})) + // end result, same as in gRFC: the inactive bucket contains the number of + // successes and failures since the last time the timer triggered. +} diff --git a/xds/internal/balancer/outlierdetection/callcounter_test.go b/xds/internal/balancer/outlierdetection/callcounter_test.go new file mode 100644 index 00000000000..638e252e146 --- /dev/null +++ b/xds/internal/balancer/outlierdetection/callcounter_test.go @@ -0,0 +1,99 @@ +/* + * + * Copyright 2022 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package outlierdetection + +import ( + "sync/atomic" + "testing" + + "github.com/google/go-cmp/cmp" +) + +func (b1 *bucket) Equal(b2 *bucket) bool { + if b1 == nil && b2 == nil { + return true + } + if (b1 != nil) != (b2 != nil) { + return false + } + if b1.numSuccesses != b2.numSuccesses { + return false + } + if b1.numFailures != b2.numFailures { + return false + } + return b1.requestVolume == b2.requestVolume +} + +func (cc1 *callCounter) Equal(cc2 *callCounter) bool { + if cc1 == nil && cc2 == nil { + return true + } + if (cc1 != nil) != (cc2 != nil) { + return false + } + ab1 := (*bucket)(atomic.LoadPointer(&cc1.activeBucket)) + ab2 := (*bucket)(atomic.LoadPointer(&cc2.activeBucket)) + if !ab1.Equal(ab2) { + return false + } + return cc1.inactiveBucket.Equal(cc2.inactiveBucket) +} + +// TestClear tests that clear on the call counter clears (everything set to 0) +// the active and inactive buckets. +func (s) TestClear(t *testing.T) { + cc := newCallCounter() + ab := (*bucket)(atomic.LoadPointer(&cc.activeBucket)) + ab.numSuccesses = 1 + ab.numFailures = 2 + ab.requestVolume = 3 + cc.inactiveBucket.numSuccesses = 4 + cc.inactiveBucket.numFailures = 5 + cc.inactiveBucket.requestVolume = 9 + cc.clear() + // Both the active and inactive buckets should be cleared. + ccWant := newCallCounter() + if diff := cmp.Diff(cc, ccWant); diff != "" { + t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) + } +} + +// TestSwap tests that swap() on the callCounter successfully has the desired +// end result of inactive bucket containing the previous active buckets data, +// and the active bucket being cleared. +func (s) TestSwap(t *testing.T) { + cc := newCallCounter() + ab := (*bucket)(atomic.LoadPointer(&cc.activeBucket)) + ab.numSuccesses = 1 + ab.numFailures = 2 + ab.requestVolume = 3 + cc.inactiveBucket.numSuccesses = 4 + cc.inactiveBucket.numFailures = 5 + cc.inactiveBucket.requestVolume = 9 + + cc.swap() + // Inactive should pick up active's data, active should be cleared. + ccWant := newCallCounter() + ccWant.inactiveBucket.numSuccesses = 1 + ccWant.inactiveBucket.numFailures = 2 + ccWant.inactiveBucket.requestVolume = 3 + if diff := cmp.Diff(cc, ccWant); diff != "" { + t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) + } +} diff --git a/xds/internal/balancer/outlierdetection/subconn_wrapper.go b/xds/internal/balancer/outlierdetection/subconn_wrapper.go new file mode 100644 index 00000000000..aef343e7243 --- /dev/null +++ b/xds/internal/balancer/outlierdetection/subconn_wrapper.go @@ -0,0 +1,62 @@ +/* + * + * Copyright 2022 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package outlierdetection + +import ( + "unsafe" + + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/internal/buffer" + "google.golang.org/grpc/resolver" +) + +type subConnWrapper struct { + balancer.SubConn + + // "The subchannel wrappers created by the outlier_detection LB policy will + // hold a reference to its map entry in the LB policy, if that map entry + // exists." - A50 + obj unsafe.Pointer // *object + // These two pieces of state will reach eventual consistency due to sync in + // run(), and child will always have the correctly updated SubConnState. + latestState balancer.SubConnState + ejected bool + + scUpdateCh *buffer.Unbounded + + addresses []resolver.Address +} + +// eject(): "The wrapper will report a state update with the TRANSIENT_FAILURE +// state, and will stop passing along updates from the underlying subchannel." +func (scw *subConnWrapper) eject() { + scw.scUpdateCh.Put(&ejectedUpdate{ + scw: scw, + ejected: true, + }) +} + +// uneject(): "The wrapper will report a state update with the latest update +// from the underlying subchannel, and resume passing along updates from the +// underlying subchannel." +func (scw *subConnWrapper) uneject() { + scw.scUpdateCh.Put(&ejectedUpdate{ + scw: scw, + ejected: false, + }) +} From 32ea80ab29cde7e0041936482cb7beb3db930224 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Tue, 28 Jun 2022 18:05:29 -0400 Subject: [PATCH 02/22] e2e test and bug fix --- test/xds/xds_client_outlier_detection_test.go | 143 +++++++++++++++++- .../balancer/outlierdetection/balancer.go | 143 +++++------------- .../outlierdetection/balancer_test.go | 24 +-- 3 files changed, 179 insertions(+), 131 deletions(-) diff --git a/test/xds/xds_client_outlier_detection_test.go b/test/xds/xds_client_outlier_detection_test.go index d500de03c80..d304d35a31a 100644 --- a/test/xds/xds_client_outlier_detection_test.go +++ b/test/xds/xds_client_outlier_detection_test.go @@ -20,19 +20,31 @@ package xds_test import ( "context" + "errors" "fmt" + "strings" "testing" + "time" + v3clusterpb "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3" + v3endpointpb "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3" + v3listenerpb "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3" + v3routepb "github.com/envoyproxy/go-control-plane/envoy/config/route/v3" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/internal" "google.golang.org/grpc/internal/envconfig" + "google.golang.org/grpc/internal/stubserver" "google.golang.org/grpc/internal/testutils/xds/e2e" - testgrpc "google.golang.org/grpc/test/grpc_testing" testpb "google.golang.org/grpc/test/grpc_testing" + "google.golang.org/protobuf/types/known/durationpb" + "google.golang.org/protobuf/types/known/wrapperspb" ) +// TestOutlierDetection tests an xDS configured ClientConn with an Outlier +// Detection present in the system which is a logical no-op. An RPC should +// proceed as normal. func (s) TestOutlierDetection(t *testing.T) { oldOD := envconfig.XDSOutlierDetection envconfig.XDSOutlierDetection = true @@ -74,3 +86,132 @@ func (s) TestOutlierDetection(t *testing.T) { t.Fatalf("rpc EmptyCall() failed: %v", err) } } + +// defaultClientResourcesSpecifyingMultipleBackendsAndOutlierDetection returns +// xDS resources which correspond to multiple upstreams, corresponding different +// backends listening on different localhost:port combinations. The resources +// also configure an Outlier Detection Balancer set up with Failure Percentage +// Algorithm, which ejects endpoints based on failure rate. +func defaultClientResourcesSpecifyingMultipleBackendsAndOutlierDetection(params e2e.ResourceParams, ports []uint32) e2e.UpdateOptions { + routeConfigName := "route-" + params.DialTarget + clusterName := "cluster-" + params.DialTarget + endpointsName := "endpoints-" + params.DialTarget + return e2e.UpdateOptions{ + NodeID: params.NodeID, + Listeners: []*v3listenerpb.Listener{e2e.DefaultClientListener(params.DialTarget, routeConfigName)}, + Routes: []*v3routepb.RouteConfiguration{e2e.DefaultRouteConfig(routeConfigName, params.DialTarget, clusterName)}, + Clusters: []*v3clusterpb.Cluster{defaultClusterWithOutlierDetection(clusterName, endpointsName, params.SecLevel)}, + Endpoints: []*v3endpointpb.ClusterLoadAssignment{e2e.DefaultEndpoint(endpointsName, params.Host, ports)}, + } +} + +func defaultClusterWithOutlierDetection(clusterName, edsServiceName string, secLevel e2e.SecurityLevel) *v3clusterpb.Cluster { + cluster := e2e.DefaultCluster(clusterName, edsServiceName, secLevel) + cluster.OutlierDetection = &v3clusterpb.OutlierDetection{ + Interval: &durationpb.Duration{ + Nanos: 500000000, + }, + BaseEjectionTime: &durationpb.Duration{Seconds: 30}, + MaxEjectionTime: &durationpb.Duration{Seconds: 300}, + MaxEjectionPercent: &wrapperspb.UInt32Value{Value: 1}, + FailurePercentageThreshold: &wrapperspb.UInt32Value{Value: 50}, + EnforcingFailurePercentage: &wrapperspb.UInt32Value{Value: 100}, + FailurePercentageRequestVolume: &wrapperspb.UInt32Value{Value: 1}, + FailurePercentageMinimumHosts: &wrapperspb.UInt32Value{Value: 1}, + } + return cluster +} + +// TestOutlierDetectionWithOutlier tests the Outlier Detection Balancer e2e. It +// spins up three backends, one which consistently errors, and configures the +// ClientConn using xDS to connect to all three of those backends. The Outlier +// Detection Balancer should eject the connection to the backend which +// constantly errors, and thus RPC's should mainly go to backend 1 and 2. +func (s) TestOutlierDetectionWithOutlier(t *testing.T) { + oldOD := envconfig.XDSOutlierDetection + envconfig.XDSOutlierDetection = true + internal.RegisterOutlierDetectionBalancerForTesting() + defer func() { + envconfig.XDSOutlierDetection = oldOD + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + managementServer, nodeID, _, resolver, cleanup := e2e.SetupManagementServer(t) + defer cleanup() + + // counters for how many times backends got called + var count1, count2, count3 int + + // Working backend 1. + port1, cleanup1 := startTestService(t, &stubserver.StubServer{ + EmptyCallF: func(context.Context, *testpb.Empty) (*testpb.Empty, error) { + count1++ + return &testpb.Empty{}, nil + }, + Address: "localhost:0", + }) + defer cleanup1() + + // Working backend 2. + port2, cleanup2 := startTestService(t, &stubserver.StubServer{ + EmptyCallF: func(context.Context, *testpb.Empty) (*testpb.Empty, error) { + count2++ + return &testpb.Empty{}, nil + }, + Address: "localhost:0", + }) + defer cleanup2() + // Backend 3 that will always return an error and eventually ejected. + port3, cleanup3 := startTestService(t, &stubserver.StubServer{ + EmptyCallF: func(context.Context, *testpb.Empty) (*testpb.Empty, error) { + count3++ + return nil, errors.New("some error") + }, + Address: "localhost:0", + }) + defer cleanup3() + + const serviceName = "my-service-client-side-xds" + resources := defaultClientResourcesSpecifyingMultipleBackendsAndOutlierDetection(e2e.ResourceParams{ + DialTarget: serviceName, + NodeID: nodeID, + Host: "localhost", + SecLevel: e2e.SecurityLevelNone, + }, []uint32{port1, port2, port3}) + ctx, cancel := context.WithTimeout(context.Background(), time.Second*100) + defer cancel() + if err := managementServer.Update(ctx, resources); err != nil { + t.Fatal(err) + } + + cc, err := grpc.Dial(fmt.Sprintf("xds:///%s", serviceName), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(resolver)) + if err != nil { + t.Fatalf("failed to dial local test server: %v", err) + } + defer cc.Close() + + client := testgrpc.NewTestServiceClient(cc) + for i := 0; i < 2000; i++ { + // Can either error or not depending on the backend called. + if _, err := client.EmptyCall(ctx, &testpb.Empty{}, grpc.WaitForReady(true)); err != nil && !strings.Contains(err.Error(), "some error") { + t.Fatalf("rpc EmptyCall() failed: %v", err) + } + time.Sleep(time.Millisecond) + } + + // Backend 1 should've gotten more than 1/3rd of the load as backend 3 + // should get ejected, leaving only 1 and 2. + if count1 < 700 { + t.Fatalf("backend 1 should've gotten more than 1/3rd of the load") + } + // Backend 2 should've gotten more than 1/3rd of the load as backend 3 + // should get ejected, leaving only 1 and 2. + if count2 < 700 { + t.Fatalf("backend 2 should've gotten more than 1/3rd of the load") + } + // Backend 3 should've gotten less than 1/3rd of the load since it gets + // ejected. + if count3 > 650 { + t.Fatalf("backend 1 should've gotten more than 1/3rd of the load") + } +} diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 6c99f94e049..1331afed9e7 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -67,12 +67,11 @@ func init() { type bb struct{} func (bb) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Balancer { - am := resolver.NewAddressMap() b := &outlierDetectionBalancer{ cc: cc, bOpts: bOpts, closed: grpcsync.NewEvent(), - odAddrs: am, + odAddrs: make(map[string]*object), scWrappers: make(map[balancer.SubConn]*subConnWrapper), scUpdateCh: buffer.NewUnbounded(), pickerUpdateCh: buffer.NewUnbounded(), @@ -174,7 +173,7 @@ type outlierDetectionBalancer struct { // running the interval timer algorithm which uses odAddrs heavily. This // will cause undefined behavior for the interval timer algorithm. mu sync.Mutex - odAddrs *resolver.AddressMap + odAddrs map[string]*object odCfg *LBConfig scWrappers map[balancer.SubConn]*subConnWrapper timerStartTime time.Time @@ -218,14 +217,14 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt // When the outlier_detection LB policy receives an address update, it will // create a map entry for each subchannel address in the list, and remove // each map entry for a subchannel address not in the list. - addrs := make(map[resolver.Address]bool) + addrs := make(map[string]bool, len(s.ResolverState.Addresses)) for _, addr := range s.ResolverState.Addresses { - addrs[addr] = true - b.odAddrs.Set(addr, newObject()) + addrs[addr.Addr] = true + b.odAddrs[addr.Addr] = newObject() } - for _, addr := range b.odAddrs.Keys() { + for addr := range b.odAddrs { if !addrs[addr] { - b.odAddrs.Delete(addr) + delete(b.odAddrs, addr) } } @@ -235,7 +234,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt var interval time.Duration if b.timerStartTime.IsZero() { b.timerStartTime = time.Now() - for _, obj := range b.objects() { + for _, obj := range b.odAddrs { obj.callCounter.clear() } interval = b.odCfg.Interval @@ -433,13 +432,7 @@ func (b *outlierDetectionBalancer) NewSubConn(addrs []resolver.Address, opts bal if len(addrs) != 1 { return scw, nil } - - val, ok := b.odAddrs.Get(addrs[0]) - if !ok { - return scw, nil - } - - obj, ok := val.(*object) + obj, ok := b.odAddrs[addrs[0].Addr] if !ok { return scw, nil } @@ -468,16 +461,12 @@ func (b *outlierDetectionBalancer) RemoveSubConn(sc balancer.SubConn) { // appendIfPresent appends the scw to the address, if the address is present in // the Outlier Detection balancers address map. Returns nil if not present, and // the map entry if present. -func (b *outlierDetectionBalancer) appendIfPresent(addr resolver.Address, scw *subConnWrapper) *object { - val, ok := b.odAddrs.Get(addr) +func (b *outlierDetectionBalancer) appendIfPresent(addr string, scw *subConnWrapper) *object { + obj, ok := b.odAddrs[addr] if !ok { return nil } - obj, ok := val.(*object) - if !ok { - // shouldn't happen, logical no-op - return nil - } + obj.sws = append(obj.sws, scw) atomic.StorePointer(&scw.obj, unsafe.Pointer(obj)) return obj @@ -498,19 +487,6 @@ func (b *outlierDetectionBalancer) removeSubConnFromAddressesMapEntry(scw *subCo } } -// sameAddrForMap returns if two addresses are the same in regards to subchannel -// uniqueness/identity (i.e. what the addresses map is keyed on - address -// string, Server Name, and Attributes). -func sameAddrForMap(oldAddr resolver.Address, newAddr resolver.Address) bool { - if oldAddr.Addr != newAddr.Addr { - return false - } - if oldAddr.ServerName != newAddr.ServerName { - return false - } - return oldAddr.Attributes.Equal(newAddr.Attributes) -} - func (b *outlierDetectionBalancer) UpdateAddresses(sc balancer.SubConn, addrs []resolver.Address) { scw, ok := sc.(*subConnWrapper) if !ok { @@ -527,16 +503,15 @@ func (b *outlierDetectionBalancer) UpdateAddresses(sc balancer.SubConn, addrs [] // old address/new address). if len(scw.addresses) == 1 { if len(addrs) == 1 { // single address to single address - // If everything we care for in regards to address specificity for a - // list of SubConn's (Addr, ServerName, Attributes) is the same, - // then there is nothing to do past this point. - if sameAddrForMap(scw.addresses[0], addrs[0]) { + // If the updated address is the same, then there is nothing to do + // past this point. + if scw.addresses[0].Addr == addrs[0].Addr { return } // 1. Remove Subchannel from Addresses map entry if present in Addresses map. b.removeSubConnFromAddressesMapEntry(scw) // 2. Add Subchannel to Addresses map entry if new address present in map. - obj := b.appendIfPresent(addrs[0], scw) + obj := b.appendIfPresent(addrs[0].Addr, scw) // 3. Relay state with eject() recalculated (using the corresponding // map entry to see if it's currently ejected). if obj == nil { // uneject unconditionally because could have come from an ejected address @@ -562,7 +537,7 @@ func (b *outlierDetectionBalancer) UpdateAddresses(sc balancer.SubConn, addrs [] } else { if len(addrs) == 1 { // multiple addresses to single address // 1. Add Subchannel to Addresses map entry if new address present in map. - obj := b.appendIfPresent(addrs[0], scw) + obj := b.appendIfPresent(addrs[0].Addr, scw) if obj != nil && !obj.latestEjectionTimestamp.IsZero() { scw.eject() } @@ -580,23 +555,6 @@ func (b *outlierDetectionBalancer) Target() string { return b.cc.Target() } -// objects returns a list of objects corresponding to every address in the address map. -func (b *outlierDetectionBalancer) objects() []*object { - var objs []*object - for _, addr := range b.odAddrs.Keys() { - val, ok := b.odAddrs.Get(addr) - if !ok { // Shouldn't happen - continue - } - obj, ok := val.(*object) - if !ok { - continue - } - objs = append(objs, obj) - } - return objs -} - func max(x, y int64) int64 { if x < y { return y @@ -618,7 +576,7 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { // 2. For each address, swap the call counter's buckets in that address's // map entry. - for _, obj := range b.objects() { + for _, obj := range b.odAddrs { obj.callCounter.swap() } @@ -635,15 +593,7 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { } // 5. For each address in the map: - for _, addr := range b.odAddrs.Keys() { - val, ok := b.odAddrs.Get(addr) - if !ok { - continue - } - obj, ok := val.(*object) - if !ok { - continue - } + for addr, obj := range b.odAddrs { // If the address is not ejected and the multiplier is greater than 0, // decrease the multiplier by 1. if obj.latestEjectionTimestamp.IsZero() && obj.ejectionTimeMultiplier > 0 { @@ -658,6 +608,7 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { b.unejectAddress(addr) } } + // This conditional only for testing (since the interval timer algorithm is // called manually), will never hit in production. if b.intervalTimer != nil { @@ -761,7 +712,7 @@ func (b *outlierDetectionBalancer) run() { // the map that have request volume of at least requestVolume. func (b *outlierDetectionBalancer) numAddrsWithAtLeastRequestVolume() uint32 { var numAddrs uint32 - for _, obj := range b.objects() { + for _, obj := range b.odAddrs { if uint32(obj.callCounter.inactiveBucket.requestVolume) >= b.odCfg.SuccessRateEjection.RequestVolume { numAddrs++ } @@ -777,21 +728,20 @@ func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume( // success_rate_ejection.request_volume. var totalFractionOfSuccessfulRequests float64 var mean float64 - - for _, obj := range b.objects() { + for _, obj := range b.odAddrs { // "of at least success_rate_ejection.request_volume" if uint32(obj.callCounter.inactiveBucket.requestVolume) >= b.odCfg.SuccessRateEjection.RequestVolume { totalFractionOfSuccessfulRequests += float64(obj.callCounter.inactiveBucket.numSuccesses) / float64(obj.callCounter.inactiveBucket.requestVolume) } } - mean = totalFractionOfSuccessfulRequests / float64(b.odAddrs.Len()) + mean = totalFractionOfSuccessfulRequests / float64(len(b.odAddrs)) var sumOfSquares float64 - for _, obj := range b.objects() { + for _, obj := range b.odAddrs { devFromMean := (float64(obj.callCounter.inactiveBucket.numSuccesses) / float64(obj.callCounter.inactiveBucket.requestVolume)) - mean sumOfSquares += devFromMean * devFromMean } - variance := sumOfSquares / float64(b.odAddrs.Len()) + variance := sumOfSquares / float64(len(b.odAddrs)) return mean, math.Sqrt(variance) } @@ -810,20 +760,12 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { mean, stddev := b.meanAndStdDevOfSuccessesAtLeastRequestVolume() // 3. For each address: - for _, addr := range b.odAddrs.Keys() { - val, ok := b.odAddrs.Get(addr) - if !ok { - continue - } - obj, ok := val.(*object) - if !ok { - continue - } + for addr, obj := range b.odAddrs { ccb := obj.callCounter.inactiveBucket sre := b.odCfg.SuccessRateEjection // i. If the percentage of ejected addresses is greater than // max_ejection_percent, stop. - if float64(b.numAddrsEjected)/float64(b.odAddrs.Len())*100 > float64(b.odCfg.MaxEjectionPercent) { + if float64(b.numAddrsEjected)/float64(len(b.odAddrs))*100 > float64(b.odCfg.MaxEjectionPercent) { return } @@ -850,25 +792,17 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { // 1. If the number of addresses is less than // failure_percentage_ejection.minimum_hosts, stop. - if uint32(b.odAddrs.Len()) < b.odCfg.FailurePercentageEjection.MinimumHosts { + if uint32(len(b.odAddrs)) < b.odCfg.FailurePercentageEjection.MinimumHosts { return } // 2. For each address: - for _, addr := range b.odAddrs.Keys() { - val, ok := b.odAddrs.Get(addr) - if !ok { - continue - } - obj, ok := val.(*object) - if !ok { - continue - } + for addr, obj := range b.odAddrs { ccb := obj.callCounter.inactiveBucket fpe := b.odCfg.FailurePercentageEjection // i. If the percentage of ejected addresses is greater than // max_ejection_percent, stop. - if float64(b.numAddrsEjected)/float64(b.odAddrs.Len())*100 > float64(b.odCfg.MaxEjectionPercent) { + if float64(b.numAddrsEjected)/float64(len(b.odAddrs))*100 > float64(b.odCfg.MaxEjectionPercent) { return } // ii. If the address's total request volume is less than @@ -891,16 +825,11 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { } } -func (b *outlierDetectionBalancer) ejectAddress(addr resolver.Address) { - val, ok := b.odAddrs.Get(addr) - if !ok { // Shouldn't happen - return - } - obj, ok := val.(*object) +func (b *outlierDetectionBalancer) ejectAddress(addr string) { + obj, ok := b.odAddrs[addr] if !ok { // Shouldn't happen return } - b.numAddrsEjected++ // To eject an address, set the current ejection timestamp to the timestamp @@ -914,12 +843,8 @@ func (b *outlierDetectionBalancer) ejectAddress(addr resolver.Address) { } } -func (b *outlierDetectionBalancer) unejectAddress(addr resolver.Address) { - val, ok := b.odAddrs.Get(addr) - if !ok { // Shouldn't happen - return - } - obj, ok := val.(*object) +func (b *outlierDetectionBalancer) unejectAddress(addr string) { + obj, ok := b.odAddrs[addr] if !ok { // Shouldn't happen return } diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index 16ae32cdc83..b84831eecad 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -969,16 +969,10 @@ func (s) TestPicker(t *testing.T) { pi.Done(balancer.DoneInfo{}) pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) od.mu.Lock() - val, ok := od.odAddrs.Get(resolver.Address{ - Addr: "address1", - }) + obj, ok := od.odAddrs["address1"] if !ok { t.Fatal("map entry for address: address1 not present in map") } - obj, ok := val.(*object) - if !ok { - t.Fatal("map value isn't obj type") - } bucketWant := &bucket{ numSuccesses: 1, numFailures: 1, @@ -1045,16 +1039,10 @@ func (s) TestPicker(t *testing.T) { pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) od.mu.Lock() - val, ok := od.odAddrs.Get(resolver.Address{ - Addr: "address1", - }) + obj, ok := od.odAddrs["address1"] if !ok { t.Fatal("map entry for address: address1 not present in map") } - obj, ok := val.(*object) - if !ok { - t.Fatal("map value isn't obj type") - } // The active bucket should be cleared because the interval timer // algorithm didn't run in between ClientConn updates and the picker @@ -1100,16 +1088,10 @@ func (s) TestPicker(t *testing.T) { pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) od.mu.Lock() - val, ok := od.odAddrs.Get(resolver.Address{ - Addr: "address1", - }) + obj, ok := od.odAddrs["address1"] if !ok { t.Fatal("map entry for address: address1 not present in map") } - obj, ok := val.(*object) - if !ok { - t.Fatal("map value isn't obj type") - } // The active bucket should be cleared because the interval timer // algorithm didn't run in between ClientConn updates and the picker From 73528ce65faa883f78528a0b3fb86dd7cb65bfb3 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Wed, 6 Jul 2022 19:46:56 -0400 Subject: [PATCH 03/22] Responded to Easwar's comments --- test/xds/xds_client_outlier_detection_test.go | 33 ++-- .../balancer/outlierdetection/balancer.go | 163 +++++++++--------- .../outlierdetection/balancer_test.go | 8 +- .../balancer/outlierdetection/callcounter.go | 7 +- .../outlierdetection/subconn_wrapper.go | 20 ++- 5 files changed, 122 insertions(+), 109 deletions(-) diff --git a/test/xds/xds_client_outlier_detection_test.go b/test/xds/xds_client_outlier_detection_test.go index d304d35a31a..56660ae9e78 100644 --- a/test/xds/xds_client_outlier_detection_test.go +++ b/test/xds/xds_client_outlier_detection_test.go @@ -42,10 +42,13 @@ import ( "google.golang.org/protobuf/types/known/wrapperspb" ) -// TestOutlierDetection tests an xDS configured ClientConn with an Outlier -// Detection present in the system which is a logical no-op. An RPC should -// proceed as normal. -func (s) TestOutlierDetection(t *testing.T) { +// TestOutlierDetection_NoopConfig tests the scenario where the Outlier +// Detection feature is enabled on the gRPC client, but it receives no Outlier +// Detection configuration from the management server. This should result in a +// no-op Outlier Detection configuration being used to configure the Outlier +// Detection balancer. This test verifies that an RPC is able to proceed +// normally with this configuration. +func (s) TestOutlierDetection_NoopConfig(t *testing.T) { oldOD := envconfig.XDSOutlierDetection envconfig.XDSOutlierDetection = true internal.RegisterOutlierDetectionBalancerForTesting() @@ -54,7 +57,7 @@ func (s) TestOutlierDetection(t *testing.T) { internal.UnregisterOutlierDetectionBalancerForTesting() }() - managementServer, nodeID, _, resolver, cleanup1 := e2e.SetupManagementServer(t) + managementServer, nodeID, _, resolver, cleanup1 := e2e.SetupManagementServer(t, nil) defer cleanup1() port, cleanup2 := startTestService(t, nil) @@ -87,12 +90,12 @@ func (s) TestOutlierDetection(t *testing.T) { } } -// defaultClientResourcesSpecifyingMultipleBackendsAndOutlierDetection returns -// xDS resources which correspond to multiple upstreams, corresponding different -// backends listening on different localhost:port combinations. The resources -// also configure an Outlier Detection Balancer set up with Failure Percentage -// Algorithm, which ejects endpoints based on failure rate. -func defaultClientResourcesSpecifyingMultipleBackendsAndOutlierDetection(params e2e.ResourceParams, ports []uint32) e2e.UpdateOptions { +// defaultClientResourcesMultipleBackendsAndOD returns xDS resources which +// correspond to multiple upstreams, corresponding different backends listening +// on different localhost:port combinations. The resources also configure an +// Outlier Detection Balancer set up with Failure Percentage Algorithm, which +// ejects endpoints based on failure rate. +func defaultClientResourcesMultipleBackendsAndOD(params e2e.ResourceParams, ports []uint32) e2e.UpdateOptions { routeConfigName := "route-" + params.DialTarget clusterName := "cluster-" + params.DialTarget endpointsName := "endpoints-" + params.DialTarget @@ -108,9 +111,7 @@ func defaultClientResourcesSpecifyingMultipleBackendsAndOutlierDetection(params func defaultClusterWithOutlierDetection(clusterName, edsServiceName string, secLevel e2e.SecurityLevel) *v3clusterpb.Cluster { cluster := e2e.DefaultCluster(clusterName, edsServiceName, secLevel) cluster.OutlierDetection = &v3clusterpb.OutlierDetection{ - Interval: &durationpb.Duration{ - Nanos: 500000000, - }, + Interval: &durationpb.Duration{Nanos: 50000000}, // .5 seconds BaseEjectionTime: &durationpb.Duration{Seconds: 30}, MaxEjectionTime: &durationpb.Duration{Seconds: 300}, MaxEjectionPercent: &wrapperspb.UInt32Value{Value: 1}, @@ -136,7 +137,7 @@ func (s) TestOutlierDetectionWithOutlier(t *testing.T) { internal.UnregisterOutlierDetectionBalancerForTesting() }() - managementServer, nodeID, _, resolver, cleanup := e2e.SetupManagementServer(t) + managementServer, nodeID, _, resolver, cleanup := e2e.SetupManagementServer(t, nil) defer cleanup() // counters for how many times backends got called @@ -172,7 +173,7 @@ func (s) TestOutlierDetectionWithOutlier(t *testing.T) { defer cleanup3() const serviceName = "my-service-client-side-xds" - resources := defaultClientResourcesSpecifyingMultipleBackendsAndOutlierDetection(e2e.ResourceParams{ + resources := defaultClientResourcesMultipleBackendsAndOD(e2e.ResourceParams{ DialTarget: serviceName, NodeID: nodeID, Host: "localhost", diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 1331afed9e7..b33719d235f 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -71,7 +71,7 @@ func (bb) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Ba cc: cc, bOpts: bOpts, closed: grpcsync.NewEvent(), - odAddrs: make(map[string]*object), + addrs: make(map[string]*addressInfo), scWrappers: make(map[balancer.SubConn]*subConnWrapper), scUpdateCh: buffer.NewUnbounded(), pickerUpdateCh: buffer.NewUnbounded(), @@ -165,16 +165,16 @@ type outlierDetectionBalancer struct { // intersplicing certain operations. // // ex 1: interval timer goes off, outlier detection algorithm starts running - // based on knobs in odCfg. in the middle of running the algorithm, a - // ClientConn update comes in and writes to odCfg. This causes undefined + // based on knobs in cfg. in the middle of running the algorithm, a + // ClientConn update comes in and writes to cfg. This causes undefined // behavior for the interval timer algorithm. // - // ex 2: Updating the odAddrs map from UpdateAddresses in the middle of - // running the interval timer algorithm which uses odAddrs heavily. This - // will cause undefined behavior for the interval timer algorithm. + // ex 2: Updating the addrs map from UpdateAddresses in the middle of + // running the interval timer algorithm which uses addrs heavily. This will + // cause undefined behavior for the interval timer algorithm. mu sync.Mutex - odAddrs map[string]*object - odCfg *LBConfig + addrs map[string]*addressInfo + cfg *LBConfig scWrappers map[balancer.SubConn]*subConnWrapper timerStartTime time.Time intervalTimer *time.Timer @@ -186,7 +186,7 @@ type outlierDetectionBalancer struct { // noopConfig returns whether this balancer is configured with a logical no-op // configuration or not. func (b *outlierDetectionBalancer) noopConfig() bool { - return b.odCfg.SuccessRateEjection == nil && b.odCfg.FailurePercentageEjection == nil + return b.cfg.SuccessRateEjection == nil && b.cfg.FailurePercentageEjection == nil } func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnState) error { @@ -201,7 +201,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt return fmt.Errorf("balancer %q not registered", lbCfg.ChildPolicy.Name) } - if b.child == nil || b.odCfg.ChildPolicy.Name != lbCfg.ChildPolicy.Name { + if b.child == nil || b.cfg.ChildPolicy.Name != lbCfg.ChildPolicy.Name { b.childMu.Lock() if b.child != nil { b.child.Close() @@ -212,7 +212,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt } b.mu.Lock() - b.odCfg = lbCfg + b.cfg = lbCfg // When the outlier_detection LB policy receives an address update, it will // create a map entry for each subchannel address in the list, and remove @@ -220,11 +220,11 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt addrs := make(map[string]bool, len(s.ResolverState.Addresses)) for _, addr := range s.ResolverState.Addresses { addrs[addr.Addr] = true - b.odAddrs[addr.Addr] = newObject() + b.addrs[addr.Addr] = newObject() } - for addr := range b.odAddrs { + for addr := range b.addrs { if !addrs[addr] { - delete(b.odAddrs, addr) + delete(b.addrs, addr) } } @@ -234,16 +234,16 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt var interval time.Duration if b.timerStartTime.IsZero() { b.timerStartTime = time.Now() - for _, obj := range b.odAddrs { + for _, obj := range b.addrs { obj.callCounter.clear() } - interval = b.odCfg.Interval + interval = b.cfg.Interval } else { // If the timer start timestamp is set, instead cancel the existing // timer and start the timer for the configured interval minus the // difference between the current time and the previous start timestamp, // or 0 if that would be negative. - interval = b.odCfg.Interval - (now().Sub(b.timerStartTime)) + interval = b.cfg.Interval - (now().Sub(b.timerStartTime)) if interval < 0 { interval = 0 } @@ -274,7 +274,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt defer b.childMu.Unlock() return b.child.UpdateClientConnState(balancer.ClientConnState{ ResolverState: s.ResolverState, - BalancerConfig: b.odCfg.ChildPolicy.Config, + BalancerConfig: b.cfg.ChildPolicy.Config, }) } @@ -337,10 +337,9 @@ func (b *outlierDetectionBalancer) ExitIdle() { // in ClusterImpl. I guess we should do that here? } -// "The outlier_detection LB policy will provide a picker that delegates to the -// child policy's picker, and when the request finishes, increment the -// corresponding counter in the map entry referenced by the subchannel wrapper -// that was picked." - A50 +// wrappedPicker delegates to the child policy's picker, and when the request +// finishes, it increments the corresponding counter in the map entry referenced +// by the subConnWrapper that was picked. type wrappedPicker struct { childPicker balancer.Picker noopPicker bool @@ -381,21 +380,21 @@ func incrementCounter(sc balancer.SubConn, info balancer.DoneInfo) { return } - // scw.obj and callCounter.activeBucket can be written to concurrently (the - // pointers themselves). Thus, protect the reads here with atomics to - // prevent data corruption. There exists a race in which you read the object - // or active bucket pointer and then that pointer points to deprecated - // memory. If this goroutine yields the processor, in between reading the - // object pointer and writing to the active bucket, UpdateAddresses can - // switch the obj the scw points to. Writing to an outdated addresses is a - // very small race and tolerable. After reading callCounter.activeBucket in - // this picker a swap call can concurrently change what activeBucket points - // to. A50 says to swap the pointer, but I decided to make create new memory - // for both active and inactive bucket, and have this race instead write to - // deprecated memory. If you swap the pointers, this write would write to - // the inactive buckets memory, which is read throughout in the interval - // timers algorithm. - obj := (*object)(atomic.LoadPointer(&scw.obj)) + // scw.addressInfo and callCounter.activeBucket can be written to + // concurrently (the pointers themselves). Thus, protect the reads here with + // atomics to prevent data corruption. There exists a race in which you read + // the addressInfo or active bucket pointer and then that pointer points to + // deprecated memory. If this goroutine yields the processor, in between + // reading the addressInfo pointer and writing to the active bucket, + // UpdateAddresses can switch the addressInfo the scw points to. Writing to + // an outdated addresses is a very small race and tolerable. After reading + // callCounter.activeBucket in this picker a swap call can concurrently + // change what activeBucket points to. A50 says to swap the pointer, but I + // decided to make create new memory for both active and inactive bucket, + // and have this race instead write to deprecated memory. If you swap the + // pointers, this write would write to the inactive buckets memory, which is + // read throughout in the interval timers algorithm. + obj := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) if obj == nil { return } @@ -432,12 +431,12 @@ func (b *outlierDetectionBalancer) NewSubConn(addrs []resolver.Address, opts bal if len(addrs) != 1 { return scw, nil } - obj, ok := b.odAddrs[addrs[0].Addr] + obj, ok := b.addrs[addrs[0].Addr] if !ok { return scw, nil } obj.sws = append(obj.sws, scw) - atomic.StorePointer(&scw.obj, unsafe.Pointer(obj)) + atomic.StorePointer(&scw.addressInfo, unsafe.Pointer(obj)) // "If that address is currently ejected, that subchannel wrapper's eject // method will be called." - A50 @@ -461,21 +460,21 @@ func (b *outlierDetectionBalancer) RemoveSubConn(sc balancer.SubConn) { // appendIfPresent appends the scw to the address, if the address is present in // the Outlier Detection balancers address map. Returns nil if not present, and // the map entry if present. -func (b *outlierDetectionBalancer) appendIfPresent(addr string, scw *subConnWrapper) *object { - obj, ok := b.odAddrs[addr] +func (b *outlierDetectionBalancer) appendIfPresent(addr string, scw *subConnWrapper) *addressInfo { + obj, ok := b.addrs[addr] if !ok { return nil } obj.sws = append(obj.sws, scw) - atomic.StorePointer(&scw.obj, unsafe.Pointer(obj)) + atomic.StorePointer(&scw.addressInfo, unsafe.Pointer(obj)) return obj } // removeSubConnFromAddressesMapEntry removes the scw from it's map entry if // present. func (b *outlierDetectionBalancer) removeSubConnFromAddressesMapEntry(scw *subConnWrapper) { - obj := (*object)(atomic.LoadPointer(&scw.obj)) + obj := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) if obj == nil { return } @@ -527,7 +526,7 @@ func (b *outlierDetectionBalancer) UpdateAddresses(sc balancer.SubConn, addrs [] // 1. Remove Subchannel from Addresses map entry if present in Addresses map. b.removeSubConnFromAddressesMapEntry(scw) // 2. Clear the Subchannel wrapper's Call Counter entry. - obj := (*object)(atomic.LoadPointer(&scw.obj)) + obj := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) if obj != nil { obj.callCounter.clear() } @@ -576,24 +575,24 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { // 2. For each address, swap the call counter's buckets in that address's // map entry. - for _, obj := range b.odAddrs { + for _, obj := range b.addrs { obj.callCounter.swap() } // 3. If the success_rate_ejection configuration field is set, run the // success rate algorithm. - if b.odCfg.SuccessRateEjection != nil { + if b.cfg.SuccessRateEjection != nil { b.successRateAlgorithm() } // 4. If the failure_percentage_ejection configuration field is set, run the // failure percentage algorithm. - if b.odCfg.FailurePercentageEjection != nil { + if b.cfg.FailurePercentageEjection != nil { b.failurePercentageAlgorithm() } // 5. For each address in the map: - for addr, obj := range b.odAddrs { + for addr, obj := range b.addrs { // If the address is not ejected and the multiplier is greater than 0, // decrease the multiplier by 1. if obj.latestEjectionTimestamp.IsZero() && obj.ejectionTimeMultiplier > 0 { @@ -604,7 +603,7 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { // ejection_timestamp + min(base_ejection_time (type: time.Time) * // multiplier (type: int), max(base_ejection_time (type: time.Time), // max_ejection_time (type: time.Time))), un-eject the address. - if !obj.latestEjectionTimestamp.IsZero() && now().After(obj.latestEjectionTimestamp.Add(time.Duration(min(b.odCfg.BaseEjectionTime.Nanoseconds()*obj.ejectionTimeMultiplier, max(b.odCfg.BaseEjectionTime.Nanoseconds(), b.odCfg.MaxEjectionTime.Nanoseconds()))))) { // need to way to inject a desired bool here at a certain point in tests, mock time.Now to return a late time, mock time.After to always return true... + if !obj.latestEjectionTimestamp.IsZero() && now().After(obj.latestEjectionTimestamp.Add(time.Duration(min(b.cfg.BaseEjectionTime.Nanoseconds()*obj.ejectionTimeMultiplier, max(b.cfg.BaseEjectionTime.Nanoseconds(), b.cfg.MaxEjectionTime.Nanoseconds()))))) { b.unejectAddress(addr) } } @@ -614,7 +613,7 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { if b.intervalTimer != nil { b.intervalTimer.Stop() } - b.intervalTimer = afterFunc(b.odCfg.Interval, func() { + b.intervalTimer = afterFunc(b.cfg.Interval, func() { b.intervalTimerAlgorithm() }) } @@ -712,8 +711,8 @@ func (b *outlierDetectionBalancer) run() { // the map that have request volume of at least requestVolume. func (b *outlierDetectionBalancer) numAddrsWithAtLeastRequestVolume() uint32 { var numAddrs uint32 - for _, obj := range b.odAddrs { - if uint32(obj.callCounter.inactiveBucket.requestVolume) >= b.odCfg.SuccessRateEjection.RequestVolume { + for _, obj := range b.addrs { + if uint32(obj.callCounter.inactiveBucket.requestVolume) >= b.cfg.SuccessRateEjection.RequestVolume { numAddrs++ } } @@ -728,20 +727,20 @@ func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume( // success_rate_ejection.request_volume. var totalFractionOfSuccessfulRequests float64 var mean float64 - for _, obj := range b.odAddrs { + for _, obj := range b.addrs { // "of at least success_rate_ejection.request_volume" - if uint32(obj.callCounter.inactiveBucket.requestVolume) >= b.odCfg.SuccessRateEjection.RequestVolume { + if uint32(obj.callCounter.inactiveBucket.requestVolume) >= b.cfg.SuccessRateEjection.RequestVolume { totalFractionOfSuccessfulRequests += float64(obj.callCounter.inactiveBucket.numSuccesses) / float64(obj.callCounter.inactiveBucket.requestVolume) } } - mean = totalFractionOfSuccessfulRequests / float64(len(b.odAddrs)) + mean = totalFractionOfSuccessfulRequests / float64(len(b.addrs)) var sumOfSquares float64 - for _, obj := range b.odAddrs { + for _, obj := range b.addrs { devFromMean := (float64(obj.callCounter.inactiveBucket.numSuccesses) / float64(obj.callCounter.inactiveBucket.requestVolume)) - mean sumOfSquares += devFromMean * devFromMean } - variance := sumOfSquares / float64(len(b.odAddrs)) + variance := sumOfSquares / float64(len(b.addrs)) return mean, math.Sqrt(variance) } @@ -750,7 +749,7 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { // 1. If the number of addresses with request volume of at least // success_rate_ejection.request_volume is less than // success_rate_ejection.minimum_hosts, stop. - if b.numAddrsWithAtLeastRequestVolume() < b.odCfg.SuccessRateEjection.MinimumHosts { // TODO: O(n) search, is there a way to optimize this? + if b.numAddrsWithAtLeastRequestVolume() < b.cfg.SuccessRateEjection.MinimumHosts { // TODO: O(n) search, is there a way to optimize this? return } @@ -760,12 +759,12 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { mean, stddev := b.meanAndStdDevOfSuccessesAtLeastRequestVolume() // 3. For each address: - for addr, obj := range b.odAddrs { + for addr, obj := range b.addrs { ccb := obj.callCounter.inactiveBucket - sre := b.odCfg.SuccessRateEjection + sre := b.cfg.SuccessRateEjection // i. If the percentage of ejected addresses is greater than // max_ejection_percent, stop. - if float64(b.numAddrsEjected)/float64(len(b.odAddrs))*100 > float64(b.odCfg.MaxEjectionPercent) { + if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { return } @@ -792,17 +791,17 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { // 1. If the number of addresses is less than // failure_percentage_ejection.minimum_hosts, stop. - if uint32(len(b.odAddrs)) < b.odCfg.FailurePercentageEjection.MinimumHosts { + if uint32(len(b.addrs)) < b.cfg.FailurePercentageEjection.MinimumHosts { return } // 2. For each address: - for addr, obj := range b.odAddrs { + for addr, obj := range b.addrs { ccb := obj.callCounter.inactiveBucket - fpe := b.odCfg.FailurePercentageEjection + fpe := b.cfg.FailurePercentageEjection // i. If the percentage of ejected addresses is greater than // max_ejection_percent, stop. - if float64(b.numAddrsEjected)/float64(len(b.odAddrs))*100 > float64(b.odCfg.MaxEjectionPercent) { + if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { return } // ii. If the address's total request volume is less than @@ -814,11 +813,11 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { // 2c. If the address's failure percentage is greater than // failure_percentage_ejection.threshold failurePercentage := (float64(ccb.numFailures) / float64(ccb.requestVolume)) * 100 - if failurePercentage > float64(b.odCfg.FailurePercentageEjection.Threshold) { + if failurePercentage > float64(b.cfg.FailurePercentageEjection.Threshold) { // then choose a random integer in [0, 100). If that number is less // than failiure_percentage_ejection.enforcement_percentage, eject // that address. - if uint32(grpcrand.Int31n(100)) < b.odCfg.FailurePercentageEjection.EnforcementPercentage { + if uint32(grpcrand.Int31n(100)) < b.cfg.FailurePercentageEjection.EnforcementPercentage { b.ejectAddress(addr) } } @@ -826,7 +825,7 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { } func (b *outlierDetectionBalancer) ejectAddress(addr string) { - obj, ok := b.odAddrs[addr] + obj, ok := b.addrs[addr] if !ok { // Shouldn't happen return } @@ -844,7 +843,7 @@ func (b *outlierDetectionBalancer) ejectAddress(addr string) { } func (b *outlierDetectionBalancer) unejectAddress(addr string) { - obj, ok := b.odAddrs[addr] + obj, ok := b.addrs[addr] if !ok { // Shouldn't happen return } @@ -861,23 +860,31 @@ func (b *outlierDetectionBalancer) unejectAddress(addr string) { } } -type object struct { - // The call result counter object +// addressInfo contains the runtime information about an address that pertains +// to Outlier Detection, including the counter for successful/failing RPC's, and +// also information about whether the addresses has been ejected, and the +// SubConns that are present that use this address. This struct and all of it's +// fields is protected by outlierDetectionBalancer.mu in the case where it +// accessed through the address map. In the case of Picker callbacks, the writes +// to the activeBucket of callCounter are protected by atomically loading and +// storing unsafe.Pointers (see further explanation in incrementCounter()). +type addressInfo struct { + // The call result counter object. callCounter *callCounter - // The latest ejection timestamp, or null if the address is currently not - // ejected - latestEjectionTimestamp time.Time // We represent the branching logic on the null with a time.Zero() value + // The latest ejection timestamp, or zero if the address is currently not + // ejected. + latestEjectionTimestamp time.Time - // The current ejection time multiplier, starting at 0 + // The current ejection time multiplier, starting at 0. ejectionTimeMultiplier int64 - // A list of subchannel wrapper objects that correspond to this address + // A list of subchannel wrapper objects that correspond to this address. sws []*subConnWrapper } -func newObject() *object { - return &object{ +func newObject() *addressInfo { + return &addressInfo{ callCounter: newCallCounter(), sws: make([]*subConnWrapper, 0), } diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index b84831eecad..181b3946ab7 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -399,7 +399,7 @@ func (tb *testClusterImplBalancer) waitForSubConnUpdate(ctx context.Context, wan return err } gotSCS := scs.(subConnWithState) - if !cmp.Equal(gotSCS, wantSCS, cmp.AllowUnexported(subConnWithState{}, testutils.TestSubConn{}, subConnWrapper{}, object{}), cmpopts.IgnoreFields(subConnWrapper{}, "scUpdateCh")) { + if !cmp.Equal(gotSCS, wantSCS, cmp.AllowUnexported(subConnWithState{}, testutils.TestSubConn{}, subConnWrapper{}, addressInfo{}), cmpopts.IgnoreFields(subConnWrapper{}, "scUpdateCh")) { return fmt.Errorf("received SubConnState: %+v, want %+v", gotSCS, wantSCS) } return nil @@ -969,7 +969,7 @@ func (s) TestPicker(t *testing.T) { pi.Done(balancer.DoneInfo{}) pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) od.mu.Lock() - obj, ok := od.odAddrs["address1"] + obj, ok := od.addrs["address1"] if !ok { t.Fatal("map entry for address: address1 not present in map") } @@ -1039,7 +1039,7 @@ func (s) TestPicker(t *testing.T) { pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) od.mu.Lock() - obj, ok := od.odAddrs["address1"] + obj, ok := od.addrs["address1"] if !ok { t.Fatal("map entry for address: address1 not present in map") } @@ -1088,7 +1088,7 @@ func (s) TestPicker(t *testing.T) { pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) od.mu.Lock() - obj, ok := od.odAddrs["address1"] + obj, ok := od.addrs["address1"] if !ok { t.Fatal("map entry for address: address1 not present in map") } diff --git a/xds/internal/balancer/outlierdetection/callcounter.go b/xds/internal/balancer/outlierdetection/callcounter.go index 2c4667d7d1c..7de43792cd7 100644 --- a/xds/internal/balancer/outlierdetection/callcounter.go +++ b/xds/internal/balancer/outlierdetection/callcounter.go @@ -35,10 +35,11 @@ func newCallCounter() *callCounter { } } +// callCounter has two buckets, which each count successes and failures. The +// activeBucket is used to actively count any finished RPC's, and the +// inactiveBucket is populated with this activeBucket's data every interval for +// use by the Outlier Detection algorithm. type callCounter struct { - // "The object contains two buckets, and each bucket has a number counting - // successes, and another counting failures." - A50 - // activeBucket updates every time a call finishes (from picker passed to // Client Conn), so protect pointer read with atomic load of unsafe.Pointer // so picker does not have to grab a mutex per RPC, the critical path. diff --git a/xds/internal/balancer/outlierdetection/subconn_wrapper.go b/xds/internal/balancer/outlierdetection/subconn_wrapper.go index aef343e7243..f2ecbae605e 100644 --- a/xds/internal/balancer/outlierdetection/subconn_wrapper.go +++ b/xds/internal/balancer/outlierdetection/subconn_wrapper.go @@ -25,13 +25,17 @@ import ( "google.golang.org/grpc/resolver" ) +// subConnWrapper wraps every created SubConn in the Outlier Detection Balancer. +// It is used to store whether the SubConn has been ejected or not, and also to +// store the latest state for use when the SubConn gets unejected. It also +// stores the addresses the SubConn was created with to support any change in +// address(es). type subConnWrapper struct { balancer.SubConn - // "The subchannel wrappers created by the outlier_detection LB policy will - // hold a reference to its map entry in the LB policy, if that map entry - // exists." - A50 - obj unsafe.Pointer // *object + // addressInfo is a pointer to the subConnWrapper's corresponding address + // map entry, if the map entry exists. + addressInfo unsafe.Pointer // *addressInfo // These two pieces of state will reach eventual consistency due to sync in // run(), and child will always have the correctly updated SubConnState. latestState balancer.SubConnState @@ -42,8 +46,8 @@ type subConnWrapper struct { addresses []resolver.Address } -// eject(): "The wrapper will report a state update with the TRANSIENT_FAILURE -// state, and will stop passing along updates from the underlying subchannel." +// eject causes the wrapper to report a state update with the TRANSIENT_FAILURE +// state, and to stop passing along updates from the underlying subchannel. func (scw *subConnWrapper) eject() { scw.scUpdateCh.Put(&ejectedUpdate{ scw: scw, @@ -51,9 +55,9 @@ func (scw *subConnWrapper) eject() { }) } -// uneject(): "The wrapper will report a state update with the latest update +// uneject causes the wrapper to report a state update with the latest update // from the underlying subchannel, and resume passing along updates from the -// underlying subchannel." +// underlying subchannel. func (scw *subConnWrapper) uneject() { scw.scUpdateCh.Put(&ejectedUpdate{ scw: scw, From e580349f485e54149ae2f43d6da6d09eacaf12ed Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Tue, 12 Jul 2022 16:00:13 -0400 Subject: [PATCH 04/22] Responded to Doug's comments --- .../balancer/outlierdetection/balancer.go | 395 ++++++++++-------- .../outlierdetection/balancer_test.go | 12 +- 2 files changed, 216 insertions(+), 191 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index b33719d235f..7718de02b0d 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -198,7 +198,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt // Reject whole config if any errors, don't persist it for later bb := balancer.Get(lbCfg.ChildPolicy.Name) if bb == nil { - return fmt.Errorf("balancer %q not registered", lbCfg.ChildPolicy.Name) + return fmt.Errorf("outlier detection: child balancer %q not registered", lbCfg.ChildPolicy.Name) } if b.child == nil || b.cfg.ChildPolicy.Name != lbCfg.ChildPolicy.Name { @@ -206,7 +206,6 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt if b.child != nil { b.child.Close() } - // What if this is nil? Seems fine b.child = bb.Build(b, b.bOpts) b.childMu.Unlock() } @@ -220,7 +219,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt addrs := make(map[string]bool, len(s.ResolverState.Addresses)) for _, addr := range s.ResolverState.Addresses { addrs[addr.Addr] = true - b.addrs[addr.Addr] = newObject() + b.addrs[addr.Addr] = newAddressInfo() } for addr := range b.addrs { if !addrs[addr] { @@ -234,8 +233,8 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt var interval time.Duration if b.timerStartTime.IsZero() { b.timerStartTime = time.Now() - for _, obj := range b.addrs { - obj.callCounter.clear() + for _, addrInfo := range b.addrs { + addrInfo.callCounter.clear() } interval = b.cfg.Interval } else { @@ -394,11 +393,11 @@ func incrementCounter(sc balancer.SubConn, info balancer.DoneInfo) { // and have this race instead write to deprecated memory. If you swap the // pointers, this write would write to the inactive buckets memory, which is // read throughout in the interval timers algorithm. - obj := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) - if obj == nil { + addrInfo := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) + if addrInfo == nil { return } - ab := (*bucket)(atomic.LoadPointer(&obj.callCounter.activeBucket)) + ab := (*bucket)(atomic.LoadPointer(&addrInfo.callCounter.activeBucket)) if info.Err == nil { atomic.AddInt64(&ab.numSuccesses, 1) @@ -431,16 +430,16 @@ func (b *outlierDetectionBalancer) NewSubConn(addrs []resolver.Address, opts bal if len(addrs) != 1 { return scw, nil } - obj, ok := b.addrs[addrs[0].Addr] + addrInfo, ok := b.addrs[addrs[0].Addr] if !ok { return scw, nil } - obj.sws = append(obj.sws, scw) - atomic.StorePointer(&scw.addressInfo, unsafe.Pointer(obj)) + addrInfo.sws = append(addrInfo.sws, scw) + atomic.StorePointer(&scw.addressInfo, unsafe.Pointer(addrInfo)) // "If that address is currently ejected, that subchannel wrapper's eject // method will be called." - A50 - if !obj.latestEjectionTimestamp.IsZero() { + if !addrInfo.latestEjectionTimestamp.IsZero() { scw.eject() } return scw, nil @@ -461,26 +460,26 @@ func (b *outlierDetectionBalancer) RemoveSubConn(sc balancer.SubConn) { // the Outlier Detection balancers address map. Returns nil if not present, and // the map entry if present. func (b *outlierDetectionBalancer) appendIfPresent(addr string, scw *subConnWrapper) *addressInfo { - obj, ok := b.addrs[addr] + addrInfo, ok := b.addrs[addr] if !ok { return nil } - obj.sws = append(obj.sws, scw) - atomic.StorePointer(&scw.addressInfo, unsafe.Pointer(obj)) - return obj + addrInfo.sws = append(addrInfo.sws, scw) + atomic.StorePointer(&scw.addressInfo, unsafe.Pointer(addrInfo)) + return addrInfo } // removeSubConnFromAddressesMapEntry removes the scw from it's map entry if // present. func (b *outlierDetectionBalancer) removeSubConnFromAddressesMapEntry(scw *subConnWrapper) { - obj := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) - if obj == nil { + addrInfo := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) + if addrInfo == nil { return } - for i, sw := range obj.sws { + for i, sw := range addrInfo.sws { if scw == sw { - obj.sws = append(obj.sws[:i], obj.sws[i+1:]...) + addrInfo.sws = append(addrInfo.sws[:i], addrInfo.sws[i+1:]...) break } } @@ -500,48 +499,45 @@ func (b *outlierDetectionBalancer) UpdateAddresses(sc balancer.SubConn, addrs [] // Note that 0 addresses is a valid update/state for a SubConn to be in. // This is correctly handled by this algorithm (handled as part of a non singular // old address/new address). - if len(scw.addresses) == 1 { - if len(addrs) == 1 { // single address to single address - // If the updated address is the same, then there is nothing to do - // past this point. - if scw.addresses[0].Addr == addrs[0].Addr { - return - } - // 1. Remove Subchannel from Addresses map entry if present in Addresses map. - b.removeSubConnFromAddressesMapEntry(scw) - // 2. Add Subchannel to Addresses map entry if new address present in map. - obj := b.appendIfPresent(addrs[0].Addr, scw) - // 3. Relay state with eject() recalculated (using the corresponding - // map entry to see if it's currently ejected). - if obj == nil { // uneject unconditionally because could have come from an ejected address - scw.eject() - } else { - if obj.latestEjectionTimestamp.IsZero() { // relay new updated subconn state - scw.uneject() - } else { - scw.eject() - } - } - } else { // single address to multiple addresses - // 1. Remove Subchannel from Addresses map entry if present in Addresses map. - b.removeSubConnFromAddressesMapEntry(scw) - // 2. Clear the Subchannel wrapper's Call Counter entry. - obj := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) - if obj != nil { - obj.callCounter.clear() - } - // 3. Uneject the Subchannel in case it was previously ejected. - scw.uneject() + switch { + case len(scw.addresses) == 1 && len(addrs) == 1: // single address to single address + // If the updated address is the same, then there is nothing to do + // past this point. + if scw.addresses[0].Addr == addrs[0].Addr { + return } - } else { - if len(addrs) == 1 { // multiple addresses to single address - // 1. Add Subchannel to Addresses map entry if new address present in map. - obj := b.appendIfPresent(addrs[0].Addr, scw) - if obj != nil && !obj.latestEjectionTimestamp.IsZero() { + // 1. Remove Subchannel from Addresses map entry if present in Addresses map. + b.removeSubConnFromAddressesMapEntry(scw) + // 2. Add Subchannel to Addresses map entry if new address present in map. + addrInfo := b.appendIfPresent(addrs[0].Addr, scw) + // 3. Relay state with eject() recalculated (using the corresponding + // map entry to see if it's currently ejected). + if addrInfo == nil { // uneject unconditionally because could have come from an ejected address + scw.uneject() + } else { + if addrInfo.latestEjectionTimestamp.IsZero() { // relay new updated subconn state + scw.uneject() + } else { scw.eject() } - } // else is multiple to multiple - no op, continued to be ignored by outlier detection. - } + } + case len(scw.addresses) == 1: // single address to multiple/no addresses + // 1. Remove Subchannel from Addresses map entry if present in Addresses map. + b.removeSubConnFromAddressesMapEntry(scw) + // 2. Clear the Subchannel wrapper's Call Counter entry. + addrInfo := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) + if addrInfo != nil { + addrInfo.callCounter.clear() + } + // 3. Uneject the Subchannel in case it was previously ejected. + scw.uneject() + case len(addrs) == 1: // multiple/no addresses to single address + // 1. Add Subchannel to Addresses map entry if new address present in map. + addrInfo := b.appendIfPresent(addrs[0].Addr, scw) + if addrInfo != nil && !addrInfo.latestEjectionTimestamp.IsZero() { + scw.eject() + } + } // otherwise multiple/no addresses to multiple/no addresses; ignore scw.addresses = addrs } @@ -568,6 +564,123 @@ func min(x, y int64) int64 { return y } +// handleSubConnUpdate stores the recent state and forward the update +// if the SubConn is not ejected. +func (b *outlierDetectionBalancer) handleSubConnUpdate(u *scUpdate) { + scw := u.scw + scw.latestState = u.state + b.childMu.Lock() + if !scw.ejected && b.child != nil { + b.child.UpdateSubConnState(scw, u.state) + } + b.childMu.Unlock() +} + +// handleEjectedUpdate handles any SubConns that get ejected/unejected, and +// forwards the appropriate corresponding subConnState to the child policy. +func (b *outlierDetectionBalancer) handleEjectedUpdate(u *ejectedUpdate) { + scw := u.scw + scw.ejected = u.ejected + var stateToUpdate balancer.SubConnState + if u.ejected { + // "The wrapper will report a state update with the + // TRANSIENT_FAILURE state, and will stop passing along + // updates from the underlying subchannel." + stateToUpdate = balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + } + } else { + // "The wrapper will report a state update with the latest + // update from the underlying subchannel, and resume passing + // along updates from the underlying subchannel." + stateToUpdate = scw.latestState // If this has never been written to will send connectivity IDLE which seems fine to me + } + b.childMu.Lock() + if b.child != nil { + b.child.UpdateSubConnState(scw, stateToUpdate) + } + b.childMu.Unlock() +} + +// handleChildStateUpdate forwards the picker update wrapped in a wrapped picker +// with the noop picker bit present. +func (b *outlierDetectionBalancer) handleChildStateUpdate(u balancer.State) { + b.childState = u + b.mu.Lock() + noopCfg := b.noopConfig() + b.mu.Unlock() + b.recentPickerNoop = noopCfg + b.cc.UpdateState(balancer.State{ + ConnectivityState: b.childState.ConnectivityState, + // The outlier_detection LB policy will provide a picker that delegates to + // the child policy's picker, and when the request finishes, increment the + // corresponding counter in the map entry referenced by the subchannel + // wrapper that was picked. + Picker: &wrappedPicker{ + childPicker: b.childState.Picker, + // If both the `success_rate_ejection` and + // `failure_percentage_ejection` fields are unset in the + // configuration, the picker should not do that counting. + noopPicker: noopCfg, + }, + }) +} + +// handleLBConfigUpdate compares whether the new config is a noop config or not, +// to the noop bit in the picker if present. It updates the picker if this bit +// changed compared to the picker currently in use. +func (b *outlierDetectionBalancer) handleLBConfigUpdate(u *LBConfig) { + noopCfg := u.SuccessRateEjection == nil && u.FailurePercentageEjection == nil + if b.childState.Picker != nil && noopCfg != b.recentPickerNoop { + b.recentPickerNoop = noopCfg + b.cc.UpdateState(balancer.State{ + ConnectivityState: b.childState.ConnectivityState, + // The outlier_detection LB policy will provide a picker that delegates to + // the child policy's picker, and when the request finishes, increment the + // corresponding counter in the map entry referenced by the subchannel + // wrapper that was picked. + Picker: &wrappedPicker{ + childPicker: b.childState.Picker, + // If both the `success_rate_ejection` and + // `failure_percentage_ejection` fields are unset in the + // configuration, the picker should not do that counting. + noopPicker: noopCfg, + }, + }) + } +} + +func (b *outlierDetectionBalancer) run() { + for { + select { + case update := <-b.scUpdateCh.Get(): + b.scUpdateCh.Load() + switch u := update.(type) { + case *scUpdate: + b.handleSubConnUpdate(u) + case *ejectedUpdate: + b.handleEjectedUpdate(u) + } + case update := <-b.pickerUpdateCh.Get(): + b.pickerUpdateCh.Load() + if b.closed.HasFired() { // don't send picker updates to grpc after the balancer has been closed + return + } + switch u := update.(type) { + case balancer.State: + b.handleChildStateUpdate(u) + case *LBConfig: + b.handleLBConfigUpdate(u) + } + case <-b.closed.Done(): + return + } + } +} + +// intervalTimerAlgorithm ejects and unejects addresses based on the outlier +// detection configuration and data about each address from the previous +// interval. func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { b.mu.Lock() defer b.mu.Unlock() @@ -575,8 +688,8 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { // 2. For each address, swap the call counter's buckets in that address's // map entry. - for _, obj := range b.addrs { - obj.callCounter.swap() + for _, addrInfo := range b.addrs { + addrInfo.callCounter.swap() } // 3. If the success_rate_ejection configuration field is set, run the @@ -592,19 +705,22 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { } // 5. For each address in the map: - for addr, obj := range b.addrs { + for _, addrInfo := range b.addrs { // If the address is not ejected and the multiplier is greater than 0, // decrease the multiplier by 1. - if obj.latestEjectionTimestamp.IsZero() && obj.ejectionTimeMultiplier > 0 { - obj.ejectionTimeMultiplier-- + if addrInfo.latestEjectionTimestamp.IsZero() && addrInfo.ejectionTimeMultiplier > 0 { + addrInfo.ejectionTimeMultiplier-- continue } + et := b.cfg.BaseEjectionTime.Nanoseconds() * addrInfo.ejectionTimeMultiplier + met := max(b.cfg.BaseEjectionTime.Nanoseconds(), b.cfg.MaxEjectionTime.Nanoseconds()) + curTimeAfterEt := now().After(addrInfo.latestEjectionTimestamp.Add(time.Duration(min(et, met)))) // If the address is ejected, and the current time is after // ejection_timestamp + min(base_ejection_time (type: time.Time) * // multiplier (type: int), max(base_ejection_time (type: time.Time), // max_ejection_time (type: time.Time))), un-eject the address. - if !obj.latestEjectionTimestamp.IsZero() && now().After(obj.latestEjectionTimestamp.Add(time.Duration(min(b.cfg.BaseEjectionTime.Nanoseconds()*obj.ejectionTimeMultiplier, max(b.cfg.BaseEjectionTime.Nanoseconds(), b.cfg.MaxEjectionTime.Nanoseconds()))))) { - b.unejectAddress(addr) + if !addrInfo.latestEjectionTimestamp.IsZero() && curTimeAfterEt { + b.unejectAddress(addrInfo) } } @@ -618,108 +734,19 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { }) } -func (b *outlierDetectionBalancer) run() { - for { - select { - case update := <-b.scUpdateCh.Get(): - b.scUpdateCh.Load() - switch u := update.(type) { - case *scUpdate: - scw := u.scw - scw.latestState = u.state - b.childMu.Lock() - if !scw.ejected && b.child != nil { - b.child.UpdateSubConnState(scw, u.state) - } - b.childMu.Unlock() - case *ejectedUpdate: - scw := u.scw - scw.ejected = u.ejected - var stateToUpdate balancer.SubConnState - if u.ejected { - // "The wrapper will report a state update with the - // TRANSIENT_FAILURE state, and will stop passing along - // updates from the underlying subchannel." - stateToUpdate = balancer.SubConnState{ - ConnectivityState: connectivity.TransientFailure, - } - } else { - // "The wrapper will report a state update with the latest - // update from the underlying subchannel, and resume passing - // along updates from the underlying subchannel." - stateToUpdate = scw.latestState // If this has never been written to will send connectivity IDLE which seems fine to me - } - b.childMu.Lock() - if b.child != nil { - b.child.UpdateSubConnState(scw, stateToUpdate) - } - b.childMu.Unlock() - } - case update := <-b.pickerUpdateCh.Get(): - b.pickerUpdateCh.Load() - if b.closed.HasFired() { // don't send picker updates to grpc after the balancer has been closed - return - } - switch u := update.(type) { - case balancer.State: - b.childState = u - b.mu.Lock() - noopCfg := b.noopConfig() - b.mu.Unlock() - b.recentPickerNoop = noopCfg - b.cc.UpdateState(balancer.State{ - ConnectivityState: b.childState.ConnectivityState, - // The outlier_detection LB policy will provide a picker that delegates to - // the child policy's picker, and when the request finishes, increment the - // corresponding counter in the map entry referenced by the subchannel - // wrapper that was picked. - Picker: &wrappedPicker{ - childPicker: b.childState.Picker, - // If both the `success_rate_ejection` and - // `failure_percentage_ejection` fields are unset in the - // configuration, the picker should not do that counting. - noopPicker: noopCfg, - }, - }) - case *LBConfig: - noopCfg := u.SuccessRateEjection == nil && u.FailurePercentageEjection == nil - if b.childState.Picker != nil && noopCfg != b.recentPickerNoop { - b.recentPickerNoop = noopCfg - b.cc.UpdateState(balancer.State{ - ConnectivityState: b.childState.ConnectivityState, - // The outlier_detection LB policy will provide a picker that delegates to - // the child policy's picker, and when the request finishes, increment the - // corresponding counter in the map entry referenced by the subchannel - // wrapper that was picked. - Picker: &wrappedPicker{ - childPicker: b.childState.Picker, - // If both the `success_rate_ejection` and - // `failure_percentage_ejection` fields are unset in the - // configuration, the picker should not do that counting. - noopPicker: noopCfg, - }, - }) - } - } - case <-b.closed.Done(): - return - } - } -} - // numAddrsWithAtLeastRequestVolume returns the number of addresses present in // the map that have request volume of at least requestVolume. func (b *outlierDetectionBalancer) numAddrsWithAtLeastRequestVolume() uint32 { var numAddrs uint32 - for _, obj := range b.addrs { - if uint32(obj.callCounter.inactiveBucket.requestVolume) >= b.cfg.SuccessRateEjection.RequestVolume { + for _, addrInfo := range b.addrs { + if uint32(addrInfo.callCounter.inactiveBucket.requestVolume) >= b.cfg.SuccessRateEjection.RequestVolume { numAddrs++ } } return numAddrs } -// meanAndStdDevOfSucceseesAtLeastRequestVolume returns the mean and std dev of +// meanAndStdDevOfSuccessesAtLeastRequestVolume returns the mean and std dev of // the number of requests of addresses that have at least requestVolume. func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume() (float64, float64) { // 2. Calculate the mean and standard deviation of the fractions of @@ -727,16 +754,16 @@ func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume( // success_rate_ejection.request_volume. var totalFractionOfSuccessfulRequests float64 var mean float64 - for _, obj := range b.addrs { + for _, addrInfo := range b.addrs { // "of at least success_rate_ejection.request_volume" - if uint32(obj.callCounter.inactiveBucket.requestVolume) >= b.cfg.SuccessRateEjection.RequestVolume { - totalFractionOfSuccessfulRequests += float64(obj.callCounter.inactiveBucket.numSuccesses) / float64(obj.callCounter.inactiveBucket.requestVolume) + if uint32(addrInfo.callCounter.inactiveBucket.requestVolume) >= b.cfg.SuccessRateEjection.RequestVolume { + totalFractionOfSuccessfulRequests += float64(addrInfo.callCounter.inactiveBucket.numSuccesses) / float64(addrInfo.callCounter.inactiveBucket.requestVolume) } } mean = totalFractionOfSuccessfulRequests / float64(len(b.addrs)) var sumOfSquares float64 - for _, obj := range b.addrs { - devFromMean := (float64(obj.callCounter.inactiveBucket.numSuccesses) / float64(obj.callCounter.inactiveBucket.requestVolume)) - mean + for _, addrInfo := range b.addrs { + devFromMean := (float64(addrInfo.callCounter.inactiveBucket.numSuccesses) / float64(addrInfo.callCounter.inactiveBucket.requestVolume)) - mean sumOfSquares += devFromMean * devFromMean } @@ -745,6 +772,9 @@ func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume( } +// syccessRateAlgorithm ejects any addresses where the success rate falls below +// the other addresses according to mean and standard deviation, and if overall +// applicable from other set heuristics. func (b *outlierDetectionBalancer) successRateAlgorithm() { // 1. If the number of addresses with request volume of at least // success_rate_ejection.request_volume is less than @@ -759,8 +789,8 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { mean, stddev := b.meanAndStdDevOfSuccessesAtLeastRequestVolume() // 3. For each address: - for addr, obj := range b.addrs { - ccb := obj.callCounter.inactiveBucket + for _, addrInfo := range b.addrs { + ccb := addrInfo.callCounter.inactiveBucket sre := b.cfg.SuccessRateEjection // i. If the percentage of ejected addresses is greater than // max_ejection_percent, stop. @@ -782,12 +812,15 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { // than success_rate_ejection.enforcement_percentage, eject that // address. if uint32(grpcrand.Int31n(100)) < sre.EnforcementPercentage { - b.ejectAddress(addr) + b.ejectAddress(addrInfo) } } } } +// failurePercentageAlgorithm ejects any addresses where the failure percentage +// rate exceeds a set enforcement percentage, if overall applicable from other +// set heuristics. func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { // 1. If the number of addresses is less than // failure_percentage_ejection.minimum_hosts, stop. @@ -796,8 +829,8 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { } // 2. For each address: - for addr, obj := range b.addrs { - ccb := obj.callCounter.inactiveBucket + for _, addrInfo := range b.addrs { + ccb := addrInfo.callCounter.inactiveBucket fpe := b.cfg.FailurePercentageEjection // i. If the percentage of ejected addresses is greater than // max_ejection_percent, stop. @@ -818,35 +851,27 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { // than failiure_percentage_ejection.enforcement_percentage, eject // that address. if uint32(grpcrand.Int31n(100)) < b.cfg.FailurePercentageEjection.EnforcementPercentage { - b.ejectAddress(addr) + b.ejectAddress(addrInfo) } } } } -func (b *outlierDetectionBalancer) ejectAddress(addr string) { - obj, ok := b.addrs[addr] - if !ok { // Shouldn't happen - return - } +func (b *outlierDetectionBalancer) ejectAddress(addrInfo *addressInfo) { b.numAddrsEjected++ // To eject an address, set the current ejection timestamp to the timestamp // that was recorded when the timer fired, increase the ejection time // multiplier by 1, and call eject() on each subchannel wrapper in that // address's subchannel wrapper list. - obj.latestEjectionTimestamp = b.timerStartTime - obj.ejectionTimeMultiplier++ - for _, sbw := range obj.sws { + addrInfo.latestEjectionTimestamp = b.timerStartTime + addrInfo.ejectionTimeMultiplier++ + for _, sbw := range addrInfo.sws { sbw.eject() } } -func (b *outlierDetectionBalancer) unejectAddress(addr string) { - obj, ok := b.addrs[addr] - if !ok { // Shouldn't happen - return - } +func (b *outlierDetectionBalancer) unejectAddress(addrInfo *addressInfo) { b.numAddrsEjected-- // To un-eject an address, set the current ejection timestamp to null @@ -854,8 +879,8 @@ func (b *outlierDetectionBalancer) unejectAddress(addr string) { // logically equivalent in practice to the time zero value) and call // uneject() on each subchannel wrapper in that address's subchannel wrapper // list. - obj.latestEjectionTimestamp = time.Time{} - for _, sbw := range obj.sws { + addrInfo.latestEjectionTimestamp = time.Time{} + for _, sbw := range addrInfo.sws { sbw.uneject() } } @@ -883,7 +908,7 @@ type addressInfo struct { sws []*subConnWrapper } -func newObject() *addressInfo { +func newAddressInfo() *addressInfo { return &addressInfo{ callCounter: newCallCounter(), sws: make([]*subConnWrapper, 0), diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index 181b3946ab7..5f94cc1a82c 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -969,7 +969,7 @@ func (s) TestPicker(t *testing.T) { pi.Done(balancer.DoneInfo{}) pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) od.mu.Lock() - obj, ok := od.addrs["address1"] + addrInfo, ok := od.addrs["address1"] if !ok { t.Fatal("map entry for address: address1 not present in map") } @@ -978,7 +978,7 @@ func (s) TestPicker(t *testing.T) { numFailures: 1, requestVolume: 2, } - if diff := cmp.Diff((*bucket)(obj.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call from picker + if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call from picker t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) } od.mu.Unlock() @@ -1039,7 +1039,7 @@ func (s) TestPicker(t *testing.T) { pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) od.mu.Lock() - obj, ok := od.addrs["address1"] + addrInfo, ok := od.addrs["address1"] if !ok { t.Fatal("map entry for address: address1 not present in map") } @@ -1049,7 +1049,7 @@ func (s) TestPicker(t *testing.T) { // should not count, as the outlier detection balancer is configured // with a no-op configuration. bucketWant := &bucket{} - if diff := cmp.Diff((*bucket)(obj.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call + if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) } od.mu.Unlock() @@ -1088,7 +1088,7 @@ func (s) TestPicker(t *testing.T) { pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) od.mu.Lock() - obj, ok := od.addrs["address1"] + addrInfo, ok := od.addrs["address1"] if !ok { t.Fatal("map entry for address: address1 not present in map") } @@ -1098,7 +1098,7 @@ func (s) TestPicker(t *testing.T) { // should not count, as the outlier detection balancer is configured // with a no-op configuration. bucketWant := &bucket{} - if diff := cmp.Diff((*bucket)(obj.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call + if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) } od.mu.Unlock() From ac8d33b120c69ff6ae49e46c20390749ae1fd693 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Tue, 26 Jul 2022 16:16:32 -0400 Subject: [PATCH 05/22] Responded to Doug's comments --- .../balancer/outlierdetection/balancer.go | 46 ++++++++++--------- .../outlierdetection/balancer_test.go | 2 +- .../balancer/outlierdetection/callcounter.go | 12 ++--- .../balancer/outlierdetection/logging.go | 34 ++++++++++++++ 4 files changed, 66 insertions(+), 28 deletions(-) create mode 100644 xds/internal/balancer/outlierdetection/logging.go diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 7718de02b0d..461ebffef31 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -36,6 +36,7 @@ import ( "google.golang.org/grpc/internal" "google.golang.org/grpc/internal/buffer" "google.golang.org/grpc/internal/envconfig" + "google.golang.org/grpc/internal/grpclog" "google.golang.org/grpc/internal/grpcrand" "google.golang.org/grpc/internal/grpcsync" "google.golang.org/grpc/resolver" @@ -76,6 +77,8 @@ func (bb) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Ba scUpdateCh: buffer.NewUnbounded(), pickerUpdateCh: buffer.NewUnbounded(), } + b.logger = prefixLogger(b) + b.logger.Infof("Created") go b.run() return b } @@ -153,6 +156,7 @@ type outlierDetectionBalancer struct { closed *grpcsync.Event cc balancer.ClientConn bOpts balancer.BuildOptions + logger *grpclog.PrefixLogger // childMu protects child and also updates to the child (to uphold the // balancer.Balancer API guarantee of synchronous calls). It also protects @@ -192,6 +196,7 @@ func (b *outlierDetectionBalancer) noopConfig() bool { func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnState) error { lbCfg, ok := s.BalancerConfig.(*LBConfig) if !ok { + b.logger.Errorf("received config with unexpected type %T: %v", s.BalancerConfig, s.BalancerConfig) return balancer.ErrBadResolverState } @@ -242,7 +247,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt // timer and start the timer for the configured interval minus the // difference between the current time and the previous start timestamp, // or 0 if that would be negative. - interval = b.cfg.Interval - (now().Sub(b.timerStartTime)) + interval = b.cfg.Interval - now().Sub(b.timerStartTime) if interval < 0 { interval = 0 } @@ -278,11 +283,12 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt } func (b *outlierDetectionBalancer) ResolverError(err error) { - if b.child != nil { - b.childMu.Lock() - defer b.childMu.Unlock() - b.child.ResolverError(err) + if b.child == nil { + return } + b.childMu.Lock() + defer b.childMu.Unlock() + b.child.ResolverError(err) } func (b *outlierDetectionBalancer) UpdateSubConnState(sc balancer.SubConn, state balancer.SubConnState) { @@ -290,7 +296,9 @@ func (b *outlierDetectionBalancer) UpdateSubConnState(sc balancer.SubConn, state defer b.mu.Unlock() scw, ok := b.scWrappers[sc] if !ok { - // Return, shouldn't happen if passed up scw + // Shouldn't happen if passed down a SubConnWrapper to child on SubConn + // creation. + b.logger.Errorf("UpdateSubConnState called with SubConn that has no corresponding SubConnWrapper") return } if state.ConnectivityState == connectivity.Shutdown { @@ -300,7 +308,6 @@ func (b *outlierDetectionBalancer) UpdateSubConnState(sc balancer.SubConn, state scw: scw, state: state, }) - } func (b *outlierDetectionBalancer) Close() { @@ -358,7 +365,6 @@ func (wp *wrappedPicker) Pick(info balancer.PickInfo) (balancer.PickResult, erro pr.Done(di) } } - // Shouldn't happen, defensive programming. scw, ok := pr.SubConn.(*subConnWrapper) if !ok { return balancer.PickResult{ @@ -400,16 +406,15 @@ func incrementCounter(sc balancer.SubConn, info balancer.DoneInfo) { ab := (*bucket)(atomic.LoadPointer(&addrInfo.callCounter.activeBucket)) if info.Err == nil { - atomic.AddInt64(&ab.numSuccesses, 1) + atomic.AddUint32(&ab.numSuccesses, 1) } else { - atomic.AddInt64(&ab.numFailures, 1) + atomic.AddUint32(&ab.numFailures, 1) } - atomic.AddInt64(&ab.requestVolume, 1) + atomic.AddUint32(&ab.requestVolume, 1) } func (b *outlierDetectionBalancer) UpdateState(s balancer.State) { b.pickerUpdateCh.Put(s) - } func (b *outlierDetectionBalancer) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) { @@ -470,7 +475,7 @@ func (b *outlierDetectionBalancer) appendIfPresent(addr string, scw *subConnWrap return addrInfo } -// removeSubConnFromAddressesMapEntry removes the scw from it's map entry if +// removeSubConnFromAddressesMapEntry removes the scw from its map entry if // present. func (b *outlierDetectionBalancer) removeSubConnFromAddressesMapEntry(scw *subConnWrapper) { addrInfo := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) @@ -739,7 +744,7 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { func (b *outlierDetectionBalancer) numAddrsWithAtLeastRequestVolume() uint32 { var numAddrs uint32 for _, addrInfo := range b.addrs { - if uint32(addrInfo.callCounter.inactiveBucket.requestVolume) >= b.cfg.SuccessRateEjection.RequestVolume { + if addrInfo.callCounter.inactiveBucket.requestVolume >= b.cfg.SuccessRateEjection.RequestVolume { numAddrs++ } } @@ -756,7 +761,7 @@ func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume( var mean float64 for _, addrInfo := range b.addrs { // "of at least success_rate_ejection.request_volume" - if uint32(addrInfo.callCounter.inactiveBucket.requestVolume) >= b.cfg.SuccessRateEjection.RequestVolume { + if addrInfo.callCounter.inactiveBucket.requestVolume >= b.cfg.SuccessRateEjection.RequestVolume { totalFractionOfSuccessfulRequests += float64(addrInfo.callCounter.inactiveBucket.numSuccesses) / float64(addrInfo.callCounter.inactiveBucket.requestVolume) } } @@ -769,17 +774,16 @@ func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume( variance := sumOfSquares / float64(len(b.addrs)) return mean, math.Sqrt(variance) - } -// syccessRateAlgorithm ejects any addresses where the success rate falls below +// successRateAlgorithm ejects any addresses where the success rate falls below // the other addresses according to mean and standard deviation, and if overall // applicable from other set heuristics. func (b *outlierDetectionBalancer) successRateAlgorithm() { // 1. If the number of addresses with request volume of at least // success_rate_ejection.request_volume is less than // success_rate_ejection.minimum_hosts, stop. - if b.numAddrsWithAtLeastRequestVolume() < b.cfg.SuccessRateEjection.MinimumHosts { // TODO: O(n) search, is there a way to optimize this? + if b.numAddrsWithAtLeastRequestVolume() < b.cfg.SuccessRateEjection.MinimumHosts { return } @@ -800,7 +804,7 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { // ii. If the address's total request volume is less than // success_rate_ejection.request_volume, continue to the next address. - if ccb.requestVolume < int64(sre.RequestVolume) { + if ccb.requestVolume < sre.RequestVolume { continue } @@ -840,7 +844,7 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { // ii. If the address's total request volume is less than // failure_percentage_ejection.request_volume, continue to the next // address. - if uint32(ccb.requestVolume) < fpe.RequestVolume { + if ccb.requestVolume < fpe.RequestVolume { continue } // 2c. If the address's failure percentage is greater than @@ -888,7 +892,7 @@ func (b *outlierDetectionBalancer) unejectAddress(addrInfo *addressInfo) { // addressInfo contains the runtime information about an address that pertains // to Outlier Detection, including the counter for successful/failing RPC's, and // also information about whether the addresses has been ejected, and the -// SubConns that are present that use this address. This struct and all of it's +// SubConns that are present that use this address. This struct and all of its // fields is protected by outlierDetectionBalancer.mu in the case where it // accessed through the address map. In the case of Picker callbacks, the writes // to the activeBucket of callCounter are protected by atomically loading and diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index 5f94cc1a82c..bb57d16bcde 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -2000,7 +2000,7 @@ func (s) TestConcurrentOperations(t *testing.T) { // Call balancer.Balancers synchronously in this goroutine, upholding the // balancer.Balancer API guarantee. - od.UpdateSubConnState(scw1, balancer.SubConnState{ + od.UpdateSubConnState(scw1.(*subConnWrapper).SubConn, balancer.SubConnState{ ConnectivityState: connectivity.Connecting, }) od.ResolverError(errors.New("some error")) diff --git a/xds/internal/balancer/outlierdetection/callcounter.go b/xds/internal/balancer/outlierdetection/callcounter.go index 7de43792cd7..25163a9c420 100644 --- a/xds/internal/balancer/outlierdetection/callcounter.go +++ b/xds/internal/balancer/outlierdetection/callcounter.go @@ -23,9 +23,9 @@ import ( ) type bucket struct { - numSuccesses int64 - numFailures int64 - requestVolume int64 // numSuccesses + numFailures, needed because this number will be used in interval timer algorithm + numSuccesses uint32 + numFailures uint32 + requestVolume uint32 // numSuccesses + numFailures, needed because this number will be used in interval timer algorithm } func newCallCounter() *callCounter { @@ -69,9 +69,9 @@ func (cc *callCounter) swap() { // dealing with picker race condition. See the wrappedPicker explanation for // the write to activeBucket for a more in depth explanation. cc.inactiveBucket = &bucket{ - numSuccesses: atomic.LoadInt64(&ab.numSuccesses), - numFailures: atomic.LoadInt64(&ab.numFailures), - requestVolume: atomic.LoadInt64(&ab.requestVolume), + numSuccesses: atomic.LoadUint32(&ab.numSuccesses), + numFailures: atomic.LoadUint32(&ab.numFailures), + requestVolume: atomic.LoadUint32(&ab.requestVolume), } atomic.StorePointer(&cc.activeBucket, unsafe.Pointer(&bucket{})) // end result, same as in gRFC: the inactive bucket contains the number of diff --git a/xds/internal/balancer/outlierdetection/logging.go b/xds/internal/balancer/outlierdetection/logging.go new file mode 100644 index 00000000000..705b0cb6918 --- /dev/null +++ b/xds/internal/balancer/outlierdetection/logging.go @@ -0,0 +1,34 @@ +/* + * + * Copyright 2022 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package outlierdetection + +import ( + "fmt" + + "google.golang.org/grpc/grpclog" + internalgrpclog "google.golang.org/grpc/internal/grpclog" +) + +const prefix = "[outlier-detection-lb %p] " + +var logger = grpclog.Component("xds") + +func prefixLogger(p *outlierDetectionBalancer) *internalgrpclog.PrefixLogger { + return internalgrpclog.NewPrefixLogger(logger, fmt.Sprintf(prefix, p)) +} From 0ba32dffe54ebc33037244adf6ce8a34a27b8167 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Tue, 26 Jul 2022 18:52:24 -0400 Subject: [PATCH 06/22] Changes discussed in chat --- .../balancer/outlierdetection/balancer.go | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 461ebffef31..82a431c564b 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -253,22 +253,27 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt } } + if b.intervalTimer != nil { + b.intervalTimer.Stop() + } if !b.noopConfig() { - if b.intervalTimer != nil { - b.intervalTimer.Stop() - } b.intervalTimer = afterFunc(interval, func() { b.intervalTimerAlgorithm() }) } else { // "If a config is provided with both the `success_rate_ejection` and // `failure_percentage_ejection` fields unset, skip starting the timer and - // unset the timer start timestamp." + // do the following:" + // "Unset the timer start timestamp." b.timerStartTime = time.Time{} - // Should we stop the timer here as well? Not defined in gRFC but I feel - // like it might make sense as you don't want to eject addresses. Also - // how will addresses eventually get unejected in this case if only one - // more pass of the interval timer after no-op configuration comes in? + for _, addrInfo := range b.addrs { + // "Uneject all currently ejected addresses." + if !addrInfo.latestEjectionTimestamp.IsZero() { + b.unejectAddress(addrInfo) + } + // "Reset each address's ejection time multiplier to 0." + addrInfo.ejectionTimeMultiplier = 0 + } } b.mu.Unlock() b.pickerUpdateCh.Put(lbCfg) From 370cbb87ae8abd1b91d3ea1e87792ca6e2794e42 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Tue, 26 Jul 2022 19:39:17 -0400 Subject: [PATCH 07/22] Added test for new functionality and bug fix --- .../balancer/outlierdetection/balancer.go | 4 +- .../outlierdetection/balancer_test.go | 223 +++++++++++++++++- 2 files changed, 215 insertions(+), 12 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 82a431c564b..3b9c658f273 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -224,7 +224,9 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt addrs := make(map[string]bool, len(s.ResolverState.Addresses)) for _, addr := range s.ResolverState.Addresses { addrs[addr.Addr] = true - b.addrs[addr.Addr] = newAddressInfo() + if _, ok := b.addrs[addr.Addr]; !ok { + b.addrs[addr.Addr] = newAddressInfo() + } } for addr := range b.addrs { if !addrs[addr] { diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index bb57d16bcde..3693fd616cd 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -1017,7 +1017,7 @@ func (s) TestPicker(t *testing.T) { // The connectivity state sent to the Client Conn should be the persisted // recent state received from the last UpdateState() call, which in this - // case is connecting. + // case is ready. select { case <-ctx.Done(): t.Fatal("timeout waiting for picker update on ClientConn, should have updated because no-op config changed on UpdateClientConnState") @@ -1044,12 +1044,17 @@ func (s) TestPicker(t *testing.T) { t.Fatal("map entry for address: address1 not present in map") } - // The active bucket should be cleared because the interval timer - // algorithm didn't run in between ClientConn updates and the picker - // should not count, as the outlier detection balancer is configured - // with a no-op configuration. - bucketWant := &bucket{} + // The active bucket should be the same as before the no-op + // configuration came in because the interval timer algorithm didn't run + // in between ClientConn updates and the picker should not count, as the + // outlier detection balancer is configured with a no-op configuration. + bucketWant := &bucket{ + numSuccesses: 1, + numFailures: 1, + requestVolume: 2, + } if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call + od.mu.Unlock() t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) } od.mu.Unlock() @@ -1093,11 +1098,15 @@ func (s) TestPicker(t *testing.T) { t.Fatal("map entry for address: address1 not present in map") } - // The active bucket should be cleared because the interval timer - // algorithm didn't run in between ClientConn updates and the picker - // should not count, as the outlier detection balancer is configured - // with a no-op configuration. - bucketWant := &bucket{} + // The active bucket should be the same as before the no-op + // configuration came in because the interval timer algorithm didn't run + // in between ClientConn updates and the picker should not count, as the + // outlier detection balancer is configured with a no-op configuration. + bucketWant := &bucket{ + numSuccesses: 1, + numFailures: 1, + requestVolume: 2, + } if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) } @@ -1336,9 +1345,201 @@ func (s) TestEjectUnejectSuccessRate(t *testing.T) { }); err != nil { t.Fatalf("Error waiting for Sub Conn update: %v", err) } + } +} + +// TestEjectUnejectSuccessRateFromNoopConfig tests that any ejected Addresses +// are unejected upon the receipt of a no-op Outlier Detection Configuration. +func (s) TestEjectUnejectSuccessRateFromNoopConfig(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + // Setup the outlier detection balancer to a point where it will be in a + // situation to potentially eject addresses. + od, tcc := setup(t) + defer func() { + od.Close() + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 500, + EnforcementPercentage: 100, + MinimumHosts: 3, + RequestVolume: 3, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + child := od.child.(*testClusterImplBalancer) + if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: testClusterImplBalancerConfig{}, + }); err != nil { + t.Fatalf("Error waiting for Client Conn update: %v", err) + } + + scw1, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address1", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + + scw2, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address2", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + scw3, err := od.NewSubConn([]resolver.Address{ + { + Addr: "address3", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) } + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &rrPicker{ + scs: []balancer.SubConn{scw1, scw2, scw3}, + }, + }) + + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case picker := <-tcc.NewPickerCh: + // Set two of the upstream addresses to have five successes each, and + // one of the upstream addresses to have five failures. This should + // cause the address which has five failures to be ejected according the + // SuccessRateAlgorithm. + for i := 0; i < 2; i++ { + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + } + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + + // should eject address that always errored. + od.intervalTimerAlgorithm() + // Due to the address being ejected, the SubConn with that address + // should be ejected, meaning a TRANSIENT_FAILURE connectivity state + // gets reported to the child. + if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + sc: scw3, + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected + }); err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + // Only one address should be ejected. + sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer sCancel() + if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { + t.Fatalf("Only one SubConn update should have been sent (only one SubConn got ejected)") + } + // Now that an address is ejected, SubConn updates for SubConns using + // that address should not be forwarded downward. These SubConn updates + // will be cached to update the child sometime in the future when the + // address gets unejected. + od.UpdateSubConnState(pi.SubConn, balancer.SubConnState{ + ConnectivityState: connectivity.Connecting, + }) + if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { + t.Fatalf("SubConn update should not have been forwarded (the SubConn is ejected)") + } + // Update the Outlier Detection Balancer with a no-op configuration. + // This should cause any ejected addresses to become unejected. + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: testClusterImplBalancerConfig{}, + }, + }, + }) + // unejected SubConn should report latest persisted state - which is + // connecting from earlier. + if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + sc: scw3, + state: balancer.SubConnState{ConnectivityState: connectivity.Connecting}, + }); err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + } } // TestEjectFailureRate tests the functionality of the interval timer From aa00a9a6d7c09796c8e012bda0936a9d56a4fd76 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Fri, 29 Jul 2022 12:31:22 -0400 Subject: [PATCH 08/22] Responded to some of Easwar's comments and updated picker synchronously --- .../balancer/outlierdetection/balancer.go | 72 +++++++++++++------ .../outlierdetection/subconn_wrapper.go | 24 ++++--- 2 files changed, 63 insertions(+), 33 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 3b9c658f273..8e20c3297f7 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -142,9 +142,15 @@ type scUpdate struct { state balancer.SubConnState } -type ejectedUpdate struct { - scw *subConnWrapper - ejected bool // true for ejected, false for unejected +type ejectionUpdate struct { + scw *subConnWrapper + isEjected bool // true for ejected, false for unejected +} + +type lbCfgUpdate struct { + lbCfg *LBConfig + // to make sure picker is updated synchronously. + done chan struct{} } type outlierDetectionBalancer struct { @@ -166,7 +172,8 @@ type outlierDetectionBalancer struct { child balancer.Balancer // mu guards access to a lot of the core LB Policy State. It also prevents - // intersplicing certain operations. + // intersplicing certain operations. Some of the undefined behaviors this + // mutex protects are: // // ex 1: interval timer goes off, outlier detection algorithm starts running // based on knobs in cfg. in the middle of running the algorithm, a @@ -176,12 +183,13 @@ type outlierDetectionBalancer struct { // ex 2: Updating the addrs map from UpdateAddresses in the middle of // running the interval timer algorithm which uses addrs heavily. This will // cause undefined behavior for the interval timer algorithm. - mu sync.Mutex - addrs map[string]*addressInfo - cfg *LBConfig - scWrappers map[balancer.SubConn]*subConnWrapper - timerStartTime time.Time - intervalTimer *time.Timer + mu sync.Mutex + addrs map[string]*addressInfo + cfg *LBConfig + scWrappers map[balancer.SubConn]*subConnWrapper + timerStartTime time.Time + intervalTimer *time.Timer + inhibitPickerUpdates bool scUpdateCh *buffer.Unbounded pickerUpdateCh *buffer.Unbounded @@ -217,7 +225,6 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt b.mu.Lock() b.cfg = lbCfg - // When the outlier_detection LB policy receives an address update, it will // create a map entry for each subchannel address in the list, and remove // each map entry for a subchannel address not in the list. @@ -277,16 +284,29 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt addrInfo.ejectionTimeMultiplier = 0 } } - b.mu.Unlock() - b.pickerUpdateCh.Put(lbCfg) + // Inhibit picker updates until run() processes the lbCfgUpdate. This + // makes sure a single picker update gets sent out synchronously as a result + // of this UpdateClientConnState call, and this picker update contains the + // most updated no-op config bit and most recent state from the child. + b.inhibitPickerUpdates = true + b.mu.Unlock() // then pass the address list along to the child policy. b.childMu.Lock() - defer b.childMu.Unlock() - return b.child.UpdateClientConnState(balancer.ClientConnState{ + err := b.child.UpdateClientConnState(balancer.ClientConnState{ ResolverState: s.ResolverState, BalancerConfig: b.cfg.ChildPolicy.Config, }) + b.childMu.Unlock() + + done := make(chan struct{}) + b.pickerUpdateCh.Put(lbCfgUpdate{ + lbCfg: lbCfg, + done: done, + }) + // To make sure Picker updated synchronously. + <-done + return err } func (b *outlierDetectionBalancer) ResolverError(err error) { @@ -590,11 +610,11 @@ func (b *outlierDetectionBalancer) handleSubConnUpdate(u *scUpdate) { // handleEjectedUpdate handles any SubConns that get ejected/unejected, and // forwards the appropriate corresponding subConnState to the child policy. -func (b *outlierDetectionBalancer) handleEjectedUpdate(u *ejectedUpdate) { +func (b *outlierDetectionBalancer) handleEjectedUpdate(u *ejectionUpdate) { scw := u.scw - scw.ejected = u.ejected + scw.ejected = u.isEjected var stateToUpdate balancer.SubConnState - if u.ejected { + if u.isEjected { // "The wrapper will report a state update with the // TRANSIENT_FAILURE state, and will stop passing along // updates from the underlying subchannel." @@ -619,6 +639,10 @@ func (b *outlierDetectionBalancer) handleEjectedUpdate(u *ejectedUpdate) { func (b *outlierDetectionBalancer) handleChildStateUpdate(u balancer.State) { b.childState = u b.mu.Lock() + if b.inhibitPickerUpdates { + b.mu.Unlock() + return + } noopCfg := b.noopConfig() b.mu.Unlock() b.recentPickerNoop = noopCfg @@ -641,8 +665,10 @@ func (b *outlierDetectionBalancer) handleChildStateUpdate(u balancer.State) { // handleLBConfigUpdate compares whether the new config is a noop config or not, // to the noop bit in the picker if present. It updates the picker if this bit // changed compared to the picker currently in use. -func (b *outlierDetectionBalancer) handleLBConfigUpdate(u *LBConfig) { - noopCfg := u.SuccessRateEjection == nil && u.FailurePercentageEjection == nil +func (b *outlierDetectionBalancer) handleLBConfigUpdate(u lbCfgUpdate) { + lbCfg := u.lbCfg + done := u.done + noopCfg := lbCfg.SuccessRateEjection == nil && lbCfg.FailurePercentageEjection == nil if b.childState.Picker != nil && noopCfg != b.recentPickerNoop { b.recentPickerNoop = noopCfg b.cc.UpdateState(balancer.State{ @@ -660,6 +686,8 @@ func (b *outlierDetectionBalancer) handleLBConfigUpdate(u *LBConfig) { }, }) } + b.inhibitPickerUpdates = false + close(done) } func (b *outlierDetectionBalancer) run() { @@ -670,7 +698,7 @@ func (b *outlierDetectionBalancer) run() { switch u := update.(type) { case *scUpdate: b.handleSubConnUpdate(u) - case *ejectedUpdate: + case *ejectionUpdate: b.handleEjectedUpdate(u) } case update := <-b.pickerUpdateCh.Get(): @@ -681,7 +709,7 @@ func (b *outlierDetectionBalancer) run() { switch u := update.(type) { case balancer.State: b.handleChildStateUpdate(u) - case *LBConfig: + case lbCfgUpdate: b.handleLBConfigUpdate(u) } case <-b.closed.Done(): diff --git a/xds/internal/balancer/outlierdetection/subconn_wrapper.go b/xds/internal/balancer/outlierdetection/subconn_wrapper.go index f2ecbae605e..8e25eb788b1 100644 --- a/xds/internal/balancer/outlierdetection/subconn_wrapper.go +++ b/xds/internal/balancer/outlierdetection/subconn_wrapper.go @@ -25,11 +25,9 @@ import ( "google.golang.org/grpc/resolver" ) -// subConnWrapper wraps every created SubConn in the Outlier Detection Balancer. -// It is used to store whether the SubConn has been ejected or not, and also to -// store the latest state for use when the SubConn gets unejected. It also -// stores the addresses the SubConn was created with to support any change in -// address(es). +// subConnWrapper wraps every created SubConn in the Outlier Detection Balancer, +// to help track the latest state update from the underlying SubConn, and also +// whether or not this SubConn is ejected. type subConnWrapper struct { balancer.SubConn @@ -38,20 +36,24 @@ type subConnWrapper struct { addressInfo unsafe.Pointer // *addressInfo // These two pieces of state will reach eventual consistency due to sync in // run(), and child will always have the correctly updated SubConnState. + // latestState is the latest state update from the underlying SubConn. This + // is used whenever a SubConn gets unejected. latestState balancer.SubConnState ejected bool scUpdateCh *buffer.Unbounded + // addresses is the list of address(es) this SubConn was created with to + // help support any change in address(es) addresses []resolver.Address } // eject causes the wrapper to report a state update with the TRANSIENT_FAILURE // state, and to stop passing along updates from the underlying subchannel. func (scw *subConnWrapper) eject() { - scw.scUpdateCh.Put(&ejectedUpdate{ - scw: scw, - ejected: true, + scw.scUpdateCh.Put(&ejectionUpdate{ + scw: scw, + isEjected: true, }) } @@ -59,8 +61,8 @@ func (scw *subConnWrapper) eject() { // from the underlying subchannel, and resume passing along updates from the // underlying subchannel. func (scw *subConnWrapper) uneject() { - scw.scUpdateCh.Put(&ejectedUpdate{ - scw: scw, - ejected: false, + scw.scUpdateCh.Put(&ejectionUpdate{ + scw: scw, + isEjected: false, }) } From 364f403766cfe3db184c9ca2508904ea1812fbc1 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Fri, 29 Jul 2022 13:53:46 -0400 Subject: [PATCH 09/22] Pruned comments --- .../balancer/outlierdetection/balancer.go | 120 ++---------------- 1 file changed, 8 insertions(+), 112 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 8e20c3297f7..5c535342a04 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -225,9 +225,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt b.mu.Lock() b.cfg = lbCfg - // When the outlier_detection LB policy receives an address update, it will - // create a map entry for each subchannel address in the list, and remove - // each map entry for a subchannel address not in the list. + addrs := make(map[string]bool, len(s.ResolverState.Addresses)) for _, addr := range s.ResolverState.Addresses { addrs[addr.Addr] = true @@ -241,9 +239,6 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt } } - // When a new config is provided, if the timer start timestamp is unset, set - // it to the current time and start the timer for the configured interval, - // then for each address, reset the call counters. var interval time.Duration if b.timerStartTime.IsZero() { b.timerStartTime = time.Now() @@ -252,10 +247,6 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt } interval = b.cfg.Interval } else { - // If the timer start timestamp is set, instead cancel the existing - // timer and start the timer for the configured interval minus the - // difference between the current time and the previous start timestamp, - // or 0 if that would be negative. interval = b.cfg.Interval - now().Sub(b.timerStartTime) if interval < 0 { interval = 0 @@ -291,7 +282,6 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt // most updated no-op config bit and most recent state from the child. b.inhibitPickerUpdates = true b.mu.Unlock() - // then pass the address list along to the child policy. b.childMu.Lock() err := b.child.UpdateClientConnState(balancer.ClientConnState{ ResolverState: s.ResolverState, @@ -304,7 +294,6 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt lbCfg: lbCfg, done: done, }) - // To make sure Picker updated synchronously. <-done return err } @@ -372,7 +361,9 @@ func (b *outlierDetectionBalancer) ExitIdle() { // wrappedPicker delegates to the child policy's picker, and when the request // finishes, it increments the corresponding counter in the map entry referenced -// by the subConnWrapper that was picked. +// by the subConnWrapper that was picked. If both the `success_rate_ejection` +// and `failure_percentage_ejection` fields are unset in the configuration, this +// picker will not count. type wrappedPicker struct { childPicker balancer.Picker noopPicker bool @@ -445,8 +436,6 @@ func (b *outlierDetectionBalancer) UpdateState(s balancer.State) { } func (b *outlierDetectionBalancer) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) { - // "When the child policy asks for a subchannel, the outlier_detection will - // wrap the subchannel with a wrapper." - A50 sc, err := b.cc.NewSubConn(addrs, opts) if err != nil { return nil, err @@ -468,9 +457,6 @@ func (b *outlierDetectionBalancer) NewSubConn(addrs []resolver.Address, opts bal } addrInfo.sws = append(addrInfo.sws, scw) atomic.StorePointer(&scw.addressInfo, unsafe.Pointer(addrInfo)) - - // "If that address is currently ejected, that subchannel wrapper's eject - // method will be called." - A50 if !addrInfo.latestEjectionTimestamp.IsZero() { scw.eject() } @@ -538,12 +524,8 @@ func (b *outlierDetectionBalancer) UpdateAddresses(sc balancer.SubConn, addrs [] if scw.addresses[0].Addr == addrs[0].Addr { return } - // 1. Remove Subchannel from Addresses map entry if present in Addresses map. b.removeSubConnFromAddressesMapEntry(scw) - // 2. Add Subchannel to Addresses map entry if new address present in map. addrInfo := b.appendIfPresent(addrs[0].Addr, scw) - // 3. Relay state with eject() recalculated (using the corresponding - // map entry to see if it's currently ejected). if addrInfo == nil { // uneject unconditionally because could have come from an ejected address scw.uneject() } else { @@ -554,17 +536,13 @@ func (b *outlierDetectionBalancer) UpdateAddresses(sc balancer.SubConn, addrs [] } } case len(scw.addresses) == 1: // single address to multiple/no addresses - // 1. Remove Subchannel from Addresses map entry if present in Addresses map. b.removeSubConnFromAddressesMapEntry(scw) - // 2. Clear the Subchannel wrapper's Call Counter entry. addrInfo := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) if addrInfo != nil { addrInfo.callCounter.clear() } - // 3. Uneject the Subchannel in case it was previously ejected. scw.uneject() case len(addrs) == 1: // multiple/no addresses to single address - // 1. Add Subchannel to Addresses map entry if new address present in map. addrInfo := b.appendIfPresent(addrs[0].Addr, scw) if addrInfo != nil && !addrInfo.latestEjectionTimestamp.IsZero() { scw.eject() @@ -615,16 +593,10 @@ func (b *outlierDetectionBalancer) handleEjectedUpdate(u *ejectionUpdate) { scw.ejected = u.isEjected var stateToUpdate balancer.SubConnState if u.isEjected { - // "The wrapper will report a state update with the - // TRANSIENT_FAILURE state, and will stop passing along - // updates from the underlying subchannel." stateToUpdate = balancer.SubConnState{ ConnectivityState: connectivity.TransientFailure, } } else { - // "The wrapper will report a state update with the latest - // update from the underlying subchannel, and resume passing - // along updates from the underlying subchannel." stateToUpdate = scw.latestState // If this has never been written to will send connectivity IDLE which seems fine to me } b.childMu.Lock() @@ -648,16 +620,9 @@ func (b *outlierDetectionBalancer) handleChildStateUpdate(u balancer.State) { b.recentPickerNoop = noopCfg b.cc.UpdateState(balancer.State{ ConnectivityState: b.childState.ConnectivityState, - // The outlier_detection LB policy will provide a picker that delegates to - // the child policy's picker, and when the request finishes, increment the - // corresponding counter in the map entry referenced by the subchannel - // wrapper that was picked. Picker: &wrappedPicker{ childPicker: b.childState.Picker, - // If both the `success_rate_ejection` and - // `failure_percentage_ejection` fields are unset in the - // configuration, the picker should not do that counting. - noopPicker: noopCfg, + noopPicker: noopCfg, }, }) } @@ -673,16 +638,9 @@ func (b *outlierDetectionBalancer) handleLBConfigUpdate(u lbCfgUpdate) { b.recentPickerNoop = noopCfg b.cc.UpdateState(balancer.State{ ConnectivityState: b.childState.ConnectivityState, - // The outlier_detection LB policy will provide a picker that delegates to - // the child policy's picker, and when the request finishes, increment the - // corresponding counter in the map entry referenced by the subchannel - // wrapper that was picked. Picker: &wrappedPicker{ childPicker: b.childState.Picker, - // If both the `success_rate_ejection` and - // `failure_percentage_ejection` fields are unset in the - // configuration, the picker should not do that counting. - noopPicker: noopCfg, + noopPicker: noopCfg, }, }) } @@ -718,36 +676,27 @@ func (b *outlierDetectionBalancer) run() { } } -// intervalTimerAlgorithm ejects and unejects addresses based on the outlier -// detection configuration and data about each address from the previous +// intervalTimerAlgorithm ejects and unejects addresses based on the Outlier +// Detection configuration and data about each address from the previous // interval. func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { b.mu.Lock() defer b.mu.Unlock() b.timerStartTime = time.Now() - // 2. For each address, swap the call counter's buckets in that address's - // map entry. for _, addrInfo := range b.addrs { addrInfo.callCounter.swap() } - // 3. If the success_rate_ejection configuration field is set, run the - // success rate algorithm. if b.cfg.SuccessRateEjection != nil { b.successRateAlgorithm() } - // 4. If the failure_percentage_ejection configuration field is set, run the - // failure percentage algorithm. if b.cfg.FailurePercentageEjection != nil { b.failurePercentageAlgorithm() } - // 5. For each address in the map: for _, addrInfo := range b.addrs { - // If the address is not ejected and the multiplier is greater than 0, - // decrease the multiplier by 1. if addrInfo.latestEjectionTimestamp.IsZero() && addrInfo.ejectionTimeMultiplier > 0 { addrInfo.ejectionTimeMultiplier-- continue @@ -755,10 +704,6 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { et := b.cfg.BaseEjectionTime.Nanoseconds() * addrInfo.ejectionTimeMultiplier met := max(b.cfg.BaseEjectionTime.Nanoseconds(), b.cfg.MaxEjectionTime.Nanoseconds()) curTimeAfterEt := now().After(addrInfo.latestEjectionTimestamp.Add(time.Duration(min(et, met)))) - // If the address is ejected, and the current time is after - // ejection_timestamp + min(base_ejection_time (type: time.Time) * - // multiplier (type: int), max(base_ejection_time (type: time.Time), - // max_ejection_time (type: time.Time))), un-eject the address. if !addrInfo.latestEjectionTimestamp.IsZero() && curTimeAfterEt { b.unejectAddress(addrInfo) } @@ -789,13 +734,9 @@ func (b *outlierDetectionBalancer) numAddrsWithAtLeastRequestVolume() uint32 { // meanAndStdDevOfSuccessesAtLeastRequestVolume returns the mean and std dev of // the number of requests of addresses that have at least requestVolume. func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume() (float64, float64) { - // 2. Calculate the mean and standard deviation of the fractions of - // successful requests among addresses with total request volume of at least - // success_rate_ejection.request_volume. var totalFractionOfSuccessfulRequests float64 var mean float64 for _, addrInfo := range b.addrs { - // "of at least success_rate_ejection.request_volume" if addrInfo.callCounter.inactiveBucket.requestVolume >= b.cfg.SuccessRateEjection.RequestVolume { totalFractionOfSuccessfulRequests += float64(addrInfo.callCounter.inactiveBucket.numSuccesses) / float64(addrInfo.callCounter.inactiveBucket.requestVolume) } @@ -806,7 +747,6 @@ func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume( devFromMean := (float64(addrInfo.callCounter.inactiveBucket.numSuccesses) / float64(addrInfo.callCounter.inactiveBucket.requestVolume)) - mean sumOfSquares += devFromMean * devFromMean } - variance := sumOfSquares / float64(len(b.addrs)) return mean, math.Sqrt(variance) } @@ -815,41 +755,21 @@ func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume( // the other addresses according to mean and standard deviation, and if overall // applicable from other set heuristics. func (b *outlierDetectionBalancer) successRateAlgorithm() { - // 1. If the number of addresses with request volume of at least - // success_rate_ejection.request_volume is less than - // success_rate_ejection.minimum_hosts, stop. if b.numAddrsWithAtLeastRequestVolume() < b.cfg.SuccessRateEjection.MinimumHosts { return } - - // 2. Calculate the mean and standard deviation of the fractions of - // successful requests among addresses with total request volume of at least - // success_rate_ejection.request_volume. mean, stddev := b.meanAndStdDevOfSuccessesAtLeastRequestVolume() - - // 3. For each address: for _, addrInfo := range b.addrs { ccb := addrInfo.callCounter.inactiveBucket sre := b.cfg.SuccessRateEjection - // i. If the percentage of ejected addresses is greater than - // max_ejection_percent, stop. if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { return } - - // ii. If the address's total request volume is less than - // success_rate_ejection.request_volume, continue to the next address. if ccb.requestVolume < sre.RequestVolume { continue } - - // iii. If the address's success rate is less than (mean - stdev * - // (success_rate_ejection.stdev_factor / 1000)) successRate := float64(ccb.numSuccesses) / float64(ccb.requestVolume) if successRate < (mean - stddev*(float64(sre.StdevFactor)/1000)) { - // then choose a random integer in [0, 100). If that number is less - // than success_rate_ejection.enforcement_percentage, eject that - // address. if uint32(grpcrand.Int31n(100)) < sre.EnforcementPercentage { b.ejectAddress(addrInfo) } @@ -861,34 +781,21 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { // rate exceeds a set enforcement percentage, if overall applicable from other // set heuristics. func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { - // 1. If the number of addresses is less than - // failure_percentage_ejection.minimum_hosts, stop. if uint32(len(b.addrs)) < b.cfg.FailurePercentageEjection.MinimumHosts { return } - // 2. For each address: for _, addrInfo := range b.addrs { ccb := addrInfo.callCounter.inactiveBucket fpe := b.cfg.FailurePercentageEjection - // i. If the percentage of ejected addresses is greater than - // max_ejection_percent, stop. if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { return } - // ii. If the address's total request volume is less than - // failure_percentage_ejection.request_volume, continue to the next - // address. if ccb.requestVolume < fpe.RequestVolume { continue } - // 2c. If the address's failure percentage is greater than - // failure_percentage_ejection.threshold failurePercentage := (float64(ccb.numFailures) / float64(ccb.requestVolume)) * 100 if failurePercentage > float64(b.cfg.FailurePercentageEjection.Threshold) { - // then choose a random integer in [0, 100). If that number is less - // than failiure_percentage_ejection.enforcement_percentage, eject - // that address. if uint32(grpcrand.Int31n(100)) < b.cfg.FailurePercentageEjection.EnforcementPercentage { b.ejectAddress(addrInfo) } @@ -898,11 +805,6 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { func (b *outlierDetectionBalancer) ejectAddress(addrInfo *addressInfo) { b.numAddrsEjected++ - - // To eject an address, set the current ejection timestamp to the timestamp - // that was recorded when the timer fired, increase the ejection time - // multiplier by 1, and call eject() on each subchannel wrapper in that - // address's subchannel wrapper list. addrInfo.latestEjectionTimestamp = b.timerStartTime addrInfo.ejectionTimeMultiplier++ for _, sbw := range addrInfo.sws { @@ -912,12 +814,6 @@ func (b *outlierDetectionBalancer) ejectAddress(addrInfo *addressInfo) { func (b *outlierDetectionBalancer) unejectAddress(addrInfo *addressInfo) { b.numAddrsEjected-- - - // To un-eject an address, set the current ejection timestamp to null - // (doesn't he mean latest ejection timestamp?, in Golang null for time is - // logically equivalent in practice to the time zero value) and call - // uneject() on each subchannel wrapper in that address's subchannel wrapper - // list. addrInfo.latestEjectionTimestamp = time.Time{} for _, sbw := range addrInfo.sws { sbw.uneject() From 512a642426cf15a735265ce65b9532aa31f36069 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Fri, 29 Jul 2022 15:24:19 -0400 Subject: [PATCH 10/22] Added test for atomic picker update for an UpdateClientConnState call --- .../outlierdetection/balancer_test.go | 102 ++++++++++++++++++ 1 file changed, 102 insertions(+) diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index 3693fd616cd..ad9ec89e9e2 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -289,6 +289,7 @@ func init() { balancer.Register(errParseConfigBuilder{}) balancer.Register(tcibBuilder{}) balancer.Register(verifyBalancerBuilder{}) + balancer.Register(twoUpdateStateBalancerBuilder{}) } type errParseConfigBuilder struct{} @@ -318,6 +319,7 @@ func setup(t *testing.T) (*outlierDetectionBalancer, *testutils.TestClientConn) const tcibname = "testClusterImplBalancer" const verifyBalancerName = "verifyBalancer" +const twoUpdateStateBalancerName = "twoUpdateStateBalancer" type tcibBuilder struct{} @@ -2264,3 +2266,103 @@ func (vb *verifyBalancer) ExitIdle() { vb.t.Fatal("ExitIdle was called after Close(), which breaks the balancer API") } } + +// TestUpdateClientConnStateSinglePickerUpdate tests that on an +// UpdateClientConnState call on the Outlier Detection Balancer, only a single +// picker update is sent back. +func (s) TestUpdateClientConnStateSinglePickerUpdate(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + od, tcc := setup(t) + defer func() { + od.Close() + internal.UnregisterOutlierDetectionBalancerForTesting() + }() + od.UpdateClientConnState(balancer.ClientConnState{ + BalancerConfig: &LBConfig{ + Interval: 10 * time.Second, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: twoUpdateStateBalancerName, + Config: twoUpdateStateBalancerConfig{}, + }, + }, + }) + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + // Should forward the connectivity State to Client Conn. + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case state := <-tcc.NewStateCh: + if state != connectivity.Ready { + t.Fatalf("ClientConn received connectivity state %v, want %v", state, connectivity.Ready) + } + } + + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case <-tcc.NewPickerCh: + } + + sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer sCancel() + + // Should only send one update. + select { + case <-tcc.NewStateCh: + t.Fatalf("only one picker update should have gotten sent") + case <-sCtx.Done(): + } + + sCtx, sCancel = context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer sCancel() + select { + case <-tcc.NewStateCh: + t.Fatalf("only one picker update should have gotten sent") + case <-sCtx.Done(): + } +} + +type twoUpdateStateBalancerBuilder struct{} + +func (twoUpdateStateBalancerBuilder) Build(cc balancer.ClientConn, opts balancer.BuildOptions) balancer.Balancer { + return &twoUpdateStateBalancer{ + cc: cc, + } +} + +func (twoUpdateStateBalancerBuilder) Name() string { + return "twoUpdateStateBalancer" +} + +type twoUpdateStateBalancerConfig struct { + serviceconfig.LoadBalancingConfig +} + +// twoUpdateStateBalancer sends two UpdateState calls inline in +// UpdateClientConnState(). This helps to verify that only a single picker +// update gets sent upward as a result of the call. +type twoUpdateStateBalancer struct { + t *testing.T + cc balancer.ClientConn +} + +func (tusb *twoUpdateStateBalancer) UpdateClientConnState(ccs balancer.ClientConnState) error { + tusb.cc.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &testutils.TestConstPicker{}, + }) + tusb.cc.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &testutils.TestConstPicker{}, + }) + return nil +} + +func (tusb *twoUpdateStateBalancer) ResolverError(err error) {} + +func (tusb *twoUpdateStateBalancer) UpdateSubConnState(sc balancer.SubConn, state balancer.SubConnState) { +} + +func (tusb *twoUpdateStateBalancer) Close() {} From 6be8dda0d7e620884cea6d59e1d28465722cc659 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Mon, 1 Aug 2022 18:58:34 -0400 Subject: [PATCH 11/22] Responded to Easwar's comments --- .../balancer/outlierdetection/balancer.go | 101 +++++++++++------- 1 file changed, 60 insertions(+), 41 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 5c535342a04..3956f256c97 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -154,8 +154,6 @@ type lbCfgUpdate struct { } type outlierDetectionBalancer struct { - numAddrsEjected int // For fast calculations of percentage of addrs ejected - childState balancer.State recentPickerNoop bool @@ -171,18 +169,20 @@ type outlierDetectionBalancer struct { childMu sync.Mutex child balancer.Balancer - // mu guards access to a lot of the core LB Policy State. It also prevents - // intersplicing certain operations. Some of the undefined behaviors this - // mutex protects are: + // mu guards access to the following fields. It also helps to synchronize + // behaviors of the following events: config updates, firing of the interval + // timer, SubConn State updates, SubConn address updates, and child state + // updates. // - // ex 1: interval timer goes off, outlier detection algorithm starts running - // based on knobs in cfg. in the middle of running the algorithm, a - // ClientConn update comes in and writes to cfg. This causes undefined - // behavior for the interval timer algorithm. + // For example, when we receive a config update in the middle of the + // interval timer algorithm, which uses knobs present in the config, the + // balancer will wait for the interval timer algorithm to finish before + // persisting the new configuration. // - // ex 2: Updating the addrs map from UpdateAddresses in the middle of - // running the interval timer algorithm which uses addrs heavily. This will - // cause undefined behavior for the interval timer algorithm. + // Another example would be the updating of the addrs map, such as from a + // SubConn address update in the middle of the interval timer algorithm + // which uses addrs. This balancer waits for the interval timer algorithm to + // finish before making the update to the addrs map. mu sync.Mutex addrs map[string]*addressInfo cfg *LBConfig @@ -190,6 +190,7 @@ type outlierDetectionBalancer struct { timerStartTime time.Time intervalTimer *time.Timer inhibitPickerUpdates bool + numAddrsEjected int // For fast calculations of percentage of addrs ejected scUpdateCh *buffer.Unbounded pickerUpdateCh *buffer.Unbounded @@ -385,6 +386,8 @@ func (wp *wrappedPicker) Pick(info balancer.PickInfo) (balancer.PickResult, erro } scw, ok := pr.SubConn.(*subConnWrapper) if !ok { + // This can never happen, but check is present for defensive + // programming. return balancer.PickResult{ SubConn: pr.SubConn, Done: done, @@ -591,13 +594,13 @@ func (b *outlierDetectionBalancer) handleSubConnUpdate(u *scUpdate) { func (b *outlierDetectionBalancer) handleEjectedUpdate(u *ejectionUpdate) { scw := u.scw scw.ejected = u.isEjected - var stateToUpdate balancer.SubConnState + // If scw.latestState has never been written to will default to connectivity + // IDLE, which is fine. + stateToUpdate := scw.latestState if u.isEjected { stateToUpdate = balancer.SubConnState{ ConnectivityState: connectivity.TransientFailure, } - } else { - stateToUpdate = scw.latestState // If this has never been written to will send connectivity IDLE which seems fine to me } b.childMu.Lock() if b.child != nil { @@ -701,10 +704,15 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { addrInfo.ejectionTimeMultiplier-- continue } + if addrInfo.latestEjectionTimestamp.IsZero() { + // Address is already not ejected, so no need to check for whether + // to uneject the address below. + continue + } et := b.cfg.BaseEjectionTime.Nanoseconds() * addrInfo.ejectionTimeMultiplier met := max(b.cfg.BaseEjectionTime.Nanoseconds(), b.cfg.MaxEjectionTime.Nanoseconds()) curTimeAfterEt := now().After(addrInfo.latestEjectionTimestamp.Add(time.Duration(min(et, met)))) - if !addrInfo.latestEjectionTimestamp.IsZero() && curTimeAfterEt { + if curTimeAfterEt { b.unejectAddress(addrInfo) } } @@ -731,23 +739,36 @@ func (b *outlierDetectionBalancer) numAddrsWithAtLeastRequestVolume() uint32 { return numAddrs } -// meanAndStdDevOfSuccessesAtLeastRequestVolume returns the mean and std dev of -// the number of requests of addresses that have at least requestVolume. -func (b *outlierDetectionBalancer) meanAndStdDevOfSuccessesAtLeastRequestVolume() (float64, float64) { - var totalFractionOfSuccessfulRequests float64 - var mean float64 +// addrsWithAtLeastRequestVolume returns a slice of address information of all +// addresses with at least request volume defined in the success rate ejection +// configuration. +func (b *outlierDetectionBalancer) addrsWithAtLeastRequestVolume() []*addressInfo { + var addrs []*addressInfo for _, addrInfo := range b.addrs { if addrInfo.callCounter.inactiveBucket.requestVolume >= b.cfg.SuccessRateEjection.RequestVolume { - totalFractionOfSuccessfulRequests += float64(addrInfo.callCounter.inactiveBucket.numSuccesses) / float64(addrInfo.callCounter.inactiveBucket.requestVolume) + addrs = append(addrs, addrInfo) } } - mean = totalFractionOfSuccessfulRequests / float64(len(b.addrs)) + return addrs +} + +// meanAndStdDev returns the mean and std dev of the fractions of successful +// requests of the addresses passed in. +func (b *outlierDetectionBalancer) meanAndStdDev(addrs []*addressInfo) (float64, float64) { + var totalFractionOfSuccessfulRequests float64 + var mean float64 + for _, addrInfo := range addrs { + ib := addrInfo.callCounter.inactiveBucket + totalFractionOfSuccessfulRequests += float64(ib.numSuccesses) / float64(ib.requestVolume) + } + mean = totalFractionOfSuccessfulRequests / float64(len(addrs)) var sumOfSquares float64 - for _, addrInfo := range b.addrs { - devFromMean := (float64(addrInfo.callCounter.inactiveBucket.numSuccesses) / float64(addrInfo.callCounter.inactiveBucket.requestVolume)) - mean + for _, addrInfo := range addrs { + ib := addrInfo.callCounter.inactiveBucket + devFromMean := (float64(ib.numSuccesses) / float64(ib.requestVolume)) - mean sumOfSquares += devFromMean * devFromMean } - variance := sumOfSquares / float64(len(b.addrs)) + variance := sumOfSquares / float64(len(addrs)) return mean, math.Sqrt(variance) } @@ -758,19 +779,17 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { if b.numAddrsWithAtLeastRequestVolume() < b.cfg.SuccessRateEjection.MinimumHosts { return } - mean, stddev := b.meanAndStdDevOfSuccessesAtLeastRequestVolume() - for _, addrInfo := range b.addrs { - ccb := addrInfo.callCounter.inactiveBucket - sre := b.cfg.SuccessRateEjection + awalrv := b.addrsWithAtLeastRequestVolume() + mean, stddev := b.meanAndStdDev(awalrv) + for _, addrInfo := range awalrv { + bucket := addrInfo.callCounter.inactiveBucket + ejectionCfg := b.cfg.SuccessRateEjection if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { return } - if ccb.requestVolume < sre.RequestVolume { - continue - } - successRate := float64(ccb.numSuccesses) / float64(ccb.requestVolume) - if successRate < (mean - stddev*(float64(sre.StdevFactor)/1000)) { - if uint32(grpcrand.Int31n(100)) < sre.EnforcementPercentage { + successRate := float64(bucket.numSuccesses) / float64(bucket.requestVolume) + if successRate < (mean - stddev*(float64(ejectionCfg.StdevFactor)/1000)) { + if uint32(grpcrand.Int31n(100)) < ejectionCfg.EnforcementPercentage { b.ejectAddress(addrInfo) } } @@ -786,17 +805,17 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { } for _, addrInfo := range b.addrs { - ccb := addrInfo.callCounter.inactiveBucket - fpe := b.cfg.FailurePercentageEjection + bucket := addrInfo.callCounter.inactiveBucket + ejectionCfg := b.cfg.FailurePercentageEjection if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { return } - if ccb.requestVolume < fpe.RequestVolume { + if bucket.requestVolume < ejectionCfg.RequestVolume { continue } - failurePercentage := (float64(ccb.numFailures) / float64(ccb.requestVolume)) * 100 + failurePercentage := (float64(bucket.numFailures) / float64(bucket.requestVolume)) * 100 if failurePercentage > float64(b.cfg.FailurePercentageEjection.Threshold) { - if uint32(grpcrand.Int31n(100)) < b.cfg.FailurePercentageEjection.EnforcementPercentage { + if uint32(grpcrand.Int31n(100)) < ejectionCfg.EnforcementPercentage { b.ejectAddress(addrInfo) } } From 8e0d085485b994d53d79d4a13e5e7023b307bd99 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Mon, 15 Aug 2022 21:05:13 -0400 Subject: [PATCH 12/22] fix correctness issue and get rid of allocation --- .../balancer/outlierdetection/balancer.go | 26 ++++++++++------- .../outlierdetection/balancer_test.go | 15 ++++------ .../balancer/outlierdetection/callcounter.go | 29 +++++-------------- .../outlierdetection/callcounter_test.go | 17 ++++------- 4 files changed, 36 insertions(+), 51 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 3956f256c97..00d48356ac2 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -431,7 +431,6 @@ func incrementCounter(sc balancer.SubConn, info balancer.DoneInfo) { } else { atomic.AddUint32(&ab.numFailures, 1) } - atomic.AddUint32(&ab.requestVolume, 1) } func (b *outlierDetectionBalancer) UpdateState(s balancer.State) { @@ -732,7 +731,9 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { func (b *outlierDetectionBalancer) numAddrsWithAtLeastRequestVolume() uint32 { var numAddrs uint32 for _, addrInfo := range b.addrs { - if addrInfo.callCounter.inactiveBucket.requestVolume >= b.cfg.SuccessRateEjection.RequestVolume { + bucket := addrInfo.callCounter.inactiveBucket + rv := bucket.numSuccesses + bucket.numFailures + if rv >= b.cfg.SuccessRateEjection.RequestVolume { numAddrs++ } } @@ -745,7 +746,9 @@ func (b *outlierDetectionBalancer) numAddrsWithAtLeastRequestVolume() uint32 { func (b *outlierDetectionBalancer) addrsWithAtLeastRequestVolume() []*addressInfo { var addrs []*addressInfo for _, addrInfo := range b.addrs { - if addrInfo.callCounter.inactiveBucket.requestVolume >= b.cfg.SuccessRateEjection.RequestVolume { + bucket := addrInfo.callCounter.inactiveBucket + rv := bucket.numSuccesses + bucket.numFailures + if rv >= b.cfg.SuccessRateEjection.RequestVolume { addrs = append(addrs, addrInfo) } } @@ -758,14 +761,16 @@ func (b *outlierDetectionBalancer) meanAndStdDev(addrs []*addressInfo) (float64, var totalFractionOfSuccessfulRequests float64 var mean float64 for _, addrInfo := range addrs { - ib := addrInfo.callCounter.inactiveBucket - totalFractionOfSuccessfulRequests += float64(ib.numSuccesses) / float64(ib.requestVolume) + bucket := addrInfo.callCounter.inactiveBucket + rv := bucket.numSuccesses + bucket.numFailures + totalFractionOfSuccessfulRequests += float64(bucket.numSuccesses) / float64(rv) } mean = totalFractionOfSuccessfulRequests / float64(len(addrs)) var sumOfSquares float64 for _, addrInfo := range addrs { - ib := addrInfo.callCounter.inactiveBucket - devFromMean := (float64(ib.numSuccesses) / float64(ib.requestVolume)) - mean + bucket := addrInfo.callCounter.inactiveBucket + rv := bucket.numSuccesses + bucket.numFailures + devFromMean := (float64(bucket.numSuccesses) / float64(rv)) - mean sumOfSquares += devFromMean * devFromMean } variance := sumOfSquares / float64(len(addrs)) @@ -787,7 +792,7 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { return } - successRate := float64(bucket.numSuccesses) / float64(bucket.requestVolume) + successRate := float64(bucket.numSuccesses) / float64(bucket.numSuccesses+bucket.numFailures) if successRate < (mean - stddev*(float64(ejectionCfg.StdevFactor)/1000)) { if uint32(grpcrand.Int31n(100)) < ejectionCfg.EnforcementPercentage { b.ejectAddress(addrInfo) @@ -810,10 +815,11 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { return } - if bucket.requestVolume < ejectionCfg.RequestVolume { + rv := bucket.numSuccesses + bucket.numFailures + if rv < ejectionCfg.RequestVolume { continue } - failurePercentage := (float64(bucket.numFailures) / float64(bucket.requestVolume)) * 100 + failurePercentage := (float64(bucket.numFailures) / float64(rv)) * 100 if failurePercentage > float64(b.cfg.FailurePercentageEjection.Threshold) { if uint32(grpcrand.Int31n(100)) < ejectionCfg.EnforcementPercentage { b.ejectAddress(addrInfo) diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index ad9ec89e9e2..bf6a20c518a 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -976,9 +976,8 @@ func (s) TestPicker(t *testing.T) { t.Fatal("map entry for address: address1 not present in map") } bucketWant := &bucket{ - numSuccesses: 1, - numFailures: 1, - requestVolume: 2, + numSuccesses: 1, + numFailures: 1, } if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call from picker t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) @@ -1051,9 +1050,8 @@ func (s) TestPicker(t *testing.T) { // in between ClientConn updates and the picker should not count, as the // outlier detection balancer is configured with a no-op configuration. bucketWant := &bucket{ - numSuccesses: 1, - numFailures: 1, - requestVolume: 2, + numSuccesses: 1, + numFailures: 1, } if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call od.mu.Unlock() @@ -1105,9 +1103,8 @@ func (s) TestPicker(t *testing.T) { // in between ClientConn updates and the picker should not count, as the // outlier detection balancer is configured with a no-op configuration. bucketWant := &bucket{ - numSuccesses: 1, - numFailures: 1, - requestVolume: 2, + numSuccesses: 1, + numFailures: 1, } if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) diff --git a/xds/internal/balancer/outlierdetection/callcounter.go b/xds/internal/balancer/outlierdetection/callcounter.go index 25163a9c420..525f09dd1d3 100644 --- a/xds/internal/balancer/outlierdetection/callcounter.go +++ b/xds/internal/balancer/outlierdetection/callcounter.go @@ -23,9 +23,8 @@ import ( ) type bucket struct { - numSuccesses uint32 - numFailures uint32 - requestVolume uint32 // numSuccesses + numFailures, needed because this number will be used in interval timer algorithm + numSuccesses uint32 + numFailures uint32 } func newCallCounter() *callCounter { @@ -43,7 +42,7 @@ type callCounter struct { // activeBucket updates every time a call finishes (from picker passed to // Client Conn), so protect pointer read with atomic load of unsafe.Pointer // so picker does not have to grab a mutex per RPC, the critical path. - activeBucket unsafe.Pointer + activeBucket unsafe.Pointer // bucket inactiveBucket *bucket } @@ -57,23 +56,11 @@ func (cc *callCounter) clear() { // failures since the last time the timer triggered. Those numbers are used to // evaluate the ejection criteria." - A50 func (cc *callCounter) swap() { - ab := (*bucket)(atomic.LoadPointer(&cc.activeBucket)) - // Don't do it exactly like defined but the same logically, as picker reads - // ref to active bucket so instead of swapping the pointers (inducing race - // conditions where picker writes to inactive bucket which is being used for - // outlier detection algorithm, copy active bucket to new memory on heap, - // picker updates which race simply write to deprecated heap memory - // activeBucket used to point to). The other options is to do this as - // defined, swap the pointers and have atomic reads of the Inactive Bucket - // in interval timer algorithm, but I think this is cleaner in regards to - // dealing with picker race condition. See the wrappedPicker explanation for - // the write to activeBucket for a more in depth explanation. + ib := cc.inactiveBucket + *ib = bucket{} + ab := (*bucket)(atomic.SwapPointer(&cc.activeBucket, unsafe.Pointer(ib))) cc.inactiveBucket = &bucket{ - numSuccesses: atomic.LoadUint32(&ab.numSuccesses), - numFailures: atomic.LoadUint32(&ab.numFailures), - requestVolume: atomic.LoadUint32(&ab.requestVolume), + numSuccesses: atomic.LoadUint32(&ab.numSuccesses), + numFailures: atomic.LoadUint32(&ab.numFailures), } - atomic.StorePointer(&cc.activeBucket, unsafe.Pointer(&bucket{})) - // end result, same as in gRFC: the inactive bucket contains the number of - // successes and failures since the last time the timer triggered. } diff --git a/xds/internal/balancer/outlierdetection/callcounter_test.go b/xds/internal/balancer/outlierdetection/callcounter_test.go index 638e252e146..8e4f5f29b5f 100644 --- a/xds/internal/balancer/outlierdetection/callcounter_test.go +++ b/xds/internal/balancer/outlierdetection/callcounter_test.go @@ -20,6 +20,7 @@ package outlierdetection import ( "sync/atomic" "testing" + "unsafe" "github.com/google/go-cmp/cmp" ) @@ -34,10 +35,7 @@ func (b1 *bucket) Equal(b2 *bucket) bool { if b1.numSuccesses != b2.numSuccesses { return false } - if b1.numFailures != b2.numFailures { - return false - } - return b1.requestVolume == b2.requestVolume + return b1.numFailures == b2.numFailures } func (cc1 *callCounter) Equal(cc2 *callCounter) bool { @@ -62,10 +60,8 @@ func (s) TestClear(t *testing.T) { ab := (*bucket)(atomic.LoadPointer(&cc.activeBucket)) ab.numSuccesses = 1 ab.numFailures = 2 - ab.requestVolume = 3 cc.inactiveBucket.numSuccesses = 4 cc.inactiveBucket.numFailures = 5 - cc.inactiveBucket.requestVolume = 9 cc.clear() // Both the active and inactive buckets should be cleared. ccWant := newCallCounter() @@ -82,17 +78,16 @@ func (s) TestSwap(t *testing.T) { ab := (*bucket)(atomic.LoadPointer(&cc.activeBucket)) ab.numSuccesses = 1 ab.numFailures = 2 - ab.requestVolume = 3 cc.inactiveBucket.numSuccesses = 4 cc.inactiveBucket.numFailures = 5 - cc.inactiveBucket.requestVolume = 9 - + ib := cc.inactiveBucket cc.swap() - // Inactive should pick up active's data, active should be cleared. + // Inactive should pick up active's data, active should be swapped to zeroed + // inactive. ccWant := newCallCounter() ccWant.inactiveBucket.numSuccesses = 1 ccWant.inactiveBucket.numFailures = 2 - ccWant.inactiveBucket.requestVolume = 3 + atomic.StorePointer(&ccWant.activeBucket, unsafe.Pointer(ib)) if diff := cmp.Diff(cc, ccWant); diff != "" { t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) } From 15d53cf75dd3955469039d2e734f497022d22bb4 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Tue, 16 Aug 2022 16:31:48 -0400 Subject: [PATCH 13/22] First part of Easwar's comments --- test/xds/xds_client_outlier_detection_test.go | 19 ++++++++++--------- .../balancer/outlierdetection/balancer.go | 3 ++- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/test/xds/xds_client_outlier_detection_test.go b/test/xds/xds_client_outlier_detection_test.go index 56660ae9e78..b53439cf66c 100644 --- a/test/xds/xds_client_outlier_detection_test.go +++ b/test/xds/xds_client_outlier_detection_test.go @@ -90,12 +90,12 @@ func (s) TestOutlierDetection_NoopConfig(t *testing.T) { } } -// defaultClientResourcesMultipleBackendsAndOD returns xDS resources which -// correspond to multiple upstreams, corresponding different backends listening -// on different localhost:port combinations. The resources also configure an +// clientResourcesMultipleBackendsAndOD returns xDS resources which correspond +// to multiple upstreams, corresponding different backends listening on +// different localhost:port combinations. The resources also configure an // Outlier Detection Balancer set up with Failure Percentage Algorithm, which // ejects endpoints based on failure rate. -func defaultClientResourcesMultipleBackendsAndOD(params e2e.ResourceParams, ports []uint32) e2e.UpdateOptions { +func clientResourcesMultipleBackendsAndOD(params e2e.ResourceParams, ports []uint32) e2e.UpdateOptions { routeConfigName := "route-" + params.DialTarget clusterName := "cluster-" + params.DialTarget endpointsName := "endpoints-" + params.DialTarget @@ -103,12 +103,12 @@ func defaultClientResourcesMultipleBackendsAndOD(params e2e.ResourceParams, port NodeID: params.NodeID, Listeners: []*v3listenerpb.Listener{e2e.DefaultClientListener(params.DialTarget, routeConfigName)}, Routes: []*v3routepb.RouteConfiguration{e2e.DefaultRouteConfig(routeConfigName, params.DialTarget, clusterName)}, - Clusters: []*v3clusterpb.Cluster{defaultClusterWithOutlierDetection(clusterName, endpointsName, params.SecLevel)}, + Clusters: []*v3clusterpb.Cluster{clusterWithOutlierDetection(clusterName, endpointsName, params.SecLevel)}, Endpoints: []*v3endpointpb.ClusterLoadAssignment{e2e.DefaultEndpoint(endpointsName, params.Host, ports)}, } } -func defaultClusterWithOutlierDetection(clusterName, edsServiceName string, secLevel e2e.SecurityLevel) *v3clusterpb.Cluster { +func clusterWithOutlierDetection(clusterName, edsServiceName string, secLevel e2e.SecurityLevel) *v3clusterpb.Cluster { cluster := e2e.DefaultCluster(clusterName, edsServiceName, secLevel) cluster.OutlierDetection = &v3clusterpb.OutlierDetection{ Interval: &durationpb.Duration{Nanos: 50000000}, // .5 seconds @@ -140,7 +140,7 @@ func (s) TestOutlierDetectionWithOutlier(t *testing.T) { managementServer, nodeID, _, resolver, cleanup := e2e.SetupManagementServer(t, nil) defer cleanup() - // counters for how many times backends got called + // Counters for how many times backends got called. var count1, count2, count3 int // Working backend 1. @@ -162,6 +162,7 @@ func (s) TestOutlierDetectionWithOutlier(t *testing.T) { Address: "localhost:0", }) defer cleanup2() + // Backend 3 that will always return an error and eventually ejected. port3, cleanup3 := startTestService(t, &stubserver.StubServer{ EmptyCallF: func(context.Context, *testpb.Empty) (*testpb.Empty, error) { @@ -173,13 +174,13 @@ func (s) TestOutlierDetectionWithOutlier(t *testing.T) { defer cleanup3() const serviceName = "my-service-client-side-xds" - resources := defaultClientResourcesMultipleBackendsAndOD(e2e.ResourceParams{ + resources := clientResourcesMultipleBackendsAndOD(e2e.ResourceParams{ DialTarget: serviceName, NodeID: nodeID, Host: "localhost", SecLevel: e2e.SecurityLevelNone, }, []uint32{port1, port2, port3}) - ctx, cancel := context.WithTimeout(context.Background(), time.Second*100) + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() if err := managementServer.Update(ctx, resources); err != nil { t.Fatal(err) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 00d48356ac2..8d89f456a52 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -209,7 +209,8 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt return balancer.ErrBadResolverState } - // Reject whole config if any errors, don't persist it for later + // Reject whole config if child policy doesn't exist, don't persist it for + // later. bb := balancer.Get(lbCfg.ChildPolicy.Name) if bb == nil { return fmt.Errorf("outlier detection: child balancer %q not registered", lbCfg.ChildPolicy.Name) From 36b2e0b0d5b678e3716a41ccb0098c6af54c20fb Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Thu, 18 Aug 2022 09:24:08 -0400 Subject: [PATCH 14/22] switched child to gsb, switched testing to use stubs, added e2e tests, and got to all of Easwar's comments --- .../balancer/outlierdetection/balancer.go | 191 +- .../outlierdetection/balancer_test.go | 2348 +++++++---------- .../balancer/outlierdetection/callcounter.go | 8 +- 3 files changed, 1011 insertions(+), 1536 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 8d89f456a52..35053e9f19c 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -34,6 +34,7 @@ import ( "google.golang.org/grpc/balancer" "google.golang.org/grpc/connectivity" "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/balancer/gracefulswitch" "google.golang.org/grpc/internal/buffer" "google.golang.org/grpc/internal/envconfig" "google.golang.org/grpc/internal/grpclog" @@ -79,6 +80,7 @@ func (bb) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Ba } b.logger = prefixLogger(b) b.logger.Infof("Created") + b.child = gracefulswitch.NewBalancer(b, bOpts) go b.run() return b } @@ -156,6 +158,7 @@ type lbCfgUpdate struct { type outlierDetectionBalancer struct { childState balancer.State recentPickerNoop bool + firstPickerSent bool closed *grpcsync.Event cc balancer.ClientConn @@ -167,7 +170,7 @@ type outlierDetectionBalancer struct { // against run() reading that the child is not nil for SubConn updates, and // then UpdateClientConnState or Close writing to the the child. childMu sync.Mutex - child balancer.Balancer + child *gracefulswitch.Balancer // mu guards access to the following fields. It also helps to synchronize // behaviors of the following events: config updates, firing of the interval @@ -197,11 +200,49 @@ type outlierDetectionBalancer struct { } // noopConfig returns whether this balancer is configured with a logical no-op -// configuration or not. +// configuration or not. Caller must hold b.mu. func (b *outlierDetectionBalancer) noopConfig() bool { return b.cfg.SuccessRateEjection == nil && b.cfg.FailurePercentageEjection == nil } +// onCountingConfig handles logic required specifically on the receipt of a +// configuration which will count RPC's. Caller must hold b.mu. +func (b *outlierDetectionBalancer) onCountingConfig() { + var interval time.Duration + if b.timerStartTime.IsZero() { + b.timerStartTime = time.Now() + for _, addrInfo := range b.addrs { + addrInfo.callCounter.clear() + } + interval = b.cfg.Interval + } else { + interval = b.cfg.Interval - now().Sub(b.timerStartTime) + if interval < 0 { + interval = 0 + } + } + b.intervalTimer = afterFunc(interval, b.intervalTimerAlgorithm) +} + +// onCountingConfig handles logic required specifically on the receipt of a +// configuration which specifies the balancer to be a noop. Caller must hold +// b.mu. +func (b *outlierDetectionBalancer) onNoopConfig() { + // "If a config is provided with both the `success_rate_ejection` and + // `failure_percentage_ejection` fields unset, skip starting the timer and + // do the following:" + // "Unset the timer start timestamp." + b.timerStartTime = time.Time{} + for _, addrInfo := range b.addrs { + // "Uneject all currently ejected addresses." + if !addrInfo.latestEjectionTimestamp.IsZero() { + b.unejectAddress(addrInfo) + } + // "Reset each address's ejection time multiplier to 0." + addrInfo.ejectionTimeMultiplier = 0 + } +} + func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnState) error { lbCfg, ok := s.BalancerConfig.(*LBConfig) if !ok { @@ -216,16 +257,23 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt return fmt.Errorf("outlier detection: child balancer %q not registered", lbCfg.ChildPolicy.Name) } - if b.child == nil || b.cfg.ChildPolicy.Name != lbCfg.ChildPolicy.Name { + if b.cfg == nil || b.cfg.ChildPolicy.Name != lbCfg.ChildPolicy.Name { b.childMu.Lock() - if b.child != nil { - b.child.Close() + err := b.child.SwitchTo(bb) + if err != nil { + b.childMu.Unlock() + return err } - b.child = bb.Build(b, b.bOpts) b.childMu.Unlock() } b.mu.Lock() + // Inhibit child picker updates until this UpdateClientConnState() call + // completes. If needed, a picker update containing the no-op config bit + // determined from this config and most recent state from the child will be + // sent synchronously upward at the end of this UpdateClientConnState() + // call. + b.inhibitPickerUpdates = true b.cfg = lbCfg addrs := make(map[string]bool, len(s.ResolverState.Addresses)) @@ -241,49 +289,17 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt } } - var interval time.Duration - if b.timerStartTime.IsZero() { - b.timerStartTime = time.Now() - for _, addrInfo := range b.addrs { - addrInfo.callCounter.clear() - } - interval = b.cfg.Interval - } else { - interval = b.cfg.Interval - now().Sub(b.timerStartTime) - if interval < 0 { - interval = 0 - } - } - if b.intervalTimer != nil { b.intervalTimer.Stop() } + if !b.noopConfig() { - b.intervalTimer = afterFunc(interval, func() { - b.intervalTimerAlgorithm() - }) + b.onCountingConfig() } else { - // "If a config is provided with both the `success_rate_ejection` and - // `failure_percentage_ejection` fields unset, skip starting the timer and - // do the following:" - // "Unset the timer start timestamp." - b.timerStartTime = time.Time{} - for _, addrInfo := range b.addrs { - // "Uneject all currently ejected addresses." - if !addrInfo.latestEjectionTimestamp.IsZero() { - b.unejectAddress(addrInfo) - } - // "Reset each address's ejection time multiplier to 0." - addrInfo.ejectionTimeMultiplier = 0 - } + b.onNoopConfig() } - - // Inhibit picker updates until run() processes the lbCfgUpdate. This - // makes sure a single picker update gets sent out synchronously as a result - // of this UpdateClientConnState call, and this picker update contains the - // most updated no-op config bit and most recent state from the child. - b.inhibitPickerUpdates = true b.mu.Unlock() + b.childMu.Lock() err := b.child.UpdateClientConnState(balancer.ClientConnState{ ResolverState: s.ResolverState, @@ -297,6 +313,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt done: done, }) <-done + return err } @@ -337,7 +354,6 @@ func (b *outlierDetectionBalancer) Close() { b.childMu.Unlock() } - // Any other cleanup needs to happen (subconns, other resources?) b.mu.Lock() defer b.mu.Unlock() if b.intervalTimer != nil { @@ -349,16 +365,9 @@ func (b *outlierDetectionBalancer) ExitIdle() { if b.child == nil { return } - if ei, ok := b.child.(balancer.ExitIdler); ok { - b.childMu.Lock() - defer b.childMu.Unlock() - ei.ExitIdle() - return - } - - // Fallback for children handled in clusterimpl balancer Removing SubConns - // is defined in API and also in graceful switch balancer, but already done - // in ClusterImpl. I guess we should do that here? + b.childMu.Lock() + defer b.childMu.Unlock() + b.child.ExitIdle() } // wrappedPicker delegates to the child policy's picker, and when the request @@ -416,11 +425,9 @@ func incrementCounter(sc balancer.SubConn, info balancer.DoneInfo) { // UpdateAddresses can switch the addressInfo the scw points to. Writing to // an outdated addresses is a very small race and tolerable. After reading // callCounter.activeBucket in this picker a swap call can concurrently - // change what activeBucket points to. A50 says to swap the pointer, but I - // decided to make create new memory for both active and inactive bucket, - // and have this race instead write to deprecated memory. If you swap the - // pointers, this write would write to the inactive buckets memory, which is - // read throughout in the interval timers algorithm. + // change what activeBucket points to. A50 says to swap the pointer, which + // will cause this race to write to deprecated memory the interval timer + // algorithm will never read, which makes this race alright. addrInfo := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) if addrInfo == nil { return @@ -479,7 +486,7 @@ func (b *outlierDetectionBalancer) RemoveSubConn(sc balancer.SubConn) { // appendIfPresent appends the scw to the address, if the address is present in // the Outlier Detection balancers address map. Returns nil if not present, and -// the map entry if present. +// the map entry if present. Caller must hold b.mu. func (b *outlierDetectionBalancer) appendIfPresent(addr string, scw *subConnWrapper) *addressInfo { addrInfo, ok := b.addrs[addr] if !ok { @@ -492,7 +499,7 @@ func (b *outlierDetectionBalancer) appendIfPresent(addr string, scw *subConnWrap } // removeSubConnFromAddressesMapEntry removes the scw from its map entry if -// present. +// present. Caller must hold b.mu. func (b *outlierDetectionBalancer) removeSubConnFromAddressesMapEntry(scw *subConnWrapper) { addrInfo := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) if addrInfo == nil { @@ -501,7 +508,7 @@ func (b *outlierDetectionBalancer) removeSubConnFromAddressesMapEntry(scw *subCo for i, sw := range addrInfo.sws { if scw == sw { addrInfo.sws = append(addrInfo.sws[:i], addrInfo.sws[i+1:]...) - break + return } } } @@ -531,12 +538,12 @@ func (b *outlierDetectionBalancer) UpdateAddresses(sc balancer.SubConn, addrs [] addrInfo := b.appendIfPresent(addrs[0].Addr, scw) if addrInfo == nil { // uneject unconditionally because could have come from an ejected address scw.uneject() + break + } + if addrInfo.latestEjectionTimestamp.IsZero() { // relay new updated subconn state + scw.uneject() } else { - if addrInfo.latestEjectionTimestamp.IsZero() { // relay new updated subconn state - scw.uneject() - } else { - scw.eject() - } + scw.eject() } case len(scw.addresses) == 1: // single address to multiple/no addresses b.removeSubConnFromAddressesMapEntry(scw) @@ -621,6 +628,7 @@ func (b *outlierDetectionBalancer) handleChildStateUpdate(u balancer.State) { noopCfg := b.noopConfig() b.mu.Unlock() b.recentPickerNoop = noopCfg + b.firstPickerSent = true b.cc.UpdateState(balancer.State{ ConnectivityState: b.childState.ConnectivityState, Picker: &wrappedPicker{ @@ -637,8 +645,9 @@ func (b *outlierDetectionBalancer) handleLBConfigUpdate(u lbCfgUpdate) { lbCfg := u.lbCfg done := u.done noopCfg := lbCfg.SuccessRateEjection == nil && lbCfg.FailurePercentageEjection == nil - if b.childState.Picker != nil && noopCfg != b.recentPickerNoop { + if b.childState.Picker != nil && noopCfg != b.recentPickerNoop || b.childState.Picker != nil && !b.firstPickerSent { b.recentPickerNoop = noopCfg + b.firstPickerSent = true b.cc.UpdateState(balancer.State{ ConnectivityState: b.childState.ConnectivityState, Picker: &wrappedPicker{ @@ -722,28 +731,12 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { if b.intervalTimer != nil { b.intervalTimer.Stop() } - b.intervalTimer = afterFunc(b.cfg.Interval, func() { - b.intervalTimerAlgorithm() - }) -} - -// numAddrsWithAtLeastRequestVolume returns the number of addresses present in -// the map that have request volume of at least requestVolume. -func (b *outlierDetectionBalancer) numAddrsWithAtLeastRequestVolume() uint32 { - var numAddrs uint32 - for _, addrInfo := range b.addrs { - bucket := addrInfo.callCounter.inactiveBucket - rv := bucket.numSuccesses + bucket.numFailures - if rv >= b.cfg.SuccessRateEjection.RequestVolume { - numAddrs++ - } - } - return numAddrs + b.intervalTimer = afterFunc(b.cfg.Interval, b.intervalTimerAlgorithm) } // addrsWithAtLeastRequestVolume returns a slice of address information of all // addresses with at least request volume defined in the success rate ejection -// configuration. +// configuration. Caller must hold b.mu. func (b *outlierDetectionBalancer) addrsWithAtLeastRequestVolume() []*addressInfo { var addrs []*addressInfo for _, addrInfo := range b.addrs { @@ -757,7 +750,7 @@ func (b *outlierDetectionBalancer) addrsWithAtLeastRequestVolume() []*addressInf } // meanAndStdDev returns the mean and std dev of the fractions of successful -// requests of the addresses passed in. +// requests of the addresses passed in. Caller must hold b.mu. func (b *outlierDetectionBalancer) meanAndStdDev(addrs []*addressInfo) (float64, float64) { var totalFractionOfSuccessfulRequests float64 var mean float64 @@ -780,14 +773,14 @@ func (b *outlierDetectionBalancer) meanAndStdDev(addrs []*addressInfo) (float64, // successRateAlgorithm ejects any addresses where the success rate falls below // the other addresses according to mean and standard deviation, and if overall -// applicable from other set heuristics. +// applicable from other set heuristics. Caller must hold b.mu. func (b *outlierDetectionBalancer) successRateAlgorithm() { - if b.numAddrsWithAtLeastRequestVolume() < b.cfg.SuccessRateEjection.MinimumHosts { + addrsToConsider := b.addrsWithAtLeastRequestVolume() + if len(addrsToConsider) < int(b.cfg.SuccessRateEjection.MinimumHosts) { return } - awalrv := b.addrsWithAtLeastRequestVolume() - mean, stddev := b.meanAndStdDev(awalrv) - for _, addrInfo := range awalrv { + mean, stddev := b.meanAndStdDev(addrsToConsider) + for _, addrInfo := range addrsToConsider { bucket := addrInfo.callCounter.inactiveBucket ejectionCfg := b.cfg.SuccessRateEjection if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { @@ -804,7 +797,7 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { // failurePercentageAlgorithm ejects any addresses where the failure percentage // rate exceeds a set enforcement percentage, if overall applicable from other -// set heuristics. +// set heuristics. Caller must hold b.mu. func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { if uint32(len(b.addrs)) < b.cfg.FailurePercentageEjection.MinimumHosts { return @@ -829,6 +822,7 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { } } +// Caller must hold b.mu. func (b *outlierDetectionBalancer) ejectAddress(addrInfo *addressInfo) { b.numAddrsEjected++ addrInfo.latestEjectionTimestamp = b.timerStartTime @@ -838,6 +832,7 @@ func (b *outlierDetectionBalancer) ejectAddress(addrInfo *addressInfo) { } } +// Caller must hold b.mu. func (b *outlierDetectionBalancer) unejectAddress(addrInfo *addressInfo) { b.numAddrsEjected-- addrInfo.latestEjectionTimestamp = time.Time{} @@ -847,13 +842,11 @@ func (b *outlierDetectionBalancer) unejectAddress(addrInfo *addressInfo) { } // addressInfo contains the runtime information about an address that pertains -// to Outlier Detection, including the counter for successful/failing RPC's, and -// also information about whether the addresses has been ejected, and the -// SubConns that are present that use this address. This struct and all of its -// fields is protected by outlierDetectionBalancer.mu in the case where it -// accessed through the address map. In the case of Picker callbacks, the writes -// to the activeBucket of callCounter are protected by atomically loading and -// storing unsafe.Pointers (see further explanation in incrementCounter()). +// to Outlier Detection. This struct and all of its fields is protected by +// outlierDetectionBalancer.mu in the case where it is accessed through the +// address map. In the case of Picker callbacks, the writes to the activeBucket +// of callCounter are protected by atomically loading and storing +// unsafe.Pointers (see further explanation in incrementCounter()). type addressInfo struct { // The call result counter object. callCounter *callCounter diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index bf6a20c518a..0373c94b333 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -30,15 +30,22 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" + "google.golang.org/grpc" "google.golang.org/grpc/balancer" "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/balancer/stub" "google.golang.org/grpc/internal/grpcsync" "google.golang.org/grpc/internal/grpctest" internalserviceconfig "google.golang.org/grpc/internal/serviceconfig" + "google.golang.org/grpc/internal/stubserver" "google.golang.org/grpc/internal/testutils" + "google.golang.org/grpc/peer" "google.golang.org/grpc/resolver" + "google.golang.org/grpc/resolver/manual" "google.golang.org/grpc/serviceconfig" + testpb "google.golang.org/grpc/test/grpc_testing" "google.golang.org/grpc/xds/internal/balancer/clusterimpl" ) @@ -58,6 +65,12 @@ func Test(t *testing.T) { // TestParseConfig verifies the ParseConfig() method in the Outlier Detection // Balancer. func (s) TestParseConfig(t *testing.T) { + stub.Register(errParseConfigName, stub.BalancerFuncs{ + ParseConfig: func(json.RawMessage) (serviceconfig.LoadBalancingConfig, error) { + return nil, errors.New("some error") + }, + }) + parser := bb{} tests := []struct { @@ -284,197 +297,79 @@ func (lbc *LBConfig) Equal(lbc2 *LBConfig) bool { return cmp.Equal(lbc.ChildPolicy, lbc2.ChildPolicy) } -func init() { - balancer.Register(errParseConfigBuilder{}) - balancer.Register(errParseConfigBuilder{}) - balancer.Register(tcibBuilder{}) - balancer.Register(verifyBalancerBuilder{}) - balancer.Register(twoUpdateStateBalancerBuilder{}) -} - -type errParseConfigBuilder struct{} - -func (errParseConfigBuilder) Build(cc balancer.ClientConn, opts balancer.BuildOptions) balancer.Balancer { - return nil -} - -func (errParseConfigBuilder) Name() string { - return "errParseConfigBalancer" -} +const errParseConfigName = "errParseConfigBalancer" +const tcibname = "testClusterImplBalancer" +const verifyBalancerName = "verifyBalancer" -func (errParseConfigBuilder) ParseConfig(c json.RawMessage) (serviceconfig.LoadBalancingConfig, error) { - return nil, errors.New("some error") +type subConnWithState struct { + sc balancer.SubConn + state balancer.SubConnState } -func setup(t *testing.T) (*outlierDetectionBalancer, *testutils.TestClientConn) { +func setup(t *testing.T) (*outlierDetectionBalancer, *testutils.TestClientConn, func()) { t.Helper() + internal.RegisterOutlierDetectionBalancerForTesting() builder := balancer.Get(Name) if builder == nil { t.Fatalf("balancer.Get(%q) returned nil", Name) } tcc := testutils.NewTestClientConn(t) odB := builder.Build(tcc, balancer.BuildOptions{}) - return odB.(*outlierDetectionBalancer), tcc -} - -const tcibname = "testClusterImplBalancer" -const verifyBalancerName = "verifyBalancer" -const twoUpdateStateBalancerName = "twoUpdateStateBalancer" - -type tcibBuilder struct{} - -func (tcibBuilder) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Balancer { - return &testClusterImplBalancer{ - ccsCh: testutils.NewChannel(), - scStateCh: testutils.NewChannel(), - resolverErrCh: testutils.NewChannel(), - closeCh: testutils.NewChannel(), - exitIdleCh: testutils.NewChannel(), - cc: cc, + return odB.(*outlierDetectionBalancer), tcc, func() { + odB.Close() + internal.UnregisterOutlierDetectionBalancerForTesting() } } -func (tcibBuilder) Name() string { - return tcibname -} - -type testClusterImplBalancerConfig struct { +type balancerConfig struct { serviceconfig.LoadBalancingConfig } -type testClusterImplBalancer struct { - // ccsCh is a channel used to signal the receipt of a ClientConn update. - ccsCh *testutils.Channel - // scStateCh is a channel used to signal the receipt of a SubConn update. - scStateCh *testutils.Channel - // resolverErrCh is a channel used to signal a resolver error. - resolverErrCh *testutils.Channel - // closeCh is a channel used to signal the closing of this balancer. - closeCh *testutils.Channel - exitIdleCh *testutils.Channel - // cc is the balancer.ClientConn passed to this test balancer as part of the - // Build() call. - cc balancer.ClientConn -} - -type subConnWithState struct { - sc balancer.SubConn - state balancer.SubConnState -} - -func (tb *testClusterImplBalancer) UpdateClientConnState(ccs balancer.ClientConnState) error { - tb.ccsCh.Send(ccs) - return nil -} - -func (tb *testClusterImplBalancer) ResolverError(err error) { - tb.resolverErrCh.Send(err) -} - -func (tb *testClusterImplBalancer) UpdateSubConnState(sc balancer.SubConn, state balancer.SubConnState) { - tb.scStateCh.Send(subConnWithState{sc: sc, state: state}) -} - -func (tb *testClusterImplBalancer) Close() { - tb.closeCh.Send(struct{}{}) -} - -// waitForClientConnUpdate verifies if the testClusterImplBalancer receives the -// provided ClientConnState within a reasonable amount of time. -func (tb *testClusterImplBalancer) waitForClientConnUpdate(ctx context.Context, wantCCS balancer.ClientConnState) error { - ccs, err := tb.ccsCh.Receive(ctx) - if err != nil { - return err - } - gotCCS := ccs.(balancer.ClientConnState) - if diff := cmp.Diff(gotCCS, wantCCS, cmpopts.IgnoreFields(resolver.State{}, "Attributes")); diff != "" { - return fmt.Errorf("received unexpected ClientConnState, diff (-got +want): %v", diff) - } - return nil -} - -// waitForSubConnUpdate verifies if the testClusterImplBalancer receives the -// provided SubConn and SubConnState within a reasonable amount of time. -func (tb *testClusterImplBalancer) waitForSubConnUpdate(ctx context.Context, wantSCS subConnWithState) error { - scs, err := tb.scStateCh.Receive(ctx) - if err != nil { - return err - } - gotSCS := scs.(subConnWithState) - if !cmp.Equal(gotSCS, wantSCS, cmp.AllowUnexported(subConnWithState{}, testutils.TestSubConn{}, subConnWrapper{}, addressInfo{}), cmpopts.IgnoreFields(subConnWrapper{}, "scUpdateCh")) { - return fmt.Errorf("received SubConnState: %+v, want %+v", gotSCS, wantSCS) - } - return nil -} - -// waitForClose verifies if the testClusterImplBalancer receives a Close call -// within a reasonable amount of time. -func (tb *testClusterImplBalancer) waitForClose(ctx context.Context) error { - if _, err := tb.closeCh.Receive(ctx); err != nil { - return err - } - return nil -} +// TestChildBasicOperations tests basic operations of the Outlier Detection +// Balancer and it's interaction with it's child. On the first receipt of a good +// config, the balancer is expected to eventually create a child and send the +// child it's configuration. When a new configuration comes in that changes the +// child's type which reports READY immediately, the first child balancer should +// be closed and the second child balancer should receive it's first config +// update. When the Outlier Detection Balancer itself is closed, this second +// child balancer should also be closed. +func (s) TestChildBasicOperations(t *testing.T) { + bc := balancerConfig{} -// TestUpdateClientConnState invokes the UpdateClientConnState method on the -// odBalancer with different inputs and verifies that the child balancer is built -// and updated properly. -func (s) TestUpdateClientConnState(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - od, _ := setup(t) - defer func() { - od.Close() // this will leak a goroutine otherwise - internal.UnregisterOutlierDetectionBalancerForTesting() - }() + ccsCh := testutils.NewChannel() + closeCh := testutils.NewChannel() - od.UpdateClientConnState(balancer.ClientConnState{ - BalancerConfig: &LBConfig{ - Interval: 10 * time.Second, - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, - SuccessRateEjection: &SuccessRateEjection{ - StdevFactor: 1900, - EnforcementPercentage: 100, - MinimumHosts: 5, - RequestVolume: 100, - }, - FailurePercentageEjection: &FailurePercentageEjection{ - Threshold: 85, - EnforcementPercentage: 5, - MinimumHosts: 5, - RequestVolume: 50, - }, - ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: testClusterImplBalancerConfig{}, - }, + stub.Register(tcibname, stub.BalancerFuncs{ + UpdateClientConnState: func(bd *stub.BalancerData, ccs balancer.ClientConnState) error { + ccsCh.Send(ccs.BalancerConfig) + return nil + }, + Close: func(bd *stub.BalancerData) { + closeCh.Send(nil) }, }) - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - // The child balancer should be created and forwarded the ClientConn update - // from the first successful UpdateClientConnState call. - if err := od.child.(*testClusterImplBalancer).waitForClientConnUpdate(ctx, balancer.ClientConnState{ - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } -} + stub.Register(verifyBalancerName, stub.BalancerFuncs{ + UpdateClientConnState: func(bd *stub.BalancerData, _ balancer.ClientConnState) error { + // UpdateState inline to READY to complete graceful switch process + // synchronously from any UpdateClientConnState call. + bd.ClientConn.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &testutils.TestConstPicker{}, + }) + ccsCh.Send(nil) + return nil + }, + Close: func(bd *stub.BalancerData) { + closeCh.Send(nil) + }, + }) -// TestUpdateClientConnStateDifferentType invokes the UpdateClientConnState -// method on the odBalancer with two different types and verifies that the child -// balancer is built and updated properly on the first, and the second update -// closes the child and builds a new one. -func (s) TestUpdateClientConnStateDifferentType(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - od, _ := setup(t) - defer func() { - od.Close() // this will leak a goroutine otherwise - internal.UnregisterOutlierDetectionBalancerForTesting() - }() + od, tcc, _ := setup(t) + defer internal.UnregisterOutlierDetectionBalancerForTesting() + // This first config update should a child to be built and forwarded it's + // first update. od.UpdateClientConnState(balancer.ClientConnState{ BalancerConfig: &LBConfig{ Interval: 10 * time.Second, @@ -495,63 +390,24 @@ func (s) TestUpdateClientConnStateDifferentType(t *testing.T) { }, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: tcibname, - Config: testClusterImplBalancerConfig{}, + Config: bc, }, }, }) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - ciChild := od.child.(*testClusterImplBalancer) - // The child balancer should be created and forwarded the ClientConn update - // from the first successful UpdateClientConnState call. - if err := ciChild.waitForClientConnUpdate(ctx, balancer.ClientConnState{ - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) + cr, err := ccsCh.Receive(ctx) + if err != nil { + t.Fatalf("timed out waiting for UpdateClientConnState on the first child balancer: %v", err) } - - od.UpdateClientConnState(balancer.ClientConnState{ - BalancerConfig: &LBConfig{ - Interval: 10 * time.Second, - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, - SuccessRateEjection: &SuccessRateEjection{ - StdevFactor: 1900, - EnforcementPercentage: 100, - MinimumHosts: 5, - RequestVolume: 100, - }, - FailurePercentageEjection: &FailurePercentageEjection{ - Threshold: 85, - EnforcementPercentage: 5, - MinimumHosts: 5, - RequestVolume: 50, - }, - ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: verifyBalancerName, - Config: verifyBalancerConfig{}, - }, - }, - }) - - // Verify previous child balancer closed. - if err := ciChild.waitForClose(ctx); err != nil { - t.Fatalf("Error waiting for Close() call on child balancer %v", err) + if _, ok := cr.(balancerConfig); !ok { + t.Fatalf("config passed to child should be balancerConfig type") } -} - -// TestUpdateState tests that an UpdateState call gets forwarded to the -// ClientConn. -func (s) TestUpdateState(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - od, tcc := setup(t) - defer func() { - od.Close() - internal.UnregisterOutlierDetectionBalancerForTesting() - }() + // This Update Client Conn State call should cause the first child balancer + // to close, and a new child to be created and also forwarded it's first + // config update. od.UpdateClientConnState(balancer.ClientConnState{ BalancerConfig: &LBConfig{ Interval: 10 * time.Second, @@ -571,20 +427,14 @@ func (s) TestUpdateState(t *testing.T) { RequestVolume: 50, }, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: testClusterImplBalancerConfig{}, + Name: verifyBalancerName, + Config: balancerConfig{}, }, }, }) - od.UpdateState(balancer.State{ - ConnectivityState: connectivity.Ready, - Picker: &testutils.TestConstPicker{}, - }) - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - - // Should forward the connectivity State to Client Conn. + // Verify inline UpdateState() call from the new child eventually makes it's + // way to the Test Client Conn. select { case <-ctx.Done(): t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") @@ -594,78 +444,69 @@ func (s) TestUpdateState(t *testing.T) { } } - select { - case <-ctx.Done(): - t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") - case picker := <-tcc.NewPickerCh: - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{}) + // Verify the first child balancer closed. + _, err = closeCh.Receive(ctx) + if err != nil { + t.Fatalf("timed out waiting for the first child balancer to be closed: %v", err) } -} - -// TestClose tests the Close operation on the Outlier Detection Balancer. The -// Close operation should close the child. -func (s) TestClose(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - defer func() { - internal.UnregisterOutlierDetectionBalancerForTesting() - }() - od, _ := setup(t) - - od.UpdateClientConnState(balancer.ClientConnState{ // could pull this out to the setup() call as well, maybe a wrapper on what is currently there... - BalancerConfig: &LBConfig{ - Interval: 10 * time.Second, - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, - SuccessRateEjection: &SuccessRateEjection{ - StdevFactor: 1900, - EnforcementPercentage: 100, - MinimumHosts: 5, - RequestVolume: 100, - }, - FailurePercentageEjection: &FailurePercentageEjection{ - Threshold: 85, - EnforcementPercentage: 5, - MinimumHosts: 5, - RequestVolume: 50, - }, - ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: testClusterImplBalancerConfig{}, - }, - }, - }) - - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - ciChild := od.child.(*testClusterImplBalancer) - if err := ciChild.waitForClientConnUpdate(ctx, balancer.ClientConnState{ - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) + // Verify the second child balancer received it's first config update. + _, err = ccsCh.Receive(ctx) + if err != nil { + t.Fatalf("timed out waiting for UpdateClientConnState on the second child balancer: %v", err) } - + // Closing the Outlier Detection Balancer should close the newly created + // child. od.Close() - - // Verify child balancer closed. - if err := ciChild.waitForClose(ctx); err != nil { - t.Fatalf("Error waiting for Close() call on child balancer %v", err) + _, err = closeCh.Receive(ctx) + if err != nil { + t.Fatalf("timed out waiting for the second child balancer to be closed: %v", err) } } // TestUpdateAddresses tests the functionality of UpdateAddresses and any -// changes in the addresses/plurality of those addresses for a SubConn. +// changes in the addresses/plurality of those addresses for a SubConn. The +// Balancer is set up with two upstreams, with one of the upstreams being +// ejected. Switching a SubConn's address list to the ejected address should +// cause the SubConn to be ejected, if not already. Switching the address list +// from single to plural should cause this SubConn to be unejected, since the +// SubConn is no longer being tracked by Outlier Detection. Then, switching this +// SubConn back to the single ejected address should reeject the SubConn. func (s) TestUpdateAddresses(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - od, tcc := setup(t) - defer func() { - od.Close() - internal.UnregisterOutlierDetectionBalancerForTesting() - }() + scsCh := testutils.NewChannel() + var scw1, scw2 balancer.SubConn + var err error + stub.Register(tcibname, stub.BalancerFuncs{ + UpdateClientConnState: func(bd *stub.BalancerData, _ balancer.ClientConnState) error { + scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{ + { + Addr: "address1", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{ + { + Addr: "address2", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + return nil + }, + UpdateSubConnState: func(_ *stub.BalancerData, sc balancer.SubConn, state balancer.SubConnState) { + scsCh.Send(subConnWithState{ + sc: sc, + state: state, + }) + }}) + + od, tcc, cleanup := setup(t) + defer cleanup() od.UpdateClientConnState(balancer.ClientConnState{ ResolverState: resolver.State{ @@ -683,7 +524,7 @@ func (s) TestUpdateAddresses(t *testing.T) { BaseEjectionTime: 30 * time.Second, MaxEjectionTime: 300 * time.Second, MaxEjectionPercent: 10, - FailurePercentageEjection: &FailurePercentageEjection{ // have this eject one but not the other + FailurePercentageEjection: &FailurePercentageEjection{ Threshold: 50, EnforcementPercentage: 100, MinimumHosts: 2, @@ -691,58 +532,13 @@ func (s) TestUpdateAddresses(t *testing.T) { }, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: tcibname, - Config: testClusterImplBalancerConfig{}, + Config: balancerConfig{}, }, }, }) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - child := od.child.(*testClusterImplBalancer) - if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - }, - }, - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } - - scw1, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address1", - }, - }, balancer.NewSubConnOptions{}) - - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - select { - case <-ctx.Done(): - t.Fatal("timeout waiting for NewSubConn to be called on test Client Conn") - case <-tcc.NewSubConnCh: - } - _, ok := scw1.(*subConnWrapper) - if !ok { - t.Fatalf("SubConn passed downward should have been a subConnWrapper") - } - - scw2, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address2", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } od.UpdateState(balancer.State{ ConnectivityState: connectivity.Ready, @@ -779,11 +575,15 @@ func (s) TestUpdateAddresses(t *testing.T) { od.intervalTimerAlgorithm() // verify UpdateSubConnState() got called with TRANSIENT_FAILURE for // child with address that was ejected. - if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + gotSCWS, err := scsCh.Receive(ctx) + if err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + if err = scwsEqual(gotSCWS.(subConnWithState), subConnWithState{ sc: scw2, - state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, }); err != nil { - t.Fatalf("Error waiting for Sub Conn update: %v", err) + t.Fatalf("Error in Sub Conn update: %v", err) } } @@ -795,19 +595,23 @@ func (s) TestUpdateAddresses(t *testing.T) { }, }) - // verify that update addresses gets forwarded to ClientConn. + // Verify that update addresses gets forwarded to ClientConn. select { case <-ctx.Done(): t.Fatal("timeout while waiting for a UpdateState call on the ClientConn") case <-tcc.UpdateAddressesAddrsCh: } - // verify scw1 got ejected (UpdateSubConnState called with TRANSIENT + // Verify scw1 got ejected (UpdateSubConnState called with TRANSIENT // FAILURE). - if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + gotSCWS, err := scsCh.Receive(ctx) + if err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + if err = scwsEqual(gotSCWS.(subConnWithState), subConnWithState{ sc: scw1, - state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, }); err != nil { - t.Fatalf("Error waiting for Sub Conn update: %v", err) + t.Fatalf("Error in Sub Conn update: %v", err) } // Update scw1 to multiple addresses. This should cause scw1 to get @@ -820,12 +624,16 @@ func (s) TestUpdateAddresses(t *testing.T) { Addr: "address2", }, }) - // verify scw2 got unejected (UpdateSubConnState called with recent state). - if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + // Verify scw1 got unejected (UpdateSubConnState called with recent state). + gotSCWS, err = scsCh.Receive(ctx) + if err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + if err = scwsEqual(gotSCWS.(subConnWithState), subConnWithState{ sc: scw1, - state: balancer.SubConnState{ConnectivityState: connectivity.Idle}, // If you uneject a SubConn that hasn't received a UpdateSubConnState, IDLE is recent state. This seems fine or is this wrong? + state: balancer.SubConnState{ConnectivityState: connectivity.Idle}, }); err != nil { - t.Fatalf("Error waiting for Sub Conn update: %v", err) + t.Fatalf("Error in Sub Conn update: %v", err) } // Update scw1 to a different multiple addresses list. A change of addresses @@ -842,879 +650,46 @@ func (s) TestUpdateAddresses(t *testing.T) { // Verify no downstream effects. sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) defer cancel() - if err := od.child.(*testClusterImplBalancer).waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { + if _, err := scsCh.Receive(sCtx); err == nil { t.Fatalf("no SubConn update should have been sent (no SubConn got ejected/unejected)") } // Update scw1 back to a single address, which is ejected. This should cause // the SubConn to be re-ejected. - od.UpdateAddresses(scw1, []resolver.Address{ { Addr: "address2", }, }) - // verify scw1 got ejected (UpdateSubConnState called with TRANSIENT FAILURE). - if err := child.waitForSubConnUpdate(ctx, subConnWithState{ + // Verify scw1 got ejected (UpdateSubConnState called with TRANSIENT FAILURE). + gotSCWS, err = scsCh.Receive(ctx) + if err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + if err = scwsEqual(gotSCWS.(subConnWithState), subConnWithState{ sc: scw1, - state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, }); err != nil { - t.Fatalf("Error waiting for Sub Conn update: %v", err) + t.Fatalf("Error in Sub Conn update: %v", err) } } -// Three important pieces of functionality run() synchronizes in regards to UpdateState calls towards grpc: - -// 1. On a config update, checking if the picker was actually created or not. -// 2. Keeping track of the most recent connectivity state sent from the child (determined by UpdateState()). -// * This will always forward up with most recent noopCfg bit -// 3. Keeping track of the most recent no-op config bit (determined by UpdateClientConnState()) -// * Will only forward up if no-op config bit changed and picker was already created - -// TestPicker tests the Picker updates sent upward to grpc from the Outlier -// Detection Balancer. Two things can trigger a picker update, an -// UpdateClientConnState call (can flip the no-op config bit that affects -// Picker) and an UpdateState call (determines the connectivity state sent -// upward). -func (s) TestPicker(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - od, tcc := setup(t) - defer func() { - od.Close() - internal.UnregisterOutlierDetectionBalancerForTesting() - }() - - od.UpdateClientConnState(balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - }, - }, - BalancerConfig: &LBConfig{ // TODO: S/ variable - Interval: 10 * time.Second, - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, - SuccessRateEjection: &SuccessRateEjection{ - StdevFactor: 1900, - EnforcementPercentage: 100, - MinimumHosts: 5, - RequestVolume: 100, - }, - FailurePercentageEjection: &FailurePercentageEjection{ - Threshold: 85, - EnforcementPercentage: 5, - MinimumHosts: 5, - RequestVolume: 50, - }, - ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: testClusterImplBalancerConfig{}, - }, - }, - }) - - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - - child := od.child.(*testClusterImplBalancer) - if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - }, - }, - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } - - scw, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address1", - }, - }, balancer.NewSubConnOptions{}) - - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - od.UpdateState(balancer.State{ - ConnectivityState: connectivity.Ready, - Picker: &testutils.TestConstPicker{ - SC: scw, - }, - }) - - // Should forward the connectivity State to Client Conn. - select { - case <-ctx.Done(): - t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") - case state := <-tcc.NewStateCh: - if state != connectivity.Ready { - t.Fatalf("ClientConn received connectivity state %v, want %v", state, connectivity.Ready) - } - } - - select { - case <-ctx.Done(): - t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") - case picker := <-tcc.NewPickerCh: - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - od.mu.Lock() - addrInfo, ok := od.addrs["address1"] - if !ok { - t.Fatal("map entry for address: address1 not present in map") - } - bucketWant := &bucket{ - numSuccesses: 1, - numFailures: 1, - } - if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call from picker - t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) - } - od.mu.Unlock() - } - - // UpdateClientConnState with a noop config - od.UpdateClientConnState(balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - }, - }, - BalancerConfig: &LBConfig{ - Interval: 1<<63 - 1, - ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: testClusterImplBalancerConfig{}, - }, - }, - }) - - if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - }, - }, - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } - - // The connectivity state sent to the Client Conn should be the persisted - // recent state received from the last UpdateState() call, which in this - // case is ready. - select { - case <-ctx.Done(): - t.Fatal("timeout waiting for picker update on ClientConn, should have updated because no-op config changed on UpdateClientConnState") - case state := <-tcc.NewStateCh: - if state != connectivity.Ready { - t.Fatalf("ClientConn received connectivity state %v, want %v", state, connectivity.Ready) - } - } - - select { - case <-ctx.Done(): - t.Fatal("timeout waiting for picker update on ClientConn, should have updated because no-op config changed on UpdateClientConnState") - case picker := <-tcc.NewPickerCh: - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - od.mu.Lock() - addrInfo, ok := od.addrs["address1"] - if !ok { - t.Fatal("map entry for address: address1 not present in map") - } - - // The active bucket should be the same as before the no-op - // configuration came in because the interval timer algorithm didn't run - // in between ClientConn updates and the picker should not count, as the - // outlier detection balancer is configured with a no-op configuration. - bucketWant := &bucket{ - numSuccesses: 1, - numFailures: 1, - } - if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call - od.mu.Unlock() - t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) - } - od.mu.Unlock() - } - - // UpdateState with a connecting state. This new most recent connectivity - // state should be forwarded to the Client Conn, alongside the most recent - // noop config bit which is true. - od.UpdateState(balancer.State{ - ConnectivityState: connectivity.Connecting, - Picker: &testutils.TestConstPicker{ - SC: scw, - }, - }) - - // Should forward the most recent connectivity State to Client Conn. - select { - case <-ctx.Done(): - t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") - case state := <-tcc.NewStateCh: - if state != connectivity.Connecting { - t.Fatalf("ClientConn received connectivity state %v, want %v", state, connectivity.Connecting) - } - } - - // Should forward the picker containing the most recent no-op config bit. - select { - case <-ctx.Done(): - t.Fatal("timeout while waiting for a UpdateState call on the ClientConn") - case picker := <-tcc.NewPickerCh: - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - od.mu.Lock() - addrInfo, ok := od.addrs["address1"] - if !ok { - t.Fatal("map entry for address: address1 not present in map") - } - - // The active bucket should be the same as before the no-op - // configuration came in because the interval timer algorithm didn't run - // in between ClientConn updates and the picker should not count, as the - // outlier detection balancer is configured with a no-op configuration. - bucketWant := &bucket{ - numSuccesses: 1, - numFailures: 1, - } - if diff := cmp.Diff((*bucket)(addrInfo.callCounter.activeBucket), bucketWant); diff != "" { // no need for atomic read because not concurrent with Done() call - t.Fatalf("callCounter is different than expected, diff (-got +want): %v", diff) - } - od.mu.Unlock() - } -} - -type rrPicker struct { - scs []balancer.SubConn - next int -} - -func (rrp *rrPicker) Pick(balancer.PickInfo) (balancer.PickResult, error) { - sc := rrp.scs[rrp.next] - rrp.next = (rrp.next + 1) % len(rrp.scs) - return balancer.PickResult{SubConn: sc}, nil -} - -// TestEjectUnejectSuccessRate tests the functionality of the interval timer -// algorithm of ejecting/unejecting SubConns when configured with -// SuccessRateEjection. It also tests a desired invariant of a SubConnWrapper -// being ejected or unejected, which is to either forward or not forward SubConn -// updates from grpc. -func (s) TestEjectUnejectSuccessRate(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - // Setup the outlier detection balancer to a point where it will be in a - // situation to potentially eject addresses. - od, tcc := setup(t) - defer func() { - od.Close() - internal.UnregisterOutlierDetectionBalancerForTesting() - }() - - od.UpdateClientConnState(balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, - }, - }, - BalancerConfig: &LBConfig{ - Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, - SuccessRateEjection: &SuccessRateEjection{ - StdevFactor: 500, - EnforcementPercentage: 100, - MinimumHosts: 3, - RequestVolume: 3, - }, - ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: testClusterImplBalancerConfig{}, - }, - }, - }) - - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - child := od.child.(*testClusterImplBalancer) - if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, - }, - }, - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } - - scw1, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address1", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - scw2, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address2", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - scw3, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address3", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - od.UpdateState(balancer.State{ - ConnectivityState: connectivity.Ready, - Picker: &rrPicker{ - scs: []balancer.SubConn{scw1, scw2, scw3}, - }, - }) - - select { - case <-ctx.Done(): - t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") - case picker := <-tcc.NewPickerCh: - // Set each of the three upstream addresses to have five successes each. - // This should cause none of the addresses to be ejected as none of them - // are outliers according to the success rate algorithm. - for i := 0; i < 3; i++ { - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - } - - od.intervalTimerAlgorithm() - - // verify no UpdateSubConnState() call on the child, as no addresses got - // ejected (ejected address will cause an UpdateSubConnState call). - sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) - defer cancel() - if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { - t.Fatalf("no SubConn update should have been sent (no SubConn got ejected)") - } - - // Since no addresses are ejected, a SubConn update should forward down - // to the child. - od.UpdateSubConnState(scw1.(*subConnWrapper).SubConn, balancer.SubConnState{ - ConnectivityState: connectivity.Connecting, - }) - - if err := child.waitForSubConnUpdate(ctx, subConnWithState{ - sc: scw1, - state: balancer.SubConnState{ConnectivityState: connectivity.Connecting}, - }); err != nil { - t.Fatalf("Error waiting for Sub Conn update: %v", err) - } - - // Set two of the upstream addresses to have five successes each, and - // one of the upstream addresses to have five failures. This should - // cause the address which has five failures to be ejected according the - // SuccessRateAlgorithm. - for i := 0; i < 2; i++ { - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - } - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - - // should eject address that always errored. - od.intervalTimerAlgorithm() - // Due to the address being ejected, the SubConn with that address - // should be ejected, meaning a TRANSIENT_FAILURE connectivity state - // gets reported to the child. - if err := child.waitForSubConnUpdate(ctx, subConnWithState{ - sc: scw3, - state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected - }); err != nil { - t.Fatalf("Error waiting for Sub Conn update: %v", err) - } - // Only one address should be ejected. - sCtx, cancel = context.WithTimeout(context.Background(), defaultTestShortTimeout) - defer cancel() - if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { - t.Fatalf("Only one SubConn update should have been sent (only one SubConn got ejected)") - } - - // Now that an address is ejected, SubConn updates for SubConns using - // that address should not be forwarded downward. These SubConn updates - // will be cached to update the child sometime in the future when the - // address gets unejected. - od.UpdateSubConnState(pi.SubConn, balancer.SubConnState{ - ConnectivityState: connectivity.Connecting, - }) - if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { - t.Fatalf("SubConn update should not have been forwarded (the SubConn is ejected)") - } - - // Override now to cause the interval timer algorithm to always uneject a SubConn. - defer func(n func() time.Time) { - now = n - }(now) - - now = func() time.Time { - return time.Now().Add(time.Second * 1000) // will cause to always uneject addresses which are ejected - } - od.intervalTimerAlgorithm() - - // unejected SubConn should report latest persisted state - which is - // connecting from earlier. - if err := child.waitForSubConnUpdate(ctx, subConnWithState{ - sc: scw3, - state: balancer.SubConnState{ConnectivityState: connectivity.Connecting}, - }); err != nil { - t.Fatalf("Error waiting for Sub Conn update: %v", err) - } - } -} - -// TestEjectUnejectSuccessRateFromNoopConfig tests that any ejected Addresses -// are unejected upon the receipt of a no-op Outlier Detection Configuration. -func (s) TestEjectUnejectSuccessRateFromNoopConfig(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - // Setup the outlier detection balancer to a point where it will be in a - // situation to potentially eject addresses. - od, tcc := setup(t) - defer func() { - od.Close() - internal.UnregisterOutlierDetectionBalancerForTesting() - }() - - od.UpdateClientConnState(balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, - }, - }, - BalancerConfig: &LBConfig{ - Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, - SuccessRateEjection: &SuccessRateEjection{ - StdevFactor: 500, - EnforcementPercentage: 100, - MinimumHosts: 3, - RequestVolume: 3, - }, - ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: testClusterImplBalancerConfig{}, - }, - }, - }) - - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - child := od.child.(*testClusterImplBalancer) - if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, - }, - }, - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } - - scw1, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address1", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - scw2, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address2", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - scw3, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address3", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - od.UpdateState(balancer.State{ - ConnectivityState: connectivity.Ready, - Picker: &rrPicker{ - scs: []balancer.SubConn{scw1, scw2, scw3}, - }, - }) - - select { - case <-ctx.Done(): - t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") - case picker := <-tcc.NewPickerCh: - // Set two of the upstream addresses to have five successes each, and - // one of the upstream addresses to have five failures. This should - // cause the address which has five failures to be ejected according the - // SuccessRateAlgorithm. - for i := 0; i < 2; i++ { - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - } - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - - // should eject address that always errored. - od.intervalTimerAlgorithm() - // Due to the address being ejected, the SubConn with that address - // should be ejected, meaning a TRANSIENT_FAILURE connectivity state - // gets reported to the child. - if err := child.waitForSubConnUpdate(ctx, subConnWithState{ - sc: scw3, - state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected - }); err != nil { - t.Fatalf("Error waiting for Sub Conn update: %v", err) - } - // Only one address should be ejected. - sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) - defer sCancel() - if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { - t.Fatalf("Only one SubConn update should have been sent (only one SubConn got ejected)") - } - // Now that an address is ejected, SubConn updates for SubConns using - // that address should not be forwarded downward. These SubConn updates - // will be cached to update the child sometime in the future when the - // address gets unejected. - od.UpdateSubConnState(pi.SubConn, balancer.SubConnState{ - ConnectivityState: connectivity.Connecting, - }) - if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { - t.Fatalf("SubConn update should not have been forwarded (the SubConn is ejected)") - } - // Update the Outlier Detection Balancer with a no-op configuration. - // This should cause any ejected addresses to become unejected. - od.UpdateClientConnState(balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, - }, - }, - BalancerConfig: &LBConfig{ - Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. - ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: testClusterImplBalancerConfig{}, - }, - }, - }) - // unejected SubConn should report latest persisted state - which is - // connecting from earlier. - if err := child.waitForSubConnUpdate(ctx, subConnWithState{ - sc: scw3, - state: balancer.SubConnState{ConnectivityState: connectivity.Connecting}, - }); err != nil { - t.Fatalf("Error waiting for Sub Conn update: %v", err) - } - } -} - -// TestEjectFailureRate tests the functionality of the interval timer -// algorithm of ejecting SubConns when configured with -// FailurePercentageEjection. -func (s) TestEjectFailureRate(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - od, tcc := setup(t) - defer func() { - internal.UnregisterOutlierDetectionBalancerForTesting() - defer od.Close() - }() - - od.UpdateClientConnState(balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, - }, - }, - BalancerConfig: &LBConfig{ - Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, - FailurePercentageEjection: &FailurePercentageEjection{ - Threshold: 50, - EnforcementPercentage: 100, - MinimumHosts: 3, - RequestVolume: 3, - }, - ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: testClusterImplBalancerConfig{}, - }, - }, - }) - - child := od.child.(*testClusterImplBalancer) - - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - if err := child.waitForClientConnUpdate(ctx, balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, - }, - }, - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } - - scw1, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address1", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - scw2, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address2", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - scw3, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address3", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - - od.UpdateState(balancer.State{ - ConnectivityState: connectivity.Ready, - Picker: &rrPicker{ - scs: []balancer.SubConn{scw1, scw2, scw3}, - }, - }) - - select { - case <-ctx.Done(): - t.Fatal("timeout while waiting for a UpdateState call on the ClientConn") - case picker := <-tcc.NewPickerCh: - // Set each upstream address to have five successes each. This should - // cause none of the addresses to be ejected as none of them are below - // the failure percentage threshold. - for i := 0; i < 3; i++ { - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - } - - od.intervalTimerAlgorithm() - - sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) - defer cancel() - if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { - t.Fatalf("no SubConn update should have been sent (no SubConn got ejected)") - } - - // Set two upstream addresses to have five successes each, and one - // upstream address to have five failures. This should cause the address - // with five failures to be ejected according to the Failure Percentage - // Algorithm. - for i := 0; i < 2; i++ { - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - } - pi, err := picker.Pick(balancer.PickInfo{}) - if err != nil { - t.Fatalf("Picker.Pick should not have errored") - } - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - - // should eject address that always errored. - od.intervalTimerAlgorithm() +func scwsEqual(gotSCWS subConnWithState, wantSCWS subConnWithState) error { + if !cmp.Equal(gotSCWS, wantSCWS, cmp.AllowUnexported(subConnWithState{}, testutils.TestSubConn{}, subConnWrapper{}, addressInfo{}), cmpopts.IgnoreFields(subConnWrapper{}, "scUpdateCh")) { + return fmt.Errorf("received SubConnState: %+v, want %+v", gotSCWS, wantSCWS) + } + return nil +} - // verify UpdateSubConnState() got called with TRANSIENT_FAILURE for - // child in address that was ejected. - if err := child.waitForSubConnUpdate(ctx, subConnWithState{ - sc: scw3, - state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, // Represents ejected - }); err != nil { - t.Fatalf("Error waiting for Sub Conn update: %v", err) - } +type rrPicker struct { + scs []balancer.SubConn + next int +} - // verify only one address got ejected - sCtx, cancel = context.WithTimeout(context.Background(), defaultTestShortTimeout) - defer cancel() - if err := child.waitForSubConnUpdate(sCtx, subConnWithState{}); err == nil { - t.Fatalf("Only one SubConn update should have been sent (only one SubConn got ejected)") - } - } +func (rrp *rrPicker) Pick(balancer.PickInfo) (balancer.PickResult, error) { + sc := rrp.scs[rrp.next] + rrp.next = (rrp.next + 1) % len(rrp.scs) + return balancer.PickResult{SubConn: sc}, nil } // TestDurationOfInterval tests the configured interval timer. On the first @@ -1725,12 +700,12 @@ func (s) TestEjectFailureRate(t *testing.T) { // and the previous start timestamp. For a no-op configuration, the timer should // not be configured at all. func (s) TestDurationOfInterval(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - od, _ := setup(t) + stub.Register(tcibname, stub.BalancerFuncs{}) + + od, _, cleanup := setup(t) defer func(af func(d time.Duration, f func()) *time.Timer) { - od.Close() + cleanup() afterFunc = af - internal.UnregisterOutlierDetectionBalancerForTesting() }(afterFunc) durationChan := testutils.NewChannel() @@ -1759,18 +734,12 @@ func (s) TestDurationOfInterval(t *testing.T) { }, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: tcibname, - Config: testClusterImplBalancerConfig{}, + Config: balancerConfig{}, }, }, }) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - if err := od.child.(*testClusterImplBalancer).waitForClientConnUpdate(ctx, balancer.ClientConnState{ - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } - d, err := durationChan.Receive(ctx) if err != nil { t.Fatalf("Error receiving duration from afterFunc() call: %v", err) @@ -1812,18 +781,11 @@ func (s) TestDurationOfInterval(t *testing.T) { }, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: tcibname, - Config: testClusterImplBalancerConfig{}, + Config: balancerConfig{}, }, }, }) - ctx, cancel = context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - if err := od.child.(*testClusterImplBalancer).waitForClientConnUpdate(ctx, balancer.ClientConnState{ - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } d, err = durationChan.Receive(ctx) if err != nil { t.Fatalf("Error receiving duration from afterFunc() call: %v", err) @@ -1840,18 +802,13 @@ func (s) TestDurationOfInterval(t *testing.T) { Interval: 10 * time.Second, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: tcibname, - Config: testClusterImplBalancerConfig{}, + Config: balancerConfig{}, }, }, }) ctx, cancel = context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - if err := od.child.(*testClusterImplBalancer).waitForClientConnUpdate(ctx, balancer.ClientConnState{ - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } // No timer should have been started. sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) defer cancel() @@ -1861,15 +818,54 @@ func (s) TestDurationOfInterval(t *testing.T) { } } -// TestConcurrentPickerCountsWithIntervalTimer tests concurrent picker updates -// (writing to the callCounter) and the interval timer algorithm, which reads -// the callCounter. -func (s) TestConcurrentPickerCountsWithIntervalTimer(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - od, tcc := setup(t) +// TestEjectUnejectSuccessRate tests the functionality of the interval timer +// algorithm of ejecting/unejecting SubConns when configured with +// SuccessRateEjection. It also tests a desired invariant of a SubConnWrapper +// being ejected or unejected, which is to either forward or not forward SubConn +// updates received from grpc. +func (s) TestEjectUnejectSuccessRate(t *testing.T) { + scsCh := testutils.NewChannel() + var scw1, scw2, scw3 balancer.SubConn + var err error + stub.Register(tcibname, stub.BalancerFuncs{ + UpdateClientConnState: func(bd *stub.BalancerData, _ balancer.ClientConnState) error { + scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{ + { + Addr: "address1", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{ + { + Addr: "address2", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + scw3, err = bd.ClientConn.NewSubConn([]resolver.Address{ + { + Addr: "address3", + }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + return nil + }, + UpdateSubConnState: func(_ *stub.BalancerData, sc balancer.SubConn, state balancer.SubConnState) { + scsCh.Send(subConnWithState{ + sc: sc, + state: state, + }) + }, + }) + + od, tcc, cleanup := setup(t) defer func() { - defer od.Close() - internal.UnregisterOutlierDetectionBalancerForTesting() + cleanup() }() od.UpdateClientConnState(balancer.ClientConnState{ @@ -1891,12 +887,6 @@ func (s) TestConcurrentPickerCountsWithIntervalTimer(t *testing.T) { BaseEjectionTime: 30 * time.Second, MaxEjectionTime: 300 * time.Second, MaxEjectionPercent: 10, - SuccessRateEjection: &SuccessRateEjection{ // Have both Success Rate and Failure Percentage to step through all the interval timer code - StdevFactor: 500, - EnforcementPercentage: 100, - MinimumHosts: 3, - RequestVolume: 3, - }, FailurePercentageEjection: &FailurePercentageEjection{ Threshold: 50, EnforcementPercentage: 100, @@ -1905,61 +895,236 @@ func (s) TestConcurrentPickerCountsWithIntervalTimer(t *testing.T) { }, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: tcibname, - Config: testClusterImplBalancerConfig{}, + Config: balancerConfig{}, }, }, }) + od.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &rrPicker{ + scs: []balancer.SubConn{scw1, scw2, scw3}, + }, + }) + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - if err := od.child.(*testClusterImplBalancer).waitForClientConnUpdate(ctx, balancer.ClientConnState{ - ResolverState: resolver.State{ - Addresses: []resolver.Address{ + + select { + case <-ctx.Done(): + t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") + case picker := <-tcc.NewPickerCh: + // Set each of the three upstream addresses to have five successes each. + // This should cause none of the addresses to be ejected as none of them + // are outliers according to the success rate algorithm. + for i := 0; i < 3; i++ { + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + } + + od.intervalTimerAlgorithm() + + // verify no UpdateSubConnState() call on the child, as no addresses got + // ejected (ejected address will cause an UpdateSubConnState call). + sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + if _, err := scsCh.Receive(sCtx); err == nil { + t.Fatalf("no SubConn update should have been sent (no SubConn got ejected)") + } + + // Since no addresses are ejected, a SubConn update should forward down + // to the child. + od.UpdateSubConnState(scw1.(*subConnWrapper).SubConn, balancer.SubConnState{ + ConnectivityState: connectivity.Connecting, + }) + + gotSCWS, err := scsCh.Receive(ctx) + if err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + if err = scwsEqual(gotSCWS.(subConnWithState), subConnWithState{ + sc: scw1, + state: balancer.SubConnState{ConnectivityState: connectivity.Connecting}, + }); err != nil { + t.Fatalf("Error in Sub Conn update: %v", err) + } + + // Set two of the upstream addresses to have five successes each, and + // one of the upstream addresses to have five failures. This should + // cause the address which has five failures to be ejected according the + // SuccessRateAlgorithm. + for i := 0; i < 2; i++ { + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + } + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + + // should eject address that always errored. + od.intervalTimerAlgorithm() + // Due to the address being ejected, the SubConn with that address + // should be ejected, meaning a TRANSIENT_FAILURE connectivity state + // gets reported to the child. + gotSCWS, err = scsCh.Receive(ctx) + if err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + if err = scwsEqual(gotSCWS.(subConnWithState), subConnWithState{ + sc: scw3, + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, + }); err != nil { + t.Fatalf("Error in Sub Conn update: %v", err) + } + // Only one address should be ejected. + sCtx, cancel = context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + if _, err := scsCh.Receive(sCtx); err == nil { + t.Fatalf("Only one SubConn update should have been sent (only one SubConn got ejected)") + } + + // Now that an address is ejected, SubConn updates for SubConns using + // that address should not be forwarded downward. These SubConn updates + // will be cached to update the child sometime in the future when the + // address gets unejected. + od.UpdateSubConnState(pi.SubConn, balancer.SubConnState{ + ConnectivityState: connectivity.Connecting, + }) + sCtx, cancel = context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + if _, err := scsCh.Receive(sCtx); err == nil { + t.Fatalf("SubConn update should not have been forwarded (the SubConn is ejected)") + } + + // Override now to cause the interval timer algorithm to always uneject + // a SubConn. + defer func(n func() time.Time) { + now = n + }(now) + + now = func() time.Time { + return time.Now().Add(time.Second * 1000) // will cause to always uneject addresses which are ejected + } + od.intervalTimerAlgorithm() + + // unejected SubConn should report latest persisted state - which is + // connecting from earlier. + gotSCWS, err = scsCh.Receive(ctx) + if err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + if err = scwsEqual(gotSCWS.(subConnWithState), subConnWithState{ + sc: scw3, + state: balancer.SubConnState{ConnectivityState: connectivity.Connecting}, + }); err != nil { + t.Fatalf("Error in Sub Conn update: %v", err) + } + } +} + +// TestEjectFailureRate tests the functionality of the interval timer algorithm +// of ejecting SubConns when configured with FailurePercentageEjection. It also +// tests the functionality of unejecting SubConns when the balancer flips to a +// noop configuration. +func (s) TestEjectFailureRate(t *testing.T) { + scsCh := testutils.NewChannel() + var scw1, scw2, scw3 balancer.SubConn + var err error + stub.Register(tcibname, stub.BalancerFuncs{ + UpdateClientConnState: func(bd *stub.BalancerData, _ balancer.ClientConnState) error { + if scw1 != nil { // UpdateClientConnState was already called, no need to recreate SubConns. + return nil + } + scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{ { Addr: "address1", }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{ { Addr: "address2", }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + scw3, err = bd.ClientConn.NewSubConn([]resolver.Address{ { Addr: "address3", }, - }, + }, balancer.NewSubConnOptions{}) + if err != nil { + t.Fatalf("error in od.NewSubConn call: %v", err) + } + return nil }, - BalancerConfig: testClusterImplBalancerConfig{}, - }); err != nil { - t.Fatalf("Error waiting for Client Conn update: %v", err) - } - - scw1, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address1", + UpdateSubConnState: func(_ *stub.BalancerData, sc balancer.SubConn, state balancer.SubConnState) { + scsCh.Send(subConnWithState{ + sc: sc, + state: state, + }) }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } + }) - scw2, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address2", - }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } + od, tcc, cleanup := setup(t) + defer func() { + cleanup() + }() - scw3, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address3", + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + SuccessRateEjection: &SuccessRateEjection{ + StdevFactor: 500, + EnforcementPercentage: 100, + MinimumHosts: 3, + RequestVolume: 3, + }, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: balancerConfig{}, + }, }, - }, balancer.NewSubConnOptions{}) - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } + }) od.UpdateState(balancer.State{ ConnectivityState: connectivity.Ready, @@ -1968,38 +1133,121 @@ func (s) TestConcurrentPickerCountsWithIntervalTimer(t *testing.T) { }, }) - var picker balancer.Picker + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + select { case <-ctx.Done(): t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") - case picker = <-tcc.NewPickerCh: - } - - // Spawn a goroutine that constantly picks and invokes the Done callback - // counting for successful and failing RPC's. - finished := make(chan struct{}) - go func() { - // constantly update the picker to test for no race conditions causing - // corrupted memory (have concurrent pointer reads/writes). - for { - select { - case <-finished: - return - default: + case picker := <-tcc.NewPickerCh: + // Set each upstream address to have five successes each. This should + // cause none of the addresses to be ejected as none of them are below + // the failure percentage threshold. + for i := 0; i < 3; i++ { + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") } + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + } + + od.intervalTimerAlgorithm() + sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + if _, err := scsCh.Receive(sCtx); err == nil { + t.Fatalf("no SubConn update should have been sent (no SubConn got ejected)") + } + + // Set two upstream addresses to have five successes each, and one + // upstream address to have five failures. This should cause the address + // with five failures to be ejected according to the Failure Percentage + // Algorithm. + for i := 0; i < 2; i++ { pi, err := picker.Pick(balancer.PickInfo{}) if err != nil { - continue + t.Fatalf("Picker.Pick should not have errored") } pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - time.Sleep(1 * time.Nanosecond) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) + pi.Done(balancer.DoneInfo{}) } - }() + pi, err := picker.Pick(balancer.PickInfo{}) + if err != nil { + t.Fatalf("Picker.Pick should not have errored") + } + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - od.intervalTimerAlgorithm() // causes two swaps on the callCounter - od.intervalTimerAlgorithm() - close(finished) + // should eject address that always errored. + od.intervalTimerAlgorithm() + + // verify UpdateSubConnState() got called with TRANSIENT_FAILURE for + // child in address that was ejected. + gotSCWS, err := scsCh.Receive(ctx) + if err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + if err = scwsEqual(gotSCWS.(subConnWithState), subConnWithState{ + sc: scw3, + state: balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}, + }); err != nil { + t.Fatalf("Error in Sub Conn update: %v", err) + } + + // verify only one address got ejected. + sCtx, cancel = context.WithTimeout(context.Background(), defaultTestShortTimeout) + defer cancel() + if _, err := scsCh.Receive(sCtx); err == nil { + t.Fatalf("Only one SubConn update should have been sent (only one SubConn got ejected)") + } + + // upon the Outlier Detection balancer being reconfigured with a noop + // configuration, every ejected SubConn should be unejected. + od.UpdateClientConnState(balancer.ClientConnState{ + ResolverState: resolver.State{ + Addresses: []resolver.Address{ + { + Addr: "address1", + }, + { + Addr: "address2", + }, + { + Addr: "address3", + }, + }, + }, + BalancerConfig: &LBConfig{ + Interval: 1<<63 - 1, + BaseEjectionTime: 30 * time.Second, + MaxEjectionTime: 300 * time.Second, + MaxEjectionPercent: 10, + ChildPolicy: &internalserviceconfig.BalancerConfig{ + Name: tcibname, + Config: balancerConfig{}, + }, + }, + }) + gotSCWS, err = scsCh.Receive(ctx) + if err != nil { + t.Fatalf("Error waiting for Sub Conn update: %v", err) + } + if err = scwsEqual(gotSCWS.(subConnWithState), subConnWithState{ + sc: scw3, + state: balancer.SubConnState{ConnectivityState: connectivity.Idle}, + }); err != nil { + t.Fatalf("Error in Sub Conn update: %v", err) + } + } } // TestConcurrentOperations calls different operations on the balancer in @@ -2007,11 +1255,37 @@ func (s) TestConcurrentPickerCountsWithIntervalTimer(t *testing.T) { // uses a child balancer which verifies that no operations on the child get // called after the child balancer is closed. func (s) TestConcurrentOperations(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - od, tcc := setup(t) + closed := grpcsync.NewEvent() + stub.Register(verifyBalancerName, stub.BalancerFuncs{ + UpdateClientConnState: func(*stub.BalancerData, balancer.ClientConnState) error { + if closed.HasFired() { + t.Fatal("UpdateClientConnState was called after Close(), which breaks the balancer API") + } + return nil + }, + ResolverError: func(*stub.BalancerData, error) { + if closed.HasFired() { + t.Fatal("ResolverError was called after Close(), which breaks the balancer API") + } + }, + UpdateSubConnState: func(*stub.BalancerData, balancer.SubConn, balancer.SubConnState) { + if closed.HasFired() { + t.Fatal("UpdateSubConnState was called after Close(), which breaks the balancer API") + } + }, + Close: func(*stub.BalancerData) { + closed.Fire() + }, + ExitIdle: func(*stub.BalancerData) { + if closed.HasFired() { + t.Fatal("ExitIdle was called after Close(), which breaks the balancer API") + } + }, + }) + + od, tcc, cleanup := setup(t) defer func() { - od.Close() - internal.UnregisterOutlierDetectionBalancerForTesting() + cleanup() }() od.UpdateClientConnState(balancer.ClientConnState{ @@ -2047,13 +1321,10 @@ func (s) TestConcurrentOperations(t *testing.T) { }, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: verifyBalancerName, - Config: verifyBalancerConfig{}, + Config: balancerConfig{}, }, }, }) - - od.child.(*verifyBalancer).t = t - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -2135,7 +1406,7 @@ func (s) TestConcurrentOperations(t *testing.T) { } }() - // call Outlier Detection's balancer.ClientConn operations asynchrously. + // call Outlier Detection's balancer.ClientConn operations asynchronously. // balancer.ClientConn operations have no guarantee from the API to be // called synchronously. wg.Add(1) @@ -2180,7 +1451,7 @@ func (s) TestConcurrentOperations(t *testing.T) { }() // Call balancer.Balancers synchronously in this goroutine, upholding the - // balancer.Balancer API guarantee. + // balancer.Balancer API guarantee of synchronous calls. od.UpdateClientConnState(balancer.ClientConnState{ // This will delete addresses and flip to no op ResolverState: resolver.State{ Addresses: []resolver.Address{ @@ -2193,7 +1464,7 @@ func (s) TestConcurrentOperations(t *testing.T) { Interval: 1<<63 - 1, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: verifyBalancerName, - Config: verifyBalancerConfig{}, + Config: balancerConfig{}, }, }, }) @@ -2210,156 +1481,367 @@ func (s) TestConcurrentOperations(t *testing.T) { wg.Wait() } -type verifyBalancerConfig struct { - serviceconfig.LoadBalancingConfig -} - -type verifyBalancerBuilder struct{} +// Setup spins up three test backends, each listening on a port on localhost. +// Two of the backends are configured to always reply with an empty response and +// no error and one is configured to always return an error. +func setupBackends(t *testing.T) ([]string, []*stubserver.StubServer) { + t.Helper() -func (verifyBalancerBuilder) Build(cc balancer.ClientConn, opts balancer.BuildOptions) balancer.Balancer { - return &verifyBalancer{ - closed: grpcsync.NewEvent(), + backends := make([]*stubserver.StubServer, 3) + addresses := make([]string, 3) + // Construct and start 2 working backends. + for i := 0; i < 2; i++ { + backend := &stubserver.StubServer{ + EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { + return &testpb.Empty{}, nil + }, + } + if err := backend.StartServer(); err != nil { + t.Fatalf("Failed to start backend: %v", err) + } + t.Logf("Started good TestService backend at: %q", backend.Address) + backends[i] = backend + addresses[i] = backend.Address } -} -func (verifyBalancerBuilder) Name() string { - return verifyBalancerName -} + // Construct and start a failing backend. + backend := &stubserver.StubServer{ + EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { + return nil, errors.New("some error") + }, + } + if err := backend.StartServer(); err != nil { + t.Fatalf("Failed to start backend: %v", err) + } + t.Logf("Started good TestService backend at: %q", backend.Address) + backends[2] = backend + addresses[2] = backend.Address -// verifyBalancer is a balancer that verifies after a Close() call, -// no other balancer.Balancer methods are called afterward. -type verifyBalancer struct { - closed *grpcsync.Event - // To fail the test if any balancer.Balancer operation gets called after - // Close(). - t *testing.T + return addresses, backends } -func (vb *verifyBalancer) UpdateClientConnState(ccs balancer.ClientConnState) error { - if vb.closed.HasFired() { - vb.t.Fatal("UpdateClientConnState was called after Close(), which breaks the balancer API") +// checkRoundRobinRPCs verifies that EmptyCall RPCs on the given ClientConn, +// connected to a server exposing the test.grpc_testing.TestService, are +// roundrobin-ed across the given backend addresses. +// +// Returns a non-nil error if context deadline expires before RPCs start to get +// roundrobin-ed across the given backends. +func checkRoundRobinRPCs(ctx context.Context, client testpb.TestServiceClient, addrs []resolver.Address) error { + wantAddrCount := make(map[string]int) + for _, addr := range addrs { + wantAddrCount[addr.Addr]++ + } + for ; ctx.Err() == nil; <-time.After(time.Millisecond) { + // Perform 3 iterations. + var iterations [][]string + for i := 0; i < 3; i++ { + iteration := make([]string, len(addrs)) + for c := 0; c < len(addrs); c++ { + var peer peer.Peer + client.EmptyCall(ctx, &testpb.Empty{}, grpc.Peer(&peer)) + if peer.Addr != nil { + iteration[c] = peer.Addr.String() + } + } + iterations = append(iterations, iteration) + } + // Ensure the the first iteration contains all addresses in addrs. + gotAddrCount := make(map[string]int) + for _, addr := range iterations[0] { + gotAddrCount[addr]++ + } + if diff := cmp.Diff(gotAddrCount, wantAddrCount); diff != "" { + logger.Infof("non-roundrobin, got address count in one iteration: %v, want: %v, Diff: %s", gotAddrCount, wantAddrCount, diff) + continue + } + // Ensure all three iterations contain the same addresses. + if !cmp.Equal(iterations[0], iterations[1]) || !cmp.Equal(iterations[0], iterations[2]) { + logger.Infof("non-roundrobin, first iter: %v, second iter: %v, third iter: %v", iterations[0], iterations[1], iterations[2]) + continue + } + return nil } - return nil + return fmt.Errorf("timeout when waiting for roundrobin distribution of RPCs across addresses: %v", addrs) } -func (vb *verifyBalancer) ResolverError(err error) { - if vb.closed.HasFired() { - vb.t.Fatal("ResolverError was called after Close(), which breaks the balancer API") +// TestOutlierDetectionAlgorithmsE2E tests the Outlier Detection Success Rate +// and Failure Percentage algorithms in an e2e fashion. The Outlier Detection +// Balancer is configured as the top level LB Policy of the channel with a Round +// Robin child, and connects to three upstreams. Two of the upstreams are healthy and +// one is unhealthy. The two algorithms should at some point eject the failing +// upstream, causing RPC's to not be routed to those two upstreams, and only be +// Round Robined across the two healthy upstreams. Other than the intervals the +// two unhealthy upstreams are ejected, RPC's should regularly round robin +// across all three upstreams. +func (s) TestOutlierDetectionAlgorithmsE2E(t *testing.T) { + tests := []struct { + name string + odscJSON string + }{ + { + name: "Success Rate Algorithm", + odscJSON: ` +{ + "loadBalancingConfig": [ + { + "outlier_detection_experimental": { + "interval": 50000000, + "baseEjectionTime": 100000000, + "maxEjectionTime": 300000000000, + "maxEjectionPercent": 33, + "successRateEjection": { + "stdevFactor": 50, + "enforcementPercentage": 100, + "minimumHosts": 3, + "requestVolume": 5 + }, + "childPolicy": [ + { + "round_robin": {} + } + ] + } + } + ] +}`, + }, + { + name: "Failure Percentage Algorithm", + odscJSON: ` +{ + "loadBalancingConfig": [ + { + "outlier_detection_experimental": { + "interval": 50000000, + "baseEjectionTime": 100000000, + "maxEjectionTime": 300000000000, + "maxEjectionPercent": 33, + "failurePercentageEjection": { + "threshold": 50, + "enforcementPercentage": 100, + "minimumHosts": 3, + "requestVolume": 5 + }, + "childPolicy": [ + { + "round_robin": {} + } + ] + } + } + ] +}`, + }, } -} + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + defer internal.UnregisterOutlierDetectionBalancerForTesting() + addresses, backends := setupBackends(t) + defer func() { + for _, backend := range backends { + backend.Stop() + } + }() -func (vb *verifyBalancer) UpdateSubConnState(sc balancer.SubConn, state balancer.SubConnState) { - if vb.closed.HasFired() { - vb.t.Fatal("UpdateSubConnState was called after Close(), which breaks the balancer API") - } -} + // The addresses which don't return errors. + okAddresses := []resolver.Address{ + { + Addr: addresses[0], + }, + { + Addr: addresses[1], + }, + } -func (vb *verifyBalancer) Close() { - vb.closed.Fire() -} + // The full list of addresses. + fullAddresses := []resolver.Address{ + { + Addr: addresses[0], + }, + { + Addr: addresses[1], + }, + { + Addr: addresses[2], + }, + } + + mr := manual.NewBuilderWithScheme("od-e2e") + defer mr.Close() + + sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(test.odscJSON) + + mr.InitialState(resolver.State{ + Addresses: fullAddresses, + ServiceConfig: sc, + }) + + cc, err := grpc.Dial(mr.Scheme()+":///", grpc.WithResolvers(mr), grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + t.Fatalf("grpc.Dial() failed: %v", err) + } + defer cc.Close() + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + testServiceClient := testpb.NewTestServiceClient(cc) + + // At first, due to no statistics on each of the backends, the 3 + // upstreams should all be round robined across. + if err = checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } + + // After calling the three upstreams, one of them constantly error + // and should eventually be ejected for a period of time. This + // period of time should cause the RPC's to be round robined only + // across the two that are healthy. + if err = checkRoundRobinRPCs(ctx, testServiceClient, okAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } -func (vb *verifyBalancer) ExitIdle() { - if vb.closed.HasFired() { - vb.t.Fatal("ExitIdle was called after Close(), which breaks the balancer API") + // The failing upstream isn't ejected indefinitely, and eventually + // should be unejected in subsequent iterations of the interval + // algorithm as per the spec for the two specific algorithms. + if err = checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } + }) } } -// TestUpdateClientConnStateSinglePickerUpdate tests that on an -// UpdateClientConnState call on the Outlier Detection Balancer, only a single -// picker update is sent back. -func (s) TestUpdateClientConnStateSinglePickerUpdate(t *testing.T) { +// TestNoopConfiguration tests the Outlier Detection Balancer configured with a +// noop configuration. The noop configuration should cause the Outlier Detection +// Balancer to not count RPC's, and thus never eject any upstreams and continue +// to route to every upstream connected to, even if they continuously error. +// Once the Outlier Detection Balancer gets reconfigured with configuration +// requiring counting RPC's, the Outlier Detection Balancer should start +// ejecting any upstreams as specified in the configuration. +func (s) TestNoopConfiguration(t *testing.T) { internal.RegisterOutlierDetectionBalancerForTesting() - od, tcc := setup(t) + defer internal.UnregisterOutlierDetectionBalancerForTesting() + addresses, backends := setupBackends(t) defer func() { - od.Close() - internal.UnregisterOutlierDetectionBalancerForTesting() + for _, backend := range backends { + backend.Stop() + } }() - od.UpdateClientConnState(balancer.ClientConnState{ - BalancerConfig: &LBConfig{ - Interval: 10 * time.Second, - ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: twoUpdateStateBalancerName, - Config: twoUpdateStateBalancerConfig{}, - }, - }, - }) - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - // Should forward the connectivity State to Client Conn. - select { - case <-ctx.Done(): - t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") - case state := <-tcc.NewStateCh: - if state != connectivity.Ready { - t.Fatalf("ClientConn received connectivity state %v, want %v", state, connectivity.Ready) - } - } + mr := manual.NewBuilderWithScheme("od-e2e") + defer mr.Close() - select { - case <-ctx.Done(): - t.Fatalf("timeout while waiting for a UpdateState call on the ClientConn") - case <-tcc.NewPickerCh: + // The addresses which don't return errors. + okAddresses := []resolver.Address{ + { + Addr: addresses[0], + }, + { + Addr: addresses[1], + }, } - sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) - defer sCancel() - - // Should only send one update. - select { - case <-tcc.NewStateCh: - t.Fatalf("only one picker update should have gotten sent") - case <-sCtx.Done(): + // The full list of addresses. + fullAddresses := []resolver.Address{ + { + Addr: addresses[0], + }, + { + Addr: addresses[1], + }, + { + Addr: addresses[2], + }, } - sCtx, sCancel = context.WithTimeout(context.Background(), defaultTestShortTimeout) - defer sCancel() - select { - case <-tcc.NewStateCh: - t.Fatalf("only one picker update should have gotten sent") - case <-sCtx.Done(): + noopODServiceConfigJSON := ` +{ + "loadBalancingConfig": [ + { + "outlier_detection_experimental": { + "interval": 50000000, + "baseEjectionTime": 100000000, + "maxEjectionTime": 300000000000, + "maxEjectionPercent": 33, + "childPolicy": [ + { + "round_robin": {} + } + ] + } + } + ] +}` + sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(noopODServiceConfigJSON) + + mr.InitialState(resolver.State{ + Addresses: fullAddresses, + ServiceConfig: sc, + }) + cc, err := grpc.Dial(mr.Scheme()+":///", grpc.WithResolvers(mr), grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + t.Fatalf("grpc.Dial() failed: %v", err) } -} - -type twoUpdateStateBalancerBuilder struct{} - -func (twoUpdateStateBalancerBuilder) Build(cc balancer.ClientConn, opts balancer.BuildOptions) balancer.Balancer { - return &twoUpdateStateBalancer{ - cc: cc, + defer cc.Close() + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + testServiceClient := testpb.NewTestServiceClient(cc) + + for i := 0; i < 2; i++ { + // Since the Outlier Detection Balancer starts with a noop + // configuration, it shouldn't count RPCs or eject any upstreams. Thus, + // even though an upstream it connects to constantly errors, it should + // continue to Round Robin across every upstream. + if err := checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } } -} - -func (twoUpdateStateBalancerBuilder) Name() string { - return "twoUpdateStateBalancer" -} - -type twoUpdateStateBalancerConfig struct { - serviceconfig.LoadBalancingConfig -} -// twoUpdateStateBalancer sends two UpdateState calls inline in -// UpdateClientConnState(). This helps to verify that only a single picker -// update gets sent upward as a result of the call. -type twoUpdateStateBalancer struct { - t *testing.T - cc balancer.ClientConn -} - -func (tusb *twoUpdateStateBalancer) UpdateClientConnState(ccs balancer.ClientConnState) error { - tusb.cc.UpdateState(balancer.State{ - ConnectivityState: connectivity.Ready, - Picker: &testutils.TestConstPicker{}, - }) - tusb.cc.UpdateState(balancer.State{ - ConnectivityState: connectivity.Ready, - Picker: &testutils.TestConstPicker{}, + // Reconfigure the Outlier Detection Balancer with a configuration that + // specifies to count RPC's and eject upstreams. Due to the balancer no + // longer being a noop, it should eject any unhealthy addresses as specified + // by the failure percentage portion of the configuration. + countingODServiceConfigJSON := ` +{ + "loadBalancingConfig": [ + { + "outlier_detection_experimental": { + "interval": 50000000, + "baseEjectionTime": 100000000, + "maxEjectionTime": 300000000000, + "maxEjectionPercent": 33, + "failurePercentageEjection": { + "threshold": 50, + "enforcementPercentage": 100, + "minimumHosts": 3, + "requestVolume": 5 + }, + "childPolicy": [ + { + "round_robin": {} + } + ] + } + } + ] +}` + sc = internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(countingODServiceConfigJSON) + + mr.UpdateState(resolver.State{ + Addresses: fullAddresses, + ServiceConfig: sc, }) - return nil -} -func (tusb *twoUpdateStateBalancer) ResolverError(err error) {} + // At first on the reconfigured balancer, the balancer has no stats + // collected about upstreams. Thus, it should at first route across the full + // upstream list. + if err = checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } -func (tusb *twoUpdateStateBalancer) UpdateSubConnState(sc balancer.SubConn, state balancer.SubConnState) { + // Now that the reconfigured balancer has data about the failing upstream, + // it should eject the upstream and only route across the two healthy + // upstreams. + if err = checkRoundRobinRPCs(ctx, testServiceClient, okAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } } - -func (tusb *twoUpdateStateBalancer) Close() {} diff --git a/xds/internal/balancer/outlierdetection/callcounter.go b/xds/internal/balancer/outlierdetection/callcounter.go index 525f09dd1d3..1f53f3cd300 100644 --- a/xds/internal/balancer/outlierdetection/callcounter.go +++ b/xds/internal/balancer/outlierdetection/callcounter.go @@ -34,10 +34,10 @@ func newCallCounter() *callCounter { } } -// callCounter has two buckets, which each count successes and failures. The -// activeBucket is used to actively count any finished RPC's, and the +// callCounter has two buckets, which each count successful and failing RPC's. +// The activeBucket is used to actively count any finished RPC's, and the // inactiveBucket is populated with this activeBucket's data every interval for -// use by the Outlier Detection algorithm. +// use by the Outlier Detection algorithm. Caller must hold b.mu. type callCounter struct { // activeBucket updates every time a call finishes (from picker passed to // Client Conn), so protect pointer read with atomic load of unsafe.Pointer @@ -54,7 +54,7 @@ func (cc *callCounter) clear() { // "When the timer triggers, the inactive bucket is zeroed and swapped with the // active bucket. Then the inactive bucket contains the number of successes and // failures since the last time the timer triggered. Those numbers are used to -// evaluate the ejection criteria." - A50 +// evaluate the ejection criteria." - A50. Caller must hold b.mu. func (cc *callCounter) swap() { ib := cc.inactiveBucket *ib = bucket{} From 1b1b1fc1f7c26794f7320d8c817e994a63ed5500 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Wed, 24 Aug 2022 13:54:21 -0400 Subject: [PATCH 15/22] Change in Failure Percentage Ejection spec --- .../balancer/outlierdetection/balancer.go | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 35053e9f19c..38907e320c7 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -735,14 +735,13 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { } // addrsWithAtLeastRequestVolume returns a slice of address information of all -// addresses with at least request volume defined in the success rate ejection -// configuration. Caller must hold b.mu. -func (b *outlierDetectionBalancer) addrsWithAtLeastRequestVolume() []*addressInfo { +// addresses with at least request volume passed in. Caller must hold b.mu. +func (b *outlierDetectionBalancer) addrsWithAtLeastRequestVolume(requestVolume uint32) []*addressInfo { var addrs []*addressInfo for _, addrInfo := range b.addrs { bucket := addrInfo.callCounter.inactiveBucket rv := bucket.numSuccesses + bucket.numFailures - if rv >= b.cfg.SuccessRateEjection.RequestVolume { + if rv >= requestVolume { addrs = append(addrs, addrInfo) } } @@ -775,7 +774,7 @@ func (b *outlierDetectionBalancer) meanAndStdDev(addrs []*addressInfo) (float64, // the other addresses according to mean and standard deviation, and if overall // applicable from other set heuristics. Caller must hold b.mu. func (b *outlierDetectionBalancer) successRateAlgorithm() { - addrsToConsider := b.addrsWithAtLeastRequestVolume() + addrsToConsider := b.addrsWithAtLeastRequestVolume(b.cfg.SuccessRateEjection.RequestVolume) if len(addrsToConsider) < int(b.cfg.SuccessRateEjection.MinimumHosts) { return } @@ -799,21 +798,18 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { // rate exceeds a set enforcement percentage, if overall applicable from other // set heuristics. Caller must hold b.mu. func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { - if uint32(len(b.addrs)) < b.cfg.FailurePercentageEjection.MinimumHosts { + addrsToConsider := b.addrsWithAtLeastRequestVolume(b.cfg.FailurePercentageEjection.RequestVolume) + if len(addrsToConsider) < int(b.cfg.FailurePercentageEjection.MinimumHosts) { return } - for _, addrInfo := range b.addrs { + for _, addrInfo := range addrsToConsider { bucket := addrInfo.callCounter.inactiveBucket ejectionCfg := b.cfg.FailurePercentageEjection if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { return } - rv := bucket.numSuccesses + bucket.numFailures - if rv < ejectionCfg.RequestVolume { - continue - } - failurePercentage := (float64(bucket.numFailures) / float64(rv)) * 100 + failurePercentage := (float64(bucket.numFailures) / float64(bucket.numSuccesses+bucket.numFailures)) * 100 if failurePercentage > float64(b.cfg.FailurePercentageEjection.Threshold) { if uint32(grpcrand.Int31n(100)) < ejectionCfg.EnforcementPercentage { b.ejectAddress(addrInfo) From 35c9bae2142ed5e635928a7c8a7242140c717eb8 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Thu, 25 Aug 2022 12:58:01 -0400 Subject: [PATCH 16/22] Another spec change --- xds/internal/balancer/outlierdetection/balancer.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 38907e320c7..1805f8ba7a7 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -782,7 +782,7 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { for _, addrInfo := range addrsToConsider { bucket := addrInfo.callCounter.inactiveBucket ejectionCfg := b.cfg.SuccessRateEjection - if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { + if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 >= float64(b.cfg.MaxEjectionPercent) { return } successRate := float64(bucket.numSuccesses) / float64(bucket.numSuccesses+bucket.numFailures) @@ -806,7 +806,7 @@ func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { for _, addrInfo := range addrsToConsider { bucket := addrInfo.callCounter.inactiveBucket ejectionCfg := b.cfg.FailurePercentageEjection - if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 > float64(b.cfg.MaxEjectionPercent) { + if float64(b.numAddrsEjected)/float64(len(b.addrs))*100 >= float64(b.cfg.MaxEjectionPercent) { return } failurePercentage := (float64(bucket.numFailures) / float64(bucket.numSuccesses+bucket.numFailures)) * 100 From 0a4d618937fa8d85f2ecca523507003a8903726d Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Thu, 25 Aug 2022 20:13:19 -0400 Subject: [PATCH 17/22] Responded to Easwar's comments --- .../balancer/outlierdetection/balancer.go | 119 ++- .../outlierdetection/balancer_test.go | 822 ++++-------------- .../balancer/outlierdetection/callcounter.go | 8 +- .../e2e_test/outlierdetection_test.go | 394 +++++++++ 4 files changed, 665 insertions(+), 678 deletions(-) create mode 100644 xds/internal/balancer/outlierdetection/e2e_test/outlierdetection_test.go diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 1805f8ba7a7..8e35fb43d1c 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -156,19 +156,30 @@ type lbCfgUpdate struct { } type outlierDetectionBalancer struct { - childState balancer.State + // These fields are safe to be accessed without holding any mutex because + // they are synchronized in run(), which makes these field accesses happen + // serially. + // + // childState is the latest balancer state received from the child. + childState balancer.State + // recentPickerNoop represents whether the most recent picker sent upward to + // the balancer.ClientConn is a noop picker, which doesn't count RPC's. Used + // to suppress redundant picker updates. recentPickerNoop bool - firstPickerSent bool closed *grpcsync.Event cc balancer.ClientConn bOpts balancer.BuildOptions logger *grpclog.PrefixLogger - // childMu protects child and also updates to the child (to uphold the - // balancer.Balancer API guarantee of synchronous calls). It also protects - // against run() reading that the child is not nil for SubConn updates, and - // then UpdateClientConnState or Close writing to the the child. + // childMu protects child and also guarantees updates to the child are sent + // synchronously (to uphold the balancer.Balancer API guarantee of + // synchronous calls). + // + // For example, run() could read that the child is not nil while processing + // SubConn updates, and then Close() could write to the the child, clearing + // the child, making it nil, then you try and update a cleared and already + // closed child, which breaks the balancer.Balancer API. childMu sync.Mutex child *gracefulswitch.Balancer @@ -186,28 +197,38 @@ type outlierDetectionBalancer struct { // SubConn address update in the middle of the interval timer algorithm // which uses addrs. This balancer waits for the interval timer algorithm to // finish before making the update to the addrs map. - mu sync.Mutex - addrs map[string]*addressInfo - cfg *LBConfig - scWrappers map[balancer.SubConn]*subConnWrapper - timerStartTime time.Time - intervalTimer *time.Timer - inhibitPickerUpdates bool - numAddrsEjected int // For fast calculations of percentage of addrs ejected + // + // This mutex is never held at the same time as childMu (within the context + // of a single goroutine). + mu sync.Mutex + addrs map[string]*addressInfo + cfg *LBConfig + scWrappers map[balancer.SubConn]*subConnWrapper + timerStartTime time.Time + intervalTimer *time.Timer + inhibitPickerUpdates bool + updateUnconditionally bool + numAddrsEjected int // For fast calculations of percentage of addrs ejected scUpdateCh *buffer.Unbounded pickerUpdateCh *buffer.Unbounded } // noopConfig returns whether this balancer is configured with a logical no-op -// configuration or not. Caller must hold b.mu. +// configuration or not. +// +// Caller must hold b.mu. func (b *outlierDetectionBalancer) noopConfig() bool { return b.cfg.SuccessRateEjection == nil && b.cfg.FailurePercentageEjection == nil } -// onCountingConfig handles logic required specifically on the receipt of a -// configuration which will count RPC's. Caller must hold b.mu. -func (b *outlierDetectionBalancer) onCountingConfig() { +// onIntervalConfig handles logic required specifically on the receipt of a +// configuration which specifies to count RPC's and periodically perform passive +// health checking based on heuristics defined in configuration every configured +// interval. +// +// Caller must hold b.mu. +func (b *outlierDetectionBalancer) onIntervalConfig() { var interval time.Duration if b.timerStartTime.IsZero() { b.timerStartTime = time.Now() @@ -224,9 +245,10 @@ func (b *outlierDetectionBalancer) onCountingConfig() { b.intervalTimer = afterFunc(interval, b.intervalTimerAlgorithm) } -// onCountingConfig handles logic required specifically on the receipt of a -// configuration which specifies the balancer to be a noop. Caller must hold -// b.mu. +// onNoopConfig handles logic required specifically on the receipt of a +// configuration which specifies the balancer to be a noop. +// +// Caller must hold b.mu. func (b *outlierDetectionBalancer) onNoopConfig() { // "If a config is provided with both the `success_rate_ejection` and // `failure_percentage_ejection` fields unset, skip starting the timer and @@ -257,6 +279,10 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt return fmt.Errorf("outlier detection: child balancer %q not registered", lbCfg.ChildPolicy.Name) } + // It is safe to read b.cfg here without holding the mutex, as the only + // write to b.cfg happens later in this function. This function is part of + // the balancer.Balancer API, so it is guaranteed to be called in a + // synchronous manner, so it cannot race with this read. if b.cfg == nil || b.cfg.ChildPolicy.Name != lbCfg.ChildPolicy.Name { b.childMu.Lock() err := b.child.SwitchTo(bb) @@ -274,6 +300,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt // sent synchronously upward at the end of this UpdateClientConnState() // call. b.inhibitPickerUpdates = true + b.updateUnconditionally = false b.cfg = lbCfg addrs := make(map[string]bool, len(s.ResolverState.Addresses)) @@ -293,10 +320,10 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt b.intervalTimer.Stop() } - if !b.noopConfig() { - b.onCountingConfig() - } else { + if b.noopConfig() { b.onNoopConfig() + } else { + b.onIntervalConfig() } b.mu.Unlock() @@ -486,7 +513,9 @@ func (b *outlierDetectionBalancer) RemoveSubConn(sc balancer.SubConn) { // appendIfPresent appends the scw to the address, if the address is present in // the Outlier Detection balancers address map. Returns nil if not present, and -// the map entry if present. Caller must hold b.mu. +// the map entry if present. +// +// Caller must hold b.mu. func (b *outlierDetectionBalancer) appendIfPresent(addr string, scw *subConnWrapper) *addressInfo { addrInfo, ok := b.addrs[addr] if !ok { @@ -499,7 +528,9 @@ func (b *outlierDetectionBalancer) appendIfPresent(addr string, scw *subConnWrap } // removeSubConnFromAddressesMapEntry removes the scw from its map entry if -// present. Caller must hold b.mu. +// present. +// +// Caller must hold b.mu. func (b *outlierDetectionBalancer) removeSubConnFromAddressesMapEntry(scw *subConnWrapper) { addrInfo := (*addressInfo)(atomic.LoadPointer(&scw.addressInfo)) if addrInfo == nil { @@ -622,13 +653,16 @@ func (b *outlierDetectionBalancer) handleChildStateUpdate(u balancer.State) { b.childState = u b.mu.Lock() if b.inhibitPickerUpdates { + // If a child's state is updated during the suppression of child + // updates, the synchronous handleLBConfigUpdate function with respect + // to UpdateClientConnState should return a picker unconditionally. + b.updateUnconditionally = true b.mu.Unlock() return } noopCfg := b.noopConfig() b.mu.Unlock() b.recentPickerNoop = noopCfg - b.firstPickerSent = true b.cc.UpdateState(balancer.State{ ConnectivityState: b.childState.ConnectivityState, Picker: &wrappedPicker{ @@ -643,11 +677,16 @@ func (b *outlierDetectionBalancer) handleChildStateUpdate(u balancer.State) { // changed compared to the picker currently in use. func (b *outlierDetectionBalancer) handleLBConfigUpdate(u lbCfgUpdate) { lbCfg := u.lbCfg - done := u.done noopCfg := lbCfg.SuccessRateEjection == nil && lbCfg.FailurePercentageEjection == nil - if b.childState.Picker != nil && noopCfg != b.recentPickerNoop || b.childState.Picker != nil && !b.firstPickerSent { + // If the child has sent it's first update and this config flips the noop + // bit compared to the most recent picker update sent upward, then a new + // picker with this updated bit needs to be forwarded upward. If a child + // update was received during the suppression of child updates within + // UpdateClientConnState(), then a new picker needs to be forwarded with + // this updated state, irregardless of whether this new configuration flips + // the bit. + if b.childState.Picker != nil && noopCfg != b.recentPickerNoop || b.updateUnconditionally { b.recentPickerNoop = noopCfg - b.firstPickerSent = true b.cc.UpdateState(balancer.State{ ConnectivityState: b.childState.ConnectivityState, Picker: &wrappedPicker{ @@ -657,7 +696,8 @@ func (b *outlierDetectionBalancer) handleLBConfigUpdate(u lbCfgUpdate) { }) } b.inhibitPickerUpdates = false - close(done) + b.updateUnconditionally = false + close(u.done) } func (b *outlierDetectionBalancer) run() { @@ -735,7 +775,9 @@ func (b *outlierDetectionBalancer) intervalTimerAlgorithm() { } // addrsWithAtLeastRequestVolume returns a slice of address information of all -// addresses with at least request volume passed in. Caller must hold b.mu. +// addresses with at least request volume passed in. +// +// Caller must hold b.mu. func (b *outlierDetectionBalancer) addrsWithAtLeastRequestVolume(requestVolume uint32) []*addressInfo { var addrs []*addressInfo for _, addrInfo := range b.addrs { @@ -749,7 +791,9 @@ func (b *outlierDetectionBalancer) addrsWithAtLeastRequestVolume(requestVolume u } // meanAndStdDev returns the mean and std dev of the fractions of successful -// requests of the addresses passed in. Caller must hold b.mu. +// requests of the addresses passed in. +// +// Caller must hold b.mu. func (b *outlierDetectionBalancer) meanAndStdDev(addrs []*addressInfo) (float64, float64) { var totalFractionOfSuccessfulRequests float64 var mean float64 @@ -772,7 +816,9 @@ func (b *outlierDetectionBalancer) meanAndStdDev(addrs []*addressInfo) (float64, // successRateAlgorithm ejects any addresses where the success rate falls below // the other addresses according to mean and standard deviation, and if overall -// applicable from other set heuristics. Caller must hold b.mu. +// applicable from other set heuristics. +// +// Caller must hold b.mu. func (b *outlierDetectionBalancer) successRateAlgorithm() { addrsToConsider := b.addrsWithAtLeastRequestVolume(b.cfg.SuccessRateEjection.RequestVolume) if len(addrsToConsider) < int(b.cfg.SuccessRateEjection.MinimumHosts) { @@ -796,7 +842,9 @@ func (b *outlierDetectionBalancer) successRateAlgorithm() { // failurePercentageAlgorithm ejects any addresses where the failure percentage // rate exceeds a set enforcement percentage, if overall applicable from other -// set heuristics. Caller must hold b.mu. +// set heuristics. +// +// Caller must hold b.mu. func (b *outlierDetectionBalancer) failurePercentageAlgorithm() { addrsToConsider := b.addrsWithAtLeastRequestVolume(b.cfg.FailurePercentageEjection.RequestVolume) if len(addrsToConsider) < int(b.cfg.FailurePercentageEjection.MinimumHosts) { @@ -861,6 +909,5 @@ type addressInfo struct { func newAddressInfo() *addressInfo { return &addressInfo{ callCounter: newCallCounter(), - sws: make([]*subConnWrapper, 0), } } diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index 0373c94b333..031c40b289d 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -23,6 +23,7 @@ import ( "encoding/json" "errors" "fmt" + "math" "strings" "sync" "testing" @@ -30,22 +31,16 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" - "google.golang.org/grpc" "google.golang.org/grpc/balancer" "google.golang.org/grpc/connectivity" - "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/internal" "google.golang.org/grpc/internal/balancer/stub" "google.golang.org/grpc/internal/grpcsync" "google.golang.org/grpc/internal/grpctest" internalserviceconfig "google.golang.org/grpc/internal/serviceconfig" - "google.golang.org/grpc/internal/stubserver" "google.golang.org/grpc/internal/testutils" - "google.golang.org/grpc/peer" "google.golang.org/grpc/resolver" - "google.golang.org/grpc/resolver/manual" "google.golang.org/grpc/serviceconfig" - testpb "google.golang.org/grpc/test/grpc_testing" "google.golang.org/grpc/xds/internal/balancer/clusterimpl" ) @@ -65,6 +60,7 @@ func Test(t *testing.T) { // TestParseConfig verifies the ParseConfig() method in the Outlier Detection // Balancer. func (s) TestParseConfig(t *testing.T) { + const errParseConfigName = "errParseConfigBalancer" stub.Register(errParseConfigName, stub.BalancerFuncs{ ParseConfig: func(json.RawMessage) (serviceconfig.LoadBalancingConfig, error) { return nil, errors.New("some error") @@ -297,10 +293,6 @@ func (lbc *LBConfig) Equal(lbc2 *LBConfig) bool { return cmp.Equal(lbc.ChildPolicy, lbc2.ChildPolicy) } -const errParseConfigName = "errParseConfigBalancer" -const tcibname = "testClusterImplBalancer" -const verifyBalancerName = "verifyBalancer" - type subConnWithState struct { sc balancer.SubConn state balancer.SubConnState @@ -321,25 +313,28 @@ func setup(t *testing.T) (*outlierDetectionBalancer, *testutils.TestClientConn, } } -type balancerConfig struct { +type emptyChildConfig struct { serviceconfig.LoadBalancingConfig } // TestChildBasicOperations tests basic operations of the Outlier Detection -// Balancer and it's interaction with it's child. On the first receipt of a good -// config, the balancer is expected to eventually create a child and send the -// child it's configuration. When a new configuration comes in that changes the -// child's type which reports READY immediately, the first child balancer should -// be closed and the second child balancer should receive it's first config -// update. When the Outlier Detection Balancer itself is closed, this second -// child balancer should also be closed. +// Balancer and it's interaction with it's child. The following scenarios are +// tested, in a step by step fashion: +// 1. The Outlier Detection Balancer receives it's first good configuration. The +// balancer is expected to create a child and sent the child it's configuration. +// 2. The Outlier Detection Balancer receives new configuration that specifies a +// child's type, and the new type immediately reports READY inline. The first +// child balancer should be closed and the second child balancer should receive +// a config update. +// 3. The Outlier Detection Balancer is closed. The second child balancer should +// be closed. func (s) TestChildBasicOperations(t *testing.T) { - bc := balancerConfig{} + bc := emptyChildConfig{} ccsCh := testutils.NewChannel() closeCh := testutils.NewChannel() - stub.Register(tcibname, stub.BalancerFuncs{ + stub.Register(t.Name()+"child1", stub.BalancerFuncs{ UpdateClientConnState: func(bd *stub.BalancerData, ccs balancer.ClientConnState) error { ccsCh.Send(ccs.BalancerConfig) return nil @@ -349,7 +344,7 @@ func (s) TestChildBasicOperations(t *testing.T) { }, }) - stub.Register(verifyBalancerName, stub.BalancerFuncs{ + stub.Register(t.Name()+"child2", stub.BalancerFuncs{ UpdateClientConnState: func(bd *stub.BalancerData, _ balancer.ClientConnState) error { // UpdateState inline to READY to complete graceful switch process // synchronously from any UpdateClientConnState call. @@ -389,7 +384,7 @@ func (s) TestChildBasicOperations(t *testing.T) { RequestVolume: 50, }, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, + Name: t.Name() + "child1", Config: bc, }, }, @@ -401,8 +396,8 @@ func (s) TestChildBasicOperations(t *testing.T) { if err != nil { t.Fatalf("timed out waiting for UpdateClientConnState on the first child balancer: %v", err) } - if _, ok := cr.(balancerConfig); !ok { - t.Fatalf("config passed to child should be balancerConfig type") + if _, ok := cr.(emptyChildConfig); !ok { + t.Fatalf("Received child policy config of type %T, want %T", cr, emptyChildConfig{}) } // This Update Client Conn State call should cause the first child balancer @@ -427,8 +422,8 @@ func (s) TestChildBasicOperations(t *testing.T) { RequestVolume: 50, }, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: verifyBalancerName, - Config: balancerConfig{}, + Name: t.Name() + "child2", + Config: emptyChildConfig{}, }, }, }) @@ -445,20 +440,17 @@ func (s) TestChildBasicOperations(t *testing.T) { } // Verify the first child balancer closed. - _, err = closeCh.Receive(ctx) - if err != nil { + if _, err = closeCh.Receive(ctx); err != nil { t.Fatalf("timed out waiting for the first child balancer to be closed: %v", err) } // Verify the second child balancer received it's first config update. - _, err = ccsCh.Receive(ctx) - if err != nil { + if _, err = ccsCh.Receive(ctx); err != nil { t.Fatalf("timed out waiting for UpdateClientConnState on the second child balancer: %v", err) } // Closing the Outlier Detection Balancer should close the newly created // child. od.Close() - _, err = closeCh.Receive(ctx) - if err != nil { + if _, err = closeCh.Receive(ctx); err != nil { t.Fatalf("timed out waiting for the second child balancer to be closed: %v", err) } } @@ -466,36 +458,41 @@ func (s) TestChildBasicOperations(t *testing.T) { // TestUpdateAddresses tests the functionality of UpdateAddresses and any // changes in the addresses/plurality of those addresses for a SubConn. The // Balancer is set up with two upstreams, with one of the upstreams being -// ejected. Switching a SubConn's address list to the ejected address should -// cause the SubConn to be ejected, if not already. Switching the address list -// from single to plural should cause this SubConn to be unejected, since the -// SubConn is no longer being tracked by Outlier Detection. Then, switching this -// SubConn back to the single ejected address should reeject the SubConn. +// ejected. Initially, there is one SubConn for each address. The following +// scenarios are tested, in a step by step fashion: +// 1. The SubConn not currently ejected switches addresses to the address that +// is ejected. This should cause the SubConn to get ejected. +// 2. Update this same SubConn to multiple addresses. This should cause the +// SubConn to get unejected, as it is no longer being tracked by Outlier +// Detection at that point. +// 3. Update this same SubConn to different addresses, still multiple. This +// should be a noop, as the SubConn is still no longer being tracked by Outlier +// Detection. +// 4. Update this same SubConn to the a single address which is ejected. This +// should cause the SubConn to be ejected. func (s) TestUpdateAddresses(t *testing.T) { scsCh := testutils.NewChannel() var scw1, scw2 balancer.SubConn var err error - stub.Register(tcibname, stub.BalancerFuncs{ + stub.Register(t.Name(), stub.BalancerFuncs{ UpdateClientConnState: func(bd *stub.BalancerData, _ balancer.ClientConnState) error { - scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{ - { - Addr: "address1", - }, - }, balancer.NewSubConnOptions{}) + scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address1"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } - scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{ - { - Addr: "address2", - }, - }, balancer.NewSubConnOptions{}) + scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address2"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } + bd.ClientConn.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &rrPicker{ + scs: []balancer.SubConn{scw1, scw2}, + }, + }) return nil }, UpdateSubConnState: func(_ *stub.BalancerData, sc balancer.SubConn, state balancer.SubConnState) { @@ -511,12 +508,8 @@ func (s) TestUpdateAddresses(t *testing.T) { od.UpdateClientConnState(balancer.ClientConnState{ ResolverState: resolver.State{ Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, + {Addr: "address1"}, + {Addr: "address2"}, }, }, BalancerConfig: &LBConfig{ @@ -531,8 +524,8 @@ func (s) TestUpdateAddresses(t *testing.T) { RequestVolume: 3, }, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: balancerConfig{}, + Name: t.Name(), + Config: emptyChildConfig{}, }, }, }) @@ -540,13 +533,6 @@ func (s) TestUpdateAddresses(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - od.UpdateState(balancer.State{ - ConnectivityState: connectivity.Ready, - Picker: &rrPicker{ - scs: []balancer.SubConn{scw1, scw2}, - }, - }) - // Setup the system to where one address is ejected and one address // isn't. select { @@ -555,23 +541,24 @@ func (s) TestUpdateAddresses(t *testing.T) { case picker := <-tcc.NewPickerCh: pi, err := picker.Pick(balancer.PickInfo{}) if err != nil { - t.Fatalf("Picker.Pick should not have errored") + t.Fatalf("picker.Pick failed with error: %v", err) + } + // Simulate 5 successful RPC calls on the first SubConn (the first call + // to picker.Pick). + for c := 0; c < 5; c++ { + pi.Done(balancer.DoneInfo{}) } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - // Eject the second address. pi, err = picker.Pick(balancer.PickInfo{}) if err != nil { - t.Fatalf("Picker.Pick should not have errored") + t.Fatalf("picker.Pick failed with error: %v", err) + } + // Simulate 5 failed RPC calls on the second SubConn (the second call to + // picker.Pick). Thus, when the interval timer algorithm is run, the + // second SubConn's address should be ejected, which will allow us to + // further test UpdateAddresses() logic. + for c := 0; c < 5; c++ { + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) } - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) od.intervalTimerAlgorithm() // verify UpdateSubConnState() got called with TRANSIENT_FAILURE for // child with address that was ejected. @@ -589,11 +576,7 @@ func (s) TestUpdateAddresses(t *testing.T) { // Update scw1 to another address that is currently ejected. This should // cause scw1 to get ejected. - od.UpdateAddresses(scw1, []resolver.Address{ - { - Addr: "address2", - }, - }) + od.UpdateAddresses(scw1, []resolver.Address{{Addr: "address2"}}) // Verify that update addresses gets forwarded to ClientConn. select { @@ -617,12 +600,8 @@ func (s) TestUpdateAddresses(t *testing.T) { // Update scw1 to multiple addresses. This should cause scw1 to get // unejected, as is it no longer being tracked for Outlier Detection. od.UpdateAddresses(scw1, []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, + {Addr: "address1"}, + {Addr: "address2"}, }) // Verify scw1 got unejected (UpdateSubConnState called with recent state). gotSCWS, err = scsCh.Receive(ctx) @@ -640,12 +619,8 @@ func (s) TestUpdateAddresses(t *testing.T) { // in which the plurality goes from multiple to multiple should be a no-op, // as the address continues to be ignored by outlier detection. od.UpdateAddresses(scw1, []resolver.Address{ - { - Addr: "address2", - }, - { - Addr: "address3", - }, + {Addr: "address2"}, + {Addr: "address3"}, }) // Verify no downstream effects. sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) @@ -656,11 +631,7 @@ func (s) TestUpdateAddresses(t *testing.T) { // Update scw1 back to a single address, which is ejected. This should cause // the SubConn to be re-ejected. - od.UpdateAddresses(scw1, []resolver.Address{ - { - Addr: "address2", - }, - }) + od.UpdateAddresses(scw1, []resolver.Address{{Addr: "address2"}}) // Verify scw1 got ejected (UpdateSubConnState called with TRANSIENT FAILURE). gotSCWS, err = scsCh.Receive(ctx) if err != nil { @@ -692,15 +663,17 @@ func (rrp *rrPicker) Pick(balancer.PickInfo) (balancer.PickResult, error) { return balancer.PickResult{SubConn: sc}, nil } -// TestDurationOfInterval tests the configured interval timer. On the first -// config received, the Outlier Detection balancer should configure the timer -// with whatever is directly specified on the config. On subsequent configs -// received, the Outlier Detection balancer should configure the timer with -// whatever interval is configured minus the difference between the current time -// and the previous start timestamp. For a no-op configuration, the timer should -// not be configured at all. +// TestDurationOfInterval tests the configured interval timer. +// The following scenarios are tested: +// 1. The Outlier Detection Balancer receives it's first config. The balancer +// should configure the timer with whatever is directly specified on the config. +// 2. The Outlier Detection Balancer receives a subsequent config. The balancer +// should configure with whatever interval is configured minus the difference +// between the current time and the previous start timestamp. +// 3. The Outlier Detection Balancer receives a no-op configuration. The +// balancer should not configure a timer at all. func (s) TestDurationOfInterval(t *testing.T) { - stub.Register(tcibname, stub.BalancerFuncs{}) + stub.Register(t.Name(), stub.BalancerFuncs{}) od, _, cleanup := setup(t) defer func(af func(d time.Duration, f func()) *time.Timer) { @@ -711,7 +684,7 @@ func (s) TestDurationOfInterval(t *testing.T) { durationChan := testutils.NewChannel() afterFunc = func(dur time.Duration, _ func()) *time.Timer { durationChan.Send(dur) - return time.NewTimer(1<<63 - 1) + return time.NewTimer(math.MaxInt64) } od.UpdateClientConnState(balancer.ClientConnState{ @@ -733,8 +706,8 @@ func (s) TestDurationOfInterval(t *testing.T) { RequestVolume: 50, }, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: balancerConfig{}, + Name: t.Name(), + Config: emptyChildConfig{}, }, }, }) @@ -747,10 +720,12 @@ func (s) TestDurationOfInterval(t *testing.T) { dur := d.(time.Duration) // The configured duration should be 8 seconds - what the balancer was // configured with. - if dur.Seconds() != 8 { + if dur != 8*time.Second { t.Fatalf("configured duration should have been 8 seconds to start timer") } + // Override time.Now to time.Now() + 5 seconds. This will represent 5 + // seconds already passing for the next check in UpdateClientConnState. defer func(n func() time.Time) { now = n }(now) @@ -780,8 +755,8 @@ func (s) TestDurationOfInterval(t *testing.T) { RequestVolume: 50, }, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: balancerConfig{}, + Name: t.Name(), + Config: emptyChildConfig{}, }, }, }) @@ -801,8 +776,8 @@ func (s) TestDurationOfInterval(t *testing.T) { BalancerConfig: &LBConfig{ Interval: 10 * time.Second, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: balancerConfig{}, + Name: t.Name(), + Config: emptyChildConfig{}, }, }, }) @@ -812,47 +787,45 @@ func (s) TestDurationOfInterval(t *testing.T) { // No timer should have been started. sCtx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) defer cancel() - _, err = durationChan.Receive(sCtx) - if err == nil { + if _, err = durationChan.Receive(sCtx); err == nil { t.Fatal("No timer should have started.") } } // TestEjectUnejectSuccessRate tests the functionality of the interval timer -// algorithm of ejecting/unejecting SubConns when configured with -// SuccessRateEjection. It also tests a desired invariant of a SubConnWrapper -// being ejected or unejected, which is to either forward or not forward SubConn -// updates received from grpc. +// algorithm when configured with SuccessRateEjection. The Outlier Detection +// Balancer will be set up with 3 SubConns, each with a different address. +// It tests the following scenarios, in a step by step fashion: +// 1. The three addresses each have 5 successes. The interval timer algorithm should +// not eject any of the addresses. +// 2. Two of the addresses have 5 successes, the third has five failures. The +// interval timer algorithm should eject the third address with five failures. +// 3. The interval timer algorithm is run at a later time past max ejection +// time. The interval timer algorithm should uneject the third address. func (s) TestEjectUnejectSuccessRate(t *testing.T) { scsCh := testutils.NewChannel() var scw1, scw2, scw3 balancer.SubConn var err error - stub.Register(tcibname, stub.BalancerFuncs{ + stub.Register(t.Name(), stub.BalancerFuncs{ UpdateClientConnState: func(bd *stub.BalancerData, _ balancer.ClientConnState) error { - scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{ - { - Addr: "address1", - }, - }, balancer.NewSubConnOptions{}) + scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address1"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } - scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{ - { - Addr: "address2", - }, - }, balancer.NewSubConnOptions{}) + scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address2"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } - scw3, err = bd.ClientConn.NewSubConn([]resolver.Address{ - { - Addr: "address3", - }, - }, balancer.NewSubConnOptions{}) + scw3, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address3"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } + bd.ClientConn.UpdateState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &rrPicker{ + scs: []balancer.SubConn{scw1, scw2, scw3}, + }, + }) return nil }, UpdateSubConnState: func(_ *stub.BalancerData, sc balancer.SubConn, state balancer.SubConnState) { @@ -871,15 +844,9 @@ func (s) TestEjectUnejectSuccessRate(t *testing.T) { od.UpdateClientConnState(balancer.ClientConnState{ ResolverState: resolver.State{ Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, + {Addr: "address1"}, + {Addr: "address2"}, + {Addr: "address3"}, }, }, BalancerConfig: &LBConfig{ @@ -894,19 +861,12 @@ func (s) TestEjectUnejectSuccessRate(t *testing.T) { RequestVolume: 3, }, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: balancerConfig{}, + Name: t.Name(), + Config: emptyChildConfig{}, }, }, }) - od.UpdateState(balancer.State{ - ConnectivityState: connectivity.Ready, - Picker: &rrPicker{ - scs: []balancer.SubConn{scw1, scw2, scw3}, - }, - }) - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -920,13 +880,11 @@ func (s) TestEjectUnejectSuccessRate(t *testing.T) { for i := 0; i < 3; i++ { pi, err := picker.Pick(balancer.PickInfo{}) if err != nil { - t.Fatalf("Picker.Pick should not have errored") + t.Fatalf("picker.Pick failed with error: %v", err) + } + for c := 0; c < 5; c++ { + pi.Done(balancer.DoneInfo{}) } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) } od.intervalTimerAlgorithm() @@ -958,28 +916,24 @@ func (s) TestEjectUnejectSuccessRate(t *testing.T) { // Set two of the upstream addresses to have five successes each, and // one of the upstream addresses to have five failures. This should - // cause the address which has five failures to be ejected according the - // SuccessRateAlgorithm. + // cause the address which has five failures to be ejected according to + // the SuccessRateAlgorithm. for i := 0; i < 2; i++ { pi, err := picker.Pick(balancer.PickInfo{}) if err != nil { - t.Fatalf("Picker.Pick should not have errored") + t.Fatalf("picker.Pick failed with error: %v", err) + } + for c := 0; c < 5; c++ { + pi.Done(balancer.DoneInfo{}) } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) } pi, err := picker.Pick(balancer.PickInfo{}) if err != nil { - t.Fatalf("Picker.Pick should not have errored") + t.Fatalf("picker.Pick failed with error: %v", err) + } + for c := 0; c < 5; c++ { + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) } - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) // should eject address that always errored. od.intervalTimerAlgorithm() @@ -1017,13 +971,15 @@ func (s) TestEjectUnejectSuccessRate(t *testing.T) { } // Override now to cause the interval timer algorithm to always uneject - // a SubConn. + // the ejected address. This will always uneject the ejected address + // because this time is set way past the max ejection time set in the + // configuration, which will make the next interval timer algorithm run + // uneject any ejected addresses. defer func(n func() time.Time) { now = n }(now) - now = func() time.Time { - return time.Now().Add(time.Second * 1000) // will cause to always uneject addresses which are ejected + return time.Now().Add(time.Second * 1000) } od.intervalTimerAlgorithm() @@ -1043,39 +999,34 @@ func (s) TestEjectUnejectSuccessRate(t *testing.T) { } // TestEjectFailureRate tests the functionality of the interval timer algorithm -// of ejecting SubConns when configured with FailurePercentageEjection. It also -// tests the functionality of unejecting SubConns when the balancer flips to a -// noop configuration. +// when configured with FailurePercentageEjection, and also the functionality of +// noop configuration. The Outlier Detection Balancer will be set up with 3 +// SubConns, each with a different address. It tests the following scenarios, in +// a step by step fashion: +// 1. The three addresses each have 5 successes. The interval timer algorithm +// should not eject any of the addresses. +// 2. Two of the addresses have 5 successes, the third has five failures. The +// interval timer algorithm should eject the third address with five failures. +// 3. The Outlier Detection Balancer receives a subsequent noop config update. +// The balancer should uneject all ejected addresses. func (s) TestEjectFailureRate(t *testing.T) { scsCh := testutils.NewChannel() var scw1, scw2, scw3 balancer.SubConn var err error - stub.Register(tcibname, stub.BalancerFuncs{ + stub.Register(t.Name(), stub.BalancerFuncs{ UpdateClientConnState: func(bd *stub.BalancerData, _ balancer.ClientConnState) error { if scw1 != nil { // UpdateClientConnState was already called, no need to recreate SubConns. return nil } - scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{ - { - Addr: "address1", - }, - }, balancer.NewSubConnOptions{}) + scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address1"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } - scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{ - { - Addr: "address2", - }, - }, balancer.NewSubConnOptions{}) + scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address2"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } - scw3, err = bd.ClientConn.NewSubConn([]resolver.Address{ - { - Addr: "address3", - }, - }, balancer.NewSubConnOptions{}) + scw3, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address3"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } @@ -1097,15 +1048,9 @@ func (s) TestEjectFailureRate(t *testing.T) { od.UpdateClientConnState(balancer.ClientConnState{ ResolverState: resolver.State{ Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, + {Addr: "address1"}, + {Addr: "address2"}, + {Addr: "address3"}, }, }, BalancerConfig: &LBConfig{ @@ -1120,8 +1065,8 @@ func (s) TestEjectFailureRate(t *testing.T) { RequestVolume: 3, }, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: balancerConfig{}, + Name: t.Name(), + Config: emptyChildConfig{}, }, }, }) @@ -1146,13 +1091,11 @@ func (s) TestEjectFailureRate(t *testing.T) { for i := 0; i < 3; i++ { pi, err := picker.Pick(balancer.PickInfo{}) if err != nil { - t.Fatalf("Picker.Pick should not have errored") + t.Fatalf("picker.Pick failed with error: %v", err) + } + for c := 0; c < 5; c++ { + pi.Done(balancer.DoneInfo{}) } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) } od.intervalTimerAlgorithm() @@ -1169,23 +1112,19 @@ func (s) TestEjectFailureRate(t *testing.T) { for i := 0; i < 2; i++ { pi, err := picker.Pick(balancer.PickInfo{}) if err != nil { - t.Fatalf("Picker.Pick should not have errored") + t.Fatalf("picker.Pick failed with error: %v", err) + } + for c := 0; c < 5; c++ { + pi.Done(balancer.DoneInfo{}) } - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) - pi.Done(balancer.DoneInfo{}) } pi, err := picker.Pick(balancer.PickInfo{}) if err != nil { - t.Fatalf("Picker.Pick should not have errored") + t.Fatalf("picker.Pick failed with error: %v", err) + } + for c := 0; c < 5; c++ { + pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) } - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) - pi.Done(balancer.DoneInfo{Err: errors.New("some error")}) // should eject address that always errored. od.intervalTimerAlgorithm() @@ -1215,15 +1154,9 @@ func (s) TestEjectFailureRate(t *testing.T) { od.UpdateClientConnState(balancer.ClientConnState{ ResolverState: resolver.State{ Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, + {Addr: "address1"}, + {Addr: "address2"}, + {Addr: "address3"}, }, }, BalancerConfig: &LBConfig{ @@ -1232,8 +1165,8 @@ func (s) TestEjectFailureRate(t *testing.T) { MaxEjectionTime: 300 * time.Second, MaxEjectionPercent: 10, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: tcibname, - Config: balancerConfig{}, + Name: t.Name(), + Config: emptyChildConfig{}, }, }, }) @@ -1256,7 +1189,7 @@ func (s) TestEjectFailureRate(t *testing.T) { // called after the child balancer is closed. func (s) TestConcurrentOperations(t *testing.T) { closed := grpcsync.NewEvent() - stub.Register(verifyBalancerName, stub.BalancerFuncs{ + stub.Register(t.Name(), stub.BalancerFuncs{ UpdateClientConnState: func(*stub.BalancerData, balancer.ClientConnState) error { if closed.HasFired() { t.Fatal("UpdateClientConnState was called after Close(), which breaks the balancer API") @@ -1291,15 +1224,9 @@ func (s) TestConcurrentOperations(t *testing.T) { od.UpdateClientConnState(balancer.ClientConnState{ ResolverState: resolver.State{ Addresses: []resolver.Address{ - { - Addr: "address1", - }, - { - Addr: "address2", - }, - { - Addr: "address3", - }, + {Addr: "address1"}, + {Addr: "address2"}, + {Addr: "address3"}, }, }, BalancerConfig: &LBConfig{ @@ -1320,19 +1247,15 @@ func (s) TestConcurrentOperations(t *testing.T) { RequestVolume: 3, }, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: verifyBalancerName, - Config: balancerConfig{}, + Name: t.Name(), + Config: emptyChildConfig{}, }, }, }) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - scw1, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address1", - }, - }, balancer.NewSubConnOptions{}) + scw1, err := od.NewSubConn([]resolver.Address{{Addr: "address1"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } @@ -1340,20 +1263,12 @@ func (s) TestConcurrentOperations(t *testing.T) { t.Fatalf("error in od.NewSubConn call: %v", err) } - scw2, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address2", - }, - }, balancer.NewSubConnOptions{}) + scw2, err := od.NewSubConn([]resolver.Address{{Addr: "address2"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } - scw3, err := od.NewSubConn([]resolver.Address{ - { - Addr: "address3", - }, - }, balancer.NewSubConnOptions{}) + scw3, err := od.NewSubConn([]resolver.Address{{Addr: "address3"}}, balancer.NewSubConnOptions{}) if err != nil { t.Fatalf("error in od.NewSubConn call: %v", err) } @@ -1443,28 +1358,20 @@ func (s) TestConcurrentOperations(t *testing.T) { wg.Add(1) go func() { defer wg.Done() - od.UpdateAddresses(scw2, []resolver.Address{ - { - Addr: "address3", - }, - }) + od.UpdateAddresses(scw2, []resolver.Address{{Addr: "address3"}}) }() // Call balancer.Balancers synchronously in this goroutine, upholding the // balancer.Balancer API guarantee of synchronous calls. od.UpdateClientConnState(balancer.ClientConnState{ // This will delete addresses and flip to no op ResolverState: resolver.State{ - Addresses: []resolver.Address{ - { - Addr: "address1", - }, - }, + Addresses: []resolver.Address{{Addr: "address1"}}, }, BalancerConfig: &LBConfig{ Interval: 1<<63 - 1, ChildPolicy: &internalserviceconfig.BalancerConfig{ - Name: verifyBalancerName, - Config: balancerConfig{}, + Name: t.Name(), + Config: emptyChildConfig{}, }, }, }) @@ -1480,368 +1387,3 @@ func (s) TestConcurrentOperations(t *testing.T) { close(finished) wg.Wait() } - -// Setup spins up three test backends, each listening on a port on localhost. -// Two of the backends are configured to always reply with an empty response and -// no error and one is configured to always return an error. -func setupBackends(t *testing.T) ([]string, []*stubserver.StubServer) { - t.Helper() - - backends := make([]*stubserver.StubServer, 3) - addresses := make([]string, 3) - // Construct and start 2 working backends. - for i := 0; i < 2; i++ { - backend := &stubserver.StubServer{ - EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { - return &testpb.Empty{}, nil - }, - } - if err := backend.StartServer(); err != nil { - t.Fatalf("Failed to start backend: %v", err) - } - t.Logf("Started good TestService backend at: %q", backend.Address) - backends[i] = backend - addresses[i] = backend.Address - } - - // Construct and start a failing backend. - backend := &stubserver.StubServer{ - EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { - return nil, errors.New("some error") - }, - } - if err := backend.StartServer(); err != nil { - t.Fatalf("Failed to start backend: %v", err) - } - t.Logf("Started good TestService backend at: %q", backend.Address) - backends[2] = backend - addresses[2] = backend.Address - - return addresses, backends -} - -// checkRoundRobinRPCs verifies that EmptyCall RPCs on the given ClientConn, -// connected to a server exposing the test.grpc_testing.TestService, are -// roundrobin-ed across the given backend addresses. -// -// Returns a non-nil error if context deadline expires before RPCs start to get -// roundrobin-ed across the given backends. -func checkRoundRobinRPCs(ctx context.Context, client testpb.TestServiceClient, addrs []resolver.Address) error { - wantAddrCount := make(map[string]int) - for _, addr := range addrs { - wantAddrCount[addr.Addr]++ - } - for ; ctx.Err() == nil; <-time.After(time.Millisecond) { - // Perform 3 iterations. - var iterations [][]string - for i := 0; i < 3; i++ { - iteration := make([]string, len(addrs)) - for c := 0; c < len(addrs); c++ { - var peer peer.Peer - client.EmptyCall(ctx, &testpb.Empty{}, grpc.Peer(&peer)) - if peer.Addr != nil { - iteration[c] = peer.Addr.String() - } - } - iterations = append(iterations, iteration) - } - // Ensure the the first iteration contains all addresses in addrs. - gotAddrCount := make(map[string]int) - for _, addr := range iterations[0] { - gotAddrCount[addr]++ - } - if diff := cmp.Diff(gotAddrCount, wantAddrCount); diff != "" { - logger.Infof("non-roundrobin, got address count in one iteration: %v, want: %v, Diff: %s", gotAddrCount, wantAddrCount, diff) - continue - } - // Ensure all three iterations contain the same addresses. - if !cmp.Equal(iterations[0], iterations[1]) || !cmp.Equal(iterations[0], iterations[2]) { - logger.Infof("non-roundrobin, first iter: %v, second iter: %v, third iter: %v", iterations[0], iterations[1], iterations[2]) - continue - } - return nil - } - return fmt.Errorf("timeout when waiting for roundrobin distribution of RPCs across addresses: %v", addrs) -} - -// TestOutlierDetectionAlgorithmsE2E tests the Outlier Detection Success Rate -// and Failure Percentage algorithms in an e2e fashion. The Outlier Detection -// Balancer is configured as the top level LB Policy of the channel with a Round -// Robin child, and connects to three upstreams. Two of the upstreams are healthy and -// one is unhealthy. The two algorithms should at some point eject the failing -// upstream, causing RPC's to not be routed to those two upstreams, and only be -// Round Robined across the two healthy upstreams. Other than the intervals the -// two unhealthy upstreams are ejected, RPC's should regularly round robin -// across all three upstreams. -func (s) TestOutlierDetectionAlgorithmsE2E(t *testing.T) { - tests := []struct { - name string - odscJSON string - }{ - { - name: "Success Rate Algorithm", - odscJSON: ` -{ - "loadBalancingConfig": [ - { - "outlier_detection_experimental": { - "interval": 50000000, - "baseEjectionTime": 100000000, - "maxEjectionTime": 300000000000, - "maxEjectionPercent": 33, - "successRateEjection": { - "stdevFactor": 50, - "enforcementPercentage": 100, - "minimumHosts": 3, - "requestVolume": 5 - }, - "childPolicy": [ - { - "round_robin": {} - } - ] - } - } - ] -}`, - }, - { - name: "Failure Percentage Algorithm", - odscJSON: ` -{ - "loadBalancingConfig": [ - { - "outlier_detection_experimental": { - "interval": 50000000, - "baseEjectionTime": 100000000, - "maxEjectionTime": 300000000000, - "maxEjectionPercent": 33, - "failurePercentageEjection": { - "threshold": 50, - "enforcementPercentage": 100, - "minimumHosts": 3, - "requestVolume": 5 - }, - "childPolicy": [ - { - "round_robin": {} - } - ] - } - } - ] -}`, - }, - } - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - defer internal.UnregisterOutlierDetectionBalancerForTesting() - addresses, backends := setupBackends(t) - defer func() { - for _, backend := range backends { - backend.Stop() - } - }() - - // The addresses which don't return errors. - okAddresses := []resolver.Address{ - { - Addr: addresses[0], - }, - { - Addr: addresses[1], - }, - } - - // The full list of addresses. - fullAddresses := []resolver.Address{ - { - Addr: addresses[0], - }, - { - Addr: addresses[1], - }, - { - Addr: addresses[2], - }, - } - - mr := manual.NewBuilderWithScheme("od-e2e") - defer mr.Close() - - sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(test.odscJSON) - - mr.InitialState(resolver.State{ - Addresses: fullAddresses, - ServiceConfig: sc, - }) - - cc, err := grpc.Dial(mr.Scheme()+":///", grpc.WithResolvers(mr), grpc.WithTransportCredentials(insecure.NewCredentials())) - if err != nil { - t.Fatalf("grpc.Dial() failed: %v", err) - } - defer cc.Close() - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - testServiceClient := testpb.NewTestServiceClient(cc) - - // At first, due to no statistics on each of the backends, the 3 - // upstreams should all be round robined across. - if err = checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { - t.Fatalf("error in expected round robin: %v", err) - } - - // After calling the three upstreams, one of them constantly error - // and should eventually be ejected for a period of time. This - // period of time should cause the RPC's to be round robined only - // across the two that are healthy. - if err = checkRoundRobinRPCs(ctx, testServiceClient, okAddresses); err != nil { - t.Fatalf("error in expected round robin: %v", err) - } - - // The failing upstream isn't ejected indefinitely, and eventually - // should be unejected in subsequent iterations of the interval - // algorithm as per the spec for the two specific algorithms. - if err = checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { - t.Fatalf("error in expected round robin: %v", err) - } - }) - } -} - -// TestNoopConfiguration tests the Outlier Detection Balancer configured with a -// noop configuration. The noop configuration should cause the Outlier Detection -// Balancer to not count RPC's, and thus never eject any upstreams and continue -// to route to every upstream connected to, even if they continuously error. -// Once the Outlier Detection Balancer gets reconfigured with configuration -// requiring counting RPC's, the Outlier Detection Balancer should start -// ejecting any upstreams as specified in the configuration. -func (s) TestNoopConfiguration(t *testing.T) { - internal.RegisterOutlierDetectionBalancerForTesting() - defer internal.UnregisterOutlierDetectionBalancerForTesting() - addresses, backends := setupBackends(t) - defer func() { - for _, backend := range backends { - backend.Stop() - } - }() - - mr := manual.NewBuilderWithScheme("od-e2e") - defer mr.Close() - - // The addresses which don't return errors. - okAddresses := []resolver.Address{ - { - Addr: addresses[0], - }, - { - Addr: addresses[1], - }, - } - - // The full list of addresses. - fullAddresses := []resolver.Address{ - { - Addr: addresses[0], - }, - { - Addr: addresses[1], - }, - { - Addr: addresses[2], - }, - } - - noopODServiceConfigJSON := ` -{ - "loadBalancingConfig": [ - { - "outlier_detection_experimental": { - "interval": 50000000, - "baseEjectionTime": 100000000, - "maxEjectionTime": 300000000000, - "maxEjectionPercent": 33, - "childPolicy": [ - { - "round_robin": {} - } - ] - } - } - ] -}` - sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(noopODServiceConfigJSON) - - mr.InitialState(resolver.State{ - Addresses: fullAddresses, - ServiceConfig: sc, - }) - cc, err := grpc.Dial(mr.Scheme()+":///", grpc.WithResolvers(mr), grpc.WithTransportCredentials(insecure.NewCredentials())) - if err != nil { - t.Fatalf("grpc.Dial() failed: %v", err) - } - defer cc.Close() - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - testServiceClient := testpb.NewTestServiceClient(cc) - - for i := 0; i < 2; i++ { - // Since the Outlier Detection Balancer starts with a noop - // configuration, it shouldn't count RPCs or eject any upstreams. Thus, - // even though an upstream it connects to constantly errors, it should - // continue to Round Robin across every upstream. - if err := checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { - t.Fatalf("error in expected round robin: %v", err) - } - } - - // Reconfigure the Outlier Detection Balancer with a configuration that - // specifies to count RPC's and eject upstreams. Due to the balancer no - // longer being a noop, it should eject any unhealthy addresses as specified - // by the failure percentage portion of the configuration. - countingODServiceConfigJSON := ` -{ - "loadBalancingConfig": [ - { - "outlier_detection_experimental": { - "interval": 50000000, - "baseEjectionTime": 100000000, - "maxEjectionTime": 300000000000, - "maxEjectionPercent": 33, - "failurePercentageEjection": { - "threshold": 50, - "enforcementPercentage": 100, - "minimumHosts": 3, - "requestVolume": 5 - }, - "childPolicy": [ - { - "round_robin": {} - } - ] - } - } - ] -}` - sc = internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(countingODServiceConfigJSON) - - mr.UpdateState(resolver.State{ - Addresses: fullAddresses, - ServiceConfig: sc, - }) - - // At first on the reconfigured balancer, the balancer has no stats - // collected about upstreams. Thus, it should at first route across the full - // upstream list. - if err = checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { - t.Fatalf("error in expected round robin: %v", err) - } - - // Now that the reconfigured balancer has data about the failing upstream, - // it should eject the upstream and only route across the two healthy - // upstreams. - if err = checkRoundRobinRPCs(ctx, testServiceClient, okAddresses); err != nil { - t.Fatalf("error in expected round robin: %v", err) - } -} diff --git a/xds/internal/balancer/outlierdetection/callcounter.go b/xds/internal/balancer/outlierdetection/callcounter.go index 1f53f3cd300..1489ea8ffb2 100644 --- a/xds/internal/balancer/outlierdetection/callcounter.go +++ b/xds/internal/balancer/outlierdetection/callcounter.go @@ -37,7 +37,9 @@ func newCallCounter() *callCounter { // callCounter has two buckets, which each count successful and failing RPC's. // The activeBucket is used to actively count any finished RPC's, and the // inactiveBucket is populated with this activeBucket's data every interval for -// use by the Outlier Detection algorithm. Caller must hold b.mu. +// use by the Outlier Detection algorithm. +// +// Caller must hold b.mu. type callCounter struct { // activeBucket updates every time a call finishes (from picker passed to // Client Conn), so protect pointer read with atomic load of unsafe.Pointer @@ -54,7 +56,9 @@ func (cc *callCounter) clear() { // "When the timer triggers, the inactive bucket is zeroed and swapped with the // active bucket. Then the inactive bucket contains the number of successes and // failures since the last time the timer triggered. Those numbers are used to -// evaluate the ejection criteria." - A50. Caller must hold b.mu. +// evaluate the ejection criteria." - A50. +// +// Caller must hold b.mu. func (cc *callCounter) swap() { ib := cc.inactiveBucket *ib = bucket{} diff --git a/xds/internal/balancer/outlierdetection/e2e_test/outlierdetection_test.go b/xds/internal/balancer/outlierdetection/e2e_test/outlierdetection_test.go new file mode 100644 index 00000000000..e9dc972f4ef --- /dev/null +++ b/xds/internal/balancer/outlierdetection/e2e_test/outlierdetection_test.go @@ -0,0 +1,394 @@ +/* + * + * Copyright 2022 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +// Package e2e_test contains e2e test cases for the Outlier Detection LB Policy. +package e2e_test + +import ( + "context" + "errors" + "fmt" + "testing" + "time" + + "github.com/google/go-cmp/cmp" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/grpctest" + "google.golang.org/grpc/internal/stubserver" + "google.golang.org/grpc/peer" + "google.golang.org/grpc/resolver" + "google.golang.org/grpc/resolver/manual" + "google.golang.org/grpc/serviceconfig" + testpb "google.golang.org/grpc/test/grpc_testing" + _ "google.golang.org/grpc/xds/internal/balancer/outlierdetection" // To register helper functions which register/unregister Outlier Detection LB Policy. +) + +var defaultTestTimeout = 5 * time.Second + +type s struct { + grpctest.Tester +} + +func Test(t *testing.T) { + grpctest.RunSubTests(t, s{}) +} + +// Setup spins up three test backends, each listening on a port on localhost. +// Two of the backends are configured to always reply with an empty response and +// no error and one is configured to always return an error. +func setupBackends(t *testing.T) ([]string, []*stubserver.StubServer) { + t.Helper() + + backends := make([]*stubserver.StubServer, 3) + addresses := make([]string, 3) + // Construct and start 2 working backends. + for i := 0; i < 2; i++ { + backend := &stubserver.StubServer{ + EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { + return &testpb.Empty{}, nil + }, + } + if err := backend.StartServer(); err != nil { + t.Fatalf("Failed to start backend: %v", err) + } + t.Logf("Started good TestService backend at: %q", backend.Address) + backends[i] = backend + addresses[i] = backend.Address + } + + // Construct and start a failing backend. + backend := &stubserver.StubServer{ + EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { + return nil, errors.New("some error") + }, + } + if err := backend.StartServer(); err != nil { + t.Fatalf("Failed to start backend: %v", err) + } + t.Logf("Started good TestService backend at: %q", backend.Address) + backends[2] = backend + addresses[2] = backend.Address + + return addresses, backends +} + +// checkRoundRobinRPCs verifies that EmptyCall RPCs on the given ClientConn, +// connected to a server exposing the test.grpc_testing.TestService, are +// roundrobined across the given backend addresses. +// +// Returns a non-nil error if context deadline expires before RPCs start to get +// roundrobined across the given backends. +func checkRoundRobinRPCs(ctx context.Context, client testpb.TestServiceClient, addrs []resolver.Address) error { + wantAddrCount := make(map[string]int) + for _, addr := range addrs { + wantAddrCount[addr.Addr]++ + } + for ; ctx.Err() == nil; <-time.After(time.Millisecond) { + // Perform 3 iterations. + var iterations [][]string + for i := 0; i < 3; i++ { + iteration := make([]string, len(addrs)) + for c := 0; c < len(addrs); c++ { + var peer peer.Peer + client.EmptyCall(ctx, &testpb.Empty{}, grpc.Peer(&peer)) + if peer.Addr != nil { + iteration[c] = peer.Addr.String() + } + } + iterations = append(iterations, iteration) + } + // Ensure the the first iteration contains all addresses in addrs. + gotAddrCount := make(map[string]int) + for _, addr := range iterations[0] { + gotAddrCount[addr]++ + } + if diff := cmp.Diff(gotAddrCount, wantAddrCount); diff != "" { + continue + } + // Ensure all three iterations contain the same addresses. + if !cmp.Equal(iterations[0], iterations[1]) || !cmp.Equal(iterations[0], iterations[2]) { + continue + } + return nil + } + return fmt.Errorf("timeout when waiting for roundrobin distribution of RPCs across addresses: %v", addrs) +} + +// TestOutlierDetectionAlgorithmsE2E tests the Outlier Detection Success Rate +// and Failure Percentage algorithms in an e2e fashion. The Outlier Detection +// Balancer is configured as the top level LB Policy of the channel with a Round +// Robin child, and connects to three upstreams. Two of the upstreams are healthy and +// one is unhealthy. The two algorithms should at some point eject the failing +// upstream, causing RPC's to not be routed to those two upstreams, and only be +// Round Robined across the two healthy upstreams. Other than the intervals the +// two unhealthy upstreams are ejected, RPC's should regularly round robin +// across all three upstreams. +func (s) TestOutlierDetectionAlgorithmsE2E(t *testing.T) { + tests := []struct { + name string + odscJSON string + }{ + { + name: "Success Rate Algorithm", + odscJSON: ` +{ + "loadBalancingConfig": [ + { + "outlier_detection_experimental": { + "interval": 50000000, + "baseEjectionTime": 100000000, + "maxEjectionTime": 300000000000, + "maxEjectionPercent": 33, + "successRateEjection": { + "stdevFactor": 50, + "enforcementPercentage": 100, + "minimumHosts": 3, + "requestVolume": 5 + }, + "childPolicy": [ + { + "round_robin": {} + } + ] + } + } + ] +}`, + }, + { + name: "Failure Percentage Algorithm", + odscJSON: ` +{ + "loadBalancingConfig": [ + { + "outlier_detection_experimental": { + "interval": 50000000, + "baseEjectionTime": 100000000, + "maxEjectionTime": 300000000000, + "maxEjectionPercent": 33, + "failurePercentageEjection": { + "threshold": 50, + "enforcementPercentage": 100, + "minimumHosts": 3, + "requestVolume": 5 + }, + "childPolicy": [ + { + "round_robin": {} + } + ] + } + } + ] +}`, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + defer internal.UnregisterOutlierDetectionBalancerForTesting() + addresses, backends := setupBackends(t) + defer func() { + for _, backend := range backends { + backend.Stop() + } + }() + + // The addresses which don't return errors. + okAddresses := []resolver.Address{ + {Addr: addresses[0]}, + {Addr: addresses[1]}, + } + + // The full list of addresses. + fullAddresses := []resolver.Address{ + {Addr: addresses[0]}, + {Addr: addresses[1]}, + {Addr: addresses[2]}, + } + + mr := manual.NewBuilderWithScheme("od-e2e") + defer mr.Close() + + sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(test.odscJSON) + + mr.InitialState(resolver.State{ + Addresses: fullAddresses, + ServiceConfig: sc, + }) + + cc, err := grpc.Dial(mr.Scheme()+":///", grpc.WithResolvers(mr), grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + t.Fatalf("grpc.Dial() failed: %v", err) + } + defer cc.Close() + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + testServiceClient := testpb.NewTestServiceClient(cc) + + // At first, due to no statistics on each of the backends, the 3 + // upstreams should all be round robined across. + if err = checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } + + // After calling the three upstreams, one of them constantly error + // and should eventually be ejected for a period of time. This + // period of time should cause the RPC's to be round robined only + // across the two that are healthy. + if err = checkRoundRobinRPCs(ctx, testServiceClient, okAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } + + // The failing upstream isn't ejected indefinitely, and eventually + // should be unejected in subsequent iterations of the interval + // algorithm as per the spec for the two specific algorithms. + if err = checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } + }) + } +} + +// TestNoopConfiguration tests the Outlier Detection Balancer configured with a +// noop configuration. The noop configuration should cause the Outlier Detection +// Balancer to not count RPC's, and thus never eject any upstreams and continue +// to route to every upstream connected to, even if they continuously error. +// Once the Outlier Detection Balancer gets reconfigured with configuration +// requiring counting RPC's, the Outlier Detection Balancer should start +// ejecting any upstreams as specified in the configuration. +func (s) TestNoopConfiguration(t *testing.T) { + internal.RegisterOutlierDetectionBalancerForTesting() + defer internal.UnregisterOutlierDetectionBalancerForTesting() + addresses, backends := setupBackends(t) + defer func() { + for _, backend := range backends { + backend.Stop() + } + }() + + mr := manual.NewBuilderWithScheme("od-e2e") + defer mr.Close() + + // The addresses which don't return errors. + okAddresses := []resolver.Address{ + {Addr: addresses[0]}, + {Addr: addresses[1]}, + } + + // The full list of addresses. + fullAddresses := []resolver.Address{ + {Addr: addresses[0]}, + {Addr: addresses[1]}, + {Addr: addresses[2]}, + } + + noopODServiceConfigJSON := ` +{ + "loadBalancingConfig": [ + { + "outlier_detection_experimental": { + "interval": 50000000, + "baseEjectionTime": 100000000, + "maxEjectionTime": 300000000000, + "maxEjectionPercent": 33, + "childPolicy": [ + { + "round_robin": {} + } + ] + } + } + ] +}` + sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(noopODServiceConfigJSON) + + mr.InitialState(resolver.State{ + Addresses: fullAddresses, + ServiceConfig: sc, + }) + cc, err := grpc.Dial(mr.Scheme()+":///", grpc.WithResolvers(mr), grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + t.Fatalf("grpc.Dial() failed: %v", err) + } + defer cc.Close() + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + testServiceClient := testpb.NewTestServiceClient(cc) + + for i := 0; i < 2; i++ { + // Since the Outlier Detection Balancer starts with a noop + // configuration, it shouldn't count RPCs or eject any upstreams. Thus, + // even though an upstream it connects to constantly errors, it should + // continue to Round Robin across every upstream. + if err := checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } + } + + // Reconfigure the Outlier Detection Balancer with a configuration that + // specifies to count RPC's and eject upstreams. Due to the balancer no + // longer being a noop, it should eject any unhealthy addresses as specified + // by the failure percentage portion of the configuration. + countingODServiceConfigJSON := ` +{ + "loadBalancingConfig": [ + { + "outlier_detection_experimental": { + "interval": 50000000, + "baseEjectionTime": 100000000, + "maxEjectionTime": 300000000000, + "maxEjectionPercent": 33, + "failurePercentageEjection": { + "threshold": 50, + "enforcementPercentage": 100, + "minimumHosts": 3, + "requestVolume": 5 + }, + "childPolicy": [ + { + "round_robin": {} + } + ] + } + } + ] +}` + sc = internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(countingODServiceConfigJSON) + + mr.UpdateState(resolver.State{ + Addresses: fullAddresses, + ServiceConfig: sc, + }) + + // At first on the reconfigured balancer, the balancer has no stats + // collected about upstreams. Thus, it should at first route across the full + // upstream list. + if err = checkRoundRobinRPCs(ctx, testServiceClient, fullAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } + + // Now that the reconfigured balancer has data about the failing upstream, + // it should eject the upstream and only route across the two healthy + // upstreams. + if err = checkRoundRobinRPCs(ctx, testServiceClient, okAddresses); err != nil { + t.Fatalf("error in expected round robin: %v", err) + } +} From 7fc669c63351a7297e7e8a80aa61ca69e630ce56 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Mon, 29 Aug 2022 17:04:58 -0400 Subject: [PATCH 18/22] Responded to Easwar's comments --- .../outlierdetection/balancer_test.go | 99 +++++-------------- .../e2e_test/outlierdetection_test.go | 95 +++++++----------- 2 files changed, 59 insertions(+), 135 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer_test.go b/xds/internal/balancer/outlierdetection/balancer_test.go index 031c40b289d..151684ac7fd 100644 --- a/xds/internal/balancer/outlierdetection/balancer_test.go +++ b/xds/internal/balancer/outlierdetection/balancer_test.go @@ -88,7 +88,7 @@ func (s) TestParseConfig(t *testing.T) { ] }`, wantCfg: &LBConfig{ - Interval: 1<<63 - 1, + Interval: math.MaxInt64, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: "xds_cluster_impl_experimental", Config: &clusterimpl.LBConfig{ @@ -363,26 +363,11 @@ func (s) TestChildBasicOperations(t *testing.T) { od, tcc, _ := setup(t) defer internal.UnregisterOutlierDetectionBalancerForTesting() - // This first config update should a child to be built and forwarded it's - // first update. + // This first config update should cause a child to be built and forwarded + // it's first update. od.UpdateClientConnState(balancer.ClientConnState{ BalancerConfig: &LBConfig{ - Interval: 10 * time.Second, - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, - SuccessRateEjection: &SuccessRateEjection{ - StdevFactor: 1900, - EnforcementPercentage: 100, - MinimumHosts: 5, - RequestVolume: 100, - }, - FailurePercentageEjection: &FailurePercentageEjection{ - Threshold: 85, - EnforcementPercentage: 5, - MinimumHosts: 5, - RequestVolume: 50, - }, + Interval: math.MaxInt64, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: t.Name() + "child1", Config: bc, @@ -405,22 +390,7 @@ func (s) TestChildBasicOperations(t *testing.T) { // config update. od.UpdateClientConnState(balancer.ClientConnState{ BalancerConfig: &LBConfig{ - Interval: 10 * time.Second, - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, - SuccessRateEjection: &SuccessRateEjection{ - StdevFactor: 1900, - EnforcementPercentage: 100, - MinimumHosts: 5, - RequestVolume: 100, - }, - FailurePercentageEjection: &FailurePercentageEjection{ - Threshold: 85, - EnforcementPercentage: 5, - MinimumHosts: 5, - RequestVolume: 50, - }, + Interval: math.MaxInt64, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: t.Name() + "child2", Config: emptyChildConfig{}, @@ -478,14 +448,11 @@ func (s) TestUpdateAddresses(t *testing.T) { UpdateClientConnState: func(bd *stub.BalancerData, _ balancer.ClientConnState) error { scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address1"}}, balancer.NewSubConnOptions{}) if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) + t.Errorf("error in od.NewSubConn call: %v", err) } scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address2"}}, balancer.NewSubConnOptions{}) if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) - } - if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) + t.Errorf("error in od.NewSubConn call: %v", err) } bd.ClientConn.UpdateState(balancer.State{ ConnectivityState: connectivity.Ready, @@ -689,22 +656,13 @@ func (s) TestDurationOfInterval(t *testing.T) { od.UpdateClientConnState(balancer.ClientConnState{ BalancerConfig: &LBConfig{ - Interval: 8 * time.Second, - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, + Interval: 8 * time.Second, SuccessRateEjection: &SuccessRateEjection{ StdevFactor: 1900, EnforcementPercentage: 100, MinimumHosts: 5, RequestVolume: 100, }, - FailurePercentageEjection: &FailurePercentageEjection{ - Threshold: 85, - EnforcementPercentage: 5, - MinimumHosts: 5, - RequestVolume: 50, - }, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: t.Name(), Config: emptyChildConfig{}, @@ -738,22 +696,13 @@ func (s) TestDurationOfInterval(t *testing.T) { // interval timer of ~4 seconds. od.UpdateClientConnState(balancer.ClientConnState{ BalancerConfig: &LBConfig{ - Interval: 9 * time.Second, - BaseEjectionTime: 30 * time.Second, - MaxEjectionTime: 300 * time.Second, - MaxEjectionPercent: 10, + Interval: 9 * time.Second, SuccessRateEjection: &SuccessRateEjection{ StdevFactor: 1900, EnforcementPercentage: 100, MinimumHosts: 5, RequestVolume: 100, }, - FailurePercentageEjection: &FailurePercentageEjection{ - Threshold: 85, - EnforcementPercentage: 5, - MinimumHosts: 5, - RequestVolume: 50, - }, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: t.Name(), Config: emptyChildConfig{}, @@ -810,15 +759,15 @@ func (s) TestEjectUnejectSuccessRate(t *testing.T) { UpdateClientConnState: func(bd *stub.BalancerData, _ balancer.ClientConnState) error { scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address1"}}, balancer.NewSubConnOptions{}) if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) + t.Errorf("error in od.NewSubConn call: %v", err) } scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address2"}}, balancer.NewSubConnOptions{}) if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) + t.Errorf("error in od.NewSubConn call: %v", err) } scw3, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address3"}}, balancer.NewSubConnOptions{}) if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) + t.Errorf("error in od.NewSubConn call: %v", err) } bd.ClientConn.UpdateState(balancer.State{ ConnectivityState: connectivity.Ready, @@ -850,7 +799,7 @@ func (s) TestEjectUnejectSuccessRate(t *testing.T) { }, }, BalancerConfig: &LBConfig{ - Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. + Interval: math.MaxInt64, // so the interval will never run unless called manually in test. BaseEjectionTime: 30 * time.Second, MaxEjectionTime: 300 * time.Second, MaxEjectionPercent: 10, @@ -1020,15 +969,15 @@ func (s) TestEjectFailureRate(t *testing.T) { } scw1, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address1"}}, balancer.NewSubConnOptions{}) if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) + t.Errorf("error in od.NewSubConn call: %v", err) } scw2, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address2"}}, balancer.NewSubConnOptions{}) if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) + t.Errorf("error in od.NewSubConn call: %v", err) } scw3, err = bd.ClientConn.NewSubConn([]resolver.Address{{Addr: "address3"}}, balancer.NewSubConnOptions{}) if err != nil { - t.Fatalf("error in od.NewSubConn call: %v", err) + t.Errorf("error in od.NewSubConn call: %v", err) } return nil }, @@ -1054,7 +1003,7 @@ func (s) TestEjectFailureRate(t *testing.T) { }, }, BalancerConfig: &LBConfig{ - Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. + Interval: math.MaxInt64, // so the interval will never run unless called manually in test. BaseEjectionTime: 30 * time.Second, MaxEjectionTime: 300 * time.Second, MaxEjectionPercent: 10, @@ -1160,7 +1109,7 @@ func (s) TestEjectFailureRate(t *testing.T) { }, }, BalancerConfig: &LBConfig{ - Interval: 1<<63 - 1, + Interval: math.MaxInt64, BaseEjectionTime: 30 * time.Second, MaxEjectionTime: 300 * time.Second, MaxEjectionPercent: 10, @@ -1192,18 +1141,18 @@ func (s) TestConcurrentOperations(t *testing.T) { stub.Register(t.Name(), stub.BalancerFuncs{ UpdateClientConnState: func(*stub.BalancerData, balancer.ClientConnState) error { if closed.HasFired() { - t.Fatal("UpdateClientConnState was called after Close(), which breaks the balancer API") + t.Error("UpdateClientConnState was called after Close(), which breaks the balancer API") } return nil }, ResolverError: func(*stub.BalancerData, error) { if closed.HasFired() { - t.Fatal("ResolverError was called after Close(), which breaks the balancer API") + t.Error("ResolverError was called after Close(), which breaks the balancer API") } }, UpdateSubConnState: func(*stub.BalancerData, balancer.SubConn, balancer.SubConnState) { if closed.HasFired() { - t.Fatal("UpdateSubConnState was called after Close(), which breaks the balancer API") + t.Error("UpdateSubConnState was called after Close(), which breaks the balancer API") } }, Close: func(*stub.BalancerData) { @@ -1211,7 +1160,7 @@ func (s) TestConcurrentOperations(t *testing.T) { }, ExitIdle: func(*stub.BalancerData) { if closed.HasFired() { - t.Fatal("ExitIdle was called after Close(), which breaks the balancer API") + t.Error("ExitIdle was called after Close(), which breaks the balancer API") } }, }) @@ -1230,7 +1179,7 @@ func (s) TestConcurrentOperations(t *testing.T) { }, }, BalancerConfig: &LBConfig{ - Interval: 1<<63 - 1, // so the interval will never run unless called manually in test. + Interval: math.MaxInt64, // so the interval will never run unless called manually in test. BaseEjectionTime: 30 * time.Second, MaxEjectionTime: 300 * time.Second, MaxEjectionPercent: 10, @@ -1368,7 +1317,7 @@ func (s) TestConcurrentOperations(t *testing.T) { Addresses: []resolver.Address{{Addr: "address1"}}, }, BalancerConfig: &LBConfig{ - Interval: 1<<63 - 1, + Interval: math.MaxInt64, ChildPolicy: &internalserviceconfig.BalancerConfig{ Name: t.Name(), Config: emptyChildConfig{}, diff --git a/xds/internal/balancer/outlierdetection/e2e_test/outlierdetection_test.go b/xds/internal/balancer/outlierdetection/e2e_test/outlierdetection_test.go index e9dc972f4ef..a1987bf98a9 100644 --- a/xds/internal/balancer/outlierdetection/e2e_test/outlierdetection_test.go +++ b/xds/internal/balancer/outlierdetection/e2e_test/outlierdetection_test.go @@ -53,7 +53,7 @@ func Test(t *testing.T) { // Setup spins up three test backends, each listening on a port on localhost. // Two of the backends are configured to always reply with an empty response and // no error and one is configured to always return an error. -func setupBackends(t *testing.T) ([]string, []*stubserver.StubServer) { +func setupBackends(t *testing.T) ([]string, func()) { t.Helper() backends := make([]*stubserver.StubServer, 3) @@ -82,11 +82,15 @@ func setupBackends(t *testing.T) ([]string, []*stubserver.StubServer) { if err := backend.StartServer(); err != nil { t.Fatalf("Failed to start backend: %v", err) } - t.Logf("Started good TestService backend at: %q", backend.Address) + t.Logf("Started bad TestService backend at: %q", backend.Address) backends[2] = backend addresses[2] = backend.Address - - return addresses, backends + cancel := func() { + for _, backend := range backends { + backend.Stop() + } + } + return addresses, cancel } // checkRoundRobinRPCs verifies that EmptyCall RPCs on the given ClientConn, @@ -162,11 +166,7 @@ func (s) TestOutlierDetectionAlgorithmsE2E(t *testing.T) { "minimumHosts": 3, "requestVolume": 5 }, - "childPolicy": [ - { - "round_robin": {} - } - ] + "childPolicy": [{"round_robin": {}}] } } ] @@ -189,10 +189,7 @@ func (s) TestOutlierDetectionAlgorithmsE2E(t *testing.T) { "minimumHosts": 3, "requestVolume": 5 }, - "childPolicy": [ - { - "round_robin": {} - } + "childPolicy": [{"round_robin": {}} ] } } @@ -204,31 +201,19 @@ func (s) TestOutlierDetectionAlgorithmsE2E(t *testing.T) { t.Run(test.name, func(t *testing.T) { internal.RegisterOutlierDetectionBalancerForTesting() defer internal.UnregisterOutlierDetectionBalancerForTesting() - addresses, backends := setupBackends(t) - defer func() { - for _, backend := range backends { - backend.Stop() - } - }() + addresses, cancel := setupBackends(t) + defer cancel() - // The addresses which don't return errors. - okAddresses := []resolver.Address{ - {Addr: addresses[0]}, - {Addr: addresses[1]}, - } + mr := manual.NewBuilderWithScheme("od-e2e") + defer mr.Close() + sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(test.odscJSON) // The full list of addresses. fullAddresses := []resolver.Address{ {Addr: addresses[0]}, {Addr: addresses[1]}, {Addr: addresses[2]}, } - - mr := manual.NewBuilderWithScheme("od-e2e") - defer mr.Close() - - sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(test.odscJSON) - mr.InitialState(resolver.State{ Addresses: fullAddresses, ServiceConfig: sc, @@ -249,6 +234,11 @@ func (s) TestOutlierDetectionAlgorithmsE2E(t *testing.T) { t.Fatalf("error in expected round robin: %v", err) } + // The addresses which don't return errors. + okAddresses := []resolver.Address{ + {Addr: addresses[0]}, + {Addr: addresses[1]}, + } // After calling the three upstreams, one of them constantly error // and should eventually be ejected for a period of time. This // period of time should cause the RPC's to be round robined only @@ -277,29 +267,12 @@ func (s) TestOutlierDetectionAlgorithmsE2E(t *testing.T) { func (s) TestNoopConfiguration(t *testing.T) { internal.RegisterOutlierDetectionBalancerForTesting() defer internal.UnregisterOutlierDetectionBalancerForTesting() - addresses, backends := setupBackends(t) - defer func() { - for _, backend := range backends { - backend.Stop() - } - }() + addresses, cancel := setupBackends(t) + defer cancel() mr := manual.NewBuilderWithScheme("od-e2e") defer mr.Close() - // The addresses which don't return errors. - okAddresses := []resolver.Address{ - {Addr: addresses[0]}, - {Addr: addresses[1]}, - } - - // The full list of addresses. - fullAddresses := []resolver.Address{ - {Addr: addresses[0]}, - {Addr: addresses[1]}, - {Addr: addresses[2]}, - } - noopODServiceConfigJSON := ` { "loadBalancingConfig": [ @@ -309,17 +282,18 @@ func (s) TestNoopConfiguration(t *testing.T) { "baseEjectionTime": 100000000, "maxEjectionTime": 300000000000, "maxEjectionPercent": 33, - "childPolicy": [ - { - "round_robin": {} - } - ] + "childPolicy": [{"round_robin": {}}] } } ] }` sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(noopODServiceConfigJSON) - + // The full list of addresses. + fullAddresses := []resolver.Address{ + {Addr: addresses[0]}, + {Addr: addresses[1]}, + {Addr: addresses[2]}, + } mr.InitialState(resolver.State{ Addresses: fullAddresses, ServiceConfig: sc, @@ -362,11 +336,7 @@ func (s) TestNoopConfiguration(t *testing.T) { "minimumHosts": 3, "requestVolume": 5 }, - "childPolicy": [ - { - "round_robin": {} - } - ] + "childPolicy": [{"round_robin": {}}] } } ] @@ -385,6 +355,11 @@ func (s) TestNoopConfiguration(t *testing.T) { t.Fatalf("error in expected round robin: %v", err) } + // The addresses which don't return errors. + okAddresses := []resolver.Address{ + {Addr: addresses[0]}, + {Addr: addresses[1]}, + } // Now that the reconfigured balancer has data about the failing upstream, // it should eject the upstream and only route across the two healthy // upstreams. From 414dba186092cb4766e3ccc15184225a077d9c3d Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Wed, 31 Aug 2022 21:00:12 -0400 Subject: [PATCH 19/22] Responded to Doug's comments --- xds/internal/balancer/outlierdetection/balancer.go | 11 +++++------ xds/internal/balancer/outlierdetection/callcounter.go | 4 ---- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 8e35fb43d1c..a38545617f1 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -71,7 +71,6 @@ type bb struct{} func (bb) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Balancer { b := &outlierDetectionBalancer{ cc: cc, - bOpts: bOpts, closed: grpcsync.NewEvent(), addrs: make(map[string]*addressInfo), scWrappers: make(map[balancer.SubConn]*subConnWrapper), @@ -169,12 +168,11 @@ type outlierDetectionBalancer struct { closed *grpcsync.Event cc balancer.ClientConn - bOpts balancer.BuildOptions logger *grpclog.PrefixLogger - // childMu protects child and also guarantees updates to the child are sent - // synchronously (to uphold the balancer.Balancer API guarantee of - // synchronous calls). + // childMu protects the closing of the child and also guarantees updates to + // the child are sent synchronously (to uphold the balancer.Balancer API + // guarantee of synchronous calls). // // For example, run() could read that the child is not nil while processing // SubConn updates, and then Close() could write to the the child, clearing @@ -288,7 +286,7 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt err := b.child.SwitchTo(bb) if err != nil { b.childMu.Unlock() - return err + return fmt.Errorf("outlier detection: error switching to child of type %q: %v", lbCfg.ChildPolicy.Name, err) } b.childMu.Unlock() } @@ -425,6 +423,7 @@ func (wp *wrappedPicker) Pick(info balancer.PickInfo) (balancer.PickResult, erro if !ok { // This can never happen, but check is present for defensive // programming. + logger.Errorf("Picked SubConn from child picker is not a SubConnWrapper") return balancer.PickResult{ SubConn: pr.SubConn, Done: done, diff --git a/xds/internal/balancer/outlierdetection/callcounter.go b/xds/internal/balancer/outlierdetection/callcounter.go index 1489ea8ffb2..4597f727b6e 100644 --- a/xds/internal/balancer/outlierdetection/callcounter.go +++ b/xds/internal/balancer/outlierdetection/callcounter.go @@ -38,8 +38,6 @@ func newCallCounter() *callCounter { // The activeBucket is used to actively count any finished RPC's, and the // inactiveBucket is populated with this activeBucket's data every interval for // use by the Outlier Detection algorithm. -// -// Caller must hold b.mu. type callCounter struct { // activeBucket updates every time a call finishes (from picker passed to // Client Conn), so protect pointer read with atomic load of unsafe.Pointer @@ -57,8 +55,6 @@ func (cc *callCounter) clear() { // active bucket. Then the inactive bucket contains the number of successes and // failures since the last time the timer triggered. Those numbers are used to // evaluate the ejection criteria." - A50. -// -// Caller must hold b.mu. func (cc *callCounter) swap() { ib := cc.inactiveBucket *ib = bucket{} From 8c143d196cdb5d0fcb5c0bcb48159a65d7a1df7e Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Thu, 1 Sep 2022 19:25:28 -0400 Subject: [PATCH 20/22] Wait for run() to exit in Close() using Wait Group --- .../balancer/outlierdetection/balancer.go | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index a38545617f1..105a8d0a64a 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -80,6 +80,7 @@ func (bb) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Ba b.logger = prefixLogger(b) b.logger.Infof("Created") b.child = gracefulswitch.NewBalancer(b, bOpts) + b.done.Add(1) go b.run() return b } @@ -167,17 +168,12 @@ type outlierDetectionBalancer struct { recentPickerNoop bool closed *grpcsync.Event + done sync.WaitGroup cc balancer.ClientConn logger *grpclog.PrefixLogger - // childMu protects the closing of the child and also guarantees updates to - // the child are sent synchronously (to uphold the balancer.Balancer API + // childMu guards calls into child (to uphold the balancer.Balancer API // guarantee of synchronous calls). - // - // For example, run() could read that the child is not nil while processing - // SubConn updates, and then Close() could write to the the child, clearing - // the child, making it nil, then you try and update a cleared and already - // closed child, which breaks the balancer.Balancer API. childMu sync.Mutex child *gracefulswitch.Balancer @@ -372,6 +368,7 @@ func (b *outlierDetectionBalancer) UpdateSubConnState(sc balancer.SubConn, state func (b *outlierDetectionBalancer) Close() { b.closed.Fire() + b.done.Wait() if b.child != nil { b.childMu.Lock() b.child.Close() @@ -619,11 +616,11 @@ func min(x, y int64) int64 { func (b *outlierDetectionBalancer) handleSubConnUpdate(u *scUpdate) { scw := u.scw scw.latestState = u.state - b.childMu.Lock() if !scw.ejected && b.child != nil { + b.childMu.Lock() b.child.UpdateSubConnState(scw, u.state) + b.childMu.Unlock() } - b.childMu.Unlock() } // handleEjectedUpdate handles any SubConns that get ejected/unejected, and @@ -639,11 +636,11 @@ func (b *outlierDetectionBalancer) handleEjectedUpdate(u *ejectionUpdate) { ConnectivityState: connectivity.TransientFailure, } } - b.childMu.Lock() if b.child != nil { + b.childMu.Lock() b.child.UpdateSubConnState(scw, stateToUpdate) + b.childMu.Unlock() } - b.childMu.Unlock() } // handleChildStateUpdate forwards the picker update wrapped in a wrapped picker @@ -700,6 +697,7 @@ func (b *outlierDetectionBalancer) handleLBConfigUpdate(u lbCfgUpdate) { } func (b *outlierDetectionBalancer) run() { + defer b.done.Done() for { select { case update := <-b.scUpdateCh.Get(): From bdba12a0a5201bd1c34968d7525e58d1828b44bb Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Thu, 1 Sep 2022 19:41:33 -0400 Subject: [PATCH 21/22] Got rid of the clearing of child and checking of cleared child --- .../balancer/outlierdetection/balancer.go | 25 ++++++------------- 1 file changed, 7 insertions(+), 18 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index 105a8d0a64a..cd24666f78e 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -339,9 +339,6 @@ func (b *outlierDetectionBalancer) UpdateClientConnState(s balancer.ClientConnSt } func (b *outlierDetectionBalancer) ResolverError(err error) { - if b.child == nil { - return - } b.childMu.Lock() defer b.childMu.Unlock() b.child.ResolverError(err) @@ -369,12 +366,9 @@ func (b *outlierDetectionBalancer) UpdateSubConnState(sc balancer.SubConn, state func (b *outlierDetectionBalancer) Close() { b.closed.Fire() b.done.Wait() - if b.child != nil { - b.childMu.Lock() - b.child.Close() - b.child = nil - b.childMu.Unlock() - } + b.childMu.Lock() + b.child.Close() + b.childMu.Unlock() b.mu.Lock() defer b.mu.Unlock() @@ -384,9 +378,6 @@ func (b *outlierDetectionBalancer) Close() { } func (b *outlierDetectionBalancer) ExitIdle() { - if b.child == nil { - return - } b.childMu.Lock() defer b.childMu.Unlock() b.child.ExitIdle() @@ -616,7 +607,7 @@ func min(x, y int64) int64 { func (b *outlierDetectionBalancer) handleSubConnUpdate(u *scUpdate) { scw := u.scw scw.latestState = u.state - if !scw.ejected && b.child != nil { + if !scw.ejected { b.childMu.Lock() b.child.UpdateSubConnState(scw, u.state) b.childMu.Unlock() @@ -636,11 +627,9 @@ func (b *outlierDetectionBalancer) handleEjectedUpdate(u *ejectionUpdate) { ConnectivityState: connectivity.TransientFailure, } } - if b.child != nil { - b.childMu.Lock() - b.child.UpdateSubConnState(scw, stateToUpdate) - b.childMu.Unlock() - } + b.childMu.Lock() + b.child.UpdateSubConnState(scw, stateToUpdate) + b.childMu.Unlock() } // handleChildStateUpdate forwards the picker update wrapped in a wrapped picker From 5482af7e90abec7ec4f0d9bdf486162640234391 Mon Sep 17 00:00:00 2001 From: Zach Reyes Date: Tue, 6 Sep 2022 14:19:34 -0400 Subject: [PATCH 22/22] Switched done from wait group to grpcsync.Event --- xds/internal/balancer/outlierdetection/balancer.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/xds/internal/balancer/outlierdetection/balancer.go b/xds/internal/balancer/outlierdetection/balancer.go index cd24666f78e..8e54a4a10d5 100644 --- a/xds/internal/balancer/outlierdetection/balancer.go +++ b/xds/internal/balancer/outlierdetection/balancer.go @@ -72,6 +72,7 @@ func (bb) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Ba b := &outlierDetectionBalancer{ cc: cc, closed: grpcsync.NewEvent(), + done: grpcsync.NewEvent(), addrs: make(map[string]*addressInfo), scWrappers: make(map[balancer.SubConn]*subConnWrapper), scUpdateCh: buffer.NewUnbounded(), @@ -80,7 +81,6 @@ func (bb) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Ba b.logger = prefixLogger(b) b.logger.Infof("Created") b.child = gracefulswitch.NewBalancer(b, bOpts) - b.done.Add(1) go b.run() return b } @@ -168,7 +168,7 @@ type outlierDetectionBalancer struct { recentPickerNoop bool closed *grpcsync.Event - done sync.WaitGroup + done *grpcsync.Event cc balancer.ClientConn logger *grpclog.PrefixLogger @@ -365,7 +365,7 @@ func (b *outlierDetectionBalancer) UpdateSubConnState(sc balancer.SubConn, state func (b *outlierDetectionBalancer) Close() { b.closed.Fire() - b.done.Wait() + <-b.done.Done() b.childMu.Lock() b.child.Close() b.childMu.Unlock() @@ -686,11 +686,14 @@ func (b *outlierDetectionBalancer) handleLBConfigUpdate(u lbCfgUpdate) { } func (b *outlierDetectionBalancer) run() { - defer b.done.Done() + defer b.done.Fire() for { select { case update := <-b.scUpdateCh.Get(): b.scUpdateCh.Load() + if b.closed.HasFired() { // don't send SubConn updates to child after the balancer has been closed + return + } switch u := update.(type) { case *scUpdate: b.handleSubConnUpdate(u)