Refactor roundrobin to support custom picker (#1707)
Split round_robin balancer into a base balancer (which manages SubConn and connectivity states), and a round_robin picker. A custom picker with other picking algorithms can be provided to build custom balancers.
This commit is contained in:
209
balancer/base/balancer.go
Normal file
209
balancer/base/balancer.go
Normal file
@ -0,0 +1,209 @@
|
|||||||
|
/*
|
||||||
|
*
|
||||||
|
* Copyright 2017 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 base
|
||||||
|
|
||||||
|
import (
|
||||||
|
"golang.org/x/net/context"
|
||||||
|
"google.golang.org/grpc/balancer"
|
||||||
|
"google.golang.org/grpc/connectivity"
|
||||||
|
"google.golang.org/grpc/grpclog"
|
||||||
|
"google.golang.org/grpc/resolver"
|
||||||
|
)
|
||||||
|
|
||||||
|
type baseBuilder struct {
|
||||||
|
name string
|
||||||
|
pickerBuilder PickerBuilder
|
||||||
|
}
|
||||||
|
|
||||||
|
func (bb *baseBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer {
|
||||||
|
return &baseBalancer{
|
||||||
|
cc: cc,
|
||||||
|
pickerBuilder: bb.pickerBuilder,
|
||||||
|
|
||||||
|
subConns: make(map[resolver.Address]balancer.SubConn),
|
||||||
|
scStates: make(map[balancer.SubConn]connectivity.State),
|
||||||
|
csEvltr: &connectivityStateEvaluator{},
|
||||||
|
// Initialize picker to a picker that always return
|
||||||
|
// ErrNoSubConnAvailable, because when state of a SubConn changes, we
|
||||||
|
// may call UpdateBalancerState with this picker.
|
||||||
|
picker: NewErrPicker(balancer.ErrNoSubConnAvailable),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (bb *baseBuilder) Name() string {
|
||||||
|
return bb.name
|
||||||
|
}
|
||||||
|
|
||||||
|
type baseBalancer struct {
|
||||||
|
cc balancer.ClientConn
|
||||||
|
pickerBuilder PickerBuilder
|
||||||
|
|
||||||
|
csEvltr *connectivityStateEvaluator
|
||||||
|
state connectivity.State
|
||||||
|
|
||||||
|
subConns map[resolver.Address]balancer.SubConn
|
||||||
|
scStates map[balancer.SubConn]connectivity.State
|
||||||
|
picker balancer.Picker
|
||||||
|
}
|
||||||
|
|
||||||
|
func (b *baseBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) {
|
||||||
|
if err != nil {
|
||||||
|
grpclog.Infof("base.baseBalancer: HandleResolvedAddrs called with error %v", err)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
grpclog.Infoln("base.baseBalancer: got new resolved addresses: ", addrs)
|
||||||
|
// addrsSet is the set converted from addrs, it's used for quick lookup of an address.
|
||||||
|
addrsSet := make(map[resolver.Address]struct{})
|
||||||
|
for _, a := range addrs {
|
||||||
|
addrsSet[a] = struct{}{}
|
||||||
|
if _, ok := b.subConns[a]; !ok {
|
||||||
|
// a is a new address (not existing in b.subConns).
|
||||||
|
sc, err := b.cc.NewSubConn([]resolver.Address{a}, balancer.NewSubConnOptions{})
|
||||||
|
if err != nil {
|
||||||
|
grpclog.Warningf("base.baseBalancer: failed to create new SubConn: %v", err)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
b.subConns[a] = sc
|
||||||
|
b.scStates[sc] = connectivity.Idle
|
||||||
|
sc.Connect()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for a, sc := range b.subConns {
|
||||||
|
// a was removed by resolver.
|
||||||
|
if _, ok := addrsSet[a]; !ok {
|
||||||
|
b.cc.RemoveSubConn(sc)
|
||||||
|
delete(b.subConns, a)
|
||||||
|
// Keep the state of this sc in b.scStates until sc's state becomes Shutdown.
|
||||||
|
// The entry will be deleted in HandleSubConnStateChange.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// regeneratePicker takes a snapshot of the balancer, and generates a picker
|
||||||
|
// from it. The picker is
|
||||||
|
// - errPicker with ErrTransientFailure if the balancer is in TransientFailure,
|
||||||
|
// - built by the pickerBuilder with all READY SubConns otherwise.
|
||||||
|
func (b *baseBalancer) regeneratePicker() {
|
||||||
|
if b.state == connectivity.TransientFailure {
|
||||||
|
b.picker = NewErrPicker(balancer.ErrTransientFailure)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
readySCs := make(map[resolver.Address]balancer.SubConn)
|
||||||
|
|
||||||
|
// Filter out all ready SCs from full subConn map.
|
||||||
|
for addr, sc := range b.subConns {
|
||||||
|
if st, ok := b.scStates[sc]; ok && st == connectivity.Ready {
|
||||||
|
readySCs[addr] = sc
|
||||||
|
}
|
||||||
|
}
|
||||||
|
b.picker = b.pickerBuilder.Build(readySCs)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (b *baseBalancer) HandleSubConnStateChange(sc balancer.SubConn, s connectivity.State) {
|
||||||
|
grpclog.Infof("base.baseBalancer: handle SubConn state change: %p, %v", sc, s)
|
||||||
|
oldS, ok := b.scStates[sc]
|
||||||
|
if !ok {
|
||||||
|
grpclog.Infof("base.baseBalancer: got state changes for an unknown SubConn: %p, %v", sc, s)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
b.scStates[sc] = s
|
||||||
|
switch s {
|
||||||
|
case connectivity.Idle:
|
||||||
|
sc.Connect()
|
||||||
|
case connectivity.Shutdown:
|
||||||
|
// When an address was removed by resolver, b called RemoveSubConn but
|
||||||
|
// kept the sc's state in scStates. Remove state for this sc here.
|
||||||
|
delete(b.scStates, sc)
|
||||||
|
}
|
||||||
|
|
||||||
|
oldAggrState := b.state
|
||||||
|
b.state = b.csEvltr.recordTransition(oldS, s)
|
||||||
|
|
||||||
|
// Regenerate picker when one of the following happens:
|
||||||
|
// - this sc became ready from not-ready
|
||||||
|
// - this sc became not-ready from ready
|
||||||
|
// - the aggregated state of balancer became TransientFailure from non-TransientFailure
|
||||||
|
// - the aggregated state of balancer became non-TransientFailure from TransientFailure
|
||||||
|
if (s == connectivity.Ready) != (oldS == connectivity.Ready) ||
|
||||||
|
(b.state == connectivity.TransientFailure) != (oldAggrState == connectivity.TransientFailure) {
|
||||||
|
b.regeneratePicker()
|
||||||
|
}
|
||||||
|
|
||||||
|
b.cc.UpdateBalancerState(b.state, b.picker)
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
// Close is a nop because base balancer doesn't have internal state to clean up,
|
||||||
|
// and it doesn't need to call RemoveSubConn for the SubConns.
|
||||||
|
func (b *baseBalancer) Close() {
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewErrPicker returns a picker that always returns err on Pick().
|
||||||
|
func NewErrPicker(err error) balancer.Picker {
|
||||||
|
return &errPicker{err: err}
|
||||||
|
}
|
||||||
|
|
||||||
|
type errPicker struct {
|
||||||
|
err error // Pick() always returns this err.
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *errPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
||||||
|
return nil, nil, p.err
|
||||||
|
}
|
||||||
|
|
||||||
|
// connectivityStateEvaluator gets updated by addrConns when their
|
||||||
|
// states transition, based on which it evaluates the state of
|
||||||
|
// ClientConn.
|
||||||
|
type connectivityStateEvaluator struct {
|
||||||
|
numReady uint64 // Number of addrConns in ready state.
|
||||||
|
numConnecting uint64 // Number of addrConns in connecting state.
|
||||||
|
numTransientFailure uint64 // Number of addrConns in transientFailure.
|
||||||
|
}
|
||||||
|
|
||||||
|
// recordTransition records state change happening in every subConn and based on
|
||||||
|
// that it evaluates what aggregated state should be.
|
||||||
|
// It can only transition between Ready, Connecting and TransientFailure. Other states,
|
||||||
|
// Idle and Shutdown are transitioned into by ClientConn; in the beginning of the connection
|
||||||
|
// before any subConn is created ClientConn is in idle state. In the end when ClientConn
|
||||||
|
// closes it is in Shutdown state.
|
||||||
|
//
|
||||||
|
// recordTransition should only be called synchronously from the same goroutine.
|
||||||
|
func (cse *connectivityStateEvaluator) recordTransition(oldState, newState connectivity.State) connectivity.State {
|
||||||
|
// Update counters.
|
||||||
|
for idx, state := range []connectivity.State{oldState, newState} {
|
||||||
|
updateVal := 2*uint64(idx) - 1 // -1 for oldState and +1 for new.
|
||||||
|
switch state {
|
||||||
|
case connectivity.Ready:
|
||||||
|
cse.numReady += updateVal
|
||||||
|
case connectivity.Connecting:
|
||||||
|
cse.numConnecting += updateVal
|
||||||
|
case connectivity.TransientFailure:
|
||||||
|
cse.numTransientFailure += updateVal
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Evaluate.
|
||||||
|
if cse.numReady > 0 {
|
||||||
|
return connectivity.Ready
|
||||||
|
}
|
||||||
|
if cse.numConnecting > 0 {
|
||||||
|
return connectivity.Connecting
|
||||||
|
}
|
||||||
|
return connectivity.TransientFailure
|
||||||
|
}
|
52
balancer/base/base.go
Normal file
52
balancer/base/base.go
Normal file
@ -0,0 +1,52 @@
|
|||||||
|
/*
|
||||||
|
*
|
||||||
|
* Copyright 2017 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 base defines a balancer base that can be used to build balancers with
|
||||||
|
// different picking algorithms.
|
||||||
|
//
|
||||||
|
// The base balancer creates a new SubConn for each resolved address. The
|
||||||
|
// provided picker will only be notified about READY SubConns.
|
||||||
|
//
|
||||||
|
// This package is the base of round_robin balancer, its purpose is to be used
|
||||||
|
// to build round_robin like balancers with complex picking algorithms.
|
||||||
|
// Balancers with more complicated logic should try to implement a balancer
|
||||||
|
// builder from scratch.
|
||||||
|
//
|
||||||
|
// All APIs in this package are experimental.
|
||||||
|
package base
|
||||||
|
|
||||||
|
import (
|
||||||
|
"google.golang.org/grpc/balancer"
|
||||||
|
"google.golang.org/grpc/resolver"
|
||||||
|
)
|
||||||
|
|
||||||
|
// PickerBuilder creates balancer.Picker.
|
||||||
|
type PickerBuilder interface {
|
||||||
|
// Build takes a slice of ready SubConns, and returns a picker that will be
|
||||||
|
// used by gRPC to pick a SubConn.
|
||||||
|
Build(readySCs map[resolver.Address]balancer.SubConn) balancer.Picker
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewBalancerBuilder returns a balancer builder. The balancers
|
||||||
|
// built by this builder will use the picker builder to build pickers.
|
||||||
|
func NewBalancerBuilder(name string, pb PickerBuilder) balancer.Builder {
|
||||||
|
return &baseBuilder{
|
||||||
|
name: name,
|
||||||
|
pickerBuilder: pb,
|
||||||
|
}
|
||||||
|
}
|
@ -26,145 +26,34 @@ import (
|
|||||||
|
|
||||||
"golang.org/x/net/context"
|
"golang.org/x/net/context"
|
||||||
"google.golang.org/grpc/balancer"
|
"google.golang.org/grpc/balancer"
|
||||||
"google.golang.org/grpc/connectivity"
|
"google.golang.org/grpc/balancer/base"
|
||||||
"google.golang.org/grpc/grpclog"
|
"google.golang.org/grpc/grpclog"
|
||||||
"google.golang.org/grpc/resolver"
|
"google.golang.org/grpc/resolver"
|
||||||
)
|
)
|
||||||
|
|
||||||
// newBuilder creates a new roundrobin balancer builder.
|
// newBuilder creates a new roundrobin balancer builder.
|
||||||
func newBuilder() balancer.Builder {
|
func newBuilder() balancer.Builder {
|
||||||
return &rrBuilder{}
|
return base.NewBalancerBuilder("round_robin", &rrPickerBuilder{})
|
||||||
}
|
}
|
||||||
|
|
||||||
func init() {
|
func init() {
|
||||||
balancer.Register(newBuilder())
|
balancer.Register(newBuilder())
|
||||||
}
|
}
|
||||||
|
|
||||||
type rrBuilder struct{}
|
type rrPickerBuilder struct{}
|
||||||
|
|
||||||
func (*rrBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer {
|
func (*rrPickerBuilder) Build(readySCs map[resolver.Address]balancer.SubConn) balancer.Picker {
|
||||||
return &rrBalancer{
|
grpclog.Infof("roundrobinPicker: newPicker called with readySCs: %v", readySCs)
|
||||||
cc: cc,
|
var scs []balancer.SubConn
|
||||||
subConns: make(map[resolver.Address]balancer.SubConn),
|
for _, sc := range readySCs {
|
||||||
scStates: make(map[balancer.SubConn]connectivity.State),
|
scs = append(scs, sc)
|
||||||
csEvltr: &connectivityStateEvaluator{},
|
}
|
||||||
// Initialize picker to a picker that always return
|
return &rrPicker{
|
||||||
// ErrNoSubConnAvailable, because when state of a SubConn changes, we
|
subConns: scs,
|
||||||
// may call UpdateBalancerState with this picker.
|
|
||||||
picker: newPicker([]balancer.SubConn{}, nil),
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (*rrBuilder) Name() string {
|
type rrPicker struct {
|
||||||
return "round_robin"
|
|
||||||
}
|
|
||||||
|
|
||||||
type rrBalancer struct {
|
|
||||||
cc balancer.ClientConn
|
|
||||||
|
|
||||||
csEvltr *connectivityStateEvaluator
|
|
||||||
state connectivity.State
|
|
||||||
|
|
||||||
subConns map[resolver.Address]balancer.SubConn
|
|
||||||
scStates map[balancer.SubConn]connectivity.State
|
|
||||||
picker *picker
|
|
||||||
}
|
|
||||||
|
|
||||||
func (b *rrBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) {
|
|
||||||
if err != nil {
|
|
||||||
grpclog.Infof("roundrobin.rrBalancer: HandleResolvedAddrs called with error %v", err)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
grpclog.Infoln("roundrobin.rrBalancer: got new resolved addresses: ", addrs)
|
|
||||||
// addrsSet is the set converted from addrs, it's used for quick lookup of an address.
|
|
||||||
addrsSet := make(map[resolver.Address]struct{})
|
|
||||||
for _, a := range addrs {
|
|
||||||
addrsSet[a] = struct{}{}
|
|
||||||
if _, ok := b.subConns[a]; !ok {
|
|
||||||
// a is a new address (not existing in b.subConns).
|
|
||||||
sc, err := b.cc.NewSubConn([]resolver.Address{a}, balancer.NewSubConnOptions{})
|
|
||||||
if err != nil {
|
|
||||||
grpclog.Warningf("roundrobin.rrBalancer: failed to create new SubConn: %v", err)
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
b.subConns[a] = sc
|
|
||||||
b.scStates[sc] = connectivity.Idle
|
|
||||||
sc.Connect()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
for a, sc := range b.subConns {
|
|
||||||
// a was removed by resolver.
|
|
||||||
if _, ok := addrsSet[a]; !ok {
|
|
||||||
b.cc.RemoveSubConn(sc)
|
|
||||||
delete(b.subConns, a)
|
|
||||||
// Keep the state of this sc in b.scStates until sc's state becomes Shutdown.
|
|
||||||
// The entry will be deleted in HandleSubConnStateChange.
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// regeneratePicker takes a snapshot of the balancer, and generates a picker
|
|
||||||
// from it. The picker
|
|
||||||
// - always returns ErrTransientFailure if the balancer is in TransientFailure,
|
|
||||||
// - or does round robin selection of all READY SubConns otherwise.
|
|
||||||
func (b *rrBalancer) regeneratePicker() {
|
|
||||||
if b.state == connectivity.TransientFailure {
|
|
||||||
b.picker = newPicker(nil, balancer.ErrTransientFailure)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
var readySCs []balancer.SubConn
|
|
||||||
for sc, st := range b.scStates {
|
|
||||||
if st == connectivity.Ready {
|
|
||||||
readySCs = append(readySCs, sc)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
b.picker = newPicker(readySCs, nil)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (b *rrBalancer) HandleSubConnStateChange(sc balancer.SubConn, s connectivity.State) {
|
|
||||||
grpclog.Infof("roundrobin.rrBalancer: handle SubConn state change: %p, %v", sc, s)
|
|
||||||
oldS, ok := b.scStates[sc]
|
|
||||||
if !ok {
|
|
||||||
grpclog.Infof("roundrobin.rrBalancer: got state changes for an unknown SubConn: %p, %v", sc, s)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
b.scStates[sc] = s
|
|
||||||
switch s {
|
|
||||||
case connectivity.Idle:
|
|
||||||
sc.Connect()
|
|
||||||
case connectivity.Shutdown:
|
|
||||||
// When an address was removed by resolver, b called RemoveSubConn but
|
|
||||||
// kept the sc's state in scStates. Remove state for this sc here.
|
|
||||||
delete(b.scStates, sc)
|
|
||||||
}
|
|
||||||
|
|
||||||
oldAggrState := b.state
|
|
||||||
b.state = b.csEvltr.recordTransition(oldS, s)
|
|
||||||
|
|
||||||
// Regenerate picker when one of the following happens:
|
|
||||||
// - this sc became ready from not-ready
|
|
||||||
// - this sc became not-ready from ready
|
|
||||||
// - the aggregated state of balancer became TransientFailure from non-TransientFailure
|
|
||||||
// - the aggregated state of balancer became non-TransientFailure from TransientFailure
|
|
||||||
if (s == connectivity.Ready) != (oldS == connectivity.Ready) ||
|
|
||||||
(b.state == connectivity.TransientFailure) != (oldAggrState == connectivity.TransientFailure) {
|
|
||||||
b.regeneratePicker()
|
|
||||||
}
|
|
||||||
|
|
||||||
b.cc.UpdateBalancerState(b.state, b.picker)
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
// Close is a nop because roundrobin balancer doesn't internal state to clean
|
|
||||||
// up, and it doesn't need to call RemoveSubConn for the SubConns.
|
|
||||||
func (b *rrBalancer) Close() {
|
|
||||||
}
|
|
||||||
|
|
||||||
type picker struct {
|
|
||||||
// If err is not nil, Pick always returns this err. It's immutable after
|
|
||||||
// picker is created.
|
|
||||||
err error
|
|
||||||
|
|
||||||
// subConns is the snapshot of the roundrobin balancer when this picker was
|
// subConns is the snapshot of the roundrobin balancer when this picker was
|
||||||
// created. The slice is immutable. Each Get() will do a round robin
|
// created. The slice is immutable. Each Get() will do a round robin
|
||||||
// selection from it and return the selected SubConn.
|
// selection from it and return the selected SubConn.
|
||||||
@ -174,20 +63,7 @@ type picker struct {
|
|||||||
next int
|
next int
|
||||||
}
|
}
|
||||||
|
|
||||||
func newPicker(scs []balancer.SubConn, err error) *picker {
|
func (p *rrPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
||||||
grpclog.Infof("roundrobinPicker: newPicker called with scs: %v, %v", scs, err)
|
|
||||||
if err != nil {
|
|
||||||
return &picker{err: err}
|
|
||||||
}
|
|
||||||
return &picker{
|
|
||||||
subConns: scs,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *picker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
|
||||||
if p.err != nil {
|
|
||||||
return nil, nil, p.err
|
|
||||||
}
|
|
||||||
if len(p.subConns) <= 0 {
|
if len(p.subConns) <= 0 {
|
||||||
return nil, nil, balancer.ErrNoSubConnAvailable
|
return nil, nil, balancer.ErrNoSubConnAvailable
|
||||||
}
|
}
|
||||||
@ -198,44 +74,3 @@ func (p *picker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.
|
|||||||
p.mu.Unlock()
|
p.mu.Unlock()
|
||||||
return sc, nil, nil
|
return sc, nil, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// connectivityStateEvaluator gets updated by addrConns when their
|
|
||||||
// states transition, based on which it evaluates the state of
|
|
||||||
// ClientConn.
|
|
||||||
type connectivityStateEvaluator struct {
|
|
||||||
numReady uint64 // Number of addrConns in ready state.
|
|
||||||
numConnecting uint64 // Number of addrConns in connecting state.
|
|
||||||
numTransientFailure uint64 // Number of addrConns in transientFailure.
|
|
||||||
}
|
|
||||||
|
|
||||||
// recordTransition records state change happening in every subConn and based on
|
|
||||||
// that it evaluates what aggregated state should be.
|
|
||||||
// It can only transition between Ready, Connecting and TransientFailure. Other states,
|
|
||||||
// Idle and Shutdown are transitioned into by ClientConn; in the beginning of the connection
|
|
||||||
// before any subConn is created ClientConn is in idle state. In the end when ClientConn
|
|
||||||
// closes it is in Shutdown state.
|
|
||||||
//
|
|
||||||
// recordTransition should only be called synchronously from the same goroutine.
|
|
||||||
func (cse *connectivityStateEvaluator) recordTransition(oldState, newState connectivity.State) connectivity.State {
|
|
||||||
// Update counters.
|
|
||||||
for idx, state := range []connectivity.State{oldState, newState} {
|
|
||||||
updateVal := 2*uint64(idx) - 1 // -1 for oldState and +1 for new.
|
|
||||||
switch state {
|
|
||||||
case connectivity.Ready:
|
|
||||||
cse.numReady += updateVal
|
|
||||||
case connectivity.Connecting:
|
|
||||||
cse.numConnecting += updateVal
|
|
||||||
case connectivity.TransientFailure:
|
|
||||||
cse.numTransientFailure += updateVal
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Evaluate.
|
|
||||||
if cse.numReady > 0 {
|
|
||||||
return connectivity.Ready
|
|
||||||
}
|
|
||||||
if cse.numConnecting > 0 {
|
|
||||||
return connectivity.Connecting
|
|
||||||
}
|
|
||||||
return connectivity.TransientFailure
|
|
||||||
}
|
|
||||||
|
Reference in New Issue
Block a user