add p2c peak ewma load balancer
parent
c7544d0e60
commit
63583d4744
@ -0,0 +1,37 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"zero/core/discov"
|
||||
"zero/example/rpc/remote/unary"
|
||||
"zero/rpcx"
|
||||
)
|
||||
|
||||
func main() {
|
||||
cli := rpcx.MustNewClient(rpcx.RpcClientConf{
|
||||
Etcd: discov.EtcdConf{
|
||||
Hosts: []string{"localhost:2379"},
|
||||
Key: "rpcx",
|
||||
},
|
||||
})
|
||||
greet := unary.NewGreeterClient(cli.Conn())
|
||||
ticker := time.NewTicker(time.Second)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
resp, err := greet.Greet(context.Background(), &unary.Request{
|
||||
Name: "kevin",
|
||||
})
|
||||
if err != nil {
|
||||
fmt.Println("X", err.Error())
|
||||
} else {
|
||||
fmt.Println("=>", resp.Greet)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -1,17 +1,13 @@
|
||||
{
|
||||
"Name": "rpc.unary",
|
||||
"MetricsUrl": "http://localhost:2222/add",
|
||||
"Log": {
|
||||
"Mode": "volume"
|
||||
},
|
||||
"ListenOn": "localhost:3457",
|
||||
"Auth": false,
|
||||
"Etcd": {
|
||||
"Hosts": [
|
||||
"localhost:2379"
|
||||
],
|
||||
"Key": "rpcx"
|
||||
},
|
||||
"Redis": {
|
||||
"Host": "localhost:6379",
|
||||
"Type": "node",
|
||||
"Key": "apps"
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,172 @@
|
||||
package p2c
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math"
|
||||
"math/rand"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"zero/core/timex"
|
||||
"zero/rpcx/internal/codes"
|
||||
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/balancer/base"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
const (
|
||||
Name = "p2c_ewma"
|
||||
decayTime = int64(time.Millisecond * 600)
|
||||
forcePick = int64(time.Second)
|
||||
initSuccess = 1000
|
||||
throttleSuccess = initSuccess / 2
|
||||
penalty = int64(math.MaxInt32)
|
||||
pickTimes = 3
|
||||
)
|
||||
|
||||
func init() {
|
||||
balancer.Register(newBuilder())
|
||||
}
|
||||
|
||||
type p2cPickerBuilder struct {
|
||||
}
|
||||
|
||||
func newBuilder() balancer.Builder {
|
||||
return base.NewBalancerBuilder(Name, new(p2cPickerBuilder))
|
||||
}
|
||||
|
||||
func (b *p2cPickerBuilder) Build(readySCs map[resolver.Address]balancer.SubConn) balancer.Picker {
|
||||
if len(readySCs) == 0 {
|
||||
return base.NewErrPicker(balancer.ErrNoSubConnAvailable)
|
||||
}
|
||||
|
||||
var conns []*subConn
|
||||
for addr, conn := range readySCs {
|
||||
conns = append(conns, &subConn{
|
||||
addr: addr,
|
||||
conn: conn,
|
||||
success: initSuccess,
|
||||
})
|
||||
}
|
||||
|
||||
return &p2cPicker{
|
||||
conns: conns,
|
||||
r: rand.New(rand.NewSource(time.Now().UnixNano())),
|
||||
}
|
||||
}
|
||||
|
||||
type p2cPicker struct {
|
||||
conns []*subConn
|
||||
r *rand.Rand
|
||||
lock sync.Mutex
|
||||
}
|
||||
|
||||
func (p *p2cPicker) Pick(ctx context.Context, info balancer.PickInfo) (
|
||||
conn balancer.SubConn, done func(balancer.DoneInfo), err error) {
|
||||
p.lock.Lock()
|
||||
defer p.lock.Unlock()
|
||||
|
||||
var chosen *subConn
|
||||
switch len(p.conns) {
|
||||
case 0:
|
||||
return nil, nil, balancer.ErrNoSubConnAvailable
|
||||
case 1:
|
||||
chosen = p.choose(p.conns[0], nil)
|
||||
case 2:
|
||||
chosen = p.choose(p.conns[0], p.conns[1])
|
||||
default:
|
||||
var node1, node2 *subConn
|
||||
for i := 0; i < pickTimes; i++ {
|
||||
a := p.r.Intn(len(p.conns))
|
||||
b := p.r.Intn(len(p.conns) - 1)
|
||||
if b >= a {
|
||||
b++
|
||||
}
|
||||
node1 = p.conns[a]
|
||||
node2 = p.conns[b]
|
||||
if node1.healthy() && node2.healthy() {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
chosen = p.choose(node1, node2)
|
||||
}
|
||||
|
||||
atomic.AddInt64(&chosen.inflight, 1)
|
||||
return chosen.conn, p.buildDoneFunc(chosen), nil
|
||||
}
|
||||
|
||||
func (p *p2cPicker) buildDoneFunc(c *subConn) func(info balancer.DoneInfo) {
|
||||
start := int64(timex.Now())
|
||||
return func(info balancer.DoneInfo) {
|
||||
atomic.AddInt64(&c.inflight, -1)
|
||||
now := int64(timex.Now())
|
||||
last := atomic.SwapInt64(&c.last, int64(now))
|
||||
td := now - last
|
||||
if td < 0 {
|
||||
td = 0
|
||||
}
|
||||
w := math.Exp(float64(-td) / float64(decayTime))
|
||||
lag := now - start
|
||||
if lag < 0 {
|
||||
lag = 0
|
||||
}
|
||||
olag := atomic.LoadUint64(&c.lag)
|
||||
if olag == 0 {
|
||||
w = 0
|
||||
}
|
||||
atomic.StoreUint64(&c.lag, uint64(float64(olag)*w+float64(lag)*(1-w)))
|
||||
success := initSuccess
|
||||
if info.Err != nil && !codes.Acceptable(info.Err) {
|
||||
success = 0
|
||||
}
|
||||
osucc := atomic.LoadUint64(&c.success)
|
||||
atomic.StoreUint64(&c.success, uint64(float64(osucc)*w+float64(success)*(1-w)))
|
||||
}
|
||||
}
|
||||
|
||||
func (p *p2cPicker) choose(c1, c2 *subConn) *subConn {
|
||||
start := int64(timex.Now())
|
||||
if c2 == nil {
|
||||
atomic.StoreInt64(&c1.pick, start)
|
||||
return c1
|
||||
}
|
||||
|
||||
if c1.load() > c2.load() {
|
||||
c1, c2 = c2, c1
|
||||
}
|
||||
|
||||
pick := atomic.LoadInt64(&c2.pick)
|
||||
if start-pick > forcePick && atomic.CompareAndSwapInt64(&c2.pick, pick, start) {
|
||||
return c2
|
||||
} else {
|
||||
atomic.StoreInt64(&c1.pick, start)
|
||||
return c1
|
||||
}
|
||||
}
|
||||
|
||||
type subConn struct {
|
||||
addr resolver.Address
|
||||
conn balancer.SubConn
|
||||
lag uint64
|
||||
inflight int64
|
||||
success uint64
|
||||
last int64
|
||||
pick int64
|
||||
}
|
||||
|
||||
func (c *subConn) healthy() bool {
|
||||
return atomic.LoadUint64(&c.success) > throttleSuccess
|
||||
}
|
||||
|
||||
func (c *subConn) load() int64 {
|
||||
lag := int64(math.Sqrt(float64(atomic.LoadUint64(&c.lag) + 1)))
|
||||
load := lag * atomic.LoadInt64(&c.inflight)
|
||||
if load == 0 {
|
||||
return penalty
|
||||
} else {
|
||||
return load
|
||||
}
|
||||
}
|
@ -0,0 +1,7 @@
|
||||
package p2c
|
||||
|
||||
import "testing"
|
||||
|
||||
func TestP2cPicker_Pick(t *testing.T) {
|
||||
|
||||
}
|
@ -0,0 +1,15 @@
|
||||
package codes
|
||||
|
||||
import (
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
func Acceptable(err error) bool {
|
||||
switch status.Code(err) {
|
||||
case codes.DeadlineExceeded, codes.Internal, codes.Unavailable, codes.DataLoss:
|
||||
return false
|
||||
default:
|
||||
return true
|
||||
}
|
||||
}
|
@ -1,36 +1,35 @@
|
||||
package internal
|
||||
|
||||
import (
|
||||
"zero/core/discov"
|
||||
"zero/rpcx/internal/balancer/roundrobin"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"zero/rpcx/internal/balancer/p2c"
|
||||
"zero/rpcx/internal/resolver"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
)
|
||||
|
||||
func init() {
|
||||
resolver.RegisterResolver()
|
||||
}
|
||||
|
||||
type DiscovClient struct {
|
||||
conn *grpc.ClientConn
|
||||
}
|
||||
|
||||
func NewDiscovClient(etcd discov.EtcdConf, opts ...ClientOption) (*DiscovClient, error) {
|
||||
resolver.RegisterResolver(etcd)
|
||||
opts = append(opts, WithDialOption(grpc.WithBalancerName(roundrobin.Name)))
|
||||
conn, err := dial("discov:///", opts...)
|
||||
func NewDiscovClient(endpoints []string, key string, opts ...ClientOption) (*DiscovClient, error) {
|
||||
opts = append(opts, WithDialOption(grpc.WithBalancerName(p2c.Name)))
|
||||
target := fmt.Sprintf("%s://%s/%s", resolver.DiscovScheme,
|
||||
strings.Join(endpoints, resolver.EndpointSep), key)
|
||||
conn, err := dial(target, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &DiscovClient{
|
||||
conn: conn,
|
||||
}, nil
|
||||
return &DiscovClient{conn: conn}, nil
|
||||
}
|
||||
|
||||
func (c *DiscovClient) Next() (*grpc.ClientConn, bool) {
|
||||
state := c.conn.GetState()
|
||||
if state == connectivity.Ready {
|
||||
return c.conn, true
|
||||
} else {
|
||||
return nil, false
|
||||
}
|
||||
func (c *DiscovClient) Conn() *grpc.ClientConn {
|
||||
return c.conn
|
||||
}
|
||||
|
@ -1,102 +0,0 @@
|
||||
package internal
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"zero/core/discov"
|
||||
"zero/core/logx"
|
||||
"zero/core/threading"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
)
|
||||
|
||||
const (
|
||||
coolOffTime = time.Second * 5
|
||||
retryTimes = 3
|
||||
)
|
||||
|
||||
type (
|
||||
RoundRobinSubClient struct {
|
||||
*discov.RoundRobinSubClient
|
||||
}
|
||||
|
||||
ConsistentSubClient struct {
|
||||
*discov.ConsistentSubClient
|
||||
}
|
||||
)
|
||||
|
||||
func NewRoundRobinRpcClient(endpoints []string, key string, opts ...ClientOption) (*RoundRobinSubClient, error) {
|
||||
subClient, err := discov.NewRoundRobinSubClient(endpoints, key, func(server string) (interface{}, error) {
|
||||
return dial(server, opts...)
|
||||
}, func(server string, conn interface{}) error {
|
||||
return closeConn(conn.(*grpc.ClientConn))
|
||||
}, discov.Exclusive())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
} else {
|
||||
return &RoundRobinSubClient{subClient}, nil
|
||||
}
|
||||
}
|
||||
|
||||
func NewConsistentRpcClient(endpoints []string, key string, opts ...ClientOption) (*ConsistentSubClient, error) {
|
||||
subClient, err := discov.NewConsistentSubClient(endpoints, key, func(server string) (interface{}, error) {
|
||||
return dial(server, opts...)
|
||||
}, func(server string, conn interface{}) error {
|
||||
return closeConn(conn.(*grpc.ClientConn))
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
} else {
|
||||
return &ConsistentSubClient{subClient}, nil
|
||||
}
|
||||
}
|
||||
|
||||
func (cli *RoundRobinSubClient) Next() (*grpc.ClientConn, bool) {
|
||||
return next(func() (interface{}, bool) {
|
||||
return cli.RoundRobinSubClient.Next()
|
||||
})
|
||||
}
|
||||
|
||||
func (cli *ConsistentSubClient) Next(key string) (*grpc.ClientConn, bool) {
|
||||
return next(func() (interface{}, bool) {
|
||||
return cli.ConsistentSubClient.Next(key)
|
||||
})
|
||||
}
|
||||
|
||||
func closeConn(conn *grpc.ClientConn) error {
|
||||
// why to close the conn asynchronously is because maybe another goroutine
|
||||
// is using the same conn, we can wait the coolOffTime to let the other
|
||||
// goroutine to finish using the conn.
|
||||
// after the conn unregistered, the balancer will not assign the conn,
|
||||
// but maybe the already assigned tasks are still using it.
|
||||
threading.GoSafe(func() {
|
||||
time.Sleep(coolOffTime)
|
||||
if err := conn.Close(); err != nil {
|
||||
logx.Error(err)
|
||||
}
|
||||
})
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func next(nextFn func() (interface{}, bool)) (*grpc.ClientConn, bool) {
|
||||
for i := 0; i < retryTimes; i++ {
|
||||
v, ok := nextFn()
|
||||
if !ok {
|
||||
break
|
||||
}
|
||||
|
||||
conn, yes := v.(*grpc.ClientConn)
|
||||
if !yes {
|
||||
break
|
||||
}
|
||||
|
||||
switch conn.GetState() {
|
||||
case connectivity.Ready:
|
||||
return conn, true
|
||||
}
|
||||
}
|
||||
|
||||
return nil, false
|
||||
}
|
@ -1,40 +0,0 @@
|
||||
package internal
|
||||
|
||||
import (
|
||||
"math/rand"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
type RRClient struct {
|
||||
conns []*grpc.ClientConn
|
||||
index int
|
||||
lock sync.Mutex
|
||||
}
|
||||
|
||||
func NewRRClient(endpoints []string) (*RRClient, error) {
|
||||
var conns []*grpc.ClientConn
|
||||
for _, endpoint := range endpoints {
|
||||
conn, err := dial(endpoint)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
conns = append(conns, conn)
|
||||
}
|
||||
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
return &RRClient{
|
||||
conns: conns,
|
||||
index: rand.Intn(len(conns)),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (c *RRClient) Next() *grpc.ClientConn {
|
||||
c.lock.Lock()
|
||||
defer c.lock.Unlock()
|
||||
c.index = (c.index + 1) % len(c.conns)
|
||||
return c.conns[c.index]
|
||||
}
|
@ -1,17 +0,0 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"zero/core/discov"
|
||||
"zero/core/lang"
|
||||
"zero/rpcx/internal"
|
||||
)
|
||||
|
||||
func main() {
|
||||
cli, err := internal.NewDiscovClient(discov.EtcdConf{
|
||||
Hosts: []string{"localhost:2379"},
|
||||
Key: "rpcx",
|
||||
})
|
||||
lang.Must(err)
|
||||
|
||||
cli.Next()
|
||||
}
|
Loading…
Reference in New Issue