Merge pull request #9860 from gyuho/new-balancer-april-2018
clientv3: Upgrade to round robin balancer based on gRPC 1.12 balancer API
@ -9,7 +9,7 @@ See [code changes](https://github.com/coreos/etcd/compare/v3.3.0...v3.4.0) and [
|
||||
|
||||
### Improved
|
||||
|
||||
- Rewrite [client balancer](TODO) with [new gRPC balancer interface](TODO).
|
||||
- Rewrite [client balancer](https://github.com/coreos/etcd/pull/9860) with [new gRPC balancer interface](https://github.com/coreos/etcd/issues/9106).
|
||||
- Add [backoff on watch retries on transient errors](https://github.com/coreos/etcd/pull/9840).
|
||||
- Add [jitter to watch progress notify](https://github.com/coreos/etcd/pull/9278) to prevent [spikes in `etcd_network_client_grpc_sent_bytes_total`](https://github.com/coreos/etcd/issues/9246).
|
||||
- Improve [slow request apply warning log](https://github.com/coreos/etcd/pull/9288).
|
||||
|
19
Gopkg.lock
generated
@ -100,6 +100,12 @@
|
||||
packages = ["."]
|
||||
revision = "4201258b820c74ac8e6922fc9e6b52f71fe46f8d"
|
||||
|
||||
[[projects]]
|
||||
name = "github.com/grpc-ecosystem/go-grpc-middleware"
|
||||
packages = ["util/backoffutils"]
|
||||
revision = "c250d6563d4d4c20252cd865923440e829844f4e"
|
||||
version = "v1.0.0"
|
||||
|
||||
[[projects]]
|
||||
name = "github.com/grpc-ecosystem/go-grpc-prometheus"
|
||||
packages = ["."]
|
||||
@ -335,9 +341,14 @@
|
||||
packages = [
|
||||
".",
|
||||
"balancer",
|
||||
"balancer/base",
|
||||
"balancer/roundrobin",
|
||||
"channelz",
|
||||
"codes",
|
||||
"connectivity",
|
||||
"credentials",
|
||||
"encoding",
|
||||
"encoding/proto",
|
||||
"grpclb/grpc_lb_v1/messages",
|
||||
"grpclog",
|
||||
"health",
|
||||
@ -348,13 +359,15 @@
|
||||
"naming",
|
||||
"peer",
|
||||
"resolver",
|
||||
"resolver/dns",
|
||||
"resolver/passthrough",
|
||||
"stats",
|
||||
"status",
|
||||
"tap",
|
||||
"transport"
|
||||
]
|
||||
revision = "5b3c4e850e90a4cf6a20ebd46c8b32a0a3afcb9e"
|
||||
version = "v1.7.5"
|
||||
revision = "7a6a684ca69eb4cae85ad0a484f2e531598c047b"
|
||||
version = "v1.12.2"
|
||||
|
||||
[[projects]]
|
||||
name = "gopkg.in/cheggaaa/pb.v1"
|
||||
@ -371,6 +384,6 @@
|
||||
[solve-meta]
|
||||
analyzer-name = "dep"
|
||||
analyzer-version = 1
|
||||
inputs-digest = "943bf7648c0129f59546321f569622e933f24a103b5d68525b82d5e47d52733f"
|
||||
inputs-digest = "40c72e54abdb7d6c044f2f5fabb8fc7ac2eadcab047cc22b28d26b7b145b5e1b"
|
||||
solver-name = "gps-cdcl"
|
||||
solver-version = 1
|
||||
|
@ -134,6 +134,15 @@
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"project": "github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils",
|
||||
"licenses": [
|
||||
{
|
||||
"type": "Apache License 2.0",
|
||||
"confidence": 1
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"project": "github.com/grpc-ecosystem/go-grpc-prometheus",
|
||||
"licenses": [
|
||||
|
@ -21,7 +21,6 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/auth/authpb"
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
@ -216,8 +215,8 @@ func (auth *authenticator) close() {
|
||||
auth.conn.Close()
|
||||
}
|
||||
|
||||
func newAuthenticator(endpoint string, opts []grpc.DialOption, c *Client) (*authenticator, error) {
|
||||
conn, err := grpc.Dial(endpoint, opts...)
|
||||
func newAuthenticator(ctx context.Context, target string, opts []grpc.DialOption, c *Client) (*authenticator, error) {
|
||||
conn, err := grpc.DialContext(ctx, target, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
275
clientv3/balancer/balancer.go
Normal file
@ -0,0 +1,275 @@
|
||||
// Copyright 2018 The etcd 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 balancer
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/clientv3/balancer/picker"
|
||||
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
"google.golang.org/grpc/resolver"
|
||||
_ "google.golang.org/grpc/resolver/dns" // register DNS resolver
|
||||
_ "google.golang.org/grpc/resolver/passthrough" // register passthrough resolver
|
||||
)
|
||||
|
||||
// RegisterBuilder creates and registers a builder. Since this function calls balancer.Register, it
|
||||
// must be invoked at initialization time.
|
||||
func RegisterBuilder(cfg Config) {
|
||||
bb := &builder{cfg}
|
||||
balancer.Register(bb)
|
||||
|
||||
bb.cfg.Logger.Info(
|
||||
"registered balancer",
|
||||
zap.String("policy", bb.cfg.Policy.String()),
|
||||
zap.String("name", bb.cfg.Name),
|
||||
)
|
||||
}
|
||||
|
||||
type builder struct {
|
||||
cfg Config
|
||||
}
|
||||
|
||||
// Build is called initially when creating "ccBalancerWrapper".
|
||||
// "grpc.Dial" is called to this client connection.
|
||||
// Then, resolved addresses will be handled via "HandleResolvedAddrs".
|
||||
func (b *builder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer {
|
||||
bb := &baseBalancer{
|
||||
id: strconv.FormatInt(time.Now().UnixNano(), 36),
|
||||
policy: b.cfg.Policy,
|
||||
name: b.cfg.Policy.String(),
|
||||
lg: b.cfg.Logger,
|
||||
|
||||
addrToSc: make(map[resolver.Address]balancer.SubConn),
|
||||
scToAddr: make(map[balancer.SubConn]resolver.Address),
|
||||
scToSt: make(map[balancer.SubConn]connectivity.State),
|
||||
|
||||
currentConn: nil,
|
||||
csEvltr: &connectivityStateEvaluator{},
|
||||
|
||||
// initialize picker always returns "ErrNoSubConnAvailable"
|
||||
Picker: picker.NewErr(balancer.ErrNoSubConnAvailable),
|
||||
}
|
||||
if b.cfg.Name != "" {
|
||||
bb.name = b.cfg.Name
|
||||
}
|
||||
if bb.lg == nil {
|
||||
bb.lg = zap.NewNop()
|
||||
}
|
||||
|
||||
// TODO: support multiple connections
|
||||
bb.mu.Lock()
|
||||
bb.currentConn = cc
|
||||
bb.mu.Unlock()
|
||||
|
||||
bb.lg.Info(
|
||||
"built balancer",
|
||||
zap.String("balancer-id", bb.id),
|
||||
zap.String("policy", bb.policy.String()),
|
||||
zap.String("resolver-target", cc.Target()),
|
||||
)
|
||||
return bb
|
||||
}
|
||||
|
||||
// Name implements "grpc/balancer.Builder" interface.
|
||||
func (b *builder) Name() string { return b.cfg.Name }
|
||||
|
||||
// Balancer defines client balancer interface.
|
||||
type Balancer interface {
|
||||
// Balancer is called on specified client connection. Client initiates gRPC
|
||||
// connection with "grpc.Dial(addr, grpc.WithBalancerName)", and then those resolved
|
||||
// addresses are passed to "grpc/balancer.Balancer.HandleResolvedAddrs".
|
||||
// For each resolved address, balancer calls "balancer.ClientConn.NewSubConn".
|
||||
// "grpc/balancer.Balancer.HandleSubConnStateChange" is called when connectivity state
|
||||
// changes, thus requires failover logic in this method.
|
||||
balancer.Balancer
|
||||
|
||||
// Picker calls "Pick" for every client request.
|
||||
picker.Picker
|
||||
}
|
||||
|
||||
type baseBalancer struct {
|
||||
id string
|
||||
policy picker.Policy
|
||||
name string
|
||||
lg *zap.Logger
|
||||
|
||||
mu sync.RWMutex
|
||||
|
||||
addrToSc map[resolver.Address]balancer.SubConn
|
||||
scToAddr map[balancer.SubConn]resolver.Address
|
||||
scToSt map[balancer.SubConn]connectivity.State
|
||||
|
||||
currentConn balancer.ClientConn
|
||||
currentState connectivity.State
|
||||
csEvltr *connectivityStateEvaluator
|
||||
|
||||
picker.Picker
|
||||
}
|
||||
|
||||
// HandleResolvedAddrs implements "grpc/balancer.Balancer" interface.
|
||||
// gRPC sends initial or updated resolved addresses from "Build".
|
||||
func (bb *baseBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) {
|
||||
if err != nil {
|
||||
bb.lg.Warn("HandleResolvedAddrs called with error", zap.String("balancer-id", bb.id), zap.Error(err))
|
||||
return
|
||||
}
|
||||
bb.lg.Info("resolved", zap.String("balancer-id", bb.id), zap.Strings("addresses", addrsToStrings(addrs)))
|
||||
|
||||
bb.mu.Lock()
|
||||
defer bb.mu.Unlock()
|
||||
|
||||
resolved := make(map[resolver.Address]struct{})
|
||||
for _, addr := range addrs {
|
||||
resolved[addr] = struct{}{}
|
||||
if _, ok := bb.addrToSc[addr]; !ok {
|
||||
sc, err := bb.currentConn.NewSubConn([]resolver.Address{addr}, balancer.NewSubConnOptions{})
|
||||
if err != nil {
|
||||
bb.lg.Warn("NewSubConn failed", zap.String("balancer-id", bb.id), zap.Error(err), zap.String("address", addr.Addr))
|
||||
continue
|
||||
}
|
||||
bb.addrToSc[addr] = sc
|
||||
bb.scToAddr[sc] = addr
|
||||
bb.scToSt[sc] = connectivity.Idle
|
||||
sc.Connect()
|
||||
}
|
||||
}
|
||||
|
||||
for addr, sc := range bb.addrToSc {
|
||||
if _, ok := resolved[addr]; !ok {
|
||||
// was removed by resolver or failed to create subconn
|
||||
bb.currentConn.RemoveSubConn(sc)
|
||||
delete(bb.addrToSc, addr)
|
||||
|
||||
bb.lg.Info(
|
||||
"removed subconn",
|
||||
zap.String("balancer-id", bb.id),
|
||||
zap.String("address", addr.Addr),
|
||||
zap.String("subconn", scToString(sc)),
|
||||
)
|
||||
|
||||
// Keep the state of this sc in bb.scToSt until sc's state becomes Shutdown.
|
||||
// The entry will be deleted in HandleSubConnStateChange.
|
||||
// (DO NOT) delete(bb.scToAddr, sc)
|
||||
// (DO NOT) delete(bb.scToSt, sc)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// HandleSubConnStateChange implements "grpc/balancer.Balancer" interface.
|
||||
func (bb *baseBalancer) HandleSubConnStateChange(sc balancer.SubConn, s connectivity.State) {
|
||||
bb.mu.Lock()
|
||||
defer bb.mu.Unlock()
|
||||
|
||||
old, ok := bb.scToSt[sc]
|
||||
if !ok {
|
||||
bb.lg.Warn(
|
||||
"state change for an unknown subconn",
|
||||
zap.String("balancer-id", bb.id),
|
||||
zap.String("subconn", scToString(sc)),
|
||||
zap.String("state", s.String()),
|
||||
)
|
||||
return
|
||||
}
|
||||
|
||||
bb.lg.Info(
|
||||
"state changed",
|
||||
zap.String("balancer-id", bb.id),
|
||||
zap.Bool("connected", s == connectivity.Ready),
|
||||
zap.String("subconn", scToString(sc)),
|
||||
zap.String("address", bb.scToAddr[sc].Addr),
|
||||
zap.String("old-state", old.String()),
|
||||
zap.String("new-state", s.String()),
|
||||
)
|
||||
|
||||
bb.scToSt[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 scToSt. Remove state for this sc here.
|
||||
delete(bb.scToAddr, sc)
|
||||
delete(bb.scToSt, sc)
|
||||
}
|
||||
|
||||
oldAggrState := bb.currentState
|
||||
bb.currentState = bb.csEvltr.recordTransition(old, 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) != (old == connectivity.Ready) ||
|
||||
(bb.currentState == connectivity.TransientFailure) != (oldAggrState == connectivity.TransientFailure) {
|
||||
bb.regeneratePicker()
|
||||
}
|
||||
|
||||
bb.currentConn.UpdateBalancerState(bb.currentState, bb.Picker)
|
||||
return
|
||||
}
|
||||
|
||||
func (bb *baseBalancer) regeneratePicker() {
|
||||
if bb.currentState == connectivity.TransientFailure {
|
||||
bb.lg.Info(
|
||||
"generated transient error picker",
|
||||
zap.String("balancer-id", bb.id),
|
||||
zap.String("policy", bb.policy.String()),
|
||||
)
|
||||
bb.Picker = picker.NewErr(balancer.ErrTransientFailure)
|
||||
return
|
||||
}
|
||||
|
||||
// only pass ready subconns to picker
|
||||
scs := make([]balancer.SubConn, 0)
|
||||
addrToSc := make(map[resolver.Address]balancer.SubConn)
|
||||
scToAddr := make(map[balancer.SubConn]resolver.Address)
|
||||
for addr, sc := range bb.addrToSc {
|
||||
if st, ok := bb.scToSt[sc]; ok && st == connectivity.Ready {
|
||||
scs = append(scs, sc)
|
||||
addrToSc[addr] = sc
|
||||
scToAddr[sc] = addr
|
||||
}
|
||||
}
|
||||
|
||||
switch bb.policy {
|
||||
case picker.RoundrobinBalanced:
|
||||
bb.Picker = picker.NewRoundrobinBalanced(bb.lg, scs, addrToSc, scToAddr)
|
||||
|
||||
default:
|
||||
panic(fmt.Errorf("invalid balancer picker policy (%d)", bb.policy))
|
||||
}
|
||||
|
||||
bb.lg.Info(
|
||||
"generated picker",
|
||||
zap.String("balancer-id", bb.id),
|
||||
zap.String("policy", bb.policy.String()),
|
||||
zap.Strings("subconn-ready", scsToStrings(addrToSc)),
|
||||
zap.Int("subconn-size", len(addrToSc)),
|
||||
)
|
||||
}
|
||||
|
||||
// Close implements "grpc/balancer.Balancer" interface.
|
||||
// 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 (bb *baseBalancer) Close() {
|
||||
// TODO
|
||||
}
|
308
clientv3/balancer/balancer_test.go
Normal file
@ -0,0 +1,308 @@
|
||||
// Copyright 2018 The etcd 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 balancer
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/clientv3/balancer/picker"
|
||||
"github.com/coreos/etcd/clientv3/balancer/resolver/endpoint"
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
"github.com/coreos/etcd/pkg/mock/mockserver"
|
||||
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/peer"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
// TestRoundRobinBalancedResolvableNoFailover ensures that
|
||||
// requests to a resolvable endpoint can be balanced between
|
||||
// multiple, if any, nodes. And there needs be no failover.
|
||||
func TestRoundRobinBalancedResolvableNoFailover(t *testing.T) {
|
||||
testCases := []struct {
|
||||
name string
|
||||
serverCount int
|
||||
reqN int
|
||||
network string
|
||||
}{
|
||||
{name: "rrBalanced_1", serverCount: 1, reqN: 5, network: "tcp"},
|
||||
{name: "rrBalanced_1_unix_sockets", serverCount: 1, reqN: 5, network: "unix"},
|
||||
{name: "rrBalanced_3", serverCount: 3, reqN: 7, network: "tcp"},
|
||||
{name: "rrBalanced_5", serverCount: 5, reqN: 10, network: "tcp"},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
ms, err := mockserver.StartMockServersOnNetwork(tc.serverCount, tc.network)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to start mock servers: %v", err)
|
||||
}
|
||||
defer ms.Stop()
|
||||
|
||||
var eps []string
|
||||
for _, svr := range ms.Servers {
|
||||
eps = append(eps, svr.ResolverAddress().Addr)
|
||||
}
|
||||
|
||||
rsv, err := endpoint.NewResolverGroup("nofailover")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer rsv.Close()
|
||||
rsv.SetEndpoints(eps)
|
||||
|
||||
name := genName()
|
||||
cfg := Config{
|
||||
Policy: picker.RoundrobinBalanced,
|
||||
Name: name,
|
||||
Logger: zap.NewExample(),
|
||||
}
|
||||
RegisterBuilder(cfg)
|
||||
conn, err := grpc.Dial(fmt.Sprintf("endpoint://nofailover/*"), grpc.WithInsecure(), grpc.WithBalancerName(name))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to dial mock server: %v", err)
|
||||
}
|
||||
defer conn.Close()
|
||||
cli := pb.NewKVClient(conn)
|
||||
|
||||
reqFunc := func(ctx context.Context) (picked string, err error) {
|
||||
var p peer.Peer
|
||||
_, err = cli.Range(ctx, &pb.RangeRequest{Key: []byte("/x")}, grpc.Peer(&p))
|
||||
if p.Addr != nil {
|
||||
picked = p.Addr.String()
|
||||
}
|
||||
return picked, err
|
||||
}
|
||||
|
||||
prev, switches := "", 0
|
||||
for i := 0; i < tc.reqN; i++ {
|
||||
picked, err := reqFunc(context.Background())
|
||||
if err != nil {
|
||||
t.Fatalf("#%d: unexpected failure %v", i, err)
|
||||
}
|
||||
if prev == "" {
|
||||
prev = picked
|
||||
continue
|
||||
}
|
||||
if prev != picked {
|
||||
switches++
|
||||
}
|
||||
prev = picked
|
||||
}
|
||||
if tc.serverCount > 1 && switches < tc.reqN-3 { // -3 for initial resolutions
|
||||
t.Fatalf("expected balanced loads for %d requests, got switches %d", tc.reqN, switches)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// TestRoundRobinBalancedResolvableFailoverFromServerFail ensures that
|
||||
// loads be rebalanced while one server goes down and comes back.
|
||||
func TestRoundRobinBalancedResolvableFailoverFromServerFail(t *testing.T) {
|
||||
serverCount := 5
|
||||
ms, err := mockserver.StartMockServers(serverCount)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to start mock servers: %s", err)
|
||||
}
|
||||
defer ms.Stop()
|
||||
var eps []string
|
||||
for _, svr := range ms.Servers {
|
||||
eps = append(eps, svr.ResolverAddress().Addr)
|
||||
}
|
||||
|
||||
rsv, err := endpoint.NewResolverGroup("serverfail")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer rsv.Close()
|
||||
rsv.SetEndpoints(eps)
|
||||
|
||||
name := genName()
|
||||
cfg := Config{
|
||||
Policy: picker.RoundrobinBalanced,
|
||||
Name: name,
|
||||
Logger: zap.NewExample(),
|
||||
}
|
||||
RegisterBuilder(cfg)
|
||||
conn, err := grpc.Dial(fmt.Sprintf("endpoint://serverfail/mock.server"), grpc.WithInsecure(), grpc.WithBalancerName(name))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to dial mock server: %s", err)
|
||||
}
|
||||
defer conn.Close()
|
||||
cli := pb.NewKVClient(conn)
|
||||
|
||||
reqFunc := func(ctx context.Context) (picked string, err error) {
|
||||
var p peer.Peer
|
||||
_, err = cli.Range(ctx, &pb.RangeRequest{Key: []byte("/x")}, grpc.Peer(&p))
|
||||
if p.Addr != nil {
|
||||
picked = p.Addr.String()
|
||||
}
|
||||
return picked, err
|
||||
}
|
||||
|
||||
// stop first server, loads should be redistributed
|
||||
// stopped server should never be picked
|
||||
ms.StopAt(0)
|
||||
available := make(map[string]struct{})
|
||||
for i := 1; i < serverCount; i++ {
|
||||
available[eps[i]] = struct{}{}
|
||||
}
|
||||
|
||||
reqN := 10
|
||||
prev, switches := "", 0
|
||||
for i := 0; i < reqN; i++ {
|
||||
picked, err := reqFunc(context.Background())
|
||||
if err != nil && strings.Contains(err.Error(), "transport is closing") {
|
||||
continue
|
||||
}
|
||||
if prev == "" { // first failover
|
||||
if eps[0] == picked {
|
||||
t.Fatalf("expected failover from %q, picked %q", eps[0], picked)
|
||||
}
|
||||
prev = picked
|
||||
continue
|
||||
}
|
||||
if _, ok := available[picked]; !ok {
|
||||
t.Fatalf("picked unavailable address %q (available %v)", picked, available)
|
||||
}
|
||||
if prev != picked {
|
||||
switches++
|
||||
}
|
||||
prev = picked
|
||||
}
|
||||
if switches < reqN-3 { // -3 for initial resolutions + failover
|
||||
t.Fatalf("expected balanced loads for %d requests, got switches %d", reqN, switches)
|
||||
}
|
||||
|
||||
// now failed server comes back
|
||||
ms.StartAt(0)
|
||||
|
||||
// enough time for reconnecting to recovered server
|
||||
time.Sleep(time.Second)
|
||||
|
||||
prev, switches = "", 0
|
||||
recoveredAddr, recovered := eps[0], 0
|
||||
available[recoveredAddr] = struct{}{}
|
||||
|
||||
for i := 0; i < 2*reqN; i++ {
|
||||
picked, err := reqFunc(context.Background())
|
||||
if err != nil {
|
||||
t.Fatalf("#%d: unexpected failure %v", i, err)
|
||||
}
|
||||
if prev == "" {
|
||||
prev = picked
|
||||
continue
|
||||
}
|
||||
if _, ok := available[picked]; !ok {
|
||||
t.Fatalf("#%d: picked unavailable address %q (available %v)", i, picked, available)
|
||||
}
|
||||
if prev != picked {
|
||||
switches++
|
||||
}
|
||||
if picked == recoveredAddr {
|
||||
recovered++
|
||||
}
|
||||
prev = picked
|
||||
}
|
||||
if switches < reqN-3 { // -3 for initial resolutions
|
||||
t.Fatalf("expected balanced loads for %d requests, got switches %d", reqN, switches)
|
||||
}
|
||||
if recovered < reqN/serverCount {
|
||||
t.Fatalf("recovered server %q got only %d requests", recoveredAddr, recovered)
|
||||
}
|
||||
}
|
||||
|
||||
// TestRoundRobinBalancedResolvableFailoverFromRequestFail ensures that
|
||||
// loads be rebalanced while some requests are failed.
|
||||
func TestRoundRobinBalancedResolvableFailoverFromRequestFail(t *testing.T) {
|
||||
serverCount := 5
|
||||
ms, err := mockserver.StartMockServers(serverCount)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to start mock servers: %s", err)
|
||||
}
|
||||
defer ms.Stop()
|
||||
var eps []string
|
||||
available := make(map[string]struct{})
|
||||
for _, svr := range ms.Servers {
|
||||
eps = append(eps, svr.ResolverAddress().Addr)
|
||||
available[svr.Address] = struct{}{}
|
||||
}
|
||||
rsv, err := endpoint.NewResolverGroup("requestfail")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer rsv.Close()
|
||||
rsv.SetEndpoints(eps)
|
||||
|
||||
name := genName()
|
||||
cfg := Config{
|
||||
Policy: picker.RoundrobinBalanced,
|
||||
Name: name,
|
||||
Logger: zap.NewExample(),
|
||||
}
|
||||
RegisterBuilder(cfg)
|
||||
conn, err := grpc.Dial(fmt.Sprintf("endpoint://requestfail/mock.server"), grpc.WithInsecure(), grpc.WithBalancerName(name))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to dial mock server: %s", err)
|
||||
}
|
||||
defer conn.Close()
|
||||
cli := pb.NewKVClient(conn)
|
||||
|
||||
reqFunc := func(ctx context.Context) (picked string, err error) {
|
||||
var p peer.Peer
|
||||
_, err = cli.Range(ctx, &pb.RangeRequest{Key: []byte("/x")}, grpc.Peer(&p))
|
||||
if p.Addr != nil {
|
||||
picked = p.Addr.String()
|
||||
}
|
||||
return picked, err
|
||||
}
|
||||
|
||||
reqN := 20
|
||||
prev, switches := "", 0
|
||||
for i := 0; i < reqN; i++ {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
if i%2 == 0 {
|
||||
cancel()
|
||||
}
|
||||
picked, err := reqFunc(ctx)
|
||||
if i%2 == 0 {
|
||||
if s, ok := status.FromError(err); ok && s.Code() != codes.Canceled || picked != "" {
|
||||
t.Fatalf("#%d: expected %v, got %v", i, context.Canceled, err)
|
||||
}
|
||||
continue
|
||||
}
|
||||
if prev == "" && picked != "" {
|
||||
prev = picked
|
||||
continue
|
||||
}
|
||||
if _, ok := available[picked]; !ok {
|
||||
t.Fatalf("#%d: picked unavailable address %q (available %v)", i, picked, available)
|
||||
}
|
||||
if prev != picked {
|
||||
switches++
|
||||
}
|
||||
prev = picked
|
||||
}
|
||||
if switches < reqN/2-3 { // -3 for initial resolutions + failover
|
||||
t.Fatalf("expected balanced loads for %d requests, got switches %d", reqN, switches)
|
||||
}
|
||||
}
|
36
clientv3/balancer/config.go
Normal file
@ -0,0 +1,36 @@
|
||||
// Copyright 2018 The etcd 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 balancer
|
||||
|
||||
import (
|
||||
"github.com/coreos/etcd/clientv3/balancer/picker"
|
||||
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// Config defines balancer configurations.
|
||||
type Config struct {
|
||||
// Policy configures balancer policy.
|
||||
Policy picker.Policy
|
||||
|
||||
// Name defines an additional name for balancer.
|
||||
// Useful for balancer testing to avoid register conflicts.
|
||||
// If empty, defaults to policy name.
|
||||
Name string
|
||||
|
||||
// Logger configures balancer logging.
|
||||
// If nil, logs are discarded.
|
||||
Logger *zap.Logger
|
||||
}
|
58
clientv3/balancer/connectivity.go
Normal file
@ -0,0 +1,58 @@
|
||||
// Copyright 2018 The etcd 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 balancer
|
||||
|
||||
import "google.golang.org/grpc/connectivity"
|
||||
|
||||
// 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
|
||||
}
|
@ -30,7 +30,7 @@ import (
|
||||
|
||||
var endpoints = []string{"localhost:2379", "localhost:22379", "localhost:32379"}
|
||||
|
||||
func TestBalancerGetUnblocking(t *testing.T) {
|
||||
func TestOldHealthBalancerGetUnblocking(t *testing.T) {
|
||||
hb := NewGRPC17Health(endpoints, minHealthRetryDuration, func(ep string, dopts ...grpc.DialOption) (*grpc.ClientConn, error) { return nil, nil })
|
||||
defer hb.Close()
|
||||
if addrs := <-hb.Notify(); len(addrs) != len(endpoints) {
|
||||
@ -74,7 +74,7 @@ func TestBalancerGetUnblocking(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestBalancerGetBlocking(t *testing.T) {
|
||||
func TestOldHealthBalancerGetBlocking(t *testing.T) {
|
||||
hb := NewGRPC17Health(endpoints, minHealthRetryDuration, func(ep string, dopts ...grpc.DialOption) (*grpc.ClientConn, error) { return nil, nil })
|
||||
defer hb.Close()
|
||||
if addrs := <-hb.Notify(); len(addrs) != len(endpoints) {
|
||||
@ -131,9 +131,9 @@ func TestBalancerGetBlocking(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// TestHealthBalancerGraylist checks one endpoint is tried after the other
|
||||
// TestOldHealthBalancerGraylist checks one endpoint is tried after the other
|
||||
// due to gray listing.
|
||||
func TestHealthBalancerGraylist(t *testing.T) {
|
||||
func TestOldHealthBalancerGraylist(t *testing.T) {
|
||||
var wg sync.WaitGroup
|
||||
// Use 3 endpoints so gray list doesn't fallback to all connections
|
||||
// after failing on 2 endpoints.
|
||||
@ -192,7 +192,7 @@ func TestHealthBalancerGraylist(t *testing.T) {
|
||||
// TestBalancerDoNotBlockOnClose ensures that balancer and grpc don't deadlock each other
|
||||
// due to rapid open/close conn. The deadlock causes balancer.Close() to block forever.
|
||||
// See issue: https://github.com/coreos/etcd/issues/7283 for more detail.
|
||||
func TestBalancerDoNotBlockOnClose(t *testing.T) {
|
||||
func TestOldHealthBalancerDoNotBlockOnClose(t *testing.T) {
|
||||
defer testutil.AfterTest(t)
|
||||
|
||||
kcl := newKillConnListener(t, 3)
|
||||
|
16
clientv3/balancer/picker/doc.go
Normal file
@ -0,0 +1,16 @@
|
||||
// Copyright 2018 The etcd 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 picker defines/implements client balancer picker policy.
|
||||
package picker
|
34
clientv3/balancer/picker/err.go
Normal file
@ -0,0 +1,34 @@
|
||||
// Copyright 2018 The etcd 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 picker
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"google.golang.org/grpc/balancer"
|
||||
)
|
||||
|
||||
// NewErr returns a picker that always returns err on "Pick".
|
||||
func NewErr(err error) Picker {
|
||||
return &errPicker{err: err}
|
||||
}
|
||||
|
||||
type errPicker struct {
|
||||
err error
|
||||
}
|
||||
|
||||
func (p *errPicker) Pick(context.Context, balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
||||
return nil, nil, p.err
|
||||
}
|
24
clientv3/balancer/picker/picker.go
Normal file
@ -0,0 +1,24 @@
|
||||
// Copyright 2018 The etcd 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 picker
|
||||
|
||||
import (
|
||||
"google.golang.org/grpc/balancer"
|
||||
)
|
||||
|
||||
// Picker defines balancer Picker methods.
|
||||
type Picker interface {
|
||||
balancer.Picker
|
||||
}
|
49
clientv3/balancer/picker/picker_policy.go
Normal file
@ -0,0 +1,49 @@
|
||||
// Copyright 2018 The etcd 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 picker
|
||||
|
||||
import "fmt"
|
||||
|
||||
// Policy defines balancer picker policy.
|
||||
type Policy uint8
|
||||
|
||||
const (
|
||||
// TODO: custom picker is not supported yet.
|
||||
// custom defines custom balancer picker.
|
||||
custom Policy = iota
|
||||
|
||||
// RoundrobinBalanced balance loads over multiple endpoints
|
||||
// and implements failover in roundrobin fashion.
|
||||
RoundrobinBalanced Policy = iota
|
||||
|
||||
// TODO: only send loads to pinned address "RoundrobinFailover"
|
||||
// just like how 3.3 client works
|
||||
//
|
||||
// TODO: priotize leader
|
||||
// TODO: health-check
|
||||
// TODO: weighted roundrobin
|
||||
// TODO: power of two random choice
|
||||
)
|
||||
|
||||
func (p Policy) String() string {
|
||||
switch p {
|
||||
case custom:
|
||||
panic("'custom' picker policy is not supported yet")
|
||||
case RoundrobinBalanced:
|
||||
return "etcd-client-roundrobin-balanced"
|
||||
default:
|
||||
panic(fmt.Errorf("invalid balancer picker policy (%d)", p))
|
||||
}
|
||||
}
|
92
clientv3/balancer/picker/roundrobin_balanced.go
Normal file
@ -0,0 +1,92 @@
|
||||
// Copyright 2018 The etcd 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 picker
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
|
||||
"go.uber.org/zap"
|
||||
"go.uber.org/zap/zapcore"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
// NewRoundrobinBalanced returns a new roundrobin balanced picker.
|
||||
func NewRoundrobinBalanced(
|
||||
lg *zap.Logger,
|
||||
scs []balancer.SubConn,
|
||||
addrToSc map[resolver.Address]balancer.SubConn,
|
||||
scToAddr map[balancer.SubConn]resolver.Address,
|
||||
) Picker {
|
||||
return &rrBalanced{
|
||||
lg: lg,
|
||||
scs: scs,
|
||||
addrToSc: addrToSc,
|
||||
scToAddr: scToAddr,
|
||||
}
|
||||
}
|
||||
|
||||
type rrBalanced struct {
|
||||
lg *zap.Logger
|
||||
|
||||
mu sync.RWMutex
|
||||
next int
|
||||
scs []balancer.SubConn
|
||||
|
||||
addrToSc map[resolver.Address]balancer.SubConn
|
||||
scToAddr map[balancer.SubConn]resolver.Address
|
||||
}
|
||||
|
||||
// Pick is called for every client request.
|
||||
func (rb *rrBalanced) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
||||
rb.mu.RLock()
|
||||
n := len(rb.scs)
|
||||
rb.mu.RUnlock()
|
||||
if n == 0 {
|
||||
return nil, nil, balancer.ErrNoSubConnAvailable
|
||||
}
|
||||
|
||||
rb.mu.Lock()
|
||||
cur := rb.next
|
||||
sc := rb.scs[cur]
|
||||
picked := rb.scToAddr[sc].Addr
|
||||
rb.next = (rb.next + 1) % len(rb.scs)
|
||||
rb.mu.Unlock()
|
||||
|
||||
rb.lg.Debug(
|
||||
"picked",
|
||||
zap.String("address", picked),
|
||||
zap.Int("subconn-index", cur),
|
||||
zap.Int("subconn-size", n),
|
||||
)
|
||||
|
||||
doneFunc := func(info balancer.DoneInfo) {
|
||||
// TODO: error handling?
|
||||
fss := []zapcore.Field{
|
||||
zap.Error(info.Err),
|
||||
zap.String("address", picked),
|
||||
zap.Bool("success", info.Err == nil),
|
||||
zap.Bool("bytes-sent", info.BytesSent),
|
||||
zap.Bool("bytes-received", info.BytesReceived),
|
||||
}
|
||||
if info.Err == nil {
|
||||
rb.lg.Debug("balancer done", fss...)
|
||||
} else {
|
||||
rb.lg.Warn("balancer failed", fss...)
|
||||
}
|
||||
}
|
||||
return sc, doneFunc, nil
|
||||
}
|
229
clientv3/balancer/resolver/endpoint/endpoint.go
Normal file
@ -0,0 +1,229 @@
|
||||
// Copyright 2018 The etcd 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 endpoint resolves etcd entpoints using grpc targets of the form 'endpoint://<id>/<endpoint>'.
|
||||
package endpoint
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/url"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
const scheme = "endpoint"
|
||||
|
||||
var (
|
||||
targetPrefix = fmt.Sprintf("%s://", scheme)
|
||||
|
||||
bldr *builder
|
||||
)
|
||||
|
||||
func init() {
|
||||
bldr = &builder{
|
||||
resolverGroups: make(map[string]*ResolverGroup),
|
||||
}
|
||||
resolver.Register(bldr)
|
||||
}
|
||||
|
||||
type builder struct {
|
||||
mu sync.RWMutex
|
||||
resolverGroups map[string]*ResolverGroup
|
||||
}
|
||||
|
||||
// NewResolverGroup creates a new ResolverGroup with the given id.
|
||||
func NewResolverGroup(id string) (*ResolverGroup, error) {
|
||||
return bldr.newResolverGroup(id)
|
||||
}
|
||||
|
||||
// ResolverGroup keeps all endpoints of resolvers using a common endpoint://<id>/ target
|
||||
// up-to-date.
|
||||
type ResolverGroup struct {
|
||||
mu sync.RWMutex
|
||||
id string
|
||||
endpoints []string
|
||||
resolvers []*Resolver
|
||||
}
|
||||
|
||||
func (e *ResolverGroup) addResolver(r *Resolver) {
|
||||
e.mu.Lock()
|
||||
addrs := epsToAddrs(e.endpoints...)
|
||||
e.resolvers = append(e.resolvers, r)
|
||||
e.mu.Unlock()
|
||||
r.cc.NewAddress(addrs)
|
||||
}
|
||||
|
||||
func (e *ResolverGroup) removeResolver(r *Resolver) {
|
||||
e.mu.Lock()
|
||||
for i, er := range e.resolvers {
|
||||
if er == r {
|
||||
e.resolvers = append(e.resolvers[:i], e.resolvers[i+1:]...)
|
||||
break
|
||||
}
|
||||
}
|
||||
e.mu.Unlock()
|
||||
}
|
||||
|
||||
// SetEndpoints updates the endpoints for ResolverGroup. All registered resolver are updated
|
||||
// immediately with the new endpoints.
|
||||
func (e *ResolverGroup) SetEndpoints(endpoints []string) {
|
||||
addrs := epsToAddrs(endpoints...)
|
||||
e.mu.Lock()
|
||||
e.endpoints = endpoints
|
||||
for _, r := range e.resolvers {
|
||||
r.cc.NewAddress(addrs)
|
||||
}
|
||||
e.mu.Unlock()
|
||||
}
|
||||
|
||||
// Target constructs a endpoint target using the endpoint id of the ResolverGroup.
|
||||
func (e *ResolverGroup) Target(endpoint string) string {
|
||||
return Target(e.id, endpoint)
|
||||
}
|
||||
|
||||
// Target constructs a endpoint resolver target.
|
||||
func Target(id, endpoint string) string {
|
||||
return fmt.Sprintf("%s://%s/%s", scheme, id, endpoint)
|
||||
}
|
||||
|
||||
// IsTarget checks if a given target string in an endpoint resolver target.
|
||||
func IsTarget(target string) bool {
|
||||
return strings.HasPrefix(target, "endpoint://")
|
||||
}
|
||||
|
||||
func (e *ResolverGroup) Close() {
|
||||
bldr.close(e.id)
|
||||
}
|
||||
|
||||
// Build creates or reuses an etcd resolver for the etcd cluster name identified by the authority part of the target.
|
||||
func (b *builder) Build(target resolver.Target, cc resolver.ClientConn, opts resolver.BuildOption) (resolver.Resolver, error) {
|
||||
if len(target.Authority) < 1 {
|
||||
return nil, fmt.Errorf("'etcd' target scheme requires non-empty authority identifying etcd cluster being routed to")
|
||||
}
|
||||
id := target.Authority
|
||||
es, err := b.getResolverGroup(id)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to build resolver: %v", err)
|
||||
}
|
||||
r := &Resolver{
|
||||
endpointID: id,
|
||||
cc: cc,
|
||||
}
|
||||
es.addResolver(r)
|
||||
return r, nil
|
||||
}
|
||||
|
||||
func (b *builder) newResolverGroup(id string) (*ResolverGroup, error) {
|
||||
b.mu.RLock()
|
||||
_, ok := b.resolverGroups[id]
|
||||
b.mu.RUnlock()
|
||||
if ok {
|
||||
return nil, fmt.Errorf("Endpoint already exists for id: %s", id)
|
||||
}
|
||||
|
||||
es := &ResolverGroup{id: id}
|
||||
b.mu.Lock()
|
||||
b.resolverGroups[id] = es
|
||||
b.mu.Unlock()
|
||||
return es, nil
|
||||
}
|
||||
|
||||
func (b *builder) getResolverGroup(id string) (*ResolverGroup, error) {
|
||||
b.mu.RLock()
|
||||
es, ok := b.resolverGroups[id]
|
||||
b.mu.RUnlock()
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("ResolverGroup not found for id: %s", id)
|
||||
}
|
||||
return es, nil
|
||||
}
|
||||
|
||||
func (b *builder) close(id string) {
|
||||
b.mu.Lock()
|
||||
delete(b.resolverGroups, id)
|
||||
b.mu.Unlock()
|
||||
}
|
||||
|
||||
func (r *builder) Scheme() string {
|
||||
return scheme
|
||||
}
|
||||
|
||||
// Resolver provides a resolver for a single etcd cluster, identified by name.
|
||||
type Resolver struct {
|
||||
endpointID string
|
||||
cc resolver.ClientConn
|
||||
sync.RWMutex
|
||||
}
|
||||
|
||||
// TODO: use balancer.epsToAddrs
|
||||
func epsToAddrs(eps ...string) (addrs []resolver.Address) {
|
||||
addrs = make([]resolver.Address, 0, len(eps))
|
||||
for _, ep := range eps {
|
||||
addrs = append(addrs, resolver.Address{Addr: ep})
|
||||
}
|
||||
return addrs
|
||||
}
|
||||
|
||||
func (*Resolver) ResolveNow(o resolver.ResolveNowOption) {}
|
||||
|
||||
func (r *Resolver) Close() {
|
||||
es, err := bldr.getResolverGroup(r.endpointID)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
es.removeResolver(r)
|
||||
}
|
||||
|
||||
// ParseEndpoint endpoint parses an endpoint of the form
|
||||
// (http|https)://<host>*|(unix|unixs)://<path>)
|
||||
// and returns a protocol ('tcp' or 'unix'),
|
||||
// host (or filepath if a unix socket),
|
||||
// scheme (http, https, unix, unixs).
|
||||
func ParseEndpoint(endpoint string) (proto string, host string, scheme string) {
|
||||
proto = "tcp"
|
||||
host = endpoint
|
||||
url, uerr := url.Parse(endpoint)
|
||||
if uerr != nil || !strings.Contains(endpoint, "://") {
|
||||
return proto, host, scheme
|
||||
}
|
||||
scheme = url.Scheme
|
||||
|
||||
// strip scheme:// prefix since grpc dials by host
|
||||
host = url.Host
|
||||
switch url.Scheme {
|
||||
case "http", "https":
|
||||
case "unix", "unixs":
|
||||
proto = "unix"
|
||||
host = url.Host + url.Path
|
||||
default:
|
||||
proto, host = "", ""
|
||||
}
|
||||
return proto, host, scheme
|
||||
}
|
||||
|
||||
// ParseTarget parses a endpoint://<id>/<endpoint> string and returns the parsed id and endpoint.
|
||||
// If the target is malformed, an error is returned.
|
||||
func ParseTarget(target string) (string, string, error) {
|
||||
noPrefix := strings.TrimPrefix(target, targetPrefix)
|
||||
if noPrefix == target {
|
||||
return "", "", fmt.Errorf("malformed target, %s prefix is required: %s", targetPrefix, target)
|
||||
}
|
||||
parts := strings.SplitN(noPrefix, "/", 2)
|
||||
if len(parts) != 2 {
|
||||
return "", "", fmt.Errorf("malformed target, expected %s://<id>/<endpoint>, but got %s", scheme, target)
|
||||
}
|
||||
return parts[0], parts[1], nil
|
||||
}
|
68
clientv3/balancer/utils.go
Normal file
@ -0,0 +1,68 @@
|
||||
// Copyright 2018 The etcd 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 balancer
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/url"
|
||||
"sort"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
func scToString(sc balancer.SubConn) string {
|
||||
return fmt.Sprintf("%p", sc)
|
||||
}
|
||||
|
||||
func scsToStrings(scs map[resolver.Address]balancer.SubConn) (ss []string) {
|
||||
ss = make([]string, 0, len(scs))
|
||||
for a, sc := range scs {
|
||||
ss = append(ss, fmt.Sprintf("%s (%s)", a.Addr, scToString(sc)))
|
||||
}
|
||||
sort.Strings(ss)
|
||||
return ss
|
||||
}
|
||||
|
||||
func addrsToStrings(addrs []resolver.Address) (ss []string) {
|
||||
ss = make([]string, len(addrs))
|
||||
for i := range addrs {
|
||||
ss[i] = addrs[i].Addr
|
||||
}
|
||||
sort.Strings(ss)
|
||||
return ss
|
||||
}
|
||||
|
||||
func epsToAddrs(eps ...string) (addrs []resolver.Address) {
|
||||
addrs = make([]resolver.Address, 0, len(eps))
|
||||
for _, ep := range eps {
|
||||
u, err := url.Parse(ep)
|
||||
if err != nil {
|
||||
addrs = append(addrs, resolver.Address{Addr: ep, Type: resolver.Backend})
|
||||
continue
|
||||
}
|
||||
addrs = append(addrs, resolver.Address{Addr: u.Host, Type: resolver.Backend})
|
||||
}
|
||||
return addrs
|
||||
}
|
||||
|
||||
var genN = new(uint32)
|
||||
|
||||
func genName() string {
|
||||
now := time.Now().UnixNano()
|
||||
return fmt.Sprintf("%X%X", now, atomic.AddUint32(genN, 1))
|
||||
}
|
34
clientv3/balancer/utils_test.go
Normal file
@ -0,0 +1,34 @@
|
||||
// Copyright 2018 The etcd 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 balancer
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
func Test_epsToAddrs(t *testing.T) {
|
||||
eps := []string{"https://example.com:2379", "127.0.0.1:2379"}
|
||||
exp := []resolver.Address{
|
||||
{Addr: "example.com:2379", Type: resolver.Backend},
|
||||
{Addr: "127.0.0.1:2379", Type: resolver.Backend},
|
||||
}
|
||||
rs := epsToAddrs(eps...)
|
||||
if !reflect.DeepEqual(rs, exp) {
|
||||
t.Fatalf("expected %v, got %v", exp, rs)
|
||||
}
|
||||
}
|
@ -21,13 +21,18 @@ import (
|
||||
"fmt"
|
||||
"net"
|
||||
"net/url"
|
||||
"os"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/clientv3/balancer"
|
||||
"github.com/coreos/etcd/clientv3/balancer/picker"
|
||||
"github.com/coreos/etcd/clientv3/balancer/resolver/endpoint"
|
||||
"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
"github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
@ -40,8 +45,26 @@ import (
|
||||
var (
|
||||
ErrNoAvailableEndpoints = errors.New("etcdclient: no available endpoints")
|
||||
ErrOldCluster = errors.New("etcdclient: old cluster version")
|
||||
|
||||
roundRobinBalancerName = fmt.Sprintf("etcd-%s", picker.RoundrobinBalanced.String())
|
||||
)
|
||||
|
||||
func init() {
|
||||
lg := zap.NewNop()
|
||||
if os.Getenv("ETCD_CLIENT_DEBUG") != "" {
|
||||
var err error
|
||||
lg, err = zap.NewProductionConfig().Build() // info level logging
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
balancer.RegisterBuilder(balancer.Config{
|
||||
Policy: picker.RoundrobinBalanced,
|
||||
Name: roundRobinBalancerName,
|
||||
Logger: lg,
|
||||
})
|
||||
}
|
||||
|
||||
// Client provides and manages an etcd v3 client session.
|
||||
type Client struct {
|
||||
Cluster
|
||||
@ -51,13 +74,13 @@ type Client struct {
|
||||
Auth
|
||||
Maintenance
|
||||
|
||||
conn *grpc.ClientConn
|
||||
dialerrc chan error
|
||||
conn *grpc.ClientConn
|
||||
|
||||
cfg Config
|
||||
creds *credentials.TransportCredentials
|
||||
balancer *balancer.GRPC17Health
|
||||
mu *sync.Mutex
|
||||
cfg Config
|
||||
creds *credentials.TransportCredentials
|
||||
balancer balancer.Balancer
|
||||
resolverGroup *endpoint.ResolverGroup
|
||||
mu *sync.Mutex
|
||||
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
@ -70,6 +93,8 @@ type Client struct {
|
||||
tokenCred *authTokenCredential
|
||||
|
||||
callOpts []grpc.CallOption
|
||||
|
||||
lg *zap.Logger
|
||||
}
|
||||
|
||||
// New creates a new etcdv3 client from a given configuration.
|
||||
@ -104,6 +129,9 @@ func (c *Client) Close() error {
|
||||
c.cancel()
|
||||
c.Watcher.Close()
|
||||
c.Lease.Close()
|
||||
if c.resolverGroup != nil {
|
||||
c.resolverGroup.Close()
|
||||
}
|
||||
if c.conn != nil {
|
||||
return toErr(c.ctx, c.conn.Close())
|
||||
}
|
||||
@ -126,16 +154,9 @@ func (c *Client) Endpoints() (eps []string) {
|
||||
// SetEndpoints updates client's endpoints.
|
||||
func (c *Client) SetEndpoints(eps ...string) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
c.cfg.Endpoints = eps
|
||||
c.mu.Unlock()
|
||||
c.balancer.UpdateAddrs(eps...)
|
||||
|
||||
if c.balancer.NeedUpdate() {
|
||||
select {
|
||||
case c.balancer.UpdateAddrsC() <- balancer.NotifyNext:
|
||||
case <-c.balancer.StopC():
|
||||
}
|
||||
}
|
||||
c.resolverGroup.SetEndpoints(eps)
|
||||
}
|
||||
|
||||
// Sync synchronizes client's endpoints with the known endpoints from the etcd membership.
|
||||
@ -189,28 +210,6 @@ func (cred authTokenCredential) GetRequestMetadata(ctx context.Context, s ...str
|
||||
}, nil
|
||||
}
|
||||
|
||||
func parseEndpoint(endpoint string) (proto string, host string, scheme string) {
|
||||
proto = "tcp"
|
||||
host = endpoint
|
||||
url, uerr := url.Parse(endpoint)
|
||||
if uerr != nil || !strings.Contains(endpoint, "://") {
|
||||
return proto, host, scheme
|
||||
}
|
||||
scheme = url.Scheme
|
||||
|
||||
// strip scheme:// prefix since grpc dials by host
|
||||
host = url.Host
|
||||
switch url.Scheme {
|
||||
case "http", "https":
|
||||
case "unix", "unixs":
|
||||
proto = "unix"
|
||||
host = url.Host + url.Path
|
||||
default:
|
||||
proto, host = "", ""
|
||||
}
|
||||
return proto, host, scheme
|
||||
}
|
||||
|
||||
func (c *Client) processCreds(scheme string) (creds *credentials.TransportCredentials) {
|
||||
creds = c.creds
|
||||
switch scheme {
|
||||
@ -231,10 +230,12 @@ func (c *Client) processCreds(scheme string) (creds *credentials.TransportCreden
|
||||
}
|
||||
|
||||
// dialSetupOpts gives the dial opts prior to any authentication
|
||||
func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) (opts []grpc.DialOption) {
|
||||
if c.cfg.DialTimeout > 0 {
|
||||
opts = []grpc.DialOption{grpc.WithTimeout(c.cfg.DialTimeout)}
|
||||
func (c *Client) dialSetupOpts(target string, dopts ...grpc.DialOption) (opts []grpc.DialOption, err error) {
|
||||
_, ep, err := endpoint.ParseTarget(target)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("unable to parse target: %v", err)
|
||||
}
|
||||
|
||||
if c.cfg.DialKeepAliveTime > 0 {
|
||||
params := keepalive.ClientParameters{
|
||||
Time: c.cfg.DialKeepAliveTime,
|
||||
@ -244,12 +245,12 @@ func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) (opts
|
||||
}
|
||||
opts = append(opts, dopts...)
|
||||
|
||||
f := func(host string, t time.Duration) (net.Conn, error) {
|
||||
proto, host, _ := parseEndpoint(c.balancer.Endpoint(host))
|
||||
if host == "" && endpoint != "" {
|
||||
f := func(dialEp string, t time.Duration) (net.Conn, error) {
|
||||
proto, host, _ := endpoint.ParseEndpoint(dialEp)
|
||||
if host == "" && ep != "" {
|
||||
// dialing an endpoint not in the balancer; use
|
||||
// endpoint passed into dial
|
||||
proto, host, _ = parseEndpoint(endpoint)
|
||||
proto, host, _ = endpoint.ParseEndpoint(ep)
|
||||
}
|
||||
if proto == "" {
|
||||
return nil, fmt.Errorf("unknown scheme for %q", host)
|
||||
@ -260,19 +261,12 @@ func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) (opts
|
||||
default:
|
||||
}
|
||||
dialer := &net.Dialer{Timeout: t}
|
||||
conn, err := dialer.DialContext(c.ctx, proto, host)
|
||||
if err != nil {
|
||||
select {
|
||||
case c.dialerrc <- err:
|
||||
default:
|
||||
}
|
||||
}
|
||||
return conn, err
|
||||
return dialer.DialContext(c.ctx, proto, host)
|
||||
}
|
||||
opts = append(opts, grpc.WithDialer(f))
|
||||
|
||||
creds := c.creds
|
||||
if _, _, scheme := parseEndpoint(endpoint); len(scheme) != 0 {
|
||||
if _, _, scheme := endpoint.ParseEndpoint(ep); len(scheme) != 0 {
|
||||
creds = c.processCreds(scheme)
|
||||
}
|
||||
if creds != nil {
|
||||
@ -281,7 +275,19 @@ func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) (opts
|
||||
opts = append(opts, grpc.WithInsecure())
|
||||
}
|
||||
|
||||
return opts
|
||||
// Interceptor retry and backoff.
|
||||
// TODO: Replace all of clientv3/retry.go with interceptor based retry, or with
|
||||
// https://github.com/grpc/proposal/blob/master/A6-client-retries.md#retry-policy
|
||||
// once it is available.
|
||||
rrBackoff := withBackoff(c.roundRobinQuorumBackoff(defaultBackoffWaitBetween, defaultBackoffJitterFraction))
|
||||
opts = append(opts,
|
||||
// Disable stream retry by default since go-grpc-middleware/retry does not support client streams.
|
||||
// Streams that are safe to retry are enabled individually.
|
||||
grpc.WithStreamInterceptor(c.streamClientInterceptor(c.lg, withMax(0), rrBackoff)),
|
||||
grpc.WithUnaryInterceptor(c.unaryClientInterceptor(c.lg, withMax(defaultUnaryMaxRetries), rrBackoff)),
|
||||
)
|
||||
|
||||
return opts, nil
|
||||
}
|
||||
|
||||
// Dial connects to a single endpoint using the client's config.
|
||||
@ -294,10 +300,18 @@ func (c *Client) getToken(ctx context.Context) error {
|
||||
var auth *authenticator
|
||||
|
||||
for i := 0; i < len(c.cfg.Endpoints); i++ {
|
||||
endpoint := c.cfg.Endpoints[i]
|
||||
host := getHost(endpoint)
|
||||
ep := c.cfg.Endpoints[i]
|
||||
// use dial options without dopts to avoid reusing the client balancer
|
||||
auth, err = newAuthenticator(host, c.dialSetupOpts(endpoint), c)
|
||||
var dOpts []grpc.DialOption
|
||||
_, host, _ := endpoint.ParseEndpoint(ep)
|
||||
target := c.resolverGroup.Target(host)
|
||||
dOpts, err = c.dialSetupOpts(target, c.cfg.DialOptions...)
|
||||
if err != nil {
|
||||
err = fmt.Errorf("failed to configure auth dialer: %v", err)
|
||||
continue
|
||||
}
|
||||
dOpts = append(dOpts, grpc.WithBalancerName(roundRobinBalancerName))
|
||||
auth, err = newAuthenticator(ctx, target, dOpts, c)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
@ -319,37 +333,52 @@ func (c *Client) getToken(ctx context.Context) error {
|
||||
return err
|
||||
}
|
||||
|
||||
func (c *Client) dial(endpoint string, dopts ...grpc.DialOption) (*grpc.ClientConn, error) {
|
||||
opts := c.dialSetupOpts(endpoint, dopts...)
|
||||
host := getHost(endpoint)
|
||||
func (c *Client) dial(ep string, dopts ...grpc.DialOption) (*grpc.ClientConn, error) {
|
||||
// We pass a target to DialContext of the form: endpoint://<clusterName>/<host-part> that
|
||||
// does not include scheme (http/https/unix/unixs) or path parts.
|
||||
_, host, _ := endpoint.ParseEndpoint(ep)
|
||||
target := c.resolverGroup.Target(host)
|
||||
|
||||
opts, err := c.dialSetupOpts(target, dopts...)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to configure dialer: %v", err)
|
||||
}
|
||||
|
||||
if c.Username != "" && c.Password != "" {
|
||||
c.tokenCred = &authTokenCredential{
|
||||
tokenMu: &sync.RWMutex{},
|
||||
}
|
||||
|
||||
ctx := c.ctx
|
||||
ctx, cancel := c.ctx, func() {}
|
||||
if c.cfg.DialTimeout > 0 {
|
||||
cctx, cancel := context.WithTimeout(ctx, c.cfg.DialTimeout)
|
||||
defer cancel()
|
||||
ctx = cctx
|
||||
ctx, cancel = context.WithTimeout(ctx, c.cfg.DialTimeout)
|
||||
}
|
||||
|
||||
err := c.getToken(ctx)
|
||||
err = c.getToken(ctx)
|
||||
if err != nil {
|
||||
if toErr(ctx, err) != rpctypes.ErrAuthNotEnabled {
|
||||
if err == ctx.Err() && ctx.Err() != c.ctx.Err() {
|
||||
err = context.DeadlineExceeded
|
||||
}
|
||||
cancel()
|
||||
return nil, err
|
||||
}
|
||||
} else {
|
||||
opts = append(opts, grpc.WithPerRPCCredentials(c.tokenCred))
|
||||
}
|
||||
cancel()
|
||||
}
|
||||
|
||||
opts = append(opts, c.cfg.DialOptions...)
|
||||
|
||||
conn, err := grpc.DialContext(c.ctx, host, opts...)
|
||||
dctx := c.ctx
|
||||
if c.cfg.DialTimeout > 0 {
|
||||
var cancel context.CancelFunc
|
||||
dctx, cancel = context.WithTimeout(c.ctx, c.cfg.DialTimeout)
|
||||
defer cancel() // TODO: Is this right for cases where grpc.WithBlock() is not set on the dial options?
|
||||
}
|
||||
|
||||
conn, err := grpc.DialContext(dctx, target, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -382,7 +411,6 @@ func newClient(cfg *Config) (*Client, error) {
|
||||
ctx, cancel := context.WithCancel(baseCtx)
|
||||
client := &Client{
|
||||
conn: nil,
|
||||
dialerrc: make(chan error, 1),
|
||||
cfg: *cfg,
|
||||
creds: creds,
|
||||
ctx: ctx,
|
||||
@ -390,6 +418,17 @@ func newClient(cfg *Config) (*Client, error) {
|
||||
mu: new(sync.Mutex),
|
||||
callOpts: defaultCallOpts,
|
||||
}
|
||||
|
||||
lcfg := DefaultLogConfig
|
||||
if cfg.LogConfig != nil {
|
||||
lcfg = *cfg.LogConfig
|
||||
}
|
||||
var err error
|
||||
client.lg, err = lcfg.Build()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if cfg.Username != "" && cfg.Password != "" {
|
||||
client.Username = cfg.Username
|
||||
client.Password = cfg.Password
|
||||
@ -412,40 +451,31 @@ func newClient(cfg *Config) (*Client, error) {
|
||||
client.callOpts = callOpts
|
||||
}
|
||||
|
||||
client.balancer = balancer.NewGRPC17Health(cfg.Endpoints, cfg.DialTimeout, client.dial)
|
||||
|
||||
// use Endpoints[0] so that for https:// without any tls config given, then
|
||||
// grpc will assume the certificate server name is the endpoint host.
|
||||
conn, err := client.dial(cfg.Endpoints[0], grpc.WithBalancer(client.balancer))
|
||||
// Prepare a 'endpoint://<unique-client-id>/' resolver for the client and create a endpoint target to pass
|
||||
// to dial so the client knows to use this resolver.
|
||||
client.resolverGroup, err = endpoint.NewResolverGroup(fmt.Sprintf("client-%s", strconv.FormatInt(time.Now().UnixNano(), 36)))
|
||||
if err != nil {
|
||||
client.cancel()
|
||||
client.balancer.Close()
|
||||
return nil, err
|
||||
}
|
||||
client.conn = conn
|
||||
client.resolverGroup.SetEndpoints(cfg.Endpoints)
|
||||
|
||||
// wait for a connection
|
||||
if cfg.DialTimeout > 0 {
|
||||
hasConn := false
|
||||
waitc := time.After(cfg.DialTimeout)
|
||||
select {
|
||||
case <-client.balancer.Ready():
|
||||
hasConn = true
|
||||
case <-ctx.Done():
|
||||
case <-waitc:
|
||||
}
|
||||
if !hasConn {
|
||||
err := context.DeadlineExceeded
|
||||
select {
|
||||
case err = <-client.dialerrc:
|
||||
default:
|
||||
}
|
||||
client.cancel()
|
||||
client.balancer.Close()
|
||||
conn.Close()
|
||||
return nil, err
|
||||
}
|
||||
if len(cfg.Endpoints) < 1 {
|
||||
return nil, fmt.Errorf("at least one Endpoint must is required in client config")
|
||||
}
|
||||
dialEndpoint := cfg.Endpoints[0]
|
||||
|
||||
// Use an provided endpoint target so that for https:// without any tls config given, then
|
||||
// grpc will assume the certificate server name is the endpoint host.
|
||||
conn, err := client.dial(dialEndpoint, grpc.WithBalancerName(roundRobinBalancerName))
|
||||
if err != nil {
|
||||
client.cancel()
|
||||
client.resolverGroup.Close()
|
||||
return nil, err
|
||||
}
|
||||
// TODO: With the old grpc balancer interface, we waited until the dial timeout
|
||||
// for the balancer to be ready. Is there an equivalent wait we should do with the new grpc balancer interface?
|
||||
client.conn = conn
|
||||
|
||||
client.Cluster = NewCluster(client)
|
||||
client.KV = NewKV(client)
|
||||
@ -465,6 +495,22 @@ func newClient(cfg *Config) (*Client, error) {
|
||||
return client, nil
|
||||
}
|
||||
|
||||
// roundRobinQuorumBackoff retries against quorum between each backoff.
|
||||
// This is intended for use with a round robin load balancer.
|
||||
func (c *Client) roundRobinQuorumBackoff(waitBetween time.Duration, jitterFraction float64) backoffFunc {
|
||||
return func(attempt uint) time.Duration {
|
||||
// after each round robin across quorum, backoff for our wait between duration
|
||||
n := uint(len(c.Endpoints()))
|
||||
quorum := (n/2 + 1)
|
||||
if attempt%quorum == 0 {
|
||||
c.lg.Info("backoff", zap.Uint("attempt", attempt), zap.Uint("quorum", quorum), zap.Duration("waitBetween", waitBetween), zap.Float64("jitterFraction", jitterFraction))
|
||||
return backoffutils.JitterUp(waitBetween, jitterFraction)
|
||||
}
|
||||
c.lg.Info("backoff skipped", zap.Uint("attempt", attempt), zap.Uint("quorum", quorum))
|
||||
return 0
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Client) checkVersion() (err error) {
|
||||
var wg sync.WaitGroup
|
||||
errc := make(chan error, len(c.cfg.Endpoints))
|
||||
@ -574,6 +620,26 @@ func canceledByCaller(stopCtx context.Context, err error) bool {
|
||||
return err == context.Canceled || err == context.DeadlineExceeded
|
||||
}
|
||||
|
||||
// IsConnCanceled returns true, if error is from a closed gRPC connection.
|
||||
// ref. https://github.com/grpc/grpc-go/pull/1854
|
||||
func IsConnCanceled(err error) bool {
|
||||
if err == nil {
|
||||
return false
|
||||
}
|
||||
// >= gRPC v1.10.x
|
||||
s, ok := status.FromError(err)
|
||||
if ok {
|
||||
// connection is canceled or server has already closed the connection
|
||||
return s.Code() == codes.Canceled || s.Message() == "transport is closing"
|
||||
}
|
||||
// >= gRPC v1.10.x
|
||||
if err == context.Canceled {
|
||||
return true
|
||||
}
|
||||
// <= gRPC v1.7.x returns 'errors.New("grpc: the client connection is closing")'
|
||||
return strings.Contains(err.Error(), "grpc: the client connection is closing")
|
||||
}
|
||||
|
||||
func getHost(ep string) string {
|
||||
url, uerr := url.Parse(ep)
|
||||
if uerr != nil || !strings.Contains(ep, "://") {
|
||||
|
@ -23,6 +23,8 @@ import (
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
func TestDialCancel(t *testing.T) {
|
||||
@ -80,14 +82,17 @@ func TestDialCancel(t *testing.T) {
|
||||
func TestDialTimeout(t *testing.T) {
|
||||
defer testutil.AfterTest(t)
|
||||
|
||||
// grpc.WithBlock to block until connection up or timeout
|
||||
testCfgs := []Config{
|
||||
{
|
||||
Endpoints: []string{"http://254.0.0.1:12345"},
|
||||
DialTimeout: 2 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
},
|
||||
{
|
||||
Endpoints: []string{"http://254.0.0.1:12345"},
|
||||
DialTimeout: time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
Username: "abc",
|
||||
Password: "def",
|
||||
},
|
||||
|
@ -19,6 +19,7 @@ import (
|
||||
"crypto/tls"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
@ -72,4 +73,27 @@ type Config struct {
|
||||
// Context is the default client context; it can be used to cancel grpc dial out and
|
||||
// other operations that do not have an explicit context.
|
||||
Context context.Context
|
||||
|
||||
// LogConfig configures client-side logger.
|
||||
// If nil, use the default logger.
|
||||
// TODO: configure gRPC logger
|
||||
LogConfig *zap.Config
|
||||
}
|
||||
|
||||
// DefaultLogConfig is the default client logging configuration.
|
||||
// Default log level is "Warn". Use "zap.InfoLevel" for debugging.
|
||||
// Use "/dev/null" for output paths, to discard all logs.
|
||||
var DefaultLogConfig = zap.Config{
|
||||
Level: zap.NewAtomicLevelAt(zap.WarnLevel),
|
||||
Development: false,
|
||||
Sampling: &zap.SamplingConfig{
|
||||
Initial: 100,
|
||||
Thereafter: 100,
|
||||
},
|
||||
Encoding: "json",
|
||||
EncoderConfig: zap.NewProductionEncoderConfig(),
|
||||
|
||||
// Use "/dev/null" to discard all
|
||||
OutputPaths: []string{"stderr"},
|
||||
ErrorOutputPaths: []string{"stderr"},
|
||||
}
|
||||
|
@ -87,11 +87,20 @@
|
||||
// go func() { cli.Close() }()
|
||||
// _, err := kvc.Get(ctx, "a")
|
||||
// if err != nil {
|
||||
// // with etcd clientv3 <= v3.3
|
||||
// if err == context.Canceled {
|
||||
// // grpc balancer calls 'Get' with an inflight client.Close
|
||||
// } else if err == grpc.ErrClientConnClosing {
|
||||
// // grpc balancer calls 'Get' after client.Close.
|
||||
// }
|
||||
// // with etcd clientv3 >= v3.4
|
||||
// if clientv3.IsConnCanceled(err) {
|
||||
// // gRPC client connection is closed
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// The grpc load balancer is registered statically and is shared across etcd clients.
|
||||
// To enable detailed load balancer logging, set the ETCD_CLIENT_DEBUG environment
|
||||
// variable. E.g. "ETCD_CLIENT_DEBUG=1".
|
||||
//
|
||||
package clientv3
|
||||
|
@ -25,6 +25,7 @@ import (
|
||||
"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
"github.com/coreos/etcd/integration"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
// TestBalancerUnderBlackholeKeepAliveWatch tests when watch discovers it cannot talk to
|
||||
@ -44,6 +45,7 @@ func TestBalancerUnderBlackholeKeepAliveWatch(t *testing.T) {
|
||||
ccfg := clientv3.Config{
|
||||
Endpoints: []string{eps[0]},
|
||||
DialTimeout: 1 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
DialKeepAliveTime: 1 * time.Second,
|
||||
DialKeepAliveTimeout: 500 * time.Millisecond,
|
||||
}
|
||||
@ -106,7 +108,7 @@ func TestBalancerUnderBlackholeKeepAliveWatch(t *testing.T) {
|
||||
func TestBalancerUnderBlackholeNoKeepAlivePut(t *testing.T) {
|
||||
testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error {
|
||||
_, err := cli.Put(ctx, "foo", "bar")
|
||||
if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
return errExpected
|
||||
}
|
||||
return err
|
||||
@ -116,7 +118,7 @@ func TestBalancerUnderBlackholeNoKeepAlivePut(t *testing.T) {
|
||||
func TestBalancerUnderBlackholeNoKeepAliveDelete(t *testing.T) {
|
||||
testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error {
|
||||
_, err := cli.Delete(ctx, "foo")
|
||||
if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
return errExpected
|
||||
}
|
||||
return err
|
||||
@ -129,7 +131,7 @@ func TestBalancerUnderBlackholeNoKeepAliveTxn(t *testing.T) {
|
||||
If(clientv3.Compare(clientv3.Version("foo"), "=", 0)).
|
||||
Then(clientv3.OpPut("foo", "bar")).
|
||||
Else(clientv3.OpPut("foo", "baz")).Commit()
|
||||
if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
return errExpected
|
||||
}
|
||||
return err
|
||||
@ -139,7 +141,7 @@ func TestBalancerUnderBlackholeNoKeepAliveTxn(t *testing.T) {
|
||||
func TestBalancerUnderBlackholeNoKeepAliveLinearizableGet(t *testing.T) {
|
||||
testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error {
|
||||
_, err := cli.Get(ctx, "a")
|
||||
if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
return errExpected
|
||||
}
|
||||
return err
|
||||
@ -149,7 +151,7 @@ func TestBalancerUnderBlackholeNoKeepAliveLinearizableGet(t *testing.T) {
|
||||
func TestBalancerUnderBlackholeNoKeepAliveSerializableGet(t *testing.T) {
|
||||
testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error {
|
||||
_, err := cli.Get(ctx, "a", clientv3.WithSerializable())
|
||||
if err == context.DeadlineExceeded || isServerCtxTimeout(err) {
|
||||
if isClientTimeout(err) || isServerCtxTimeout(err) {
|
||||
return errExpected
|
||||
}
|
||||
return err
|
||||
@ -172,6 +174,7 @@ func testBalancerUnderBlackholeNoKeepAlive(t *testing.T, op func(*clientv3.Clien
|
||||
ccfg := clientv3.Config{
|
||||
Endpoints: []string{eps[0]},
|
||||
DialTimeout: 1 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
}
|
||||
cli, err := clientv3.New(ccfg)
|
||||
if err != nil {
|
||||
@ -189,22 +192,23 @@ func testBalancerUnderBlackholeNoKeepAlive(t *testing.T, op func(*clientv3.Clien
|
||||
// blackhole eps[0]
|
||||
clus.Members[0].Blackhole()
|
||||
|
||||
// fail first due to blackhole, retry should succeed
|
||||
// With round robin balancer, client will make a request to a healthy endpoint
|
||||
// within a few requests.
|
||||
// TODO: first operation can succeed
|
||||
// when gRPC supports better retry on non-delivered request
|
||||
for i := 0; i < 2; i++ {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
|
||||
for i := 0; i < 5; i++ {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
|
||||
err = op(cli, ctx)
|
||||
cancel()
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
if i == 0 {
|
||||
if err != errExpected {
|
||||
t.Errorf("#%d: expected %v, got %v", i, errExpected, err)
|
||||
}
|
||||
} else if err != nil {
|
||||
} else if err == errExpected {
|
||||
t.Logf("#%d: current error %v", i, err)
|
||||
} else {
|
||||
t.Errorf("#%d: failed with error %v", i, err)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
@ -26,6 +26,7 @@ import (
|
||||
"github.com/coreos/etcd/integration"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"github.com/coreos/etcd/pkg/transport"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
var (
|
||||
@ -58,10 +59,11 @@ func TestDialTLSExpired(t *testing.T) {
|
||||
_, err = clientv3.New(clientv3.Config{
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr()},
|
||||
DialTimeout: 3 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
TLS: tls,
|
||||
})
|
||||
if err != context.DeadlineExceeded {
|
||||
t.Fatalf("expected %v, got %v", context.DeadlineExceeded, err)
|
||||
if !isClientTimeout(err) {
|
||||
t.Fatalf("expected dial timeout error, got %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
@ -72,12 +74,24 @@ func TestDialTLSNoConfig(t *testing.T) {
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, ClientTLS: &testTLSInfo, SkipCreatingClient: true})
|
||||
defer clus.Terminate(t)
|
||||
// expect "signed by unknown authority"
|
||||
_, err := clientv3.New(clientv3.Config{
|
||||
c, err := clientv3.New(clientv3.Config{
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr()},
|
||||
DialTimeout: time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
})
|
||||
if err != context.DeadlineExceeded {
|
||||
t.Fatalf("expected %v, got %v", context.DeadlineExceeded, err)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
// TODO: this should not be required when we set grpc.WithBlock()
|
||||
if c != nil {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
|
||||
_, err = c.KV.Get(ctx, "/")
|
||||
cancel()
|
||||
}
|
||||
if !isClientTimeout(err) {
|
||||
t.Fatalf("expected dial timeout error, got %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
@ -104,7 +118,11 @@ func testDialSetEndpoints(t *testing.T, setBefore bool) {
|
||||
}
|
||||
toKill := rand.Intn(len(eps))
|
||||
|
||||
cfg := clientv3.Config{Endpoints: []string{eps[toKill]}, DialTimeout: 1 * time.Second}
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{eps[toKill]},
|
||||
DialTimeout: 1 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
}
|
||||
cli, err := clientv3.New(cfg)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
@ -121,6 +139,7 @@ func testDialSetEndpoints(t *testing.T, setBefore bool) {
|
||||
if !setBefore {
|
||||
cli.SetEndpoints(eps[toKill%3], eps[(toKill+1)%3])
|
||||
}
|
||||
time.Sleep(time.Second * 2)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), integration.RequestWaitTimeout)
|
||||
if _, err = cli.Get(ctx, "foo", clientv3.WithSerializable()); err != nil {
|
||||
t.Fatal(err)
|
||||
@ -142,6 +161,7 @@ func TestSwitchSetEndpoints(t *testing.T) {
|
||||
clus.Members[0].InjectPartition(t, clus.Members[1:]...)
|
||||
|
||||
cli.SetEndpoints(eps...)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
||||
defer cancel()
|
||||
if _, err := cli.Get(ctx, "foo"); err != nil {
|
||||
@ -158,6 +178,7 @@ func TestRejectOldCluster(t *testing.T) {
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr()},
|
||||
DialTimeout: 5 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
RejectOldCluster: true,
|
||||
}
|
||||
cli, err := clientv3.New(cfg)
|
||||
|
@ -442,7 +442,7 @@ func TestKVGetErrConnClosed(t *testing.T) {
|
||||
go func() {
|
||||
defer close(donec)
|
||||
_, err := cli.Get(context.TODO(), "foo")
|
||||
if err != nil && err != context.Canceled && err != grpc.ErrClientConnClosing {
|
||||
if !clientv3.IsConnCanceled(err) {
|
||||
t.Fatalf("expected %v or %v, got %v", context.Canceled, grpc.ErrClientConnClosing, err)
|
||||
}
|
||||
}()
|
||||
@ -474,7 +474,7 @@ func TestKVNewAfterClose(t *testing.T) {
|
||||
donec := make(chan struct{})
|
||||
go func() {
|
||||
_, err := cli.Get(context.TODO(), "foo")
|
||||
if err != context.Canceled && err != grpc.ErrClientConnClosing {
|
||||
if !clientv3.IsConnCanceled(err) {
|
||||
t.Fatalf("expected %v or %v, got %v", context.Canceled, grpc.ErrClientConnClosing, err)
|
||||
}
|
||||
close(donec)
|
||||
@ -708,9 +708,10 @@ func TestKVGetRetry(t *testing.T) {
|
||||
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
clus.Members[fIdx].Restart(t)
|
||||
clus.Members[fIdx].WaitOK(t)
|
||||
|
||||
select {
|
||||
case <-time.After(5 * time.Second):
|
||||
case <-time.After(20 * time.Second):
|
||||
t.Fatalf("timed out waiting for get")
|
||||
case <-donec:
|
||||
}
|
||||
@ -750,7 +751,7 @@ func TestKVPutFailGetRetry(t *testing.T) {
|
||||
clus.Members[0].Restart(t)
|
||||
|
||||
select {
|
||||
case <-time.After(5 * time.Second):
|
||||
case <-time.After(20 * time.Second):
|
||||
t.Fatalf("timed out waiting for get")
|
||||
case <-donec:
|
||||
}
|
||||
@ -792,7 +793,7 @@ func TestKVGetStoppedServerAndClose(t *testing.T) {
|
||||
// this Get fails and triggers an asynchronous connection retry
|
||||
_, err := cli.Get(ctx, "abc")
|
||||
cancel()
|
||||
if err != nil && err != context.DeadlineExceeded {
|
||||
if err != nil && !(isCanceled(err) || isClientTimeout(err)) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -814,14 +815,15 @@ func TestKVPutStoppedServerAndClose(t *testing.T) {
|
||||
// grpc finds out the original connection is down due to the member shutdown.
|
||||
_, err := cli.Get(ctx, "abc")
|
||||
cancel()
|
||||
if err != nil && err != context.DeadlineExceeded {
|
||||
if err != nil && !(isCanceled(err) || isClientTimeout(err)) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
ctx, cancel = context.WithTimeout(context.TODO(), time.Second)
|
||||
// this Put fails and triggers an asynchronous connection retry
|
||||
_, err = cli.Put(ctx, "abc", "123")
|
||||
cancel()
|
||||
if err != nil && err != context.DeadlineExceeded {
|
||||
if err != nil && !(isCanceled(err) || isClientTimeout(err) || isUnavailable(err)) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
@ -906,7 +908,7 @@ func TestKVLargeRequests(t *testing.T) {
|
||||
maxCallSendBytesClient: 10 * 1024 * 1024,
|
||||
maxCallRecvBytesClient: 0,
|
||||
valueSize: 10 * 1024 * 1024,
|
||||
expectError: grpc.Errorf(codes.ResourceExhausted, "grpc: trying to send message larger than max "),
|
||||
expectError: grpc.Errorf(codes.ResourceExhausted, "trying to send message larger than max "),
|
||||
},
|
||||
{
|
||||
maxRequestBytesServer: 10 * 1024 * 1024,
|
||||
@ -920,7 +922,7 @@ func TestKVLargeRequests(t *testing.T) {
|
||||
maxCallSendBytesClient: 10 * 1024 * 1024,
|
||||
maxCallRecvBytesClient: 0,
|
||||
valueSize: 10*1024*1024 + 5,
|
||||
expectError: grpc.Errorf(codes.ResourceExhausted, "grpc: trying to send message larger than max "),
|
||||
expectError: grpc.Errorf(codes.ResourceExhausted, "trying to send message larger than max "),
|
||||
},
|
||||
}
|
||||
for i, test := range tests {
|
||||
@ -940,7 +942,7 @@ func TestKVLargeRequests(t *testing.T) {
|
||||
t.Errorf("#%d: expected %v, got %v", i, test.expectError, err)
|
||||
}
|
||||
} else if err != nil && !strings.HasPrefix(err.Error(), test.expectError.Error()) {
|
||||
t.Errorf("#%d: expected %v, got %v", i, test.expectError, err)
|
||||
t.Errorf("#%d: expected error starting with '%s', got '%s'", i, test.expectError.Error(), err.Error())
|
||||
}
|
||||
|
||||
// put request went through, now expects large response back
|
||||
|
@ -145,6 +145,10 @@ func TestLeaseKeepAlive(t *testing.T) {
|
||||
t.Errorf("chan is closed, want not closed")
|
||||
}
|
||||
|
||||
if kresp == nil {
|
||||
t.Fatalf("unexpected null response")
|
||||
}
|
||||
|
||||
if kresp.ID != resp.ID {
|
||||
t.Errorf("ID = %x, want %x", kresp.ID, resp.ID)
|
||||
}
|
||||
@ -292,10 +296,10 @@ func TestLeaseGrantErrConnClosed(t *testing.T) {
|
||||
go func() {
|
||||
defer close(donec)
|
||||
_, err := cli.Grant(context.TODO(), 5)
|
||||
if err != nil && err != grpc.ErrClientConnClosing && err != context.Canceled {
|
||||
if !clientv3.IsConnCanceled(err) {
|
||||
// grpc.ErrClientConnClosing if grpc-go balancer calls 'Get' after client.Close.
|
||||
// context.Canceled if grpc-go balancer calls 'Get' with an inflight client.Close.
|
||||
t.Fatalf("expected %v or %v, got %v", grpc.ErrClientConnClosing, context.Canceled, err)
|
||||
t.Fatalf("expected %v, %v or server unavailable, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err)
|
||||
}
|
||||
}()
|
||||
|
||||
@ -324,8 +328,9 @@ func TestLeaseGrantNewAfterClose(t *testing.T) {
|
||||
|
||||
donec := make(chan struct{})
|
||||
go func() {
|
||||
if _, err := cli.Grant(context.TODO(), 5); err != context.Canceled && err != grpc.ErrClientConnClosing {
|
||||
t.Fatalf("expected %v or %v, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err)
|
||||
_, err := cli.Grant(context.TODO(), 5)
|
||||
if !clientv3.IsConnCanceled(err) {
|
||||
t.Fatalf("expected %v, %v or server unavailable, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err)
|
||||
}
|
||||
close(donec)
|
||||
}()
|
||||
@ -356,8 +361,9 @@ func TestLeaseRevokeNewAfterClose(t *testing.T) {
|
||||
|
||||
donec := make(chan struct{})
|
||||
go func() {
|
||||
if _, err := cli.Revoke(context.TODO(), leaseID); err != context.Canceled && err != grpc.ErrClientConnClosing {
|
||||
t.Fatalf("expected %v or %v, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err)
|
||||
_, err := cli.Revoke(context.TODO(), leaseID)
|
||||
if !clientv3.IsConnCanceled(err) {
|
||||
t.Fatalf("expected %v, %v or server unavailable, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err)
|
||||
}
|
||||
close(donec)
|
||||
}()
|
||||
|
@ -1905,7 +1905,7 @@ func TestLeasingSessionExpire(t *testing.T) {
|
||||
}
|
||||
waitForExpireAck(t, lkv)
|
||||
clus.Members[0].Restart(t)
|
||||
|
||||
integration.WaitClientV3(t, lkv2)
|
||||
if _, err = lkv2.Put(context.TODO(), "abc", "def"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -1920,10 +1920,6 @@ func TestLeasingSessionExpire(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestLeasingSessionExpireCancel(t *testing.T) {
|
||||
defer testutil.AfterTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
tests := []func(context.Context, clientv3.KV) error{
|
||||
func(ctx context.Context, kv clientv3.KV) error {
|
||||
_, err := kv.Get(ctx, "abc")
|
||||
@ -1960,37 +1956,43 @@ func TestLeasingSessionExpireCancel(t *testing.T) {
|
||||
},
|
||||
}
|
||||
for i := range tests {
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/", concurrency.WithTTL(1))
|
||||
testutil.AssertNil(t, err)
|
||||
defer closeLKV()
|
||||
t.Run(fmt.Sprintf("test %d", i), func(t *testing.T) {
|
||||
defer testutil.AfterTest(t)
|
||||
clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
|
||||
defer clus.Terminate(t)
|
||||
|
||||
if _, err = lkv.Get(context.TODO(), "abc"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/", concurrency.WithTTL(1))
|
||||
testutil.AssertNil(t, err)
|
||||
defer closeLKV()
|
||||
|
||||
// down endpoint lkv uses for keepalives
|
||||
clus.Members[0].Stop(t)
|
||||
if err := waitForLeasingExpire(clus.Client(1), "foo/abc"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
waitForExpireAck(t, lkv)
|
||||
|
||||
ctx, cancel := context.WithCancel(context.TODO())
|
||||
errc := make(chan error, 1)
|
||||
go func() { errc <- tests[i](ctx, lkv) }()
|
||||
// some delay to get past for ctx.Err() != nil {} loops
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
cancel()
|
||||
|
||||
select {
|
||||
case err := <-errc:
|
||||
if err != ctx.Err() {
|
||||
t.Errorf("#%d: expected %v, got %v", i, ctx.Err(), err)
|
||||
if _, err = lkv.Get(context.TODO(), "abc"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
case <-time.After(5 * time.Second):
|
||||
t.Errorf("#%d: timed out waiting for cancel", i)
|
||||
}
|
||||
clus.Members[0].Restart(t)
|
||||
|
||||
// down endpoint lkv uses for keepalives
|
||||
clus.Members[0].Stop(t)
|
||||
if err := waitForLeasingExpire(clus.Client(1), "foo/abc"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
waitForExpireAck(t, lkv)
|
||||
|
||||
ctx, cancel := context.WithCancel(context.TODO())
|
||||
errc := make(chan error, 1)
|
||||
go func() { errc <- tests[i](ctx, lkv) }()
|
||||
// some delay to get past for ctx.Err() != nil {} loops
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
cancel()
|
||||
|
||||
select {
|
||||
case err := <-errc:
|
||||
if err != ctx.Err() {
|
||||
t.Errorf("#%d: expected %v of server unavailable, got %v", i, ctx.Err(), err)
|
||||
}
|
||||
case <-time.After(5 * time.Second):
|
||||
t.Errorf("#%d: timed out waiting for cancel", i)
|
||||
}
|
||||
clus.Members[0].Restart(t)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@ -2016,6 +2018,8 @@ func waitForExpireAck(t *testing.T, kv clientv3.KV) {
|
||||
cancel()
|
||||
if err == ctx.Err() {
|
||||
return
|
||||
} else if err != nil {
|
||||
t.Logf("current error: %v", err)
|
||||
}
|
||||
time.Sleep(time.Second)
|
||||
}
|
||||
|
@ -21,7 +21,6 @@ import (
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
@ -131,8 +130,8 @@ func TestMaintenanceSnapshotError(t *testing.T) {
|
||||
time.Sleep(2 * time.Second)
|
||||
|
||||
_, err = io.Copy(ioutil.Discard, rc2)
|
||||
if err != nil && err != context.DeadlineExceeded {
|
||||
t.Errorf("expected %v, got %v", context.DeadlineExceeded, err)
|
||||
if err != nil && !isClientTimeout(err) {
|
||||
t.Errorf("expected client timeout, got %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
@ -157,9 +156,10 @@ func TestMaintenanceSnapshotErrorInflight(t *testing.T) {
|
||||
b.Close()
|
||||
clus.Members[0].Restart(t)
|
||||
|
||||
cli := clus.RandClient()
|
||||
// reading snapshot with canceled context should error out
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
rc1, err := clus.RandClient().Snapshot(ctx)
|
||||
rc1, err := cli.Snapshot(ctx)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -189,7 +189,7 @@ func TestMaintenanceSnapshotErrorInflight(t *testing.T) {
|
||||
// 300ms left and expect timeout while snapshot reading is in progress
|
||||
time.Sleep(700 * time.Millisecond)
|
||||
_, err = io.Copy(ioutil.Discard, rc2)
|
||||
if err != nil && !strings.Contains(err.Error(), context.DeadlineExceeded.Error()) {
|
||||
t.Errorf("expected %v from gRPC, got %v", context.DeadlineExceeded, err)
|
||||
if err != nil && !isClientTimeout(err) {
|
||||
t.Errorf("expected client timeout, got %v", err)
|
||||
}
|
||||
}
|
||||
|
@ -26,6 +26,7 @@ import (
|
||||
"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
"github.com/coreos/etcd/integration"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
var errExpected = errors.New("expected error")
|
||||
@ -36,7 +37,7 @@ var errExpected = errors.New("expected error")
|
||||
func TestBalancerUnderNetworkPartitionPut(t *testing.T) {
|
||||
testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error {
|
||||
_, err := cli.Put(ctx, "a", "b")
|
||||
if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
return errExpected
|
||||
}
|
||||
return err
|
||||
@ -46,7 +47,7 @@ func TestBalancerUnderNetworkPartitionPut(t *testing.T) {
|
||||
func TestBalancerUnderNetworkPartitionDelete(t *testing.T) {
|
||||
testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error {
|
||||
_, err := cli.Delete(ctx, "a")
|
||||
if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
return errExpected
|
||||
}
|
||||
return err
|
||||
@ -59,7 +60,7 @@ func TestBalancerUnderNetworkPartitionTxn(t *testing.T) {
|
||||
If(clientv3.Compare(clientv3.Version("foo"), "=", 0)).
|
||||
Then(clientv3.OpPut("foo", "bar")).
|
||||
Else(clientv3.OpPut("foo", "baz")).Commit()
|
||||
if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout {
|
||||
return errExpected
|
||||
}
|
||||
return err
|
||||
@ -72,6 +73,9 @@ func TestBalancerUnderNetworkPartitionTxn(t *testing.T) {
|
||||
func TestBalancerUnderNetworkPartitionLinearizableGetWithLongTimeout(t *testing.T) {
|
||||
testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error {
|
||||
_, err := cli.Get(ctx, "a")
|
||||
if err == rpctypes.ErrTimeout {
|
||||
return errExpected
|
||||
}
|
||||
return err
|
||||
}, 7*time.Second)
|
||||
}
|
||||
@ -82,7 +86,7 @@ func TestBalancerUnderNetworkPartitionLinearizableGetWithLongTimeout(t *testing.
|
||||
func TestBalancerUnderNetworkPartitionLinearizableGetWithShortTimeout(t *testing.T) {
|
||||
testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error {
|
||||
_, err := cli.Get(ctx, "a")
|
||||
if err == context.DeadlineExceeded || isServerCtxTimeout(err) {
|
||||
if isClientTimeout(err) || isServerCtxTimeout(err) {
|
||||
return errExpected
|
||||
}
|
||||
return err
|
||||
@ -111,6 +115,7 @@ func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, c
|
||||
ccfg := clientv3.Config{
|
||||
Endpoints: []string{eps[0]},
|
||||
DialTimeout: 3 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
}
|
||||
cli, err := clientv3.New(ccfg)
|
||||
if err != nil {
|
||||
@ -123,9 +128,10 @@ func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, c
|
||||
|
||||
// add other endpoints for later endpoint switch
|
||||
cli.SetEndpoints(eps...)
|
||||
time.Sleep(time.Second * 2)
|
||||
clus.Members[0].InjectPartition(t, clus.Members[1:]...)
|
||||
|
||||
for i := 0; i < 2; i++ {
|
||||
for i := 0; i < 5; i++ {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), timeout)
|
||||
err = op(cli, ctx)
|
||||
cancel()
|
||||
@ -133,7 +139,7 @@ func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, c
|
||||
break
|
||||
}
|
||||
if err != errExpected {
|
||||
t.Errorf("#%d: expected %v, got %v", i, errExpected, err)
|
||||
t.Errorf("#%d: expected '%v', got '%v'", i, errExpected, err)
|
||||
}
|
||||
// give enough time for endpoint switch
|
||||
// TODO: remove random sleep by syncing directly with balancer
|
||||
@ -165,16 +171,14 @@ func TestBalancerUnderNetworkPartitionLinearizableGetLeaderElection(t *testing.T
|
||||
|
||||
cli, err := clientv3.New(clientv3.Config{
|
||||
Endpoints: []string{eps[(lead+1)%2]},
|
||||
DialTimeout: 1 * time.Second,
|
||||
DialTimeout: 2 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer cli.Close()
|
||||
|
||||
// wait for non-leader to be pinned
|
||||
mustWaitPinReady(t, cli)
|
||||
|
||||
// add all eps to list, so that when the original pined one fails
|
||||
// the client can switch to other available eps
|
||||
cli.SetEndpoints(eps[lead], eps[(lead+1)%2])
|
||||
@ -182,10 +186,15 @@ func TestBalancerUnderNetworkPartitionLinearizableGetLeaderElection(t *testing.T
|
||||
// isolate leader
|
||||
clus.Members[lead].InjectPartition(t, clus.Members[(lead+1)%3], clus.Members[(lead+2)%3])
|
||||
|
||||
// expects balancer endpoint switch while ongoing leader election
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), timeout)
|
||||
_, err = cli.Get(ctx, "a")
|
||||
cancel()
|
||||
// expects balancer to round robin to leader within two attempts
|
||||
for i := 0; i < 2; i++ {
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), timeout)
|
||||
_, err = cli.Get(ctx, "a")
|
||||
cancel()
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -17,7 +17,6 @@ package integration
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"reflect"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
@ -29,6 +28,7 @@ import (
|
||||
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
"google.golang.org/grpc/transport"
|
||||
)
|
||||
|
||||
// TestBalancerUnderServerShutdownWatch expects that watch client
|
||||
@ -105,7 +105,7 @@ func TestBalancerUnderServerShutdownWatch(t *testing.T) {
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout || err == rpctypes.ErrTimeoutDueToLeaderFail {
|
||||
if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout || err == rpctypes.ErrTimeoutDueToLeaderFail {
|
||||
continue
|
||||
}
|
||||
t.Fatal(err)
|
||||
@ -338,14 +338,20 @@ func testBalancerUnderServerStopInflightRangeOnRestart(t *testing.T, linearizabl
|
||||
defer close(donec)
|
||||
ctx, cancel := context.WithTimeout(context.TODO(), clientTimeout)
|
||||
readyc <- struct{}{}
|
||||
_, err := cli.Get(ctx, "abc", gops...)
|
||||
|
||||
// TODO: The new grpc load balancer will not pin to an endpoint
|
||||
// as intended by this test. But it will round robin member within
|
||||
// two attempts.
|
||||
// Remove retry loop once the new grpc load balancer provides retry.
|
||||
for i := 0; i < 2; i++ {
|
||||
_, err = cli.Get(ctx, "abc", gops...)
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
cancel()
|
||||
if err != nil {
|
||||
if linearizable && strings.Contains(err.Error(), "context deadline exceeded") {
|
||||
t.Logf("TODO: FIX THIS after balancer rewrite! %v %v", reflect.TypeOf(err), err)
|
||||
} else {
|
||||
t.Fatal(err)
|
||||
}
|
||||
t.Fatalf("unexpected error: %v", err)
|
||||
}
|
||||
}()
|
||||
|
||||
@ -373,3 +379,50 @@ func isServerCtxTimeout(err error) bool {
|
||||
code := ev.Code()
|
||||
return code == codes.DeadlineExceeded && strings.Contains(err.Error(), "context deadline exceeded")
|
||||
}
|
||||
|
||||
// In grpc v1.11.3+ dial timeouts can error out with transport.ErrConnClosing. Previously dial timeouts
|
||||
// would always error out with context.DeadlineExceeded.
|
||||
func isClientTimeout(err error) bool {
|
||||
if err == nil {
|
||||
return false
|
||||
}
|
||||
if err == context.DeadlineExceeded {
|
||||
return true
|
||||
}
|
||||
ev, ok := status.FromError(err)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
code := ev.Code()
|
||||
return code == codes.DeadlineExceeded || ev.Message() == transport.ErrConnClosing.Desc
|
||||
}
|
||||
|
||||
func isCanceled(err error) bool {
|
||||
if err == nil {
|
||||
return false
|
||||
}
|
||||
if err == context.Canceled {
|
||||
return true
|
||||
}
|
||||
ev, ok := status.FromError(err)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
code := ev.Code()
|
||||
return code == codes.Canceled
|
||||
}
|
||||
|
||||
func isUnavailable(err error) bool {
|
||||
if err == nil {
|
||||
return false
|
||||
}
|
||||
if err == context.Canceled {
|
||||
return true
|
||||
}
|
||||
ev, ok := status.FromError(err)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
code := ev.Code()
|
||||
return code == codes.Unavailable
|
||||
}
|
||||
|
@ -90,7 +90,7 @@ func TestTxnWriteFail(t *testing.T) {
|
||||
}()
|
||||
|
||||
select {
|
||||
case <-time.After(2 * clus.Members[1].ServerConfig.ReqTimeout()):
|
||||
case <-time.After(5 * clus.Members[1].ServerConfig.ReqTimeout()):
|
||||
t.Fatalf("timed out waiting for get")
|
||||
case <-getc:
|
||||
}
|
||||
|
@ -17,11 +17,13 @@ package integration
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
"github.com/coreos/etcd/integration"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
func TestUserError(t *testing.T) {
|
||||
@ -68,7 +70,11 @@ func TestUserErrorAuth(t *testing.T) {
|
||||
}
|
||||
|
||||
// wrong id or password
|
||||
cfg := clientv3.Config{Endpoints: authapi.Endpoints()}
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: authapi.Endpoints(),
|
||||
DialTimeout: 5 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
}
|
||||
cfg.Username, cfg.Password = "wrong-id", "123"
|
||||
if _, err := clientv3.New(cfg); err != rpctypes.ErrAuthFailed {
|
||||
t.Fatalf("expected %v, got %v", rpctypes.ErrAuthFailed, err)
|
||||
|
@ -30,7 +30,6 @@ import (
|
||||
mvccpb "github.com/coreos/etcd/mvcc/mvccpb"
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/metadata"
|
||||
)
|
||||
|
||||
@ -667,8 +666,9 @@ func TestWatchErrConnClosed(t *testing.T) {
|
||||
go func() {
|
||||
defer close(donec)
|
||||
ch := cli.Watch(context.TODO(), "foo")
|
||||
if wr := <-ch; grpc.ErrorDesc(wr.Err()) != grpc.ErrClientConnClosing.Error() {
|
||||
t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, grpc.ErrorDesc(wr.Err()))
|
||||
|
||||
if wr := <-ch; !isCanceled(wr.Err()) {
|
||||
t.Fatalf("expected context canceled, got %v", wr.Err())
|
||||
}
|
||||
}()
|
||||
|
||||
@ -699,8 +699,8 @@ func TestWatchAfterClose(t *testing.T) {
|
||||
donec := make(chan struct{})
|
||||
go func() {
|
||||
cli.Watch(context.TODO(), "foo")
|
||||
if err := cli.Close(); err != nil && err != grpc.ErrClientConnClosing {
|
||||
t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err)
|
||||
if err := cli.Close(); err != nil && err != context.Canceled {
|
||||
t.Fatalf("expected %v, got %v", context.Canceled, err)
|
||||
}
|
||||
close(donec)
|
||||
}()
|
||||
|
@ -466,7 +466,7 @@ func (l *lessor) recvKeepAliveLoop() (gerr error) {
|
||||
// resetRecv opens a new lease stream and starts sending keep alive requests.
|
||||
func (l *lessor) resetRecv() (pb.Lease_LeaseKeepAliveClient, error) {
|
||||
sctx, cancel := context.WithCancel(l.stopCtx)
|
||||
stream, err := l.remote.LeaseKeepAlive(sctx, l.callOpts...)
|
||||
stream, err := l.remote.LeaseKeepAlive(sctx, append(l.callOpts, withMax(0))...)
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, err
|
||||
|
@ -16,6 +16,7 @@ package clientv3
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
@ -77,7 +78,7 @@ func NewMaintenance(c *Client) Maintenance {
|
||||
dial: func(endpoint string) (pb.MaintenanceClient, func(), error) {
|
||||
conn, err := c.dial(endpoint)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
return nil, nil, fmt.Errorf("failed to dial endpoint %s with maintenance client: %v", endpoint, err)
|
||||
}
|
||||
cancel := func() { conn.Close() }
|
||||
return RetryMaintenanceClient(c, conn), cancel, nil
|
||||
@ -175,6 +176,7 @@ func (m *maintenance) Status(ctx context.Context, endpoint string) (*StatusRespo
|
||||
func (m *maintenance) HashKV(ctx context.Context, endpoint string, rev int64) (*HashKVResponse, error) {
|
||||
remote, cancel, err := m.dial(endpoint)
|
||||
if err != nil {
|
||||
|
||||
return nil, toErr(ctx, err)
|
||||
}
|
||||
defer cancel()
|
||||
@ -186,7 +188,7 @@ func (m *maintenance) HashKV(ctx context.Context, endpoint string, rev int64) (*
|
||||
}
|
||||
|
||||
func (m *maintenance) Snapshot(ctx context.Context) (io.ReadCloser, error) {
|
||||
ss, err := m.remote.Snapshot(ctx, &pb.SnapshotRequest{}, m.callOpts...)
|
||||
ss, err := m.remote.Snapshot(ctx, &pb.SnapshotRequest{}, append(m.callOpts, withMax(defaultStreamMaxRetries))...)
|
||||
if err != nil {
|
||||
return nil, toErr(ctx, err)
|
||||
}
|
||||
|
@ -16,17 +16,17 @@ package clientv3
|
||||
|
||||
import (
|
||||
"math"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
var (
|
||||
// Disable gRPC internal retrial logic
|
||||
// TODO: enable when gRPC retry is stable (FailFast=false)
|
||||
// Reference:
|
||||
// - https://github.com/grpc/grpc-go/issues/1532
|
||||
// - https://github.com/grpc/proposal/blob/master/A6-client-retries.md
|
||||
defaultFailFast = grpc.FailFast(true)
|
||||
// client-side handling retrying of request failures where data was not written to the wire or
|
||||
// where server indicates it did not process the data. gRPC default is default is "FailFast(true)"
|
||||
// but for etcd we default to "FailFast(false)" to minimize client request error responses due to
|
||||
// transient failures.
|
||||
defaultFailFast = grpc.FailFast(false)
|
||||
|
||||
// client-side request send limit, gRPC default is math.MaxInt32
|
||||
// Make sure that "client-side send limit < server-side default send/recv limit"
|
||||
@ -38,6 +38,22 @@ var (
|
||||
// because range response can easily exceed request send limits
|
||||
// Default to math.MaxInt32; writes exceeding server-side send limit fails anyway
|
||||
defaultMaxCallRecvMsgSize = grpc.MaxCallRecvMsgSize(math.MaxInt32)
|
||||
|
||||
// client-side non-streaming retry limit, only applied to requests where server responds with
|
||||
// a error code clearly indicating it was unable to process the request such as codes.Unavailable.
|
||||
// If set to 0, retry is disabled.
|
||||
defaultUnaryMaxRetries uint = 100
|
||||
|
||||
// client-side streaming retry limit, only applied to requests where server responds with
|
||||
// a error code clearly indicating it was unable to process the request such as codes.Unavailable.
|
||||
// If set to 0, retry is disabled.
|
||||
defaultStreamMaxRetries uint = ^uint(0) // max uint
|
||||
|
||||
// client-side retry backoff wait between requests.
|
||||
defaultBackoffWaitBetween = 25 * time.Millisecond
|
||||
|
||||
// client-side retry backoff default jitter fraction.
|
||||
defaultBackoffJitterFraction = 0.10
|
||||
)
|
||||
|
||||
// defaultCallOpts defines a list of default "gRPC.CallOption".
|
||||
|
@ -82,6 +82,7 @@ func TestDetectKvOrderViolation(t *testing.T) {
|
||||
clus.Members[2].Restart(t)
|
||||
// force OrderingKv to query the third member
|
||||
cli.SetEndpoints(clus.Members[2].GRPCAddr())
|
||||
time.Sleep(2 * time.Second) // FIXME: Figure out how pause SetEndpoints sufficiently that this is not needed
|
||||
|
||||
_, err = orderingKv.Get(ctx, "foo", clientv3.WithSerializable())
|
||||
if err != errOrderViolation {
|
||||
@ -147,7 +148,7 @@ func TestDetectTxnOrderViolation(t *testing.T) {
|
||||
clus.Members[2].Restart(t)
|
||||
// force OrderingKv to query the third member
|
||||
cli.SetEndpoints(clus.Members[2].GRPCAddr())
|
||||
|
||||
time.Sleep(2 * time.Second) // FIXME: Figure out how pause SetEndpoints sufficiently that this is not needed
|
||||
_, err = orderingKv.Get(ctx, "foo", clientv3.WithSerializable())
|
||||
if err != errOrderViolation {
|
||||
t.Fatalf("expected %v, got %v", errOrderViolation, err)
|
||||
|
@ -43,6 +43,8 @@ func NewOrderViolationSwitchEndpointClosure(c clientv3.Client) OrderViolationFun
|
||||
// set available endpoints back to all endpoints in to ensure
|
||||
// the client has access to all the endpoints.
|
||||
c.SetEndpoints(eps...)
|
||||
// give enough time for operation
|
||||
time.Sleep(1 * time.Second)
|
||||
violationCount++
|
||||
return nil
|
||||
}
|
||||
|
@ -122,6 +122,7 @@ func TestUnresolvableOrderViolation(t *testing.T) {
|
||||
// NewOrderViolationSwitchEndpointClosure will be able to
|
||||
// access the full list of endpoints.
|
||||
cli.SetEndpoints(eps...)
|
||||
time.Sleep(1 * time.Second) // give enough time for operation
|
||||
OrderingKv := NewKV(cli.KV, NewOrderViolationSwitchEndpointClosure(*cli))
|
||||
// set prevRev to the first member's revision of "foo" such that
|
||||
// the revision is higher than the fourth and fifth members' revision of "foo"
|
||||
@ -133,10 +134,17 @@ func TestUnresolvableOrderViolation(t *testing.T) {
|
||||
clus.Members[0].Stop(t)
|
||||
clus.Members[1].Stop(t)
|
||||
clus.Members[2].Stop(t)
|
||||
clus.Members[3].Restart(t)
|
||||
clus.Members[4].Restart(t)
|
||||
err = clus.Members[3].Restart(t)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
err = clus.Members[4].Restart(t)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
clus.Members[3].WaitStarted(t)
|
||||
cli.SetEndpoints(clus.Members[3].GRPCAddr())
|
||||
time.Sleep(1 * time.Second) // give enough time for operation
|
||||
time.Sleep(5 * time.Second) // give enough time for operation
|
||||
|
||||
_, err = OrderingKv.Get(ctx, "foo", clientv3.WithSerializable())
|
||||
if err != ErrNoGreaterRev {
|
||||
|
@ -32,467 +32,267 @@ const (
|
||||
nonRepeatable
|
||||
)
|
||||
|
||||
func (rp retryPolicy) String() string {
|
||||
switch rp {
|
||||
case repeatable:
|
||||
return "repeatable"
|
||||
case nonRepeatable:
|
||||
return "nonRepeatable"
|
||||
default:
|
||||
return "UNKNOWN"
|
||||
}
|
||||
}
|
||||
|
||||
type rpcFunc func(ctx context.Context) error
|
||||
type retryRPCFunc func(context.Context, rpcFunc, retryPolicy) error
|
||||
type retryStopErrFunc func(error) bool
|
||||
|
||||
// isSafeRetryImmutableRPC returns "true" when an immutable request is safe for retry.
|
||||
//
|
||||
// immutable requests (e.g. Get) should be retried unless it's
|
||||
// an obvious server-side error (e.g. rpctypes.ErrRequestTooLarge).
|
||||
//
|
||||
// "isRepeatableStopError" returns "true" when an immutable request
|
||||
// is interrupted by server-side or gRPC-side error and its status
|
||||
// code is not transient (!= codes.Unavailable).
|
||||
//
|
||||
// Returning "true" means retry should stop, since client cannot
|
||||
// Returning "false" means retry should stop, since client cannot
|
||||
// handle itself even with retries.
|
||||
func isRepeatableStopError(err error) bool {
|
||||
func isSafeRetryImmutableRPC(err error) bool {
|
||||
eErr := rpctypes.Error(err)
|
||||
// always stop retry on etcd errors
|
||||
if serverErr, ok := eErr.(rpctypes.EtcdError); ok && serverErr.Code() != codes.Unavailable {
|
||||
return true
|
||||
// interrupted by non-transient server-side or gRPC-side error
|
||||
// client cannot handle itself (e.g. rpctypes.ErrCompacted)
|
||||
return false
|
||||
}
|
||||
// only retry if unavailable
|
||||
ev, ok := status.FromError(err)
|
||||
if !ok {
|
||||
// all errors from RPC is typed "grpc/status.(*statusError)"
|
||||
// (ref. https://github.com/grpc/grpc-go/pull/1782)
|
||||
//
|
||||
// if the error type is not "grpc/status.(*statusError)",
|
||||
// it could be from "Dial"
|
||||
// TODO: do not retry for now
|
||||
// ref. https://github.com/grpc/grpc-go/issues/1581
|
||||
return false
|
||||
}
|
||||
return ev.Code() != codes.Unavailable
|
||||
return ev.Code() == codes.Unavailable
|
||||
}
|
||||
|
||||
// isSafeRetryMutableRPC returns "true" when a mutable request is safe for retry.
|
||||
//
|
||||
// mutable requests (e.g. Put, Delete, Txn) should only be retried
|
||||
// when the status code is codes.Unavailable when initial connection
|
||||
// has not been established (no pinned endpoint).
|
||||
// has not been established (no endpoint is up).
|
||||
//
|
||||
// "isNonRepeatableStopError" returns "true" when a mutable request
|
||||
// is interrupted by non-transient error that client cannot handle itself,
|
||||
// or transient error while the connection has already been established
|
||||
// (pinned endpoint exists).
|
||||
//
|
||||
// Returning "true" means retry should stop, otherwise it violates
|
||||
// Returning "false" means retry should stop, otherwise it violates
|
||||
// write-at-most-once semantics.
|
||||
func isNonRepeatableStopError(err error) bool {
|
||||
func isSafeRetryMutableRPC(err error) bool {
|
||||
if ev, ok := status.FromError(err); ok && ev.Code() != codes.Unavailable {
|
||||
return true
|
||||
// not safe for mutable RPCs
|
||||
// e.g. interrupted by non-transient error that client cannot handle itself,
|
||||
// or transient error while the connection has already been established
|
||||
return false
|
||||
}
|
||||
desc := rpctypes.ErrorDesc(err)
|
||||
return desc != "there is no address available" && desc != "there is no connection available"
|
||||
}
|
||||
|
||||
func (c *Client) newRetryWrapper() retryRPCFunc {
|
||||
return func(rpcCtx context.Context, f rpcFunc, rp retryPolicy) error {
|
||||
var isStop retryStopErrFunc
|
||||
switch rp {
|
||||
case repeatable:
|
||||
isStop = isRepeatableStopError
|
||||
case nonRepeatable:
|
||||
isStop = isNonRepeatableStopError
|
||||
}
|
||||
for {
|
||||
if err := readyWait(rpcCtx, c.ctx, c.balancer.ConnectNotify()); err != nil {
|
||||
return err
|
||||
}
|
||||
pinned := c.balancer.Pinned()
|
||||
err := f(rpcCtx)
|
||||
if err == nil {
|
||||
return nil
|
||||
}
|
||||
lg.Lvl(4).Infof("clientv3/retry: error %q on pinned endpoint %q", err.Error(), pinned)
|
||||
|
||||
if s, ok := status.FromError(err); ok && (s.Code() == codes.Unavailable || s.Code() == codes.DeadlineExceeded || s.Code() == codes.Internal) {
|
||||
// mark this before endpoint switch is triggered
|
||||
c.balancer.HostPortError(pinned, err)
|
||||
c.balancer.Next()
|
||||
lg.Lvl(4).Infof("clientv3/retry: switching from %q due to error %q", pinned, err.Error())
|
||||
}
|
||||
|
||||
if isStop(err) {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Client) newAuthRetryWrapper(retryf retryRPCFunc) retryRPCFunc {
|
||||
return func(rpcCtx context.Context, f rpcFunc, rp retryPolicy) error {
|
||||
for {
|
||||
pinned := c.balancer.Pinned()
|
||||
err := retryf(rpcCtx, f, rp)
|
||||
if err == nil {
|
||||
return nil
|
||||
}
|
||||
lg.Lvl(4).Infof("clientv3/auth-retry: error %q on pinned endpoint %q", err.Error(), pinned)
|
||||
// always stop retry on etcd errors other than invalid auth token
|
||||
if rpctypes.Error(err) == rpctypes.ErrInvalidAuthToken {
|
||||
gterr := c.getToken(rpcCtx)
|
||||
if gterr != nil {
|
||||
lg.Lvl(4).Infof("clientv3/auth-retry: cannot retry due to error %q(%q) on pinned endpoint %q", err.Error(), gterr.Error(), pinned)
|
||||
return err // return the original error for simplicity
|
||||
}
|
||||
continue
|
||||
}
|
||||
return err
|
||||
}
|
||||
}
|
||||
return desc == "there is no address available" || desc == "there is no connection available"
|
||||
}
|
||||
|
||||
type retryKVClient struct {
|
||||
kc pb.KVClient
|
||||
retryf retryRPCFunc
|
||||
kc pb.KVClient
|
||||
}
|
||||
|
||||
// RetryKVClient implements a KVClient.
|
||||
func RetryKVClient(c *Client) pb.KVClient {
|
||||
return &retryKVClient{
|
||||
kc: pb.NewKVClient(c.conn),
|
||||
retryf: c.newAuthRetryWrapper(c.newRetryWrapper()),
|
||||
kc: pb.NewKVClient(c.conn),
|
||||
}
|
||||
}
|
||||
func (rkv *retryKVClient) Range(ctx context.Context, in *pb.RangeRequest, opts ...grpc.CallOption) (resp *pb.RangeResponse, err error) {
|
||||
err = rkv.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rkv.kc.Range(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rkv.kc.Range(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rkv *retryKVClient) Put(ctx context.Context, in *pb.PutRequest, opts ...grpc.CallOption) (resp *pb.PutResponse, err error) {
|
||||
err = rkv.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rkv.kc.Put(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rkv.kc.Put(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rkv *retryKVClient) DeleteRange(ctx context.Context, in *pb.DeleteRangeRequest, opts ...grpc.CallOption) (resp *pb.DeleteRangeResponse, err error) {
|
||||
err = rkv.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rkv.kc.DeleteRange(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rkv.kc.DeleteRange(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rkv *retryKVClient) Txn(ctx context.Context, in *pb.TxnRequest, opts ...grpc.CallOption) (resp *pb.TxnResponse, err error) {
|
||||
// TODO: "repeatable" for read-only txn
|
||||
err = rkv.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rkv.kc.Txn(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rkv.kc.Txn(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rkv *retryKVClient) Compact(ctx context.Context, in *pb.CompactionRequest, opts ...grpc.CallOption) (resp *pb.CompactionResponse, err error) {
|
||||
err = rkv.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rkv.kc.Compact(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rkv.kc.Compact(ctx, in, opts...)
|
||||
}
|
||||
|
||||
type retryLeaseClient struct {
|
||||
lc pb.LeaseClient
|
||||
retryf retryRPCFunc
|
||||
lc pb.LeaseClient
|
||||
}
|
||||
|
||||
// RetryLeaseClient implements a LeaseClient.
|
||||
func RetryLeaseClient(c *Client) pb.LeaseClient {
|
||||
return &retryLeaseClient{
|
||||
lc: pb.NewLeaseClient(c.conn),
|
||||
retryf: c.newAuthRetryWrapper(c.newRetryWrapper()),
|
||||
lc: pb.NewLeaseClient(c.conn),
|
||||
}
|
||||
}
|
||||
|
||||
func (rlc *retryLeaseClient) LeaseTimeToLive(ctx context.Context, in *pb.LeaseTimeToLiveRequest, opts ...grpc.CallOption) (resp *pb.LeaseTimeToLiveResponse, err error) {
|
||||
err = rlc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rlc.lc.LeaseTimeToLive(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rlc.lc.LeaseTimeToLive(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rlc *retryLeaseClient) LeaseLeases(ctx context.Context, in *pb.LeaseLeasesRequest, opts ...grpc.CallOption) (resp *pb.LeaseLeasesResponse, err error) {
|
||||
err = rlc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rlc.lc.LeaseLeases(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rlc.lc.LeaseLeases(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rlc *retryLeaseClient) LeaseGrant(ctx context.Context, in *pb.LeaseGrantRequest, opts ...grpc.CallOption) (resp *pb.LeaseGrantResponse, err error) {
|
||||
err = rlc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rlc.lc.LeaseGrant(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
|
||||
return rlc.lc.LeaseGrant(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rlc *retryLeaseClient) LeaseRevoke(ctx context.Context, in *pb.LeaseRevokeRequest, opts ...grpc.CallOption) (resp *pb.LeaseRevokeResponse, err error) {
|
||||
err = rlc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rlc.lc.LeaseRevoke(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rlc.lc.LeaseRevoke(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rlc *retryLeaseClient) LeaseKeepAlive(ctx context.Context, opts ...grpc.CallOption) (stream pb.Lease_LeaseKeepAliveClient, err error) {
|
||||
err = rlc.retryf(ctx, func(rctx context.Context) error {
|
||||
stream, err = rlc.lc.LeaseKeepAlive(rctx, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return stream, err
|
||||
return rlc.lc.LeaseKeepAlive(ctx, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
type retryClusterClient struct {
|
||||
cc pb.ClusterClient
|
||||
retryf retryRPCFunc
|
||||
cc pb.ClusterClient
|
||||
}
|
||||
|
||||
// RetryClusterClient implements a ClusterClient.
|
||||
func RetryClusterClient(c *Client) pb.ClusterClient {
|
||||
return &retryClusterClient{
|
||||
cc: pb.NewClusterClient(c.conn),
|
||||
retryf: c.newRetryWrapper(),
|
||||
cc: pb.NewClusterClient(c.conn),
|
||||
}
|
||||
}
|
||||
|
||||
func (rcc *retryClusterClient) MemberList(ctx context.Context, in *pb.MemberListRequest, opts ...grpc.CallOption) (resp *pb.MemberListResponse, err error) {
|
||||
err = rcc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rcc.cc.MemberList(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rcc.cc.MemberList(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rcc *retryClusterClient) MemberAdd(ctx context.Context, in *pb.MemberAddRequest, opts ...grpc.CallOption) (resp *pb.MemberAddResponse, err error) {
|
||||
err = rcc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rcc.cc.MemberAdd(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rcc.cc.MemberAdd(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rcc *retryClusterClient) MemberRemove(ctx context.Context, in *pb.MemberRemoveRequest, opts ...grpc.CallOption) (resp *pb.MemberRemoveResponse, err error) {
|
||||
err = rcc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rcc.cc.MemberRemove(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rcc.cc.MemberRemove(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rcc *retryClusterClient) MemberUpdate(ctx context.Context, in *pb.MemberUpdateRequest, opts ...grpc.CallOption) (resp *pb.MemberUpdateResponse, err error) {
|
||||
err = rcc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rcc.cc.MemberUpdate(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rcc.cc.MemberUpdate(ctx, in, opts...)
|
||||
}
|
||||
|
||||
type retryMaintenanceClient struct {
|
||||
mc pb.MaintenanceClient
|
||||
retryf retryRPCFunc
|
||||
mc pb.MaintenanceClient
|
||||
}
|
||||
|
||||
// RetryMaintenanceClient implements a Maintenance.
|
||||
func RetryMaintenanceClient(c *Client, conn *grpc.ClientConn) pb.MaintenanceClient {
|
||||
return &retryMaintenanceClient{
|
||||
mc: pb.NewMaintenanceClient(conn),
|
||||
retryf: c.newRetryWrapper(),
|
||||
mc: pb.NewMaintenanceClient(conn),
|
||||
}
|
||||
}
|
||||
|
||||
func (rmc *retryMaintenanceClient) Alarm(ctx context.Context, in *pb.AlarmRequest, opts ...grpc.CallOption) (resp *pb.AlarmResponse, err error) {
|
||||
err = rmc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rmc.mc.Alarm(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rmc.mc.Alarm(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rmc *retryMaintenanceClient) Status(ctx context.Context, in *pb.StatusRequest, opts ...grpc.CallOption) (resp *pb.StatusResponse, err error) {
|
||||
err = rmc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rmc.mc.Status(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rmc.mc.Status(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rmc *retryMaintenanceClient) Hash(ctx context.Context, in *pb.HashRequest, opts ...grpc.CallOption) (resp *pb.HashResponse, err error) {
|
||||
err = rmc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rmc.mc.Hash(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rmc.mc.Hash(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rmc *retryMaintenanceClient) HashKV(ctx context.Context, in *pb.HashKVRequest, opts ...grpc.CallOption) (resp *pb.HashKVResponse, err error) {
|
||||
err = rmc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rmc.mc.HashKV(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rmc.mc.HashKV(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rmc *retryMaintenanceClient) Snapshot(ctx context.Context, in *pb.SnapshotRequest, opts ...grpc.CallOption) (stream pb.Maintenance_SnapshotClient, err error) {
|
||||
err = rmc.retryf(ctx, func(rctx context.Context) error {
|
||||
stream, err = rmc.mc.Snapshot(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return stream, err
|
||||
return rmc.mc.Snapshot(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rmc *retryMaintenanceClient) MoveLeader(ctx context.Context, in *pb.MoveLeaderRequest, opts ...grpc.CallOption) (resp *pb.MoveLeaderResponse, err error) {
|
||||
err = rmc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rmc.mc.MoveLeader(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rmc.mc.MoveLeader(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rmc *retryMaintenanceClient) Defragment(ctx context.Context, in *pb.DefragmentRequest, opts ...grpc.CallOption) (resp *pb.DefragmentResponse, err error) {
|
||||
err = rmc.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rmc.mc.Defragment(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rmc.mc.Defragment(ctx, in, opts...)
|
||||
}
|
||||
|
||||
type retryAuthClient struct {
|
||||
ac pb.AuthClient
|
||||
retryf retryRPCFunc
|
||||
ac pb.AuthClient
|
||||
}
|
||||
|
||||
// RetryAuthClient implements a AuthClient.
|
||||
func RetryAuthClient(c *Client) pb.AuthClient {
|
||||
return &retryAuthClient{
|
||||
ac: pb.NewAuthClient(c.conn),
|
||||
retryf: c.newRetryWrapper(),
|
||||
ac: pb.NewAuthClient(c.conn),
|
||||
}
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserList(ctx context.Context, in *pb.AuthUserListRequest, opts ...grpc.CallOption) (resp *pb.AuthUserListResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.UserList(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rac.ac.UserList(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserGet(ctx context.Context, in *pb.AuthUserGetRequest, opts ...grpc.CallOption) (resp *pb.AuthUserGetResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.UserGet(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rac.ac.UserGet(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) RoleGet(ctx context.Context, in *pb.AuthRoleGetRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleGetResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.RoleGet(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rac.ac.RoleGet(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) RoleList(ctx context.Context, in *pb.AuthRoleListRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleListResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.RoleList(rctx, in, opts...)
|
||||
return err
|
||||
}, repeatable)
|
||||
return resp, err
|
||||
return rac.ac.RoleList(ctx, in, append(opts, withRetryPolicy(repeatable))...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) AuthEnable(ctx context.Context, in *pb.AuthEnableRequest, opts ...grpc.CallOption) (resp *pb.AuthEnableResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.AuthEnable(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.AuthEnable(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) AuthDisable(ctx context.Context, in *pb.AuthDisableRequest, opts ...grpc.CallOption) (resp *pb.AuthDisableResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.AuthDisable(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.AuthDisable(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserAdd(ctx context.Context, in *pb.AuthUserAddRequest, opts ...grpc.CallOption) (resp *pb.AuthUserAddResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.UserAdd(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.UserAdd(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserDelete(ctx context.Context, in *pb.AuthUserDeleteRequest, opts ...grpc.CallOption) (resp *pb.AuthUserDeleteResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.UserDelete(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.UserDelete(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserChangePassword(ctx context.Context, in *pb.AuthUserChangePasswordRequest, opts ...grpc.CallOption) (resp *pb.AuthUserChangePasswordResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.UserChangePassword(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.UserChangePassword(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserGrantRole(ctx context.Context, in *pb.AuthUserGrantRoleRequest, opts ...grpc.CallOption) (resp *pb.AuthUserGrantRoleResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.UserGrantRole(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.UserGrantRole(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) UserRevokeRole(ctx context.Context, in *pb.AuthUserRevokeRoleRequest, opts ...grpc.CallOption) (resp *pb.AuthUserRevokeRoleResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.UserRevokeRole(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.UserRevokeRole(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) RoleAdd(ctx context.Context, in *pb.AuthRoleAddRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleAddResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.RoleAdd(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.RoleAdd(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) RoleDelete(ctx context.Context, in *pb.AuthRoleDeleteRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleDeleteResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.RoleDelete(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.RoleDelete(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) RoleGrantPermission(ctx context.Context, in *pb.AuthRoleGrantPermissionRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleGrantPermissionResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.RoleGrantPermission(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.RoleGrantPermission(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) RoleRevokePermission(ctx context.Context, in *pb.AuthRoleRevokePermissionRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleRevokePermissionResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.RoleRevokePermission(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.RoleRevokePermission(ctx, in, opts...)
|
||||
}
|
||||
|
||||
func (rac *retryAuthClient) Authenticate(ctx context.Context, in *pb.AuthenticateRequest, opts ...grpc.CallOption) (resp *pb.AuthenticateResponse, err error) {
|
||||
err = rac.retryf(ctx, func(rctx context.Context) error {
|
||||
resp, err = rac.ac.Authenticate(rctx, in, opts...)
|
||||
return err
|
||||
}, nonRepeatable)
|
||||
return resp, err
|
||||
return rac.ac.Authenticate(ctx, in, opts...)
|
||||
}
|
||||
|
382
clientv3/retry_interceptor.go
Normal file
@ -0,0 +1,382 @@
|
||||
// Copyright 2016 The etcd 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.
|
||||
|
||||
// Based on github.com/grpc-ecosystem/go-grpc-middleware/retry, but modified to support the more
|
||||
// fine grained error checking required by write-at-most-once retry semantics of etcd.
|
||||
|
||||
package clientv3
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
"github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/metadata"
|
||||
)
|
||||
|
||||
// unaryClientInterceptor returns a new retrying unary client interceptor.
|
||||
//
|
||||
// The default configuration of the interceptor is to not retry *at all*. This behaviour can be
|
||||
// changed through options (e.g. WithMax) on creation of the interceptor or on call (through grpc.CallOptions).
|
||||
func (c *Client) unaryClientInterceptor(logger *zap.Logger, optFuncs ...retryOption) grpc.UnaryClientInterceptor {
|
||||
intOpts := reuseOrNewWithCallOptions(defaultOptions, optFuncs)
|
||||
return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
|
||||
grpcOpts, retryOpts := filterCallOptions(opts)
|
||||
callOpts := reuseOrNewWithCallOptions(intOpts, retryOpts)
|
||||
// short circuit for simplicity, and avoiding allocations.
|
||||
if callOpts.max == 0 {
|
||||
return invoker(ctx, method, req, reply, cc, grpcOpts...)
|
||||
}
|
||||
var lastErr error
|
||||
for attempt := uint(0); attempt < callOpts.max; attempt++ {
|
||||
if err := waitRetryBackoff(attempt, ctx, callOpts); err != nil {
|
||||
return err
|
||||
}
|
||||
lastErr = invoker(ctx, method, req, reply, cc, grpcOpts...)
|
||||
logger.Info("retry unary intercept", zap.Uint("attempt", attempt), zap.Error(lastErr))
|
||||
if lastErr == nil {
|
||||
return nil
|
||||
}
|
||||
if isContextError(lastErr) {
|
||||
if ctx.Err() != nil {
|
||||
// its the context deadline or cancellation.
|
||||
return lastErr
|
||||
}
|
||||
// its the callCtx deadline or cancellation, in which case try again.
|
||||
continue
|
||||
}
|
||||
if callOpts.retryAuth && rpctypes.Error(lastErr) == rpctypes.ErrInvalidAuthToken {
|
||||
gterr := c.getToken(ctx)
|
||||
if gterr != nil {
|
||||
logger.Info("retry failed to fetch new auth token", zap.Error(gterr))
|
||||
return lastErr // return the original error for simplicity
|
||||
}
|
||||
continue
|
||||
}
|
||||
if !isSafeRetry(c.lg, lastErr, callOpts) {
|
||||
return lastErr
|
||||
}
|
||||
}
|
||||
return lastErr
|
||||
}
|
||||
}
|
||||
|
||||
// streamClientInterceptor returns a new retrying stream client interceptor for server side streaming calls.
|
||||
//
|
||||
// The default configuration of the interceptor is to not retry *at all*. This behaviour can be
|
||||
// changed through options (e.g. WithMax) on creation of the interceptor or on call (through grpc.CallOptions).
|
||||
//
|
||||
// Retry logic is available *only for ServerStreams*, i.e. 1:n streams, as the internal logic needs
|
||||
// to buffer the messages sent by the client. If retry is enabled on any other streams (ClientStreams,
|
||||
// BidiStreams), the retry interceptor will fail the call.
|
||||
func (c *Client) streamClientInterceptor(logger *zap.Logger, optFuncs ...retryOption) grpc.StreamClientInterceptor {
|
||||
intOpts := reuseOrNewWithCallOptions(defaultOptions, optFuncs)
|
||||
return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
|
||||
grpcOpts, retryOpts := filterCallOptions(opts)
|
||||
callOpts := reuseOrNewWithCallOptions(intOpts, retryOpts)
|
||||
// short circuit for simplicity, and avoiding allocations.
|
||||
if callOpts.max == 0 {
|
||||
return streamer(ctx, desc, cc, method, grpcOpts...)
|
||||
}
|
||||
if desc.ClientStreams {
|
||||
return nil, grpc.Errorf(codes.Unimplemented, "clientv3/retry_interceptor: cannot retry on ClientStreams, set Disable()")
|
||||
}
|
||||
newStreamer, err := streamer(ctx, desc, cc, method, grpcOpts...)
|
||||
logger.Info("retry stream intercept", zap.Error(err))
|
||||
if err != nil {
|
||||
// TODO(mwitkow): Maybe dial and transport errors should be retriable?
|
||||
return nil, err
|
||||
}
|
||||
retryingStreamer := &serverStreamingRetryingStream{
|
||||
client: c,
|
||||
ClientStream: newStreamer,
|
||||
callOpts: callOpts,
|
||||
ctx: ctx,
|
||||
streamerCall: func(ctx context.Context) (grpc.ClientStream, error) {
|
||||
return streamer(ctx, desc, cc, method, grpcOpts...)
|
||||
},
|
||||
}
|
||||
return retryingStreamer, nil
|
||||
}
|
||||
}
|
||||
|
||||
// type serverStreamingRetryingStream is the implementation of grpc.ClientStream that acts as a
|
||||
// proxy to the underlying call. If any of the RecvMsg() calls fail, it will try to reestablish
|
||||
// a new ClientStream according to the retry policy.
|
||||
type serverStreamingRetryingStream struct {
|
||||
grpc.ClientStream
|
||||
client *Client
|
||||
bufferedSends []interface{} // single message that the client can sen
|
||||
receivedGood bool // indicates whether any prior receives were successful
|
||||
wasClosedSend bool // indicates that CloseSend was closed
|
||||
ctx context.Context
|
||||
callOpts *options
|
||||
streamerCall func(ctx context.Context) (grpc.ClientStream, error)
|
||||
mu sync.RWMutex
|
||||
}
|
||||
|
||||
func (s *serverStreamingRetryingStream) setStream(clientStream grpc.ClientStream) {
|
||||
s.mu.Lock()
|
||||
s.ClientStream = clientStream
|
||||
s.mu.Unlock()
|
||||
}
|
||||
|
||||
func (s *serverStreamingRetryingStream) getStream() grpc.ClientStream {
|
||||
s.mu.RLock()
|
||||
defer s.mu.RUnlock()
|
||||
return s.ClientStream
|
||||
}
|
||||
|
||||
func (s *serverStreamingRetryingStream) SendMsg(m interface{}) error {
|
||||
s.mu.Lock()
|
||||
s.bufferedSends = append(s.bufferedSends, m)
|
||||
s.mu.Unlock()
|
||||
return s.getStream().SendMsg(m)
|
||||
}
|
||||
|
||||
func (s *serverStreamingRetryingStream) CloseSend() error {
|
||||
s.mu.Lock()
|
||||
s.wasClosedSend = true
|
||||
s.mu.Unlock()
|
||||
return s.getStream().CloseSend()
|
||||
}
|
||||
|
||||
func (s *serverStreamingRetryingStream) Header() (metadata.MD, error) {
|
||||
return s.getStream().Header()
|
||||
}
|
||||
|
||||
func (s *serverStreamingRetryingStream) Trailer() metadata.MD {
|
||||
return s.getStream().Trailer()
|
||||
}
|
||||
|
||||
func (s *serverStreamingRetryingStream) RecvMsg(m interface{}) error {
|
||||
attemptRetry, lastErr := s.receiveMsgAndIndicateRetry(m)
|
||||
if !attemptRetry {
|
||||
return lastErr // success or hard failure
|
||||
}
|
||||
// We start off from attempt 1, because zeroth was already made on normal SendMsg().
|
||||
for attempt := uint(1); attempt < s.callOpts.max; attempt++ {
|
||||
if err := waitRetryBackoff(attempt, s.ctx, s.callOpts); err != nil {
|
||||
return err
|
||||
}
|
||||
newStream, err := s.reestablishStreamAndResendBuffer(s.ctx)
|
||||
if err != nil {
|
||||
// TODO(mwitkow): Maybe dial and transport errors should be retriable?
|
||||
return err
|
||||
}
|
||||
s.setStream(newStream)
|
||||
attemptRetry, lastErr = s.receiveMsgAndIndicateRetry(m)
|
||||
//fmt.Printf("Received message and indicate: %v %v\n", attemptRetry, lastErr)
|
||||
if !attemptRetry {
|
||||
return lastErr
|
||||
}
|
||||
}
|
||||
return lastErr
|
||||
}
|
||||
|
||||
func (s *serverStreamingRetryingStream) receiveMsgAndIndicateRetry(m interface{}) (bool, error) {
|
||||
s.mu.RLock()
|
||||
wasGood := s.receivedGood
|
||||
s.mu.RUnlock()
|
||||
err := s.getStream().RecvMsg(m)
|
||||
if err == nil || err == io.EOF {
|
||||
s.mu.Lock()
|
||||
s.receivedGood = true
|
||||
s.mu.Unlock()
|
||||
return false, err
|
||||
} else if wasGood {
|
||||
// previous RecvMsg in the stream succeeded, no retry logic should interfere
|
||||
return false, err
|
||||
}
|
||||
if isContextError(err) {
|
||||
if s.ctx.Err() != nil {
|
||||
return false, err
|
||||
}
|
||||
// its the callCtx deadline or cancellation, in which case try again.
|
||||
return true, err
|
||||
}
|
||||
if s.callOpts.retryAuth && rpctypes.Error(err) == rpctypes.ErrInvalidAuthToken {
|
||||
gterr := s.client.getToken(s.ctx)
|
||||
if gterr != nil {
|
||||
s.client.lg.Info("retry failed to fetch new auth token", zap.Error(gterr))
|
||||
return false, err // return the original error for simplicity
|
||||
}
|
||||
return true, err
|
||||
|
||||
}
|
||||
return isSafeRetry(s.client.lg, err, s.callOpts), err
|
||||
}
|
||||
|
||||
func (s *serverStreamingRetryingStream) reestablishStreamAndResendBuffer(callCtx context.Context) (grpc.ClientStream, error) {
|
||||
s.mu.RLock()
|
||||
bufferedSends := s.bufferedSends
|
||||
s.mu.RUnlock()
|
||||
newStream, err := s.streamerCall(callCtx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for _, msg := range bufferedSends {
|
||||
if err := newStream.SendMsg(msg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
if err := newStream.CloseSend(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return newStream, nil
|
||||
}
|
||||
|
||||
func waitRetryBackoff(attempt uint, ctx context.Context, callOpts *options) error {
|
||||
var waitTime time.Duration = 0
|
||||
if attempt > 0 {
|
||||
waitTime = callOpts.backoffFunc(attempt)
|
||||
}
|
||||
if waitTime > 0 {
|
||||
timer := time.NewTimer(waitTime)
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
timer.Stop()
|
||||
return contextErrToGrpcErr(ctx.Err())
|
||||
case <-timer.C:
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// isSafeRetry returns "true", if request is safe for retry with the given error.
|
||||
func isSafeRetry(lg *zap.Logger, err error, callOpts *options) bool {
|
||||
if isContextError(err) {
|
||||
return false
|
||||
}
|
||||
switch callOpts.retryPolicy {
|
||||
case repeatable:
|
||||
return isSafeRetryImmutableRPC(err)
|
||||
case nonRepeatable:
|
||||
return isSafeRetryMutableRPC(err)
|
||||
default:
|
||||
lg.Warn("unrecognized retry policy", zap.String("retryPolicy", callOpts.retryPolicy.String()))
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
func isContextError(err error) bool {
|
||||
return grpc.Code(err) == codes.DeadlineExceeded || grpc.Code(err) == codes.Canceled
|
||||
}
|
||||
|
||||
func contextErrToGrpcErr(err error) error {
|
||||
switch err {
|
||||
case context.DeadlineExceeded:
|
||||
return grpc.Errorf(codes.DeadlineExceeded, err.Error())
|
||||
case context.Canceled:
|
||||
return grpc.Errorf(codes.Canceled, err.Error())
|
||||
default:
|
||||
return grpc.Errorf(codes.Unknown, err.Error())
|
||||
}
|
||||
}
|
||||
|
||||
var (
|
||||
defaultOptions = &options{
|
||||
retryPolicy: nonRepeatable,
|
||||
max: 0, // disable
|
||||
backoffFunc: backoffLinearWithJitter(50*time.Millisecond /*jitter*/, 0.10),
|
||||
retryAuth: true,
|
||||
}
|
||||
)
|
||||
|
||||
// backoffFunc denotes a family of functions that control the backoff duration between call retries.
|
||||
//
|
||||
// They are called with an identifier of the attempt, and should return a time the system client should
|
||||
// hold off for. If the time returned is longer than the `context.Context.Deadline` of the request
|
||||
// the deadline of the request takes precedence and the wait will be interrupted before proceeding
|
||||
// with the next iteration.
|
||||
type backoffFunc func(attempt uint) time.Duration
|
||||
|
||||
// withRetryPolicy sets the retry policy of this call.
|
||||
func withRetryPolicy(rp retryPolicy) retryOption {
|
||||
return retryOption{applyFunc: func(o *options) {
|
||||
o.retryPolicy = rp
|
||||
}}
|
||||
}
|
||||
|
||||
// withAuthRetry sets enables authentication retries.
|
||||
func withAuthRetry(retryAuth bool) retryOption {
|
||||
return retryOption{applyFunc: func(o *options) {
|
||||
o.retryAuth = retryAuth
|
||||
}}
|
||||
}
|
||||
|
||||
// withMax sets the maximum number of retries on this call, or this interceptor.
|
||||
func withMax(maxRetries uint) retryOption {
|
||||
return retryOption{applyFunc: func(o *options) {
|
||||
o.max = maxRetries
|
||||
}}
|
||||
}
|
||||
|
||||
// WithBackoff sets the `BackoffFunc `used to control time between retries.
|
||||
func withBackoff(bf backoffFunc) retryOption {
|
||||
return retryOption{applyFunc: func(o *options) {
|
||||
o.backoffFunc = bf
|
||||
}}
|
||||
}
|
||||
|
||||
type options struct {
|
||||
retryPolicy retryPolicy
|
||||
max uint
|
||||
backoffFunc backoffFunc
|
||||
retryAuth bool
|
||||
}
|
||||
|
||||
// retryOption is a grpc.CallOption that is local to clientv3's retry interceptor.
|
||||
type retryOption struct {
|
||||
grpc.EmptyCallOption // make sure we implement private after() and before() fields so we don't panic.
|
||||
applyFunc func(opt *options)
|
||||
}
|
||||
|
||||
func reuseOrNewWithCallOptions(opt *options, retryOptions []retryOption) *options {
|
||||
if len(retryOptions) == 0 {
|
||||
return opt
|
||||
}
|
||||
optCopy := &options{}
|
||||
*optCopy = *opt
|
||||
for _, f := range retryOptions {
|
||||
f.applyFunc(optCopy)
|
||||
}
|
||||
return optCopy
|
||||
}
|
||||
|
||||
func filterCallOptions(callOptions []grpc.CallOption) (grpcOptions []grpc.CallOption, retryOptions []retryOption) {
|
||||
for _, opt := range callOptions {
|
||||
if co, ok := opt.(retryOption); ok {
|
||||
retryOptions = append(retryOptions, co)
|
||||
} else {
|
||||
grpcOptions = append(grpcOptions, opt)
|
||||
}
|
||||
}
|
||||
return grpcOptions, retryOptions
|
||||
}
|
||||
|
||||
// BackoffLinearWithJitter waits a set period of time, allowing for jitter (fractional adjustment).
|
||||
//
|
||||
// For example waitBetween=1s and jitter=0.10 can generate waits between 900ms and 1100ms.
|
||||
func backoffLinearWithJitter(waitBetween time.Duration, jitterFraction float64) backoffFunc {
|
||||
return func(attempt uint) time.Duration {
|
||||
return backoffutils.JitterUp(waitBetween, jitterFraction)
|
||||
}
|
||||
}
|
@ -1,24 +1,151 @@
|
||||
.. _client-architecture:
|
||||
|
||||
########################
|
||||
|
||||
etcd Client Architecture
|
||||
########################
|
||||
|
||||
|
||||
.. image:: img/etcd.png
|
||||
:width: 100px
|
||||
Introduction
|
||||
============
|
||||
|
||||
etcd server has proven its robustness with years of failure injection testing. Most complex application logic is already handled by etcd server and its data stores (e.g. cluster membership is transparent to clients, with Raft-layer forwarding proposals to leader). Although server components are correct, its composition with client requires a different set of intricate protocols to guarantee its correctness and high availability under faulty conditions. Ideally, etcd server provides one logical cluster view of many physical machines, and client implements automatic failover between replicas. This documents client architectural decisions and its implementation details.
|
||||
|
||||
|
||||
Glossary
|
||||
========
|
||||
|
||||
*clientv3*: etcd Official Go client for etcd v3 API.
|
||||
|
||||
*clientv3-grpc1.0*: Official client implementation, with `grpc-go v1.0.x <https://github.com/grpc/grpc-go/releases/tag/v1.0.0>`_, which is used in latest etcd v3.1.
|
||||
|
||||
*clientv3-grpc1.7*: Official client implementation, with `grpc-go v1.7.x <https://github.com/grpc/grpc-go/releases/tag/v1.7.0>`_, which is used in latest etcd v3.2 and v3.3.
|
||||
|
||||
*clientv3-grpc1.12*: Official client implementation, with `grpc-go v1.12.x <https://github.com/grpc/grpc-go/releases/tag/v1.12.0>`_, which is used in latest etcd v3.4.
|
||||
|
||||
*Balancer*: etcd client load balancer that implements retry and failover mechanism. etcd client should automatically balance loads between multiple endpoints.
|
||||
|
||||
*Endpoints*: A list of etcd server endpoints that clients can connect to. Typically, 3 or 5 client URLs of an etcd cluster.
|
||||
|
||||
*Pinned endpoint*: When configured with multiple endpoints, <= v3.3 client balancer chooses only one endpoint to establish a TCP connection, in order to conserve total open connections to etcd cluster. In v3.4, balancer round-robins pinned endpoints for every request, thus distributing loads more evenly.
|
||||
|
||||
*Client Connection*: TCP connection that has been established to an etcd server, via gRPC Dial.
|
||||
|
||||
*Sub Connection*: gRPC SubConn interface. Each sub-connection contains a list of addresses. Balancer creates a SubConn from a list of resolved addresses. gRPC ClientConn can map to multiple SubConn (e.g. example.com resolves to ``10.10.10.1`` and ``10.10.10.2`` of two sub-connections). etcd v3.4 balancer employs internal resolver to establish one sub-connection for each endpoint.
|
||||
|
||||
*Transient disconnect*: When gRPC server returns a status error of `code Unavailable <https://godoc.org/google.golang.org/grpc/codes#Code>`_.
|
||||
|
||||
|
||||
Client Requirements
|
||||
===================
|
||||
|
||||
*Correctness*. Requests may fail in the presence of server faults. However, it never violates consistency guarantees: global ordering properties, never write corrupted data, at-most once semantics for mutable operations, watch never observes partial events, and so on.
|
||||
|
||||
*Liveness*. Servers may fail or disconnect briefly. Clients should make progress in either way. Clients should `never deadlock <https://github.com/coreos/etcd/issues/8980>`_ waiting for a server to come back from offline, unless configured to do so. Ideally, clients detect unavailable servers with HTTP/2 ping and failover to other nodes with clear error messages.
|
||||
|
||||
*Effectiveness*. Clients should operate effectively with minimum resources: previous TCP connections should be `gracefully closed <https://github.com/coreos/etcd/issues/9212>`_ after endpoint switch. Failover mechanism should effectively predict the next replica to connect, without wastefully retrying on failed nodes.
|
||||
|
||||
*Portability*. Official client should be clearly documented and its implementation be applicable to other language bindings. Error handling between different language bindings should be consistent. Since etcd is fully committed to gRPC, implementation should be closely aligned with gRPC long-term design goals (e.g. pluggable retry policy should be compatible with `gRPC retry <https://github.com/grpc/proposal/blob/master/A6-client-retries.md>`_). Upgrades between two client versions should be non-disruptive.
|
||||
|
||||
|
||||
Client Overview
|
||||
===============
|
||||
|
||||
etcd client implements the following components:
|
||||
|
||||
* balancer that establishes gRPC connections to an etcd cluster,
|
||||
* API client that sends RPCs to an etcd server, and
|
||||
* error handler that decides whether to retry a failed request or switch endpoints.
|
||||
|
||||
Languages may differ in how to establish an initial connection (e.g. configure TLS), how to encode and send Protocol Buffer messages to server, how to handle stream RPCs, and so on. However, errors returned from etcd server will be the same. So should be error handling and retry policy.
|
||||
|
||||
For example, etcd server may return ``"rpc error: code = Unavailable desc = etcdserver: request timed out"``, which is transient error that expects retries. Or return `rpc error: code = InvalidArgument desc = etcdserver: key is not provided`, which means request was invalid and should not be retried. Go client can parse errors with ``google.golang.org/grpc/status.FromError``, and Java client with ``io.grpc.Status.fromThrowable``.
|
||||
|
||||
|
||||
clientv3-grpc1.0: Balancer Overview
|
||||
-----------------------------------
|
||||
|
||||
``clientv3-grpc1.0`` maintains multiple TCP connections when configured with multiple etcd endpoints. Then pick one address and use it to send all client requests. The pinned address is maintained until the client object is closed (see *Figure 1*). When the client receives an error, it randomly picks another and retries.
|
||||
|
||||
.. image:: img/client-architecture-balancer-figure-01.png
|
||||
:align: center
|
||||
:height: 100px
|
||||
:alt: client-architecture-balancer-figure-01
|
||||
|
||||
|
||||
What is etcd Client?
|
||||
====================
|
||||
clientv3-grpc1.0: Balancer Limitation
|
||||
-------------------------------------
|
||||
|
||||
Hello World!
|
||||
``clientv3-grpc1.0`` opening multiple TCP connections may provide faster balancer failover but requires more resources. The balancer does not understand node’s health status or cluster membership. So, it is possible that balancer gets stuck with one failed or partitioned node.
|
||||
|
||||
|
||||
etcd Client Use Case?
|
||||
=====================
|
||||
clientv3-grpc1.7: Balancer Overview
|
||||
------------------------------------
|
||||
|
||||
Hello World!
|
||||
``clientv3-grpc1.7`` maintains only one TCP connection to a chosen etcd server. When given multiple cluster endpoints, a client first tries to connect to them all. As soon as one connection is up, balancer pins the address, closing others (see *Figure 2*). The pinned address is to be maintained until the client object is closed. An error, from server or client network fault, is sent to client error handler (see *Figure 3*).
|
||||
|
||||
.. image:: img/client-architecture-balancer-figure-02.png
|
||||
:align: center
|
||||
:alt: client-architecture-balancer-figure-02
|
||||
|
||||
.. image:: img/client-architecture-balancer-figure-03.png
|
||||
:align: center
|
||||
:alt: client-architecture-balancer-figure-03
|
||||
|
||||
The client error handler takes an error from gRPC server, and decides whether to retry on the same endpoint, or to switch to other addresses, based on the error code and message (see *Figure 4* and *Figure 5*).
|
||||
|
||||
.. image:: img/client-architecture-balancer-figure-04.png
|
||||
:align: center
|
||||
:alt: client-architecture-balancer-figure-04
|
||||
|
||||
.. image:: img/client-architecture-balancer-figure-05.png
|
||||
:align: center
|
||||
:alt: client-architecture-balancer-figure-05
|
||||
|
||||
Stream RPCs, such as Watch and KeepAlive, are often requested with no timeouts. Instead, client can send periodic HTTP/2 pings to check the status of a pinned endpoint; if the server does not respond to the ping, balancer switches to other endpoints (see *Figure 6*).
|
||||
|
||||
.. image:: img/client-architecture-balancer-figure-06.png
|
||||
:align: center
|
||||
:alt: client-architecture-balancer-figure-06
|
||||
|
||||
|
||||
clientv3-grpc1.7: Balancer Limitation
|
||||
-------------------------------------
|
||||
|
||||
``clientv3-grpc1.7`` balancer sends HTTP/2 keepalives to detect disconnects from streaming requests. It is a simple gRPC server ping mechanism and does not reason about cluster membership, thus unable to detect network partitions. Since partitioned gRPC server can still respond to client pings, balancer may get stuck with a partitioned node. Ideally, keepalive ping detects partition and triggers endpoint switch, before request time-out (see `issue#8673 <https://github.com/coreos/etcd/issues/8673>`_ and *Figure 7*).
|
||||
|
||||
.. image:: img/client-architecture-balancer-figure-07.png
|
||||
:align: center
|
||||
:alt: client-architecture-balancer-figure-07
|
||||
|
||||
``clientv3-grpc1.7`` balancer maintains a list of unhealthy endpoints. Disconnected addresses are added to “unhealthy” list, and considered unavailable until after wait duration, which is hard coded as dial timeout with default value 5-second. Balancer can have false positives on which endpoints are unhealthy. For instance, endpoint A may come back right after being blacklisted, but still unusable for next 5 seconds (see *Figure 8*).
|
||||
|
||||
``clientv3-grpc1.0`` suffered the same problems above.
|
||||
|
||||
.. image:: img/client-architecture-balancer-figure-08.png
|
||||
:align: center
|
||||
:alt: client-architecture-balancer-figure-08
|
||||
|
||||
Upstream gRPC Go had already migrated to new balancer interface. For example, ``clientv3-grpc1.7`` underlying balancer implementation uses new gRPC balancer and tries to be consistent with old balancer behaviors. While its compatibility has been maintained reasonably well, etcd client still `suffered from subtle breaking changes <https://github.com/grpc/grpc-go/issues/1649>`_. Furthermore, gRPC maintainer recommends to `not rely on the old balancer interface <https://github.com/grpc/grpc-go/issues/1942#issuecomment-375368665>`_. In general, to get better support from upstream, it is best to be in sync with latest gRPC releases. And new features, such as retry policy, may not be backported to gRPC 1.7 branch. Thus, both etcd server and client must migrate to latest gRPC versions.
|
||||
|
||||
|
||||
clientv3-grpc1.12: Balancer Overview
|
||||
------------------------------------
|
||||
|
||||
``clientv3-grpc1.7`` is so tightly coupled with old gRPC interface, that every single gRPC dependency upgrade broke client behavior. Majority of development and debugging efforts were devoted to fixing those client behavior changes. As a result, its implementation has become overly complicated with bad assumptions on server connectivities.
|
||||
|
||||
The primary goal of ``clientv3-grpc1.12`` is to simplify balancer failover logic; rather than maintaining a list of unhealthy endpoints, which may be stale, simply roundrobin to the next endpoint whenever client gets disconnected from the current endpoint. It does not assume endpoint status. Thus, no more complicated status tracking is needed (see *Figure 8* and above). Upgrading to ``clientv3-grpc1.12`` should be no issue; all changes were internal while keeping all the backward compatibilities.
|
||||
|
||||
Internally, when given multiple endpoints, ``clientv3-grpc1.12`` creates multiple sub-connections (one sub-connection per each endpoint), while ``clientv3-grpc1.7`` creates only one connection to a pinned endpoint (see *Figure 9*). For instance, in 5-node cluster, ``clientv3-grpc1.12`` balancer would require 5 TCP connections, while ``clientv3-grpc1.7`` only requires one. By preserving the pool of TCP connections, ``clientv3-grpc1.12`` may consume more resources but provide more flexible load balancer with better failover performance. The default balancing policy is round robin but can be easily extended to support other types of balancers (e.g. power of two, pick leader, etc.). ``clientv3-grpc1.12`` uses gRPC resolver group and implements balancer picker policy, in order to delegate complex balancing work to upstream gRPC. On the other hand, ``clientv3-grpc1.7`` manually handles each gRPC connection and balancer failover, which complicates the implementation. ``clientv3-grpc1.12`` implements retry in the gRPC interceptor chain that automatically handles gRPC internal errors and enables more advanced retry policies like backoff, while ``clientv3-grpc1.7`` manually interprets gRPC errors for retries.
|
||||
|
||||
.. image:: img/client-architecture-balancer-figure-09.png
|
||||
:align: center
|
||||
:alt: client-architecture-balancer-figure-09
|
||||
|
||||
|
||||
clientv3-grpc1.12: Balancer Limitation
|
||||
--------------------------------------
|
||||
|
||||
Improvements can be made by caching the status of each endpoint. For instance, balancer can ping each server in advance to maintain a list of healthy candidates, and use this information when doing round-robin. Or when disconnected, balancer can prioritize healthy endpoints. This may complicate the balancer implementation, thus can be addressed in later versions.
|
||||
|
||||
Client-side keepalive ping still does not reason about network partitions. Streaming request may get stuck with a partitioned node. Advanced health checking service need to be implemented to understand the cluster membership (see `issue#8673 <https://github.com/coreos/etcd/issues/8673>`_ for more detail).
|
||||
|
||||
Currently, retry logic is handled manually as an interceptor. This may be simplified via `official gRPC retries <https://github.com/grpc/proposal/blob/master/A6-client-retries.md>`_.
|
||||
|
BIN
docs/img/client-architecture-balancer-figure-01.png
Normal file
After Width: | Height: | Size: 99 KiB |
BIN
docs/img/client-architecture-balancer-figure-02.png
Normal file
After Width: | Height: | Size: 78 KiB |
BIN
docs/img/client-architecture-balancer-figure-03.png
Normal file
After Width: | Height: | Size: 74 KiB |
BIN
docs/img/client-architecture-balancer-figure-04.png
Normal file
After Width: | Height: | Size: 80 KiB |
BIN
docs/img/client-architecture-balancer-figure-05.png
Normal file
After Width: | Height: | Size: 73 KiB |
BIN
docs/img/client-architecture-balancer-figure-06.png
Normal file
After Width: | Height: | Size: 86 KiB |
BIN
docs/img/client-architecture-balancer-figure-07.png
Normal file
After Width: | Height: | Size: 73 KiB |
BIN
docs/img/client-architecture-balancer-figure-08.png
Normal file
After Width: | Height: | Size: 199 KiB |
BIN
docs/img/client-architecture-balancer-figure-09.png
Normal file
After Width: | Height: | Size: 138 KiB |
@ -1,8 +1,8 @@
|
||||
|
||||
Welcome to etcd's documentation!
|
||||
================================
|
||||
etcd Documentation
|
||||
==================
|
||||
|
||||
* :ref:`client-architecture`: etcd Architecture.
|
||||
* :ref:`client-architecture`: Describes etcd client components.
|
||||
|
||||
.. toctree::
|
||||
:maxdepth: 2
|
||||
|
@ -536,6 +536,7 @@ type member struct {
|
||||
PeerTLSInfo *transport.TLSInfo
|
||||
// ClientTLSInfo enables client TLS when set
|
||||
ClientTLSInfo *transport.TLSInfo
|
||||
DialOptions []grpc.DialOption
|
||||
|
||||
raftHandler *testutil.PauseableHandler
|
||||
s *etcdserver.EtcdServer
|
||||
@ -733,6 +734,7 @@ func NewClientV3(m *member) (*clientv3.Client, error) {
|
||||
cfg := clientv3.Config{
|
||||
Endpoints: []string{m.grpcAddr},
|
||||
DialTimeout: 5 * time.Second,
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
MaxCallSendMsgSize: m.clientMaxCallSendMsgSize,
|
||||
MaxCallRecvMsgSize: m.clientMaxCallRecvMsgSize,
|
||||
}
|
||||
@ -744,6 +746,9 @@ func NewClientV3(m *member) (*clientv3.Client, error) {
|
||||
}
|
||||
cfg.TLS = tls
|
||||
}
|
||||
if m.DialOptions != nil {
|
||||
cfg.DialOptions = append(cfg.DialOptions, m.DialOptions...)
|
||||
}
|
||||
return newClientV3(cfg)
|
||||
}
|
||||
|
||||
@ -950,6 +955,13 @@ func (m *member) Launch() error {
|
||||
}
|
||||
|
||||
func (m *member) WaitOK(t *testing.T) {
|
||||
m.WaitStarted(t)
|
||||
for m.s.Leader() == 0 {
|
||||
time.Sleep(tickDuration)
|
||||
}
|
||||
}
|
||||
|
||||
func (m *member) WaitStarted(t *testing.T) {
|
||||
cc := MustNewHTTPClient(t, []string{m.URL()}, m.ClientTLSInfo)
|
||||
kapi := client.NewKeysAPI(cc)
|
||||
for {
|
||||
@ -962,9 +974,23 @@ func (m *member) WaitOK(t *testing.T) {
|
||||
cancel()
|
||||
break
|
||||
}
|
||||
for m.s.Leader() == 0 {
|
||||
}
|
||||
|
||||
func WaitClientV3(t *testing.T, kv clientv3.KV) {
|
||||
timeout := time.Now().Add(requestTimeout)
|
||||
var err error
|
||||
for time.Now().Before(timeout) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), requestTimeout)
|
||||
_, err = kv.Get(ctx, "/")
|
||||
cancel()
|
||||
if err == nil {
|
||||
return
|
||||
}
|
||||
time.Sleep(tickDuration)
|
||||
}
|
||||
if err != nil {
|
||||
t.Fatalf("timed out waiting for client: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func (m *member) URL() string { return m.ClientURLs[0].String() }
|
||||
|
@ -88,13 +88,16 @@ func TestV3StorageQuotaApply(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
ctx, close := context.WithTimeout(context.TODO(), RequestWaitTimeout)
|
||||
defer close()
|
||||
|
||||
// small quota machine should reject put
|
||||
if _, err := kvc0.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err == nil {
|
||||
if _, err := kvc0.Put(ctx, &pb.PutRequest{Key: key, Value: smallbuf}); err == nil {
|
||||
t.Fatalf("past-quota instance should reject put")
|
||||
}
|
||||
|
||||
// large quota machine should reject put
|
||||
if _, err := kvc1.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err == nil {
|
||||
if _, err := kvc1.Put(ctx, &pb.PutRequest{Key: key, Value: smallbuf}); err == nil {
|
||||
t.Fatalf("past-quota instance should reject put")
|
||||
}
|
||||
|
||||
@ -175,12 +178,20 @@ func TestV3CorruptAlarm(t *testing.T) {
|
||||
s.Close()
|
||||
be.Close()
|
||||
|
||||
clus.Members[1].WaitOK(t)
|
||||
clus.Members[2].WaitOK(t)
|
||||
time.Sleep(time.Second * 2)
|
||||
|
||||
// Wait for cluster so Puts succeed in case member 0 was the leader.
|
||||
if _, err := clus.Client(1).Get(context.TODO(), "k"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
clus.Client(1).Put(context.TODO(), "xyz", "321")
|
||||
clus.Client(1).Put(context.TODO(), "abc", "fed")
|
||||
if _, err := clus.Client(1).Put(context.TODO(), "xyz", "321"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if _, err := clus.Client(1).Put(context.TODO(), "abc", "fed"); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Restart with corruption checking enabled.
|
||||
clus.Members[1].Stop(t)
|
||||
@ -189,12 +200,15 @@ func TestV3CorruptAlarm(t *testing.T) {
|
||||
m.CorruptCheckTime = time.Second
|
||||
m.Restart(t)
|
||||
}
|
||||
// Member 0 restarts into split brain.
|
||||
clus.WaitLeader(t)
|
||||
time.Sleep(time.Second * 2)
|
||||
|
||||
clus.Members[0].WaitStarted(t)
|
||||
resp0, err0 := clus.Client(0).Get(context.TODO(), "abc")
|
||||
if err0 != nil {
|
||||
t.Fatal(err0)
|
||||
}
|
||||
clus.Members[1].WaitStarted(t)
|
||||
resp1, err1 := clus.Client(1).Get(context.TODO(), "abc")
|
||||
if err1 != nil {
|
||||
t.Fatal(err1)
|
||||
|
@ -25,6 +25,7 @@ import (
|
||||
"github.com/coreos/etcd/pkg/testutil"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/transport"
|
||||
)
|
||||
|
||||
// TestV3MaintenanceDefragmentInflightRange ensures inflight range requests
|
||||
@ -81,8 +82,9 @@ func TestV3KVInflightRangeRequests(t *testing.T) {
|
||||
defer wg.Done()
|
||||
_, err := kvc.Range(ctx, &pb.RangeRequest{Key: []byte("foo"), Serializable: true}, grpc.FailFast(false))
|
||||
if err != nil {
|
||||
if err != nil && rpctypes.ErrorDesc(err) != context.Canceled.Error() {
|
||||
t.Fatalf("inflight request should be canceld with %v, got %v", context.Canceled, err)
|
||||
errDesc := rpctypes.ErrorDesc(err)
|
||||
if err != nil && !(errDesc == context.Canceled.Error() || errDesc == transport.ErrConnClosing.Desc) {
|
||||
t.Fatalf("inflight request should be canceled with '%v' or '%v', got '%v'", context.Canceled.Error(), transport.ErrConnClosing.Desc, errDesc)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
@ -32,7 +32,9 @@ import (
|
||||
"github.com/coreos/etcd/pkg/transport"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/metadata"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
// TestV3PutOverwrite puts a key with the v3 api to a random cluster member,
|
||||
@ -1570,6 +1572,7 @@ func TestTLSGRPCRejectSecureClient(t *testing.T) {
|
||||
defer clus.Terminate(t)
|
||||
|
||||
clus.Members[0].ClientTLSInfo = &testTLSInfo
|
||||
clus.Members[0].DialOptions = []grpc.DialOption{grpc.WithBlock()}
|
||||
client, err := NewClientV3(clus.Members[0])
|
||||
if client != nil || err == nil {
|
||||
t.Fatalf("expected no client")
|
||||
@ -1752,6 +1755,7 @@ func testTLSReload(
|
||||
continue
|
||||
}
|
||||
cli, cerr := clientv3.New(clientv3.Config{
|
||||
DialOptions: []grpc.DialOption{grpc.WithBlock()},
|
||||
Endpoints: []string{clus.Members[0].GRPCAddr()},
|
||||
DialTimeout: time.Second,
|
||||
TLS: cc,
|
||||
@ -1932,7 +1936,18 @@ func eqErrGRPC(err1 error, err2 error) bool {
|
||||
// FailFast=false works with Put.
|
||||
func waitForRestart(t *testing.T, kvc pb.KVClient) {
|
||||
req := &pb.RangeRequest{Key: []byte("_"), Serializable: true}
|
||||
if _, err := kvc.Range(context.TODO(), req, grpc.FailFast(false)); err != nil {
|
||||
t.Fatal(err)
|
||||
// TODO: Remove retry loop once the new grpc load balancer provides retry.
|
||||
var err error
|
||||
for i := 0; i < 10; i++ {
|
||||
if _, err = kvc.Range(context.TODO(), req, grpc.FailFast(false)); err != nil {
|
||||
if status, ok := status.FromError(err); ok && status.Code() == codes.Unavailable {
|
||||
time.Sleep(time.Millisecond * 250)
|
||||
} else {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
t.Fatalf("timed out waiting for restart: %v", err)
|
||||
}
|
||||
}
|
||||
|
16
pkg/mock/mockserver/doc.go
Normal file
@ -0,0 +1,16 @@
|
||||
// Copyright 2018 The etcd 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 mockserver provides mock implementations for etcdserver's server interface.
|
||||
package mockserver
|
188
pkg/mock/mockserver/mockserver.go
Normal file
@ -0,0 +1,188 @@
|
||||
// Copyright 2018 The etcd 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 mockserver
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"net"
|
||||
"os"
|
||||
"sync"
|
||||
|
||||
pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
// MockServer provides a mocked out grpc server of the etcdserver interface.
|
||||
type MockServer struct {
|
||||
ln net.Listener
|
||||
Network string
|
||||
Address string
|
||||
GrpcServer *grpc.Server
|
||||
}
|
||||
|
||||
func (ms *MockServer) ResolverAddress() resolver.Address {
|
||||
switch ms.Network {
|
||||
case "unix":
|
||||
return resolver.Address{Addr: fmt.Sprintf("unix://%s", ms.Address)}
|
||||
case "tcp":
|
||||
return resolver.Address{Addr: ms.Address}
|
||||
default:
|
||||
panic("illegal network type: " + ms.Network)
|
||||
}
|
||||
}
|
||||
|
||||
// MockServers provides a cluster of mocket out gprc servers of the etcdserver interface.
|
||||
type MockServers struct {
|
||||
mu sync.RWMutex
|
||||
Servers []*MockServer
|
||||
wg sync.WaitGroup
|
||||
}
|
||||
|
||||
// StartMockServers creates the desired count of mock servers
|
||||
// and starts them.
|
||||
func StartMockServers(count int) (ms *MockServers, err error) {
|
||||
return StartMockServersOnNetwork(count, "tcp")
|
||||
}
|
||||
|
||||
// StartMockServersOnNetwork creates mock servers on either 'tcp' or 'unix' sockets.
|
||||
func StartMockServersOnNetwork(count int, network string) (ms *MockServers, err error) {
|
||||
switch network {
|
||||
case "tcp":
|
||||
return startMockServersTcp(count)
|
||||
case "unix":
|
||||
return startMockServersUnix(count)
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported network type: %s", network)
|
||||
}
|
||||
}
|
||||
|
||||
func startMockServersTcp(count int) (ms *MockServers, err error) {
|
||||
addrs := make([]string, 0, count)
|
||||
for i := 0; i < count; i++ {
|
||||
addrs = append(addrs, "localhost:0")
|
||||
}
|
||||
return startMockServers("tcp", addrs)
|
||||
}
|
||||
|
||||
func startMockServersUnix(count int) (ms *MockServers, err error) {
|
||||
dir := os.TempDir()
|
||||
addrs := make([]string, 0, count)
|
||||
for i := 0; i < count; i++ {
|
||||
f, err := ioutil.TempFile(dir, "etcd-unix-so-")
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to allocate temp file for unix socket: %v", err)
|
||||
}
|
||||
fn := f.Name()
|
||||
err = os.Remove(fn)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to remove temp file before creating unix socket: %v", err)
|
||||
}
|
||||
addrs = append(addrs, fn)
|
||||
}
|
||||
return startMockServers("unix", addrs)
|
||||
}
|
||||
|
||||
func startMockServers(network string, addrs []string) (ms *MockServers, err error) {
|
||||
ms = &MockServers{
|
||||
Servers: make([]*MockServer, len(addrs)),
|
||||
wg: sync.WaitGroup{},
|
||||
}
|
||||
defer func() {
|
||||
if err != nil {
|
||||
ms.Stop()
|
||||
}
|
||||
}()
|
||||
for idx, addr := range addrs {
|
||||
ln, err := net.Listen(network, addr)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to listen %v", err)
|
||||
}
|
||||
ms.Servers[idx] = &MockServer{ln: ln, Network: network, Address: ln.Addr().String()}
|
||||
ms.StartAt(idx)
|
||||
}
|
||||
return ms, nil
|
||||
}
|
||||
|
||||
// StartAt restarts mock server at given index.
|
||||
func (ms *MockServers) StartAt(idx int) (err error) {
|
||||
ms.mu.Lock()
|
||||
defer ms.mu.Unlock()
|
||||
|
||||
if ms.Servers[idx].ln == nil {
|
||||
ms.Servers[idx].ln, err = net.Listen(ms.Servers[idx].Network, ms.Servers[idx].Address)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to listen %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
svr := grpc.NewServer()
|
||||
pb.RegisterKVServer(svr, &mockKVServer{})
|
||||
ms.Servers[idx].GrpcServer = svr
|
||||
|
||||
ms.wg.Add(1)
|
||||
go func(svr *grpc.Server, l net.Listener) {
|
||||
svr.Serve(l)
|
||||
}(ms.Servers[idx].GrpcServer, ms.Servers[idx].ln)
|
||||
return nil
|
||||
}
|
||||
|
||||
// StopAt stops mock server at given index.
|
||||
func (ms *MockServers) StopAt(idx int) {
|
||||
ms.mu.Lock()
|
||||
defer ms.mu.Unlock()
|
||||
|
||||
if ms.Servers[idx].ln == nil {
|
||||
return
|
||||
}
|
||||
|
||||
ms.Servers[idx].GrpcServer.Stop()
|
||||
ms.Servers[idx].GrpcServer = nil
|
||||
ms.Servers[idx].ln = nil
|
||||
ms.wg.Done()
|
||||
}
|
||||
|
||||
// Stop stops the mock server, immediately closing all open connections and listeners.
|
||||
func (ms *MockServers) Stop() {
|
||||
for idx := range ms.Servers {
|
||||
ms.StopAt(idx)
|
||||
}
|
||||
ms.wg.Wait()
|
||||
}
|
||||
|
||||
type mockKVServer struct{}
|
||||
|
||||
func (m *mockKVServer) Range(context.Context, *pb.RangeRequest) (*pb.RangeResponse, error) {
|
||||
return &pb.RangeResponse{}, nil
|
||||
}
|
||||
|
||||
func (m *mockKVServer) Put(context.Context, *pb.PutRequest) (*pb.PutResponse, error) {
|
||||
return &pb.PutResponse{}, nil
|
||||
}
|
||||
|
||||
func (m *mockKVServer) DeleteRange(context.Context, *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) {
|
||||
return &pb.DeleteRangeResponse{}, nil
|
||||
}
|
||||
|
||||
func (m *mockKVServer) Txn(context.Context, *pb.TxnRequest) (*pb.TxnResponse, error) {
|
||||
return &pb.TxnResponse{}, nil
|
||||
}
|
||||
|
||||
func (m *mockKVServer) Compact(context.Context, *pb.CompactionRequest) (*pb.CompactionResponse, error) {
|
||||
return &pb.CompactionResponse{}, nil
|
||||
}
|
@ -20,10 +20,8 @@ import (
|
||||
"sync"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
|
||||
|
||||
"golang.org/x/time/rate"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
const (
|
||||
@ -69,7 +67,7 @@ func (l *leader) recvLoop() {
|
||||
}
|
||||
if cresp.Err() != nil {
|
||||
l.loseLeader()
|
||||
if rpctypes.ErrorDesc(cresp.Err()) == grpc.ErrClientConnClosing.Error() {
|
||||
if clientv3.IsConnCanceled(cresp.Err()) {
|
||||
close(l.disconnc)
|
||||
return
|
||||
}
|
||||
|
2
test
@ -380,7 +380,7 @@ function shellcheck_pass {
|
||||
|
||||
function markdown_you_pass {
|
||||
# eschew you
|
||||
yous=$(find . -name \*.md ! -path './vendor/*' ! -path './gopath.proto/*' -exec grep -E --color "[Yy]ou[r]?[ '.,;]" {} + | grep -v /v2/ || true)
|
||||
yous=$(find . -name \*.md ! -path './vendor/*' ! -path './Documentation/v2/*' ! -path './gopath.proto/*' -exec grep -E --color "[Yy]ou[r]?[ '.,;]" {} + || true)
|
||||
if [ ! -z "$yous" ]; then
|
||||
echo -e "found 'you' in documentation:\\n${yous}"
|
||||
exit 255
|
||||
|
201
vendor/github.com/grpc-ecosystem/go-grpc-middleware/LICENSE
generated
vendored
Normal file
@ -0,0 +1,201 @@
|
||||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "[]"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright [yyyy] [name of copyright owner]
|
||||
|
||||
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.
|
183
vendor/github.com/grpc-ecosystem/go-grpc-middleware/chain.go
generated
vendored
Normal file
@ -0,0 +1,183 @@
|
||||
// Copyright 2016 Michal Witkowski. All Rights Reserved.
|
||||
// See LICENSE for licensing terms.
|
||||
|
||||
// gRPC Server Interceptor chaining middleware.
|
||||
|
||||
package grpc_middleware
|
||||
|
||||
import (
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
// ChainUnaryServer creates a single interceptor out of a chain of many interceptors.
|
||||
//
|
||||
// Execution is done in left-to-right order, including passing of context.
|
||||
// For example ChainUnaryServer(one, two, three) will execute one before two before three, and three
|
||||
// will see context changes of one and two.
|
||||
func ChainUnaryServer(interceptors ...grpc.UnaryServerInterceptor) grpc.UnaryServerInterceptor {
|
||||
n := len(interceptors)
|
||||
|
||||
if n > 1 {
|
||||
lastI := n - 1
|
||||
return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
|
||||
var (
|
||||
chainHandler grpc.UnaryHandler
|
||||
curI int
|
||||
)
|
||||
|
||||
chainHandler = func(currentCtx context.Context, currentReq interface{}) (interface{}, error) {
|
||||
if curI == lastI {
|
||||
return handler(currentCtx, currentReq)
|
||||
}
|
||||
curI++
|
||||
resp, err := interceptors[curI](currentCtx, currentReq, info, chainHandler)
|
||||
curI--
|
||||
return resp, err
|
||||
}
|
||||
|
||||
return interceptors[0](ctx, req, info, chainHandler)
|
||||
}
|
||||
}
|
||||
|
||||
if n == 1 {
|
||||
return interceptors[0]
|
||||
}
|
||||
|
||||
// n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil.
|
||||
return func(ctx context.Context, req interface{}, _ *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
|
||||
return handler(ctx, req)
|
||||
}
|
||||
}
|
||||
|
||||
// ChainStreamServer creates a single interceptor out of a chain of many interceptors.
|
||||
//
|
||||
// Execution is done in left-to-right order, including passing of context.
|
||||
// For example ChainUnaryServer(one, two, three) will execute one before two before three.
|
||||
// If you want to pass context between interceptors, use WrapServerStream.
|
||||
func ChainStreamServer(interceptors ...grpc.StreamServerInterceptor) grpc.StreamServerInterceptor {
|
||||
n := len(interceptors)
|
||||
|
||||
if n > 1 {
|
||||
lastI := n - 1
|
||||
return func(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
|
||||
var (
|
||||
chainHandler grpc.StreamHandler
|
||||
curI int
|
||||
)
|
||||
|
||||
chainHandler = func(currentSrv interface{}, currentStream grpc.ServerStream) error {
|
||||
if curI == lastI {
|
||||
return handler(currentSrv, currentStream)
|
||||
}
|
||||
curI++
|
||||
err := interceptors[curI](currentSrv, currentStream, info, chainHandler)
|
||||
curI--
|
||||
return err
|
||||
}
|
||||
|
||||
return interceptors[0](srv, stream, info, chainHandler)
|
||||
}
|
||||
}
|
||||
|
||||
if n == 1 {
|
||||
return interceptors[0]
|
||||
}
|
||||
|
||||
// n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil.
|
||||
return func(srv interface{}, stream grpc.ServerStream, _ *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
|
||||
return handler(srv, stream)
|
||||
}
|
||||
}
|
||||
|
||||
// ChainUnaryClient creates a single interceptor out of a chain of many interceptors.
|
||||
//
|
||||
// Execution is done in left-to-right order, including passing of context.
|
||||
// For example ChainUnaryClient(one, two, three) will execute one before two before three.
|
||||
func ChainUnaryClient(interceptors ...grpc.UnaryClientInterceptor) grpc.UnaryClientInterceptor {
|
||||
n := len(interceptors)
|
||||
|
||||
if n > 1 {
|
||||
lastI := n - 1
|
||||
return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
|
||||
var (
|
||||
chainHandler grpc.UnaryInvoker
|
||||
curI int
|
||||
)
|
||||
|
||||
chainHandler = func(currentCtx context.Context, currentMethod string, currentReq, currentRepl interface{}, currentConn *grpc.ClientConn, currentOpts ...grpc.CallOption) error {
|
||||
if curI == lastI {
|
||||
return invoker(currentCtx, currentMethod, currentReq, currentRepl, currentConn, currentOpts...)
|
||||
}
|
||||
curI++
|
||||
err := interceptors[curI](currentCtx, currentMethod, currentReq, currentRepl, currentConn, chainHandler, currentOpts...)
|
||||
curI--
|
||||
return err
|
||||
}
|
||||
|
||||
return interceptors[0](ctx, method, req, reply, cc, chainHandler, opts...)
|
||||
}
|
||||
}
|
||||
|
||||
if n == 1 {
|
||||
return interceptors[0]
|
||||
}
|
||||
|
||||
// n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil.
|
||||
return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
|
||||
return invoker(ctx, method, req, reply, cc, opts...)
|
||||
}
|
||||
}
|
||||
|
||||
// ChainStreamClient creates a single interceptor out of a chain of many interceptors.
|
||||
//
|
||||
// Execution is done in left-to-right order, including passing of context.
|
||||
// For example ChainStreamClient(one, two, three) will execute one before two before three.
|
||||
func ChainStreamClient(interceptors ...grpc.StreamClientInterceptor) grpc.StreamClientInterceptor {
|
||||
n := len(interceptors)
|
||||
|
||||
if n > 1 {
|
||||
lastI := n - 1
|
||||
return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
|
||||
var (
|
||||
chainHandler grpc.Streamer
|
||||
curI int
|
||||
)
|
||||
|
||||
chainHandler = func(currentCtx context.Context, currentDesc *grpc.StreamDesc, currentConn *grpc.ClientConn, currentMethod string, currentOpts ...grpc.CallOption) (grpc.ClientStream, error) {
|
||||
if curI == lastI {
|
||||
return streamer(currentCtx, currentDesc, currentConn, currentMethod, currentOpts...)
|
||||
}
|
||||
curI++
|
||||
stream, err := interceptors[curI](currentCtx, currentDesc, currentConn, currentMethod, chainHandler, currentOpts...)
|
||||
curI--
|
||||
return stream, err
|
||||
}
|
||||
|
||||
return interceptors[0](ctx, desc, cc, method, chainHandler, opts...)
|
||||
}
|
||||
}
|
||||
|
||||
if n == 1 {
|
||||
return interceptors[0]
|
||||
}
|
||||
|
||||
// n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil.
|
||||
return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
|
||||
return streamer(ctx, desc, cc, method, opts...)
|
||||
}
|
||||
}
|
||||
|
||||
// Chain creates a single interceptor out of a chain of many interceptors.
|
||||
//
|
||||
// WithUnaryServerChain is a grpc.Server config option that accepts multiple unary interceptors.
|
||||
// Basically syntactic sugar.
|
||||
func WithUnaryServerChain(interceptors ...grpc.UnaryServerInterceptor) grpc.ServerOption {
|
||||
return grpc.UnaryInterceptor(ChainUnaryServer(interceptors...))
|
||||
}
|
||||
|
||||
// WithStreamServerChain is a grpc.Server config option that accepts multiple stream interceptors.
|
||||
// Basically syntactic sugar.
|
||||
func WithStreamServerChain(interceptors ...grpc.StreamServerInterceptor) grpc.ServerOption {
|
||||
return grpc.StreamInterceptor(ChainStreamServer(interceptors...))
|
||||
}
|
69
vendor/github.com/grpc-ecosystem/go-grpc-middleware/doc.go
generated
vendored
Normal file
@ -0,0 +1,69 @@
|
||||
// Copyright 2016 Michal Witkowski. All Rights Reserved.
|
||||
// See LICENSE for licensing terms.
|
||||
|
||||
/*
|
||||
`grpc_middleware` is a collection of gRPC middleware packages: interceptors, helpers and tools.
|
||||
|
||||
Middleware
|
||||
|
||||
gRPC is a fantastic RPC middleware, which sees a lot of adoption in the Golang world. However, the
|
||||
upstream gRPC codebase is relatively bare bones.
|
||||
|
||||
This package, and most of its child packages provides commonly needed middleware for gRPC:
|
||||
client-side interceptors for retires, server-side interceptors for input validation and auth,
|
||||
functions for chaining said interceptors, metadata convenience methods and more.
|
||||
|
||||
Chaining
|
||||
|
||||
By default, gRPC doesn't allow one to have more than one interceptor either on the client nor on
|
||||
the server side. `grpc_middleware` provides convenient chaining methods
|
||||
|
||||
Simple way of turning a multiple interceptors into a single interceptor. Here's an example for
|
||||
server chaining:
|
||||
|
||||
myServer := grpc.NewServer(
|
||||
grpc.StreamInterceptor(grpc_middleware.ChainStreamServer(loggingStream, monitoringStream, authStream)),
|
||||
grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(loggingUnary, monitoringUnary, authUnary),
|
||||
)
|
||||
|
||||
These interceptors will be executed from left to right: logging, monitoring and auth.
|
||||
|
||||
Here's an example for client side chaining:
|
||||
|
||||
clientConn, err = grpc.Dial(
|
||||
address,
|
||||
grpc.WithUnaryInterceptor(grpc_middleware.ChainUnaryClient(monitoringClientUnary, retryUnary)),
|
||||
grpc.WithStreamInterceptor(grpc_middleware.ChainStreamClient(monitoringClientStream, retryStream)),
|
||||
)
|
||||
client = pb_testproto.NewTestServiceClient(clientConn)
|
||||
resp, err := client.PingEmpty(s.ctx, &myservice.Request{Msg: "hello"})
|
||||
|
||||
These interceptors will be executed from left to right: monitoring and then retry logic.
|
||||
|
||||
The retry interceptor will call every interceptor that follows it whenever when a retry happens.
|
||||
|
||||
Writing Your Own
|
||||
|
||||
Implementing your own interceptor is pretty trivial: there are interfaces for that. But the interesting
|
||||
bit exposing common data to handlers (and other middleware), similarly to HTTP Middleware design.
|
||||
For example, you may want to pass the identity of the caller from the auth interceptor all the way
|
||||
to the handling function.
|
||||
|
||||
For example, a client side interceptor example for auth looks like:
|
||||
|
||||
func FakeAuthUnaryInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
|
||||
newCtx := context.WithValue(ctx, "user_id", "john@example.com")
|
||||
return handler(newCtx, req)
|
||||
}
|
||||
|
||||
Unfortunately, it's not as easy for streaming RPCs. These have the `context.Context` embedded within
|
||||
the `grpc.ServerStream` object. To pass values through context, a wrapper (`WrappedServerStream`) is
|
||||
needed. For example:
|
||||
|
||||
func FakeAuthStreamingInterceptor(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
|
||||
newStream := grpc_middleware.WrapServerStream(stream)
|
||||
newStream.WrappedContext = context.WithValue(ctx, "user_id", "john@example.com")
|
||||
return handler(srv, stream)
|
||||
}
|
||||
*/
|
||||
package grpc_middleware
|
23
vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils/backoff.go
generated
vendored
Normal file
@ -0,0 +1,23 @@
|
||||
// Copyright 2016 Michal Witkowski. All Rights Reserved.
|
||||
// See LICENSE for licensing terms.
|
||||
|
||||
/*
|
||||
Backoff Helper Utilities
|
||||
|
||||
Implements common backoff features.
|
||||
*/
|
||||
package backoffutils
|
||||
|
||||
import (
|
||||
"math/rand"
|
||||
"time"
|
||||
)
|
||||
|
||||
// JitterUp adds random jitter to the duration.
|
||||
//
|
||||
// This adds or substracts time from the duration within a given jitter fraction.
|
||||
// For example for 10s and jitter 0.1, it will returna time within [9s, 11s])
|
||||
func JitterUp(duration time.Duration, jitter float64) time.Duration {
|
||||
multiplier := jitter * (rand.Float64()*2 - 1)
|
||||
return time.Duration(float64(duration) * (1 + multiplier))
|
||||
}
|
29
vendor/github.com/grpc-ecosystem/go-grpc-middleware/wrappers.go
generated
vendored
Normal file
@ -0,0 +1,29 @@
|
||||
// Copyright 2016 Michal Witkowski. All Rights Reserved.
|
||||
// See LICENSE for licensing terms.
|
||||
|
||||
package grpc_middleware
|
||||
|
||||
import (
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
// WrappedServerStream is a thin wrapper around grpc.ServerStream that allows modifying context.
|
||||
type WrappedServerStream struct {
|
||||
grpc.ServerStream
|
||||
// WrappedContext is the wrapper's own Context. You can assign it.
|
||||
WrappedContext context.Context
|
||||
}
|
||||
|
||||
// Context returns the wrapper's WrappedContext, overwriting the nested grpc.ServerStream.Context()
|
||||
func (w *WrappedServerStream) Context() context.Context {
|
||||
return w.WrappedContext
|
||||
}
|
||||
|
||||
// WrapServerStream returns a ServerStream that has the ability to overwrite context.
|
||||
func WrapServerStream(stream grpc.ServerStream) *WrappedServerStream {
|
||||
if existing, ok := stream.(*WrappedServerStream); ok {
|
||||
return existing
|
||||
}
|
||||
return &WrappedServerStream{ServerStream: stream, WrappedContext: stream.Context()}
|
||||
}
|
14
vendor/google.golang.org/grpc/backoff.go
generated
vendored
@ -25,14 +25,12 @@ import (
|
||||
|
||||
// DefaultBackoffConfig uses values specified for backoff in
|
||||
// https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md.
|
||||
var (
|
||||
DefaultBackoffConfig = BackoffConfig{
|
||||
MaxDelay: 120 * time.Second,
|
||||
baseDelay: 1.0 * time.Second,
|
||||
factor: 1.6,
|
||||
jitter: 0.2,
|
||||
}
|
||||
)
|
||||
var DefaultBackoffConfig = BackoffConfig{
|
||||
MaxDelay: 120 * time.Second,
|
||||
baseDelay: 1.0 * time.Second,
|
||||
factor: 1.6,
|
||||
jitter: 0.2,
|
||||
}
|
||||
|
||||
// backoffStrategy defines the methodology for backing off after a grpc
|
||||
// connection failure.
|
||||
|
16
vendor/google.golang.org/grpc/balancer.go
generated
vendored
@ -28,10 +28,12 @@ import (
|
||||
"google.golang.org/grpc/credentials"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/naming"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
// Address represents a server the client connects to.
|
||||
// This is the EXPERIMENTAL API and may be changed or extended in the future.
|
||||
//
|
||||
// Deprecated: please use package balancer.
|
||||
type Address struct {
|
||||
// Addr is the server address on which a connection will be established.
|
||||
Addr string
|
||||
@ -41,6 +43,8 @@ type Address struct {
|
||||
}
|
||||
|
||||
// BalancerConfig specifies the configurations for Balancer.
|
||||
//
|
||||
// Deprecated: please use package balancer.
|
||||
type BalancerConfig struct {
|
||||
// DialCreds is the transport credential the Balancer implementation can
|
||||
// use to dial to a remote load balancer server. The Balancer implementations
|
||||
@ -53,7 +57,8 @@ type BalancerConfig struct {
|
||||
}
|
||||
|
||||
// BalancerGetOptions configures a Get call.
|
||||
// This is the EXPERIMENTAL API and may be changed or extended in the future.
|
||||
//
|
||||
// Deprecated: please use package balancer.
|
||||
type BalancerGetOptions struct {
|
||||
// BlockingWait specifies whether Get should block when there is no
|
||||
// connected address.
|
||||
@ -61,7 +66,8 @@ type BalancerGetOptions struct {
|
||||
}
|
||||
|
||||
// Balancer chooses network addresses for RPCs.
|
||||
// This is the EXPERIMENTAL API and may be changed or extended in the future.
|
||||
//
|
||||
// Deprecated: please use package balancer.
|
||||
type Balancer interface {
|
||||
// Start does the initialization work to bootstrap a Balancer. For example,
|
||||
// this function may start the name resolution and watch the updates. It will
|
||||
@ -134,6 +140,8 @@ func downErrorf(timeout, temporary bool, format string, a ...interface{}) downEr
|
||||
|
||||
// RoundRobin returns a Balancer that selects addresses round-robin. It uses r to watch
|
||||
// the name resolution updates and updates the addresses available correspondingly.
|
||||
//
|
||||
// Deprecated: please use package balancer/roundrobin.
|
||||
func RoundRobin(r naming.Resolver) Balancer {
|
||||
return &roundRobin{r: r}
|
||||
}
|
||||
@ -310,7 +318,7 @@ func (rr *roundRobin) Get(ctx context.Context, opts BalancerGetOptions) (addr Ad
|
||||
if !opts.BlockingWait {
|
||||
if len(rr.addrs) == 0 {
|
||||
rr.mu.Unlock()
|
||||
err = Errorf(codes.Unavailable, "there is no address available")
|
||||
err = status.Errorf(codes.Unavailable, "there is no address available")
|
||||
return
|
||||
}
|
||||
// Returns the next addr on rr.addrs for failfast RPCs.
|
||||
|
44
vendor/google.golang.org/grpc/balancer/balancer.go
generated
vendored
@ -23,6 +23,7 @@ package balancer
|
||||
import (
|
||||
"errors"
|
||||
"net"
|
||||
"strings"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
@ -33,24 +34,26 @@ import (
|
||||
var (
|
||||
// m is a map from name to balancer builder.
|
||||
m = make(map[string]Builder)
|
||||
// defaultBuilder is the default balancer to use.
|
||||
defaultBuilder Builder // TODO(bar) install pickfirst as default.
|
||||
)
|
||||
|
||||
// Register registers the balancer builder to the balancer map.
|
||||
// b.Name will be used as the name registered with this builder.
|
||||
// Register registers the balancer builder to the balancer map. b.Name
|
||||
// (lowercased) will be used as the name registered with this builder.
|
||||
//
|
||||
// NOTE: this function must only be called during initialization time (i.e. in
|
||||
// an init() function), and is not thread-safe. If multiple Balancers are
|
||||
// registered with the same name, the one registered last will take effect.
|
||||
func Register(b Builder) {
|
||||
m[b.Name()] = b
|
||||
m[strings.ToLower(b.Name())] = b
|
||||
}
|
||||
|
||||
// Get returns the resolver builder registered with the given name.
|
||||
// If no builder is register with the name, the default pickfirst will
|
||||
// be used.
|
||||
// Note that the compare is done in a case-insenstive fashion.
|
||||
// If no builder is register with the name, nil will be returned.
|
||||
func Get(name string) Builder {
|
||||
if b, ok := m[name]; ok {
|
||||
if b, ok := m[strings.ToLower(name)]; ok {
|
||||
return b
|
||||
}
|
||||
return defaultBuilder
|
||||
return nil
|
||||
}
|
||||
|
||||
// SubConn represents a gRPC sub connection.
|
||||
@ -66,6 +69,11 @@ func Get(name string) Builder {
|
||||
// When the connection encounters an error, it will reconnect immediately.
|
||||
// When the connection becomes IDLE, it will not reconnect unless Connect is
|
||||
// called.
|
||||
//
|
||||
// This interface is to be implemented by gRPC. Users should not need a
|
||||
// brand new implementation of this interface. For the situations like
|
||||
// testing, the new implementation should embed this interface. This allows
|
||||
// gRPC to add new methods to this interface.
|
||||
type SubConn interface {
|
||||
// UpdateAddresses updates the addresses used in this SubConn.
|
||||
// gRPC checks if currently-connected address is still in the new list.
|
||||
@ -83,6 +91,11 @@ type SubConn interface {
|
||||
type NewSubConnOptions struct{}
|
||||
|
||||
// ClientConn represents a gRPC ClientConn.
|
||||
//
|
||||
// This interface is to be implemented by gRPC. Users should not need a
|
||||
// brand new implementation of this interface. For the situations like
|
||||
// testing, the new implementation should embed this interface. This allows
|
||||
// gRPC to add new methods to this interface.
|
||||
type ClientConn interface {
|
||||
// NewSubConn is called by balancer to create a new SubConn.
|
||||
// It doesn't block and wait for the connections to be established.
|
||||
@ -99,6 +112,9 @@ type ClientConn interface {
|
||||
// on the new picker to pick new SubConn.
|
||||
UpdateBalancerState(s connectivity.State, p Picker)
|
||||
|
||||
// ResolveNow is called by balancer to notify gRPC to do a name resolving.
|
||||
ResolveNow(resolver.ResolveNowOption)
|
||||
|
||||
// Target returns the dial target for this ClientConn.
|
||||
Target() string
|
||||
}
|
||||
@ -113,6 +129,8 @@ type BuildOptions struct {
|
||||
// to a remote load balancer server. The Balancer implementations
|
||||
// can ignore this if it doesn't need to talk to remote balancer.
|
||||
Dialer func(context.Context, string) (net.Conn, error)
|
||||
// ChannelzParentID is the entity parent's channelz unique identification number.
|
||||
ChannelzParentID int64
|
||||
}
|
||||
|
||||
// Builder creates a balancer.
|
||||
@ -131,6 +149,10 @@ type PickOptions struct{}
|
||||
type DoneInfo struct {
|
||||
// Err is the rpc error the RPC finished with. It could be nil.
|
||||
Err error
|
||||
// BytesSent indicates if any bytes have been sent to the server.
|
||||
BytesSent bool
|
||||
// BytesReceived indicates if any byte has been received from the server.
|
||||
BytesReceived bool
|
||||
}
|
||||
|
||||
var (
|
||||
@ -143,7 +165,7 @@ var (
|
||||
)
|
||||
|
||||
// Picker is used by gRPC to pick a SubConn to send an RPC.
|
||||
// Balancer is expected to generate a new picker from its snapshot everytime its
|
||||
// Balancer is expected to generate a new picker from its snapshot every time its
|
||||
// internal state has changed.
|
||||
//
|
||||
// The pickers used by gRPC can be updated by ClientConn.UpdateBalancerState().
|
||||
@ -161,7 +183,7 @@ type Picker interface {
|
||||
// If a SubConn is returned:
|
||||
// - If it is READY, gRPC will send the RPC on it;
|
||||
// - If it is not ready, or becomes not ready after it's returned, gRPC will block
|
||||
// this call until a new picker is updated and will call pick on the new picker.
|
||||
// until UpdateBalancerState() is called and will call pick on the new picker.
|
||||
//
|
||||
// If the returned error is not nil:
|
||||
// - If the error is ErrNoSubConnAvailable, gRPC will block until UpdateBalancerState()
|
||||
|
208
vendor/google.golang.org/grpc/balancer/base/balancer.go
generated
vendored
Normal file
@ -0,0 +1,208 @@
|
||||
/*
|
||||
*
|
||||
* 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)
|
||||
}
|
||||
|
||||
// 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
vendor/google.golang.org/grpc/balancer/base/base.go
generated
vendored
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,
|
||||
}
|
||||
}
|
79
vendor/google.golang.org/grpc/balancer/roundrobin/roundrobin.go
generated
vendored
Normal file
@ -0,0 +1,79 @@
|
||||
/*
|
||||
*
|
||||
* 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 roundrobin defines a roundrobin balancer. Roundrobin balancer is
|
||||
// installed as one of the default balancers in gRPC, users don't need to
|
||||
// explicitly install this balancer.
|
||||
package roundrobin
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/balancer/base"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
// Name is the name of round_robin balancer.
|
||||
const Name = "round_robin"
|
||||
|
||||
// newBuilder creates a new roundrobin balancer builder.
|
||||
func newBuilder() balancer.Builder {
|
||||
return base.NewBalancerBuilder(Name, &rrPickerBuilder{})
|
||||
}
|
||||
|
||||
func init() {
|
||||
balancer.Register(newBuilder())
|
||||
}
|
||||
|
||||
type rrPickerBuilder struct{}
|
||||
|
||||
func (*rrPickerBuilder) Build(readySCs map[resolver.Address]balancer.SubConn) balancer.Picker {
|
||||
grpclog.Infof("roundrobinPicker: newPicker called with readySCs: %v", readySCs)
|
||||
var scs []balancer.SubConn
|
||||
for _, sc := range readySCs {
|
||||
scs = append(scs, sc)
|
||||
}
|
||||
return &rrPicker{
|
||||
subConns: scs,
|
||||
}
|
||||
}
|
||||
|
||||
type rrPicker struct {
|
||||
// subConns is the snapshot of the roundrobin balancer when this picker was
|
||||
// created. The slice is immutable. Each Get() will do a round robin
|
||||
// selection from it and return the selected SubConn.
|
||||
subConns []balancer.SubConn
|
||||
|
||||
mu sync.Mutex
|
||||
next int
|
||||
}
|
||||
|
||||
func (p *rrPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
||||
if len(p.subConns) <= 0 {
|
||||
return nil, nil, balancer.ErrNoSubConnAvailable
|
||||
}
|
||||
|
||||
p.mu.Lock()
|
||||
sc := p.subConns[p.next]
|
||||
p.next = (p.next + 1) % len(p.subConns)
|
||||
p.mu.Unlock()
|
||||
return sc, nil, nil
|
||||
}
|
68
vendor/google.golang.org/grpc/balancer_conn_wrappers.go
generated
vendored
@ -19,6 +19,7 @@
|
||||
package grpc
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"google.golang.org/grpc/balancer"
|
||||
@ -73,7 +74,7 @@ func (b *scStateUpdateBuffer) load() {
|
||||
}
|
||||
}
|
||||
|
||||
// get returns the channel that receives a recvMsg in the buffer.
|
||||
// get returns the channel that the scStateUpdate will be sent to.
|
||||
//
|
||||
// Upon receiving, the caller should call load to send another
|
||||
// scStateChangeTuple onto the channel if there is any.
|
||||
@ -96,6 +97,9 @@ type ccBalancerWrapper struct {
|
||||
stateChangeQueue *scStateUpdateBuffer
|
||||
resolverUpdateCh chan *resolverUpdate
|
||||
done chan struct{}
|
||||
|
||||
mu sync.Mutex
|
||||
subConns map[*acBalancerWrapper]struct{}
|
||||
}
|
||||
|
||||
func newCCBalancerWrapper(cc *ClientConn, b balancer.Builder, bopts balancer.BuildOptions) *ccBalancerWrapper {
|
||||
@ -104,21 +108,34 @@ func newCCBalancerWrapper(cc *ClientConn, b balancer.Builder, bopts balancer.Bui
|
||||
stateChangeQueue: newSCStateUpdateBuffer(),
|
||||
resolverUpdateCh: make(chan *resolverUpdate, 1),
|
||||
done: make(chan struct{}),
|
||||
subConns: make(map[*acBalancerWrapper]struct{}),
|
||||
}
|
||||
go ccb.watcher()
|
||||
ccb.balancer = b.Build(ccb, bopts)
|
||||
return ccb
|
||||
}
|
||||
|
||||
// watcher balancer functions sequencially, so the balancer can be implemeneted
|
||||
// watcher balancer functions sequentially, so the balancer can be implemented
|
||||
// lock-free.
|
||||
func (ccb *ccBalancerWrapper) watcher() {
|
||||
for {
|
||||
select {
|
||||
case t := <-ccb.stateChangeQueue.get():
|
||||
ccb.stateChangeQueue.load()
|
||||
select {
|
||||
case <-ccb.done:
|
||||
ccb.balancer.Close()
|
||||
return
|
||||
default:
|
||||
}
|
||||
ccb.balancer.HandleSubConnStateChange(t.sc, t.state)
|
||||
case t := <-ccb.resolverUpdateCh:
|
||||
select {
|
||||
case <-ccb.done:
|
||||
ccb.balancer.Close()
|
||||
return
|
||||
default:
|
||||
}
|
||||
ccb.balancer.HandleResolvedAddrs(t.addrs, t.err)
|
||||
case <-ccb.done:
|
||||
}
|
||||
@ -126,6 +143,13 @@ func (ccb *ccBalancerWrapper) watcher() {
|
||||
select {
|
||||
case <-ccb.done:
|
||||
ccb.balancer.Close()
|
||||
ccb.mu.Lock()
|
||||
scs := ccb.subConns
|
||||
ccb.subConns = nil
|
||||
ccb.mu.Unlock()
|
||||
for acbw := range scs {
|
||||
ccb.cc.removeAddrConn(acbw.getAddrConn(), errConnDrain)
|
||||
}
|
||||
return
|
||||
default:
|
||||
}
|
||||
@ -165,33 +189,54 @@ func (ccb *ccBalancerWrapper) handleResolvedAddrs(addrs []resolver.Address, err
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) {
|
||||
grpclog.Infof("ccBalancerWrapper: new subconn: %v", addrs)
|
||||
if len(addrs) <= 0 {
|
||||
return nil, fmt.Errorf("grpc: cannot create SubConn with empty address list")
|
||||
}
|
||||
ccb.mu.Lock()
|
||||
defer ccb.mu.Unlock()
|
||||
if ccb.subConns == nil {
|
||||
return nil, fmt.Errorf("grpc: ClientConn balancer wrapper was closed")
|
||||
}
|
||||
ac, err := ccb.cc.newAddrConn(addrs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
acbw := &acBalancerWrapper{ac: ac}
|
||||
ac.mu.Lock()
|
||||
acbw.ac.mu.Lock()
|
||||
ac.acbw = acbw
|
||||
ac.mu.Unlock()
|
||||
acbw.ac.mu.Unlock()
|
||||
ccb.subConns[acbw] = struct{}{}
|
||||
return acbw, nil
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) RemoveSubConn(sc balancer.SubConn) {
|
||||
grpclog.Infof("ccBalancerWrapper: removing subconn")
|
||||
acbw, ok := sc.(*acBalancerWrapper)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
ccb.mu.Lock()
|
||||
defer ccb.mu.Unlock()
|
||||
if ccb.subConns == nil {
|
||||
return
|
||||
}
|
||||
delete(ccb.subConns, acbw)
|
||||
ccb.cc.removeAddrConn(acbw.getAddrConn(), errConnDrain)
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) UpdateBalancerState(s connectivity.State, p balancer.Picker) {
|
||||
grpclog.Infof("ccBalancerWrapper: updating state and picker called by balancer: %v, %p", s, p)
|
||||
ccb.mu.Lock()
|
||||
defer ccb.mu.Unlock()
|
||||
if ccb.subConns == nil {
|
||||
return
|
||||
}
|
||||
ccb.cc.csMgr.updateState(s)
|
||||
ccb.cc.blockingpicker.updatePicker(p)
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) ResolveNow(o resolver.ResolveNowOption) {
|
||||
ccb.cc.resolveNow(o)
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) Target() string {
|
||||
return ccb.cc.target
|
||||
}
|
||||
@ -204,9 +249,12 @@ type acBalancerWrapper struct {
|
||||
}
|
||||
|
||||
func (acbw *acBalancerWrapper) UpdateAddresses(addrs []resolver.Address) {
|
||||
grpclog.Infof("acBalancerWrapper: UpdateAddresses called with %v", addrs)
|
||||
acbw.mu.Lock()
|
||||
defer acbw.mu.Unlock()
|
||||
if len(addrs) <= 0 {
|
||||
acbw.ac.tearDown(errConnDrain)
|
||||
return
|
||||
}
|
||||
if !acbw.ac.tryUpdateAddrs(addrs) {
|
||||
cc := acbw.ac.cc
|
||||
acbw.ac.mu.Lock()
|
||||
@ -234,7 +282,7 @@ func (acbw *acBalancerWrapper) UpdateAddresses(addrs []resolver.Address) {
|
||||
ac.acbw = acbw
|
||||
ac.mu.Unlock()
|
||||
if acState != connectivity.Idle {
|
||||
ac.connect(false)
|
||||
ac.connect()
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -242,7 +290,7 @@ func (acbw *acBalancerWrapper) UpdateAddresses(addrs []resolver.Address) {
|
||||
func (acbw *acBalancerWrapper) Connect() {
|
||||
acbw.mu.Lock()
|
||||
defer acbw.mu.Unlock()
|
||||
acbw.ac.connect(false)
|
||||
acbw.ac.connect()
|
||||
}
|
||||
|
||||
func (acbw *acBalancerWrapper) getAddrConn() *addrConn {
|
||||
|
45
vendor/google.golang.org/grpc/balancer_v1_wrapper.go
generated
vendored
@ -19,6 +19,7 @@
|
||||
package grpc
|
||||
|
||||
import (
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
@ -27,6 +28,7 @@ import (
|
||||
"google.golang.org/grpc/connectivity"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/resolver"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
type balancerWrapperBuilder struct {
|
||||
@ -34,20 +36,27 @@ type balancerWrapperBuilder struct {
|
||||
}
|
||||
|
||||
func (bwb *balancerWrapperBuilder) Build(cc balancer.ClientConn, opts balancer.BuildOptions) balancer.Balancer {
|
||||
bwb.b.Start(cc.Target(), BalancerConfig{
|
||||
targetAddr := cc.Target()
|
||||
targetSplitted := strings.Split(targetAddr, ":///")
|
||||
if len(targetSplitted) >= 2 {
|
||||
targetAddr = targetSplitted[1]
|
||||
}
|
||||
|
||||
bwb.b.Start(targetAddr, BalancerConfig{
|
||||
DialCreds: opts.DialCreds,
|
||||
Dialer: opts.Dialer,
|
||||
})
|
||||
_, pickfirst := bwb.b.(*pickFirst)
|
||||
bw := &balancerWrapper{
|
||||
balancer: bwb.b,
|
||||
pickfirst: pickfirst,
|
||||
cc: cc,
|
||||
startCh: make(chan struct{}),
|
||||
conns: make(map[resolver.Address]balancer.SubConn),
|
||||
connSt: make(map[balancer.SubConn]*scState),
|
||||
csEvltr: &connectivityStateEvaluator{},
|
||||
state: connectivity.Idle,
|
||||
balancer: bwb.b,
|
||||
pickfirst: pickfirst,
|
||||
cc: cc,
|
||||
targetAddr: targetAddr,
|
||||
startCh: make(chan struct{}),
|
||||
conns: make(map[resolver.Address]balancer.SubConn),
|
||||
connSt: make(map[balancer.SubConn]*scState),
|
||||
csEvltr: &connectivityStateEvaluator{},
|
||||
state: connectivity.Idle,
|
||||
}
|
||||
cc.UpdateBalancerState(connectivity.Idle, bw)
|
||||
go bw.lbWatcher()
|
||||
@ -68,7 +77,8 @@ type balancerWrapper struct {
|
||||
balancer Balancer // The v1 balancer.
|
||||
pickfirst bool
|
||||
|
||||
cc balancer.ClientConn
|
||||
cc balancer.ClientConn
|
||||
targetAddr string // Target without the scheme.
|
||||
|
||||
// To aggregate the connectivity state.
|
||||
csEvltr *connectivityStateEvaluator
|
||||
@ -88,12 +98,11 @@ type balancerWrapper struct {
|
||||
// connections accordingly.
|
||||
func (bw *balancerWrapper) lbWatcher() {
|
||||
<-bw.startCh
|
||||
grpclog.Infof("balancerWrapper: is pickfirst: %v\n", bw.pickfirst)
|
||||
notifyCh := bw.balancer.Notify()
|
||||
if notifyCh == nil {
|
||||
// There's no resolver in the balancer. Connect directly.
|
||||
a := resolver.Address{
|
||||
Addr: bw.cc.Target(),
|
||||
Addr: bw.targetAddr,
|
||||
Type: resolver.Backend,
|
||||
}
|
||||
sc, err := bw.cc.NewSubConn([]resolver.Address{a}, balancer.NewSubConnOptions{})
|
||||
@ -103,7 +112,7 @@ func (bw *balancerWrapper) lbWatcher() {
|
||||
bw.mu.Lock()
|
||||
bw.conns[a] = sc
|
||||
bw.connSt[sc] = &scState{
|
||||
addr: Address{Addr: bw.cc.Target()},
|
||||
addr: Address{Addr: bw.targetAddr},
|
||||
s: connectivity.Idle,
|
||||
}
|
||||
bw.mu.Unlock()
|
||||
@ -165,10 +174,10 @@ func (bw *balancerWrapper) lbWatcher() {
|
||||
sc.Connect()
|
||||
}
|
||||
} else {
|
||||
oldSC.UpdateAddresses(newAddrs)
|
||||
bw.mu.Lock()
|
||||
bw.connSt[oldSC].addr = addrs[0]
|
||||
bw.mu.Unlock()
|
||||
oldSC.UpdateAddresses(newAddrs)
|
||||
}
|
||||
} else {
|
||||
var (
|
||||
@ -221,7 +230,6 @@ func (bw *balancerWrapper) lbWatcher() {
|
||||
}
|
||||
|
||||
func (bw *balancerWrapper) HandleSubConnStateChange(sc balancer.SubConn, s connectivity.State) {
|
||||
grpclog.Infof("balancerWrapper: handle subconn state change: %p, %v", sc, s)
|
||||
bw.mu.Lock()
|
||||
defer bw.mu.Unlock()
|
||||
scSt, ok := bw.connSt[sc]
|
||||
@ -249,7 +257,6 @@ func (bw *balancerWrapper) HandleSubConnStateChange(sc balancer.SubConn, s conne
|
||||
// Remove state for this sc.
|
||||
delete(bw.connSt, sc)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (bw *balancerWrapper) HandleResolvedAddrs([]resolver.Address, error) {
|
||||
@ -262,7 +269,6 @@ func (bw *balancerWrapper) HandleResolvedAddrs([]resolver.Address, error) {
|
||||
}
|
||||
// There should be a resolver inside the balancer.
|
||||
// All updates here, if any, are ignored.
|
||||
return
|
||||
}
|
||||
|
||||
func (bw *balancerWrapper) Close() {
|
||||
@ -274,7 +280,6 @@ func (bw *balancerWrapper) Close() {
|
||||
close(bw.startCh)
|
||||
}
|
||||
bw.balancer.Close()
|
||||
return
|
||||
}
|
||||
|
||||
// The picker is the balancerWrapper itself.
|
||||
@ -310,12 +315,12 @@ func (bw *balancerWrapper) Pick(ctx context.Context, opts balancer.PickOptions)
|
||||
Metadata: a.Metadata,
|
||||
}]
|
||||
if !ok && failfast {
|
||||
return nil, nil, Errorf(codes.Unavailable, "there is no connection available")
|
||||
return nil, nil, status.Errorf(codes.Unavailable, "there is no connection available")
|
||||
}
|
||||
if s, ok := bw.connSt[sc]; failfast && (!ok || s.s != connectivity.Ready) {
|
||||
// If the returned sc is not ready and RPC is failfast,
|
||||
// return error, and this RPC will fail.
|
||||
return nil, nil, Errorf(codes.Unavailable, "there is no connection available")
|
||||
return nil, nil, status.Errorf(codes.Unavailable, "there is no connection available")
|
||||
}
|
||||
}
|
||||
|
||||
|
308
vendor/google.golang.org/grpc/call.go
generated
vendored
@ -19,289 +19,75 @@
|
||||
package grpc
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"golang.org/x/net/trace"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/peer"
|
||||
"google.golang.org/grpc/stats"
|
||||
"google.golang.org/grpc/status"
|
||||
"google.golang.org/grpc/transport"
|
||||
)
|
||||
|
||||
// recvResponse receives and parses an RPC response.
|
||||
// On error, it returns the error and indicates whether the call should be retried.
|
||||
// Invoke sends the RPC request on the wire and returns after response is
|
||||
// received. This is typically called by generated code.
|
||||
//
|
||||
// TODO(zhaoq): Check whether the received message sequence is valid.
|
||||
// TODO ctx is used for stats collection and processing. It is the context passed from the application.
|
||||
func recvResponse(ctx context.Context, dopts dialOptions, t transport.ClientTransport, c *callInfo, stream *transport.Stream, reply interface{}) (err error) {
|
||||
// Try to acquire header metadata from the server if there is any.
|
||||
defer func() {
|
||||
if err != nil {
|
||||
if _, ok := err.(transport.ConnectionError); !ok {
|
||||
t.CloseStream(stream, err)
|
||||
}
|
||||
}
|
||||
}()
|
||||
c.headerMD, err = stream.Header()
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
p := &parser{r: stream}
|
||||
var inPayload *stats.InPayload
|
||||
if dopts.copts.StatsHandler != nil {
|
||||
inPayload = &stats.InPayload{
|
||||
Client: true,
|
||||
}
|
||||
}
|
||||
for {
|
||||
if c.maxReceiveMessageSize == nil {
|
||||
return Errorf(codes.Internal, "callInfo maxReceiveMessageSize field uninitialized(nil)")
|
||||
}
|
||||
if err = recv(p, dopts.codec, stream, dopts.dc, reply, *c.maxReceiveMessageSize, inPayload); err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return
|
||||
}
|
||||
}
|
||||
if inPayload != nil && err == io.EOF && stream.Status().Code() == codes.OK {
|
||||
// TODO in the current implementation, inTrailer may be handled before inPayload in some cases.
|
||||
// Fix the order if necessary.
|
||||
dopts.copts.StatsHandler.HandleRPC(ctx, inPayload)
|
||||
}
|
||||
c.trailerMD = stream.Trailer()
|
||||
return nil
|
||||
}
|
||||
// All errors returned by Invoke are compatible with the status package.
|
||||
func (cc *ClientConn) Invoke(ctx context.Context, method string, args, reply interface{}, opts ...CallOption) error {
|
||||
// allow interceptor to see all applicable call options, which means those
|
||||
// configured as defaults from dial option as well as per-call options
|
||||
opts = combine(cc.dopts.callOptions, opts)
|
||||
|
||||
// sendRequest writes out various information of an RPC such as Context and Message.
|
||||
func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, c *callInfo, callHdr *transport.CallHdr, stream *transport.Stream, t transport.ClientTransport, args interface{}, opts *transport.Options) (err error) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
// If err is connection error, t will be closed, no need to close stream here.
|
||||
if _, ok := err.(transport.ConnectionError); !ok {
|
||||
t.CloseStream(stream, err)
|
||||
}
|
||||
}
|
||||
}()
|
||||
var (
|
||||
cbuf *bytes.Buffer
|
||||
outPayload *stats.OutPayload
|
||||
)
|
||||
if compressor != nil {
|
||||
cbuf = new(bytes.Buffer)
|
||||
}
|
||||
if dopts.copts.StatsHandler != nil {
|
||||
outPayload = &stats.OutPayload{
|
||||
Client: true,
|
||||
}
|
||||
}
|
||||
hdr, data, err := encode(dopts.codec, args, compressor, cbuf, outPayload)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if c.maxSendMessageSize == nil {
|
||||
return Errorf(codes.Internal, "callInfo maxSendMessageSize field uninitialized(nil)")
|
||||
}
|
||||
if len(data) > *c.maxSendMessageSize {
|
||||
return Errorf(codes.ResourceExhausted, "grpc: trying to send message larger than max (%d vs. %d)", len(data), *c.maxSendMessageSize)
|
||||
}
|
||||
err = t.Write(stream, hdr, data, opts)
|
||||
if err == nil && outPayload != nil {
|
||||
outPayload.SentTime = time.Now()
|
||||
dopts.copts.StatsHandler.HandleRPC(ctx, outPayload)
|
||||
}
|
||||
// t.NewStream(...) could lead to an early rejection of the RPC (e.g., the service/method
|
||||
// does not exist.) so that t.Write could get io.EOF from wait(...). Leave the following
|
||||
// recvResponse to get the final status.
|
||||
if err != nil && err != io.EOF {
|
||||
return err
|
||||
}
|
||||
// Sent successfully.
|
||||
return nil
|
||||
}
|
||||
|
||||
// Invoke sends the RPC request on the wire and returns after response is received.
|
||||
// Invoke is called by generated code. Also users can call Invoke directly when it
|
||||
// is really needed in their use cases.
|
||||
func Invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) error {
|
||||
if cc.dopts.unaryInt != nil {
|
||||
return cc.dopts.unaryInt(ctx, method, args, reply, cc, invoke, opts...)
|
||||
}
|
||||
return invoke(ctx, method, args, reply, cc, opts...)
|
||||
}
|
||||
|
||||
func invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) (e error) {
|
||||
c := defaultCallInfo()
|
||||
mc := cc.GetMethodConfig(method)
|
||||
if mc.WaitForReady != nil {
|
||||
c.failFast = !*mc.WaitForReady
|
||||
func combine(o1 []CallOption, o2 []CallOption) []CallOption {
|
||||
// we don't use append because o1 could have extra capacity whose
|
||||
// elements would be overwritten, which could cause inadvertent
|
||||
// sharing (and race connditions) between concurrent calls
|
||||
if len(o1) == 0 {
|
||||
return o2
|
||||
} else if len(o2) == 0 {
|
||||
return o1
|
||||
}
|
||||
ret := make([]CallOption, len(o1)+len(o2))
|
||||
copy(ret, o1)
|
||||
copy(ret[len(o1):], o2)
|
||||
return ret
|
||||
}
|
||||
|
||||
if mc.Timeout != nil && *mc.Timeout >= 0 {
|
||||
var cancel context.CancelFunc
|
||||
ctx, cancel = context.WithTimeout(ctx, *mc.Timeout)
|
||||
defer cancel()
|
||||
}
|
||||
// Invoke sends the RPC request on the wire and returns after response is
|
||||
// received. This is typically called by generated code.
|
||||
//
|
||||
// DEPRECATED: Use ClientConn.Invoke instead.
|
||||
func Invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) error {
|
||||
return cc.Invoke(ctx, method, args, reply, opts...)
|
||||
}
|
||||
|
||||
opts = append(cc.dopts.callOptions, opts...)
|
||||
for _, o := range opts {
|
||||
if err := o.before(c); err != nil {
|
||||
return toRPCErr(err)
|
||||
}
|
||||
}
|
||||
defer func() {
|
||||
for _, o := range opts {
|
||||
o.after(c)
|
||||
}
|
||||
}()
|
||||
var unaryStreamDesc = &StreamDesc{ServerStreams: false, ClientStreams: false}
|
||||
|
||||
c.maxSendMessageSize = getMaxSize(mc.MaxReqSize, c.maxSendMessageSize, defaultClientMaxSendMessageSize)
|
||||
c.maxReceiveMessageSize = getMaxSize(mc.MaxRespSize, c.maxReceiveMessageSize, defaultClientMaxReceiveMessageSize)
|
||||
|
||||
if EnableTracing {
|
||||
c.traceInfo.tr = trace.New("grpc.Sent."+methodFamily(method), method)
|
||||
defer c.traceInfo.tr.Finish()
|
||||
c.traceInfo.firstLine.client = true
|
||||
if deadline, ok := ctx.Deadline(); ok {
|
||||
c.traceInfo.firstLine.deadline = deadline.Sub(time.Now())
|
||||
}
|
||||
c.traceInfo.tr.LazyLog(&c.traceInfo.firstLine, false)
|
||||
// TODO(dsymonds): Arrange for c.traceInfo.firstLine.remoteAddr to be set.
|
||||
defer func() {
|
||||
if e != nil {
|
||||
c.traceInfo.tr.LazyLog(&fmtStringer{"%v", []interface{}{e}}, true)
|
||||
c.traceInfo.tr.SetError()
|
||||
}
|
||||
}()
|
||||
}
|
||||
ctx = newContextWithRPCInfo(ctx, c.failFast)
|
||||
sh := cc.dopts.copts.StatsHandler
|
||||
if sh != nil {
|
||||
ctx = sh.TagRPC(ctx, &stats.RPCTagInfo{FullMethodName: method, FailFast: c.failFast})
|
||||
begin := &stats.Begin{
|
||||
Client: true,
|
||||
BeginTime: time.Now(),
|
||||
FailFast: c.failFast,
|
||||
}
|
||||
sh.HandleRPC(ctx, begin)
|
||||
defer func() {
|
||||
end := &stats.End{
|
||||
Client: true,
|
||||
EndTime: time.Now(),
|
||||
Error: e,
|
||||
}
|
||||
sh.HandleRPC(ctx, end)
|
||||
}()
|
||||
}
|
||||
topts := &transport.Options{
|
||||
Last: true,
|
||||
Delay: false,
|
||||
}
|
||||
func invoke(ctx context.Context, method string, req, reply interface{}, cc *ClientConn, opts ...CallOption) error {
|
||||
// TODO: implement retries in clientStream and make this simply
|
||||
// newClientStream, SendMsg, RecvMsg.
|
||||
firstAttempt := true
|
||||
for {
|
||||
var (
|
||||
err error
|
||||
t transport.ClientTransport
|
||||
stream *transport.Stream
|
||||
// Record the done handler from Balancer.Get(...). It is called once the
|
||||
// RPC has completed or failed.
|
||||
done func(balancer.DoneInfo)
|
||||
)
|
||||
// TODO(zhaoq): Need a formal spec of fail-fast.
|
||||
callHdr := &transport.CallHdr{
|
||||
Host: cc.authority,
|
||||
Method: method,
|
||||
}
|
||||
if cc.dopts.cp != nil {
|
||||
callHdr.SendCompress = cc.dopts.cp.Type()
|
||||
}
|
||||
if c.creds != nil {
|
||||
callHdr.Creds = c.creds
|
||||
}
|
||||
|
||||
t, done, err = cc.getTransport(ctx, c.failFast)
|
||||
csInt, err := newClientStream(ctx, unaryStreamDesc, cc, method, opts...)
|
||||
if err != nil {
|
||||
// TODO(zhaoq): Probably revisit the error handling.
|
||||
if _, ok := status.FromError(err); ok {
|
||||
return err
|
||||
}
|
||||
if err == errConnClosing || err == errConnUnavailable {
|
||||
if c.failFast {
|
||||
return Errorf(codes.Unavailable, "%v", err)
|
||||
}
|
||||
return err
|
||||
}
|
||||
cs := csInt.(*clientStream)
|
||||
if err := cs.SendMsg(req); err != nil {
|
||||
if !cs.c.failFast && cs.attempt.s.Unprocessed() && firstAttempt {
|
||||
// TODO: Add a field to header for grpc-transparent-retry-attempts
|
||||
firstAttempt = false
|
||||
continue
|
||||
}
|
||||
// All the other errors are treated as Internal errors.
|
||||
return Errorf(codes.Internal, "%v", err)
|
||||
return err
|
||||
}
|
||||
if c.traceInfo.tr != nil {
|
||||
c.traceInfo.tr.LazyLog(&payload{sent: true, msg: args}, true)
|
||||
}
|
||||
stream, err = t.NewStream(ctx, callHdr)
|
||||
if err != nil {
|
||||
if done != nil {
|
||||
if _, ok := err.(transport.ConnectionError); ok {
|
||||
// If error is connection error, transport was sending data on wire,
|
||||
// and we are not sure if anything has been sent on wire.
|
||||
// If error is not connection error, we are sure nothing has been sent.
|
||||
updateRPCInfoInContext(ctx, rpcInfo{bytesSent: true, bytesReceived: false})
|
||||
}
|
||||
done(balancer.DoneInfo{Err: err})
|
||||
}
|
||||
if _, ok := err.(transport.ConnectionError); (ok || err == transport.ErrStreamDrain) && !c.failFast {
|
||||
if err := cs.RecvMsg(reply); err != nil {
|
||||
if !cs.c.failFast && cs.attempt.s.Unprocessed() && firstAttempt {
|
||||
// TODO: Add a field to header for grpc-transparent-retry-attempts
|
||||
firstAttempt = false
|
||||
continue
|
||||
}
|
||||
return toRPCErr(err)
|
||||
return err
|
||||
}
|
||||
if peer, ok := peer.FromContext(stream.Context()); ok {
|
||||
c.peer = peer
|
||||
}
|
||||
err = sendRequest(ctx, cc.dopts, cc.dopts.cp, c, callHdr, stream, t, args, topts)
|
||||
if err != nil {
|
||||
if done != nil {
|
||||
updateRPCInfoInContext(ctx, rpcInfo{
|
||||
bytesSent: stream.BytesSent(),
|
||||
bytesReceived: stream.BytesReceived(),
|
||||
})
|
||||
done(balancer.DoneInfo{Err: err})
|
||||
}
|
||||
// Retry a non-failfast RPC when
|
||||
// i) there is a connection error; or
|
||||
// ii) the server started to drain before this RPC was initiated.
|
||||
if _, ok := err.(transport.ConnectionError); (ok || err == transport.ErrStreamDrain) && !c.failFast {
|
||||
continue
|
||||
}
|
||||
return toRPCErr(err)
|
||||
}
|
||||
err = recvResponse(ctx, cc.dopts, t, c, stream, reply)
|
||||
if err != nil {
|
||||
if done != nil {
|
||||
updateRPCInfoInContext(ctx, rpcInfo{
|
||||
bytesSent: stream.BytesSent(),
|
||||
bytesReceived: stream.BytesReceived(),
|
||||
})
|
||||
done(balancer.DoneInfo{Err: err})
|
||||
}
|
||||
if _, ok := err.(transport.ConnectionError); (ok || err == transport.ErrStreamDrain) && !c.failFast {
|
||||
continue
|
||||
}
|
||||
return toRPCErr(err)
|
||||
}
|
||||
if c.traceInfo.tr != nil {
|
||||
c.traceInfo.tr.LazyLog(&payload{sent: false, msg: reply}, true)
|
||||
}
|
||||
t.CloseStream(stream, nil)
|
||||
if done != nil {
|
||||
updateRPCInfoInContext(ctx, rpcInfo{
|
||||
bytesSent: stream.BytesSent(),
|
||||
bytesReceived: stream.BytesReceived(),
|
||||
})
|
||||
done(balancer.DoneInfo{Err: err})
|
||||
}
|
||||
return stream.Status().Err()
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
573
vendor/google.golang.org/grpc/channelz/funcs.go
generated
vendored
Normal file
@ -0,0 +1,573 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2018 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 channelz defines APIs for enabling channelz service, entry
|
||||
// registration/deletion, and accessing channelz data. It also defines channelz
|
||||
// metric struct formats.
|
||||
//
|
||||
// All APIs in this package are experimental.
|
||||
package channelz
|
||||
|
||||
import (
|
||||
"sort"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
|
||||
"google.golang.org/grpc/grpclog"
|
||||
)
|
||||
|
||||
var (
|
||||
db dbWrapper
|
||||
idGen idGenerator
|
||||
// EntryPerPage defines the number of channelz entries to be shown on a web page.
|
||||
EntryPerPage = 50
|
||||
curState int32
|
||||
)
|
||||
|
||||
// TurnOn turns on channelz data collection.
|
||||
func TurnOn() {
|
||||
if !IsOn() {
|
||||
NewChannelzStorage()
|
||||
atomic.StoreInt32(&curState, 1)
|
||||
}
|
||||
}
|
||||
|
||||
// IsOn returns whether channelz data collection is on.
|
||||
func IsOn() bool {
|
||||
return atomic.CompareAndSwapInt32(&curState, 1, 1)
|
||||
}
|
||||
|
||||
// dbWarpper wraps around a reference to internal channelz data storage, and
|
||||
// provide synchronized functionality to set and get the reference.
|
||||
type dbWrapper struct {
|
||||
mu sync.RWMutex
|
||||
DB *channelMap
|
||||
}
|
||||
|
||||
func (d *dbWrapper) set(db *channelMap) {
|
||||
d.mu.Lock()
|
||||
d.DB = db
|
||||
d.mu.Unlock()
|
||||
}
|
||||
|
||||
func (d *dbWrapper) get() *channelMap {
|
||||
d.mu.RLock()
|
||||
defer d.mu.RUnlock()
|
||||
return d.DB
|
||||
}
|
||||
|
||||
// NewChannelzStorage initializes channelz data storage and id generator.
|
||||
//
|
||||
// Note: This function is exported for testing purpose only. User should not call
|
||||
// it in most cases.
|
||||
func NewChannelzStorage() {
|
||||
db.set(&channelMap{
|
||||
topLevelChannels: make(map[int64]struct{}),
|
||||
channels: make(map[int64]*channel),
|
||||
listenSockets: make(map[int64]*listenSocket),
|
||||
normalSockets: make(map[int64]*normalSocket),
|
||||
servers: make(map[int64]*server),
|
||||
subChannels: make(map[int64]*subChannel),
|
||||
})
|
||||
idGen.reset()
|
||||
}
|
||||
|
||||
// GetTopChannels returns a slice of top channel's ChannelMetric, along with a
|
||||
// boolean indicating whether there's more top channels to be queried for.
|
||||
//
|
||||
// The arg id specifies that only top channel with id at or above it will be included
|
||||
// in the result. The returned slice is up to a length of EntryPerPage, and is
|
||||
// sorted in ascending id order.
|
||||
func GetTopChannels(id int64) ([]*ChannelMetric, bool) {
|
||||
return db.get().GetTopChannels(id)
|
||||
}
|
||||
|
||||
// GetServers returns a slice of server's ServerMetric, along with a
|
||||
// boolean indicating whether there's more servers to be queried for.
|
||||
//
|
||||
// The arg id specifies that only server with id at or above it will be included
|
||||
// in the result. The returned slice is up to a length of EntryPerPage, and is
|
||||
// sorted in ascending id order.
|
||||
func GetServers(id int64) ([]*ServerMetric, bool) {
|
||||
return db.get().GetServers(id)
|
||||
}
|
||||
|
||||
// GetServerSockets returns a slice of server's (identified by id) normal socket's
|
||||
// SocketMetric, along with a boolean indicating whether there's more sockets to
|
||||
// be queried for.
|
||||
//
|
||||
// The arg startID specifies that only sockets with id at or above it will be
|
||||
// included in the result. The returned slice is up to a length of EntryPerPage,
|
||||
// and is sorted in ascending id order.
|
||||
func GetServerSockets(id int64, startID int64) ([]*SocketMetric, bool) {
|
||||
return db.get().GetServerSockets(id, startID)
|
||||
}
|
||||
|
||||
// GetChannel returns the ChannelMetric for the channel (identified by id).
|
||||
func GetChannel(id int64) *ChannelMetric {
|
||||
return db.get().GetChannel(id)
|
||||
}
|
||||
|
||||
// GetSubChannel returns the SubChannelMetric for the subchannel (identified by id).
|
||||
func GetSubChannel(id int64) *SubChannelMetric {
|
||||
return db.get().GetSubChannel(id)
|
||||
}
|
||||
|
||||
// GetSocket returns the SocketInternalMetric for the socket (identified by id).
|
||||
func GetSocket(id int64) *SocketMetric {
|
||||
return db.get().GetSocket(id)
|
||||
}
|
||||
|
||||
// RegisterChannel registers the given channel c in channelz database with ref
|
||||
// as its reference name, and add it to the child list of its parent (identified
|
||||
// by pid). pid = 0 means no parent. It returns the unique channelz tracking id
|
||||
// assigned to this channel.
|
||||
func RegisterChannel(c Channel, pid int64, ref string) int64 {
|
||||
id := idGen.genID()
|
||||
cn := &channel{
|
||||
refName: ref,
|
||||
c: c,
|
||||
subChans: make(map[int64]string),
|
||||
nestedChans: make(map[int64]string),
|
||||
id: id,
|
||||
pid: pid,
|
||||
}
|
||||
if pid == 0 {
|
||||
db.get().addChannel(id, cn, true, pid, ref)
|
||||
} else {
|
||||
db.get().addChannel(id, cn, false, pid, ref)
|
||||
}
|
||||
return id
|
||||
}
|
||||
|
||||
// RegisterSubChannel registers the given channel c in channelz database with ref
|
||||
// as its reference name, and add it to the child list of its parent (identified
|
||||
// by pid). It returns the unique channelz tracking id assigned to this subchannel.
|
||||
func RegisterSubChannel(c Channel, pid int64, ref string) int64 {
|
||||
if pid == 0 {
|
||||
grpclog.Error("a SubChannel's parent id cannot be 0")
|
||||
return 0
|
||||
}
|
||||
id := idGen.genID()
|
||||
sc := &subChannel{
|
||||
refName: ref,
|
||||
c: c,
|
||||
sockets: make(map[int64]string),
|
||||
id: id,
|
||||
pid: pid,
|
||||
}
|
||||
db.get().addSubChannel(id, sc, pid, ref)
|
||||
return id
|
||||
}
|
||||
|
||||
// RegisterServer registers the given server s in channelz database. It returns
|
||||
// the unique channelz tracking id assigned to this server.
|
||||
func RegisterServer(s Server, ref string) int64 {
|
||||
id := idGen.genID()
|
||||
svr := &server{
|
||||
refName: ref,
|
||||
s: s,
|
||||
sockets: make(map[int64]string),
|
||||
listenSockets: make(map[int64]string),
|
||||
id: id,
|
||||
}
|
||||
db.get().addServer(id, svr)
|
||||
return id
|
||||
}
|
||||
|
||||
// RegisterListenSocket registers the given listen socket s in channelz database
|
||||
// with ref as its reference name, and add it to the child list of its parent
|
||||
// (identified by pid). It returns the unique channelz tracking id assigned to
|
||||
// this listen socket.
|
||||
func RegisterListenSocket(s Socket, pid int64, ref string) int64 {
|
||||
if pid == 0 {
|
||||
grpclog.Error("a ListenSocket's parent id cannot be 0")
|
||||
return 0
|
||||
}
|
||||
id := idGen.genID()
|
||||
ls := &listenSocket{refName: ref, s: s, id: id, pid: pid}
|
||||
db.get().addListenSocket(id, ls, pid, ref)
|
||||
return id
|
||||
}
|
||||
|
||||
// RegisterNormalSocket registers the given normal socket s in channelz database
|
||||
// with ref as its reference name, and add it to the child list of its parent
|
||||
// (identified by pid). It returns the unique channelz tracking id assigned to
|
||||
// this normal socket.
|
||||
func RegisterNormalSocket(s Socket, pid int64, ref string) int64 {
|
||||
if pid == 0 {
|
||||
grpclog.Error("a NormalSocket's parent id cannot be 0")
|
||||
return 0
|
||||
}
|
||||
id := idGen.genID()
|
||||
ns := &normalSocket{refName: ref, s: s, id: id, pid: pid}
|
||||
db.get().addNormalSocket(id, ns, pid, ref)
|
||||
return id
|
||||
}
|
||||
|
||||
// RemoveEntry removes an entry with unique channelz trakcing id to be id from
|
||||
// channelz database.
|
||||
func RemoveEntry(id int64) {
|
||||
db.get().removeEntry(id)
|
||||
}
|
||||
|
||||
// channelMap is the storage data structure for channelz.
|
||||
// Methods of channelMap can be divided in two two categories with respect to locking.
|
||||
// 1. Methods acquire the global lock.
|
||||
// 2. Methods that can only be called when global lock is held.
|
||||
// A second type of method need always to be called inside a first type of method.
|
||||
type channelMap struct {
|
||||
mu sync.RWMutex
|
||||
topLevelChannels map[int64]struct{}
|
||||
servers map[int64]*server
|
||||
channels map[int64]*channel
|
||||
subChannels map[int64]*subChannel
|
||||
listenSockets map[int64]*listenSocket
|
||||
normalSockets map[int64]*normalSocket
|
||||
}
|
||||
|
||||
func (c *channelMap) addServer(id int64, s *server) {
|
||||
c.mu.Lock()
|
||||
s.cm = c
|
||||
c.servers[id] = s
|
||||
c.mu.Unlock()
|
||||
}
|
||||
|
||||
func (c *channelMap) addChannel(id int64, cn *channel, isTopChannel bool, pid int64, ref string) {
|
||||
c.mu.Lock()
|
||||
cn.cm = c
|
||||
c.channels[id] = cn
|
||||
if isTopChannel {
|
||||
c.topLevelChannels[id] = struct{}{}
|
||||
} else {
|
||||
c.findEntry(pid).addChild(id, cn)
|
||||
}
|
||||
c.mu.Unlock()
|
||||
}
|
||||
|
||||
func (c *channelMap) addSubChannel(id int64, sc *subChannel, pid int64, ref string) {
|
||||
c.mu.Lock()
|
||||
sc.cm = c
|
||||
c.subChannels[id] = sc
|
||||
c.findEntry(pid).addChild(id, sc)
|
||||
c.mu.Unlock()
|
||||
}
|
||||
|
||||
func (c *channelMap) addListenSocket(id int64, ls *listenSocket, pid int64, ref string) {
|
||||
c.mu.Lock()
|
||||
ls.cm = c
|
||||
c.listenSockets[id] = ls
|
||||
c.findEntry(pid).addChild(id, ls)
|
||||
c.mu.Unlock()
|
||||
}
|
||||
|
||||
func (c *channelMap) addNormalSocket(id int64, ns *normalSocket, pid int64, ref string) {
|
||||
c.mu.Lock()
|
||||
ns.cm = c
|
||||
c.normalSockets[id] = ns
|
||||
c.findEntry(pid).addChild(id, ns)
|
||||
c.mu.Unlock()
|
||||
}
|
||||
|
||||
// removeEntry triggers the removal of an entry, which may not indeed delete the
|
||||
// entry, if it has to wait on the deletion of its children, or may lead to a chain
|
||||
// of entry deletion. For example, deleting the last socket of a gracefully shutting
|
||||
// down server will lead to the server being also deleted.
|
||||
func (c *channelMap) removeEntry(id int64) {
|
||||
c.mu.Lock()
|
||||
c.findEntry(id).triggerDelete()
|
||||
c.mu.Unlock()
|
||||
}
|
||||
|
||||
// c.mu must be held by the caller.
|
||||
func (c *channelMap) findEntry(id int64) entry {
|
||||
var v entry
|
||||
var ok bool
|
||||
if v, ok = c.channels[id]; ok {
|
||||
return v
|
||||
}
|
||||
if v, ok = c.subChannels[id]; ok {
|
||||
return v
|
||||
}
|
||||
if v, ok = c.servers[id]; ok {
|
||||
return v
|
||||
}
|
||||
if v, ok = c.listenSockets[id]; ok {
|
||||
return v
|
||||
}
|
||||
if v, ok = c.normalSockets[id]; ok {
|
||||
return v
|
||||
}
|
||||
return &dummyEntry{idNotFound: id}
|
||||
}
|
||||
|
||||
// c.mu must be held by the caller
|
||||
// deleteEntry simply deletes an entry from the channelMap. Before calling this
|
||||
// method, caller must check this entry is ready to be deleted, i.e removeEntry()
|
||||
// has been called on it, and no children still exist.
|
||||
// Conditionals are ordered by the expected frequency of deletion of each entity
|
||||
// type, in order to optimize performance.
|
||||
func (c *channelMap) deleteEntry(id int64) {
|
||||
var ok bool
|
||||
if _, ok = c.normalSockets[id]; ok {
|
||||
delete(c.normalSockets, id)
|
||||
return
|
||||
}
|
||||
if _, ok = c.subChannels[id]; ok {
|
||||
delete(c.subChannels, id)
|
||||
return
|
||||
}
|
||||
if _, ok = c.channels[id]; ok {
|
||||
delete(c.channels, id)
|
||||
delete(c.topLevelChannels, id)
|
||||
return
|
||||
}
|
||||
if _, ok = c.listenSockets[id]; ok {
|
||||
delete(c.listenSockets, id)
|
||||
return
|
||||
}
|
||||
if _, ok = c.servers[id]; ok {
|
||||
delete(c.servers, id)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
type int64Slice []int64
|
||||
|
||||
func (s int64Slice) Len() int { return len(s) }
|
||||
func (s int64Slice) Swap(i, j int) { s[i], s[j] = s[j], s[i] }
|
||||
func (s int64Slice) Less(i, j int) bool { return s[i] < s[j] }
|
||||
|
||||
func copyMap(m map[int64]string) map[int64]string {
|
||||
n := make(map[int64]string)
|
||||
for k, v := range m {
|
||||
n[k] = v
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
func min(a, b int) int {
|
||||
if a < b {
|
||||
return a
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
func (c *channelMap) GetTopChannels(id int64) ([]*ChannelMetric, bool) {
|
||||
c.mu.RLock()
|
||||
l := len(c.topLevelChannels)
|
||||
ids := make([]int64, 0, l)
|
||||
cns := make([]*channel, 0, min(l, EntryPerPage))
|
||||
|
||||
for k := range c.topLevelChannels {
|
||||
ids = append(ids, k)
|
||||
}
|
||||
sort.Sort(int64Slice(ids))
|
||||
idx := sort.Search(len(ids), func(i int) bool { return ids[i] >= id })
|
||||
count := 0
|
||||
var end bool
|
||||
var t []*ChannelMetric
|
||||
for i, v := range ids[idx:] {
|
||||
if count == EntryPerPage {
|
||||
break
|
||||
}
|
||||
if cn, ok := c.channels[v]; ok {
|
||||
cns = append(cns, cn)
|
||||
t = append(t, &ChannelMetric{
|
||||
NestedChans: copyMap(cn.nestedChans),
|
||||
SubChans: copyMap(cn.subChans),
|
||||
})
|
||||
count++
|
||||
}
|
||||
if i == len(ids[idx:])-1 {
|
||||
end = true
|
||||
break
|
||||
}
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
if count == 0 {
|
||||
end = true
|
||||
}
|
||||
|
||||
for i, cn := range cns {
|
||||
t[i].ChannelData = cn.c.ChannelzMetric()
|
||||
t[i].ID = cn.id
|
||||
t[i].RefName = cn.refName
|
||||
}
|
||||
return t, end
|
||||
}
|
||||
|
||||
func (c *channelMap) GetServers(id int64) ([]*ServerMetric, bool) {
|
||||
c.mu.RLock()
|
||||
l := len(c.servers)
|
||||
ids := make([]int64, 0, l)
|
||||
ss := make([]*server, 0, min(l, EntryPerPage))
|
||||
for k := range c.servers {
|
||||
ids = append(ids, k)
|
||||
}
|
||||
sort.Sort(int64Slice(ids))
|
||||
idx := sort.Search(len(ids), func(i int) bool { return ids[i] >= id })
|
||||
count := 0
|
||||
var end bool
|
||||
var s []*ServerMetric
|
||||
for i, v := range ids[idx:] {
|
||||
if count == EntryPerPage {
|
||||
break
|
||||
}
|
||||
if svr, ok := c.servers[v]; ok {
|
||||
ss = append(ss, svr)
|
||||
s = append(s, &ServerMetric{
|
||||
ListenSockets: copyMap(svr.listenSockets),
|
||||
})
|
||||
count++
|
||||
}
|
||||
if i == len(ids[idx:])-1 {
|
||||
end = true
|
||||
break
|
||||
}
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
if count == 0 {
|
||||
end = true
|
||||
}
|
||||
|
||||
for i, svr := range ss {
|
||||
s[i].ServerData = svr.s.ChannelzMetric()
|
||||
s[i].ID = svr.id
|
||||
s[i].RefName = svr.refName
|
||||
}
|
||||
return s, end
|
||||
}
|
||||
|
||||
func (c *channelMap) GetServerSockets(id int64, startID int64) ([]*SocketMetric, bool) {
|
||||
var svr *server
|
||||
var ok bool
|
||||
c.mu.RLock()
|
||||
if svr, ok = c.servers[id]; !ok {
|
||||
// server with id doesn't exist.
|
||||
c.mu.RUnlock()
|
||||
return nil, true
|
||||
}
|
||||
svrskts := svr.sockets
|
||||
l := len(svrskts)
|
||||
ids := make([]int64, 0, l)
|
||||
sks := make([]*normalSocket, 0, min(l, EntryPerPage))
|
||||
for k := range svrskts {
|
||||
ids = append(ids, k)
|
||||
}
|
||||
sort.Sort((int64Slice(ids)))
|
||||
idx := sort.Search(len(ids), func(i int) bool { return ids[i] >= id })
|
||||
count := 0
|
||||
var end bool
|
||||
for i, v := range ids[idx:] {
|
||||
if count == EntryPerPage {
|
||||
break
|
||||
}
|
||||
if ns, ok := c.normalSockets[v]; ok {
|
||||
sks = append(sks, ns)
|
||||
count++
|
||||
}
|
||||
if i == len(ids[idx:])-1 {
|
||||
end = true
|
||||
break
|
||||
}
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
if count == 0 {
|
||||
end = true
|
||||
}
|
||||
var s []*SocketMetric
|
||||
for _, ns := range sks {
|
||||
sm := &SocketMetric{}
|
||||
sm.SocketData = ns.s.ChannelzMetric()
|
||||
sm.ID = ns.id
|
||||
sm.RefName = ns.refName
|
||||
s = append(s, sm)
|
||||
}
|
||||
return s, end
|
||||
}
|
||||
|
||||
func (c *channelMap) GetChannel(id int64) *ChannelMetric {
|
||||
cm := &ChannelMetric{}
|
||||
var cn *channel
|
||||
var ok bool
|
||||
c.mu.RLock()
|
||||
if cn, ok = c.channels[id]; !ok {
|
||||
// channel with id doesn't exist.
|
||||
c.mu.RUnlock()
|
||||
return nil
|
||||
}
|
||||
cm.NestedChans = copyMap(cn.nestedChans)
|
||||
cm.SubChans = copyMap(cn.subChans)
|
||||
c.mu.RUnlock()
|
||||
cm.ChannelData = cn.c.ChannelzMetric()
|
||||
cm.ID = cn.id
|
||||
cm.RefName = cn.refName
|
||||
return cm
|
||||
}
|
||||
|
||||
func (c *channelMap) GetSubChannel(id int64) *SubChannelMetric {
|
||||
cm := &SubChannelMetric{}
|
||||
var sc *subChannel
|
||||
var ok bool
|
||||
c.mu.RLock()
|
||||
if sc, ok = c.subChannels[id]; !ok {
|
||||
// subchannel with id doesn't exist.
|
||||
c.mu.RUnlock()
|
||||
return nil
|
||||
}
|
||||
cm.Sockets = copyMap(sc.sockets)
|
||||
c.mu.RUnlock()
|
||||
cm.ChannelData = sc.c.ChannelzMetric()
|
||||
cm.ID = sc.id
|
||||
cm.RefName = sc.refName
|
||||
return cm
|
||||
}
|
||||
|
||||
func (c *channelMap) GetSocket(id int64) *SocketMetric {
|
||||
sm := &SocketMetric{}
|
||||
c.mu.RLock()
|
||||
if ls, ok := c.listenSockets[id]; ok {
|
||||
c.mu.RUnlock()
|
||||
sm.SocketData = ls.s.ChannelzMetric()
|
||||
sm.ID = ls.id
|
||||
sm.RefName = ls.refName
|
||||
return sm
|
||||
}
|
||||
if ns, ok := c.normalSockets[id]; ok {
|
||||
c.mu.RUnlock()
|
||||
sm.SocketData = ns.s.ChannelzMetric()
|
||||
sm.ID = ns.id
|
||||
sm.RefName = ns.refName
|
||||
return sm
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
type idGenerator struct {
|
||||
id int64
|
||||
}
|
||||
|
||||
func (i *idGenerator) reset() {
|
||||
atomic.StoreInt64(&i.id, 0)
|
||||
}
|
||||
|
||||
func (i *idGenerator) genID() int64 {
|
||||
return atomic.AddInt64(&i.id, 1)
|
||||
}
|
418
vendor/google.golang.org/grpc/channelz/types.go
generated
vendored
Normal file
@ -0,0 +1,418 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2018 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 channelz
|
||||
|
||||
import (
|
||||
"net"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc/connectivity"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
)
|
||||
|
||||
// entry represents a node in the channelz database.
|
||||
type entry interface {
|
||||
// addChild adds a child e, whose channelz id is id to child list
|
||||
addChild(id int64, e entry)
|
||||
// deleteChild deletes a child with channelz id to be id from child list
|
||||
deleteChild(id int64)
|
||||
// triggerDelete tries to delete self from channelz database. However, if child
|
||||
// list is not empty, then deletion from the database is on hold until the last
|
||||
// child is deleted from database.
|
||||
triggerDelete()
|
||||
// deleteSelfIfReady check whether triggerDelete() has been called before, and whether child
|
||||
// list is now empty. If both conditions are met, then delete self from database.
|
||||
deleteSelfIfReady()
|
||||
}
|
||||
|
||||
// dummyEntry is a fake entry to handle entry not found case.
|
||||
type dummyEntry struct {
|
||||
idNotFound int64
|
||||
}
|
||||
|
||||
func (d *dummyEntry) addChild(id int64, e entry) {
|
||||
// Note: It is possible for a normal program to reach here under race condition.
|
||||
// For example, there could be a race between ClientConn.Close() info being propagated
|
||||
// to addrConn and http2Client. ClientConn.Close() cancel the context and result
|
||||
// in http2Client to error. The error info is then caught by transport monitor
|
||||
// and before addrConn.tearDown() is called in side ClientConn.Close(). Therefore,
|
||||
// the addrConn will create a new transport. And when registering the new transport in
|
||||
// channelz, its parent addrConn could have already been torn down and deleted
|
||||
// from channelz tracking, and thus reach the code here.
|
||||
grpclog.Infof("attempt to add child of type %T with id %d to a parent (id=%d) that doesn't currently exist", e, id, d.idNotFound)
|
||||
}
|
||||
|
||||
func (d *dummyEntry) deleteChild(id int64) {
|
||||
// It is possible for a normal program to reach here under race condition.
|
||||
// Refer to the example described in addChild().
|
||||
grpclog.Infof("attempt to delete child with id %d from a parent (id=%d) that doesn't currently exist", id, d.idNotFound)
|
||||
}
|
||||
|
||||
func (d *dummyEntry) triggerDelete() {
|
||||
grpclog.Warningf("attempt to delete an entry (id=%d) that doesn't currently exist", d.idNotFound)
|
||||
}
|
||||
|
||||
func (*dummyEntry) deleteSelfIfReady() {
|
||||
// code should not reach here. deleteSelfIfReady is always called on an existing entry.
|
||||
}
|
||||
|
||||
// ChannelMetric defines the info channelz provides for a specific Channel, which
|
||||
// includes ChannelInternalMetric and channelz-specific data, such as channelz id,
|
||||
// child list, etc.
|
||||
type ChannelMetric struct {
|
||||
// ID is the channelz id of this channel.
|
||||
ID int64
|
||||
// RefName is the human readable reference string of this channel.
|
||||
RefName string
|
||||
// ChannelData contains channel internal metric reported by the channel through
|
||||
// ChannelzMetric().
|
||||
ChannelData *ChannelInternalMetric
|
||||
// NestedChans tracks the nested channel type children of this channel in the format of
|
||||
// a map from nested channel channelz id to corresponding reference string.
|
||||
NestedChans map[int64]string
|
||||
// SubChans tracks the subchannel type children of this channel in the format of a
|
||||
// map from subchannel channelz id to corresponding reference string.
|
||||
SubChans map[int64]string
|
||||
// Sockets tracks the socket type children of this channel in the format of a map
|
||||
// from socket channelz id to corresponding reference string.
|
||||
// Note current grpc implementation doesn't allow channel having sockets directly,
|
||||
// therefore, this is field is unused.
|
||||
Sockets map[int64]string
|
||||
}
|
||||
|
||||
// SubChannelMetric defines the info channelz provides for a specific SubChannel,
|
||||
// which includes ChannelInternalMetric and channelz-specific data, such as
|
||||
// channelz id, child list, etc.
|
||||
type SubChannelMetric struct {
|
||||
// ID is the channelz id of this subchannel.
|
||||
ID int64
|
||||
// RefName is the human readable reference string of this subchannel.
|
||||
RefName string
|
||||
// ChannelData contains subchannel internal metric reported by the subchannel
|
||||
// through ChannelzMetric().
|
||||
ChannelData *ChannelInternalMetric
|
||||
// NestedChans tracks the nested channel type children of this subchannel in the format of
|
||||
// a map from nested channel channelz id to corresponding reference string.
|
||||
// Note current grpc implementation doesn't allow subchannel to have nested channels
|
||||
// as children, therefore, this field is unused.
|
||||
NestedChans map[int64]string
|
||||
// SubChans tracks the subchannel type children of this subchannel in the format of a
|
||||
// map from subchannel channelz id to corresponding reference string.
|
||||
// Note current grpc implementation doesn't allow subchannel to have subchannels
|
||||
// as children, therefore, this field is unused.
|
||||
SubChans map[int64]string
|
||||
// Sockets tracks the socket type children of this subchannel in the format of a map
|
||||
// from socket channelz id to corresponding reference string.
|
||||
Sockets map[int64]string
|
||||
}
|
||||
|
||||
// ChannelInternalMetric defines the struct that the implementor of Channel interface
|
||||
// should return from ChannelzMetric().
|
||||
type ChannelInternalMetric struct {
|
||||
// current connectivity state of the channel.
|
||||
State connectivity.State
|
||||
// The target this channel originally tried to connect to. May be absent
|
||||
Target string
|
||||
// The number of calls started on the channel.
|
||||
CallsStarted int64
|
||||
// The number of calls that have completed with an OK status.
|
||||
CallsSucceeded int64
|
||||
// The number of calls that have a completed with a non-OK status.
|
||||
CallsFailed int64
|
||||
// The last time a call was started on the channel.
|
||||
LastCallStartedTimestamp time.Time
|
||||
//TODO: trace
|
||||
}
|
||||
|
||||
// Channel is the interface that should be satisfied in order to be tracked by
|
||||
// channelz as Channel or SubChannel.
|
||||
type Channel interface {
|
||||
ChannelzMetric() *ChannelInternalMetric
|
||||
}
|
||||
|
||||
type channel struct {
|
||||
refName string
|
||||
c Channel
|
||||
closeCalled bool
|
||||
nestedChans map[int64]string
|
||||
subChans map[int64]string
|
||||
id int64
|
||||
pid int64
|
||||
cm *channelMap
|
||||
}
|
||||
|
||||
func (c *channel) addChild(id int64, e entry) {
|
||||
switch v := e.(type) {
|
||||
case *subChannel:
|
||||
c.subChans[id] = v.refName
|
||||
case *channel:
|
||||
c.nestedChans[id] = v.refName
|
||||
default:
|
||||
grpclog.Errorf("cannot add a child (id = %d) of type %T to a channel", id, e)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *channel) deleteChild(id int64) {
|
||||
delete(c.subChans, id)
|
||||
delete(c.nestedChans, id)
|
||||
c.deleteSelfIfReady()
|
||||
}
|
||||
|
||||
func (c *channel) triggerDelete() {
|
||||
c.closeCalled = true
|
||||
c.deleteSelfIfReady()
|
||||
}
|
||||
|
||||
func (c *channel) deleteSelfIfReady() {
|
||||
if !c.closeCalled || len(c.subChans)+len(c.nestedChans) != 0 {
|
||||
return
|
||||
}
|
||||
c.cm.deleteEntry(c.id)
|
||||
// not top channel
|
||||
if c.pid != 0 {
|
||||
c.cm.findEntry(c.pid).deleteChild(c.id)
|
||||
}
|
||||
}
|
||||
|
||||
type subChannel struct {
|
||||
refName string
|
||||
c Channel
|
||||
closeCalled bool
|
||||
sockets map[int64]string
|
||||
id int64
|
||||
pid int64
|
||||
cm *channelMap
|
||||
}
|
||||
|
||||
func (sc *subChannel) addChild(id int64, e entry) {
|
||||
if v, ok := e.(*normalSocket); ok {
|
||||
sc.sockets[id] = v.refName
|
||||
} else {
|
||||
grpclog.Errorf("cannot add a child (id = %d) of type %T to a subChannel", id, e)
|
||||
}
|
||||
}
|
||||
|
||||
func (sc *subChannel) deleteChild(id int64) {
|
||||
delete(sc.sockets, id)
|
||||
sc.deleteSelfIfReady()
|
||||
}
|
||||
|
||||
func (sc *subChannel) triggerDelete() {
|
||||
sc.closeCalled = true
|
||||
sc.deleteSelfIfReady()
|
||||
}
|
||||
|
||||
func (sc *subChannel) deleteSelfIfReady() {
|
||||
if !sc.closeCalled || len(sc.sockets) != 0 {
|
||||
return
|
||||
}
|
||||
sc.cm.deleteEntry(sc.id)
|
||||
sc.cm.findEntry(sc.pid).deleteChild(sc.id)
|
||||
}
|
||||
|
||||
// SocketMetric defines the info channelz provides for a specific Socket, which
|
||||
// includes SocketInternalMetric and channelz-specific data, such as channelz id, etc.
|
||||
type SocketMetric struct {
|
||||
// ID is the channelz id of this socket.
|
||||
ID int64
|
||||
// RefName is the human readable reference string of this socket.
|
||||
RefName string
|
||||
// SocketData contains socket internal metric reported by the socket through
|
||||
// ChannelzMetric().
|
||||
SocketData *SocketInternalMetric
|
||||
}
|
||||
|
||||
// SocketInternalMetric defines the struct that the implementor of Socket interface
|
||||
// should return from ChannelzMetric().
|
||||
type SocketInternalMetric struct {
|
||||
// The number of streams that have been started.
|
||||
StreamsStarted int64
|
||||
// The number of streams that have ended successfully:
|
||||
// On client side, receiving frame with eos bit set.
|
||||
// On server side, sending frame with eos bit set.
|
||||
StreamsSucceeded int64
|
||||
// The number of streams that have ended unsuccessfully:
|
||||
// On client side, termination without receiving frame with eos bit set.
|
||||
// On server side, termination without sending frame with eos bit set.
|
||||
StreamsFailed int64
|
||||
// The number of messages successfully sent on this socket.
|
||||
MessagesSent int64
|
||||
MessagesReceived int64
|
||||
// The number of keep alives sent. This is typically implemented with HTTP/2
|
||||
// ping messages.
|
||||
KeepAlivesSent int64
|
||||
// The last time a stream was created by this endpoint. Usually unset for
|
||||
// servers.
|
||||
LastLocalStreamCreatedTimestamp time.Time
|
||||
// The last time a stream was created by the remote endpoint. Usually unset
|
||||
// for clients.
|
||||
LastRemoteStreamCreatedTimestamp time.Time
|
||||
// The last time a message was sent by this endpoint.
|
||||
LastMessageSentTimestamp time.Time
|
||||
// The last time a message was received by this endpoint.
|
||||
LastMessageReceivedTimestamp time.Time
|
||||
// The amount of window, granted to the local endpoint by the remote endpoint.
|
||||
// This may be slightly out of date due to network latency. This does NOT
|
||||
// include stream level or TCP level flow control info.
|
||||
LocalFlowControlWindow int64
|
||||
// The amount of window, granted to the remote endpoint by the local endpoint.
|
||||
// This may be slightly out of date due to network latency. This does NOT
|
||||
// include stream level or TCP level flow control info.
|
||||
RemoteFlowControlWindow int64
|
||||
// The locally bound address.
|
||||
LocalAddr net.Addr
|
||||
// The remote bound address. May be absent.
|
||||
RemoteAddr net.Addr
|
||||
// Optional, represents the name of the remote endpoint, if different than
|
||||
// the original target name.
|
||||
RemoteName string
|
||||
//TODO: socket options
|
||||
//TODO: Security
|
||||
}
|
||||
|
||||
// Socket is the interface that should be satisfied in order to be tracked by
|
||||
// channelz as Socket.
|
||||
type Socket interface {
|
||||
ChannelzMetric() *SocketInternalMetric
|
||||
}
|
||||
|
||||
type listenSocket struct {
|
||||
refName string
|
||||
s Socket
|
||||
id int64
|
||||
pid int64
|
||||
cm *channelMap
|
||||
}
|
||||
|
||||
func (ls *listenSocket) addChild(id int64, e entry) {
|
||||
grpclog.Errorf("cannot add a child (id = %d) of type %T to a listen socket", id, e)
|
||||
}
|
||||
|
||||
func (ls *listenSocket) deleteChild(id int64) {
|
||||
grpclog.Errorf("cannot delete a child (id = %d) from a listen socket", id)
|
||||
}
|
||||
|
||||
func (ls *listenSocket) triggerDelete() {
|
||||
ls.cm.deleteEntry(ls.id)
|
||||
ls.cm.findEntry(ls.pid).deleteChild(ls.id)
|
||||
}
|
||||
|
||||
func (ls *listenSocket) deleteSelfIfReady() {
|
||||
grpclog.Errorf("cannot call deleteSelfIfReady on a listen socket")
|
||||
}
|
||||
|
||||
type normalSocket struct {
|
||||
refName string
|
||||
s Socket
|
||||
id int64
|
||||
pid int64
|
||||
cm *channelMap
|
||||
}
|
||||
|
||||
func (ns *normalSocket) addChild(id int64, e entry) {
|
||||
grpclog.Errorf("cannot add a child (id = %d) of type %T to a normal socket", id, e)
|
||||
}
|
||||
|
||||
func (ns *normalSocket) deleteChild(id int64) {
|
||||
grpclog.Errorf("cannot delete a child (id = %d) from a normal socket", id)
|
||||
}
|
||||
|
||||
func (ns *normalSocket) triggerDelete() {
|
||||
ns.cm.deleteEntry(ns.id)
|
||||
ns.cm.findEntry(ns.pid).deleteChild(ns.id)
|
||||
}
|
||||
|
||||
func (ns *normalSocket) deleteSelfIfReady() {
|
||||
grpclog.Errorf("cannot call deleteSelfIfReady on a normal socket")
|
||||
}
|
||||
|
||||
// ServerMetric defines the info channelz provides for a specific Server, which
|
||||
// includes ServerInternalMetric and channelz-specific data, such as channelz id,
|
||||
// child list, etc.
|
||||
type ServerMetric struct {
|
||||
// ID is the channelz id of this server.
|
||||
ID int64
|
||||
// RefName is the human readable reference string of this server.
|
||||
RefName string
|
||||
// ServerData contains server internal metric reported by the server through
|
||||
// ChannelzMetric().
|
||||
ServerData *ServerInternalMetric
|
||||
// ListenSockets tracks the listener socket type children of this server in the
|
||||
// format of a map from socket channelz id to corresponding reference string.
|
||||
ListenSockets map[int64]string
|
||||
}
|
||||
|
||||
// ServerInternalMetric defines the struct that the implementor of Server interface
|
||||
// should return from ChannelzMetric().
|
||||
type ServerInternalMetric struct {
|
||||
// The number of incoming calls started on the server.
|
||||
CallsStarted int64
|
||||
// The number of incoming calls that have completed with an OK status.
|
||||
CallsSucceeded int64
|
||||
// The number of incoming calls that have a completed with a non-OK status.
|
||||
CallsFailed int64
|
||||
// The last time a call was started on the server.
|
||||
LastCallStartedTimestamp time.Time
|
||||
//TODO: trace
|
||||
}
|
||||
|
||||
// Server is the interface to be satisfied in order to be tracked by channelz as
|
||||
// Server.
|
||||
type Server interface {
|
||||
ChannelzMetric() *ServerInternalMetric
|
||||
}
|
||||
|
||||
type server struct {
|
||||
refName string
|
||||
s Server
|
||||
closeCalled bool
|
||||
sockets map[int64]string
|
||||
listenSockets map[int64]string
|
||||
id int64
|
||||
cm *channelMap
|
||||
}
|
||||
|
||||
func (s *server) addChild(id int64, e entry) {
|
||||
switch v := e.(type) {
|
||||
case *normalSocket:
|
||||
s.sockets[id] = v.refName
|
||||
case *listenSocket:
|
||||
s.listenSockets[id] = v.refName
|
||||
default:
|
||||
grpclog.Errorf("cannot add a child (id = %d) of type %T to a server", id, e)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *server) deleteChild(id int64) {
|
||||
delete(s.sockets, id)
|
||||
delete(s.listenSockets, id)
|
||||
s.deleteSelfIfReady()
|
||||
}
|
||||
|
||||
func (s *server) triggerDelete() {
|
||||
s.closeCalled = true
|
||||
s.deleteSelfIfReady()
|
||||
}
|
||||
|
||||
func (s *server) deleteSelfIfReady() {
|
||||
if !s.closeCalled || len(s.sockets)+len(s.listenSockets) != 0 {
|
||||
return
|
||||
}
|
||||
s.cm.deleteEntry(s.id)
|
||||
}
|
977
vendor/google.golang.org/grpc/clientconn.go
generated
vendored
88
vendor/google.golang.org/grpc/codec.go
generated
vendored
@ -19,86 +19,32 @@
|
||||
package grpc
|
||||
|
||||
import (
|
||||
"math"
|
||||
"sync"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"google.golang.org/grpc/encoding"
|
||||
_ "google.golang.org/grpc/encoding/proto" // to register the Codec for "proto"
|
||||
)
|
||||
|
||||
// baseCodec contains the functionality of both Codec and encoding.Codec, but
|
||||
// omits the name/string, which vary between the two and are not needed for
|
||||
// anything besides the registry in the encoding package.
|
||||
type baseCodec interface {
|
||||
Marshal(v interface{}) ([]byte, error)
|
||||
Unmarshal(data []byte, v interface{}) error
|
||||
}
|
||||
|
||||
var _ baseCodec = Codec(nil)
|
||||
var _ baseCodec = encoding.Codec(nil)
|
||||
|
||||
// Codec defines the interface gRPC uses to encode and decode messages.
|
||||
// Note that implementations of this interface must be thread safe;
|
||||
// a Codec's methods can be called from concurrent goroutines.
|
||||
//
|
||||
// Deprecated: use encoding.Codec instead.
|
||||
type Codec interface {
|
||||
// Marshal returns the wire format of v.
|
||||
Marshal(v interface{}) ([]byte, error)
|
||||
// Unmarshal parses the wire format into v.
|
||||
Unmarshal(data []byte, v interface{}) error
|
||||
// String returns the name of the Codec implementation. The returned
|
||||
// string will be used as part of content type in transmission.
|
||||
// String returns the name of the Codec implementation. This is unused by
|
||||
// gRPC.
|
||||
String() string
|
||||
}
|
||||
|
||||
// protoCodec is a Codec implementation with protobuf. It is the default codec for gRPC.
|
||||
type protoCodec struct {
|
||||
}
|
||||
|
||||
type cachedProtoBuffer struct {
|
||||
lastMarshaledSize uint32
|
||||
proto.Buffer
|
||||
}
|
||||
|
||||
func capToMaxInt32(val int) uint32 {
|
||||
if val > math.MaxInt32 {
|
||||
return uint32(math.MaxInt32)
|
||||
}
|
||||
return uint32(val)
|
||||
}
|
||||
|
||||
func (p protoCodec) marshal(v interface{}, cb *cachedProtoBuffer) ([]byte, error) {
|
||||
protoMsg := v.(proto.Message)
|
||||
newSlice := make([]byte, 0, cb.lastMarshaledSize)
|
||||
|
||||
cb.SetBuf(newSlice)
|
||||
cb.Reset()
|
||||
if err := cb.Marshal(protoMsg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
out := cb.Bytes()
|
||||
cb.lastMarshaledSize = capToMaxInt32(len(out))
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (p protoCodec) Marshal(v interface{}) ([]byte, error) {
|
||||
cb := protoBufferPool.Get().(*cachedProtoBuffer)
|
||||
out, err := p.marshal(v, cb)
|
||||
|
||||
// put back buffer and lose the ref to the slice
|
||||
cb.SetBuf(nil)
|
||||
protoBufferPool.Put(cb)
|
||||
return out, err
|
||||
}
|
||||
|
||||
func (p protoCodec) Unmarshal(data []byte, v interface{}) error {
|
||||
cb := protoBufferPool.Get().(*cachedProtoBuffer)
|
||||
cb.SetBuf(data)
|
||||
v.(proto.Message).Reset()
|
||||
err := cb.Unmarshal(v.(proto.Message))
|
||||
cb.SetBuf(nil)
|
||||
protoBufferPool.Put(cb)
|
||||
return err
|
||||
}
|
||||
|
||||
func (protoCodec) String() string {
|
||||
return "proto"
|
||||
}
|
||||
|
||||
var (
|
||||
protoBufferPool = &sync.Pool{
|
||||
New: func() interface{} {
|
||||
return &cachedProtoBuffer{
|
||||
Buffer: proto.Buffer{},
|
||||
lastMarshaledSize: 16,
|
||||
}
|
||||
},
|
||||
}
|
||||
)
|
||||
|
66
vendor/google.golang.org/grpc/codes/code_string.go
generated
vendored
@ -1,16 +1,62 @@
|
||||
// Code generated by "stringer -type=Code"; DO NOT EDIT.
|
||||
/*
|
||||
*
|
||||
* 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 codes
|
||||
|
||||
import "fmt"
|
||||
import "strconv"
|
||||
|
||||
const _Code_name = "OKCanceledUnknownInvalidArgumentDeadlineExceededNotFoundAlreadyExistsPermissionDeniedResourceExhaustedFailedPreconditionAbortedOutOfRangeUnimplementedInternalUnavailableDataLossUnauthenticated"
|
||||
|
||||
var _Code_index = [...]uint8{0, 2, 10, 17, 32, 48, 56, 69, 85, 102, 120, 127, 137, 150, 158, 169, 177, 192}
|
||||
|
||||
func (i Code) String() string {
|
||||
if i >= Code(len(_Code_index)-1) {
|
||||
return fmt.Sprintf("Code(%d)", i)
|
||||
func (c Code) String() string {
|
||||
switch c {
|
||||
case OK:
|
||||
return "OK"
|
||||
case Canceled:
|
||||
return "Canceled"
|
||||
case Unknown:
|
||||
return "Unknown"
|
||||
case InvalidArgument:
|
||||
return "InvalidArgument"
|
||||
case DeadlineExceeded:
|
||||
return "DeadlineExceeded"
|
||||
case NotFound:
|
||||
return "NotFound"
|
||||
case AlreadyExists:
|
||||
return "AlreadyExists"
|
||||
case PermissionDenied:
|
||||
return "PermissionDenied"
|
||||
case ResourceExhausted:
|
||||
return "ResourceExhausted"
|
||||
case FailedPrecondition:
|
||||
return "FailedPrecondition"
|
||||
case Aborted:
|
||||
return "Aborted"
|
||||
case OutOfRange:
|
||||
return "OutOfRange"
|
||||
case Unimplemented:
|
||||
return "Unimplemented"
|
||||
case Internal:
|
||||
return "Internal"
|
||||
case Unavailable:
|
||||
return "Unavailable"
|
||||
case DataLoss:
|
||||
return "DataLoss"
|
||||
case Unauthenticated:
|
||||
return "Unauthenticated"
|
||||
default:
|
||||
return "Code(" + strconv.FormatInt(int64(c), 10) + ")"
|
||||
}
|
||||
return _Code_name[_Code_index[i]:_Code_index[i+1]]
|
||||
}
|
||||
|
66
vendor/google.golang.org/grpc/codes/codes.go
generated
vendored
@ -20,11 +20,13 @@
|
||||
// consistent across various languages.
|
||||
package codes // import "google.golang.org/grpc/codes"
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
)
|
||||
|
||||
// A Code is an unsigned 32-bit error code as defined in the gRPC spec.
|
||||
type Code uint32
|
||||
|
||||
//go:generate stringer -type=Code
|
||||
|
||||
const (
|
||||
// OK is returned on success.
|
||||
OK Code = 0
|
||||
@ -32,9 +34,9 @@ const (
|
||||
// Canceled indicates the operation was canceled (typically by the caller).
|
||||
Canceled Code = 1
|
||||
|
||||
// Unknown error. An example of where this error may be returned is
|
||||
// Unknown error. An example of where this error may be returned is
|
||||
// if a Status value received from another address space belongs to
|
||||
// an error-space that is not known in this address space. Also
|
||||
// an error-space that is not known in this address space. Also
|
||||
// errors raised by APIs that do not return enough error information
|
||||
// may be converted to this error.
|
||||
Unknown Code = 2
|
||||
@ -63,15 +65,11 @@ const (
|
||||
// PermissionDenied indicates the caller does not have permission to
|
||||
// execute the specified operation. It must not be used for rejections
|
||||
// caused by exhausting some resource (use ResourceExhausted
|
||||
// instead for those errors). It must not be
|
||||
// instead for those errors). It must not be
|
||||
// used if the caller cannot be identified (use Unauthenticated
|
||||
// instead for those errors).
|
||||
PermissionDenied Code = 7
|
||||
|
||||
// Unauthenticated indicates the request does not have valid
|
||||
// authentication credentials for the operation.
|
||||
Unauthenticated Code = 16
|
||||
|
||||
// ResourceExhausted indicates some resource has been exhausted, perhaps
|
||||
// a per-user quota, or perhaps the entire file system is out of space.
|
||||
ResourceExhausted Code = 8
|
||||
@ -87,7 +85,7 @@ const (
|
||||
// (b) Use Aborted if the client should retry at a higher-level
|
||||
// (e.g., restarting a read-modify-write sequence).
|
||||
// (c) Use FailedPrecondition if the client should not retry until
|
||||
// the system state has been explicitly fixed. E.g., if an "rmdir"
|
||||
// the system state has been explicitly fixed. E.g., if an "rmdir"
|
||||
// fails because the directory is non-empty, FailedPrecondition
|
||||
// should be returned since the client should not retry unless
|
||||
// they have first fixed up the directory by deleting files from it.
|
||||
@ -116,7 +114,7 @@ const (
|
||||
// file size.
|
||||
//
|
||||
// There is a fair bit of overlap between FailedPrecondition and
|
||||
// OutOfRange. We recommend using OutOfRange (the more specific
|
||||
// OutOfRange. We recommend using OutOfRange (the more specific
|
||||
// error) when it applies so that callers who are iterating through
|
||||
// a space can easily look for an OutOfRange error to detect when
|
||||
// they are done.
|
||||
@ -126,8 +124,8 @@ const (
|
||||
// supported/enabled in this service.
|
||||
Unimplemented Code = 12
|
||||
|
||||
// Internal errors. Means some invariants expected by underlying
|
||||
// system has been broken. If you see one of these errors,
|
||||
// Internal errors. Means some invariants expected by underlying
|
||||
// system has been broken. If you see one of these errors,
|
||||
// something is very broken.
|
||||
Internal Code = 13
|
||||
|
||||
@ -141,4 +139,46 @@ const (
|
||||
|
||||
// DataLoss indicates unrecoverable data loss or corruption.
|
||||
DataLoss Code = 15
|
||||
|
||||
// Unauthenticated indicates the request does not have valid
|
||||
// authentication credentials for the operation.
|
||||
Unauthenticated Code = 16
|
||||
)
|
||||
|
||||
var strToCode = map[string]Code{
|
||||
`"OK"`: OK,
|
||||
`"CANCELLED"`:/* [sic] */ Canceled,
|
||||
`"UNKNOWN"`: Unknown,
|
||||
`"INVALID_ARGUMENT"`: InvalidArgument,
|
||||
`"DEADLINE_EXCEEDED"`: DeadlineExceeded,
|
||||
`"NOT_FOUND"`: NotFound,
|
||||
`"ALREADY_EXISTS"`: AlreadyExists,
|
||||
`"PERMISSION_DENIED"`: PermissionDenied,
|
||||
`"RESOURCE_EXHAUSTED"`: ResourceExhausted,
|
||||
`"FAILED_PRECONDITION"`: FailedPrecondition,
|
||||
`"ABORTED"`: Aborted,
|
||||
`"OUT_OF_RANGE"`: OutOfRange,
|
||||
`"UNIMPLEMENTED"`: Unimplemented,
|
||||
`"INTERNAL"`: Internal,
|
||||
`"UNAVAILABLE"`: Unavailable,
|
||||
`"DATA_LOSS"`: DataLoss,
|
||||
`"UNAUTHENTICATED"`: Unauthenticated,
|
||||
}
|
||||
|
||||
// UnmarshalJSON unmarshals b into the Code.
|
||||
func (c *Code) UnmarshalJSON(b []byte) error {
|
||||
// From json.Unmarshaler: By convention, to approximate the behavior of
|
||||
// Unmarshal itself, Unmarshalers implement UnmarshalJSON([]byte("null")) as
|
||||
// a no-op.
|
||||
if string(b) == "null" {
|
||||
return nil
|
||||
}
|
||||
if c == nil {
|
||||
return fmt.Errorf("nil receiver passed to UnmarshalJSON")
|
||||
}
|
||||
if jc, ok := strToCode[string(b)]; ok {
|
||||
*c = jc
|
||||
return nil
|
||||
}
|
||||
return fmt.Errorf("invalid code: %q", string(b))
|
||||
}
|
||||
|
27
vendor/google.golang.org/grpc/credentials/credentials.go
generated
vendored
@ -34,10 +34,8 @@ import (
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
var (
|
||||
// alpnProtoStr are the specified application level protocols for gRPC.
|
||||
alpnProtoStr = []string{"h2"}
|
||||
)
|
||||
// alpnProtoStr are the specified application level protocols for gRPC.
|
||||
var alpnProtoStr = []string{"h2"}
|
||||
|
||||
// PerRPCCredentials defines the common interface for the credentials which need to
|
||||
// attach security information to every RPC (e.g., oauth2).
|
||||
@ -45,8 +43,9 @@ type PerRPCCredentials interface {
|
||||
// GetRequestMetadata gets the current request metadata, refreshing
|
||||
// tokens if required. This should be called by the transport layer on
|
||||
// each request, and the data should be populated in headers or other
|
||||
// context. uri is the URI of the entry point for the request. When
|
||||
// supported by the underlying implementation, ctx can be used for
|
||||
// context. If a status code is returned, it will be used as the status
|
||||
// for the RPC. uri is the URI of the entry point for the request.
|
||||
// When supported by the underlying implementation, ctx can be used for
|
||||
// timeout and cancellation.
|
||||
// TODO(zhaoq): Define the set of the qualified keys instead of leaving
|
||||
// it as an arbitrary string.
|
||||
@ -74,11 +73,9 @@ type AuthInfo interface {
|
||||
AuthType() string
|
||||
}
|
||||
|
||||
var (
|
||||
// ErrConnDispatched indicates that rawConn has been dispatched out of gRPC
|
||||
// and the caller should not close rawConn.
|
||||
ErrConnDispatched = errors.New("credentials: rawConn is dispatched out of gRPC")
|
||||
)
|
||||
// ErrConnDispatched indicates that rawConn has been dispatched out of gRPC
|
||||
// and the caller should not close rawConn.
|
||||
var ErrConnDispatched = errors.New("credentials: rawConn is dispatched out of gRPC")
|
||||
|
||||
// TransportCredentials defines the common interface for all the live gRPC wire
|
||||
// protocols and supported transport security protocols (e.g., TLS, SSL).
|
||||
@ -135,15 +132,15 @@ func (c tlsCreds) Info() ProtocolInfo {
|
||||
}
|
||||
}
|
||||
|
||||
func (c *tlsCreds) ClientHandshake(ctx context.Context, addr string, rawConn net.Conn) (_ net.Conn, _ AuthInfo, err error) {
|
||||
func (c *tlsCreds) ClientHandshake(ctx context.Context, authority string, rawConn net.Conn) (_ net.Conn, _ AuthInfo, err error) {
|
||||
// use local cfg to avoid clobbering ServerName if using multiple endpoints
|
||||
cfg := cloneTLSConfig(c.config)
|
||||
if cfg.ServerName == "" {
|
||||
colonPos := strings.LastIndex(addr, ":")
|
||||
colonPos := strings.LastIndex(authority, ":")
|
||||
if colonPos == -1 {
|
||||
colonPos = len(addr)
|
||||
colonPos = len(authority)
|
||||
}
|
||||
cfg.ServerName = addr[:colonPos]
|
||||
cfg.ServerName = authority[:colonPos]
|
||||
}
|
||||
conn := tls.Client(rawConn, cfg)
|
||||
errChannel := make(chan error, 1)
|
||||
|
118
vendor/google.golang.org/grpc/encoding/encoding.go
generated
vendored
Normal file
@ -0,0 +1,118 @@
|
||||
/*
|
||||
*
|
||||
* 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 encoding defines the interface for the compressor and codec, and
|
||||
// functions to register and retrieve compressors and codecs.
|
||||
//
|
||||
// This package is EXPERIMENTAL.
|
||||
package encoding
|
||||
|
||||
import (
|
||||
"io"
|
||||
"strings"
|
||||
)
|
||||
|
||||
// Identity specifies the optional encoding for uncompressed streams.
|
||||
// It is intended for grpc internal use only.
|
||||
const Identity = "identity"
|
||||
|
||||
// Compressor is used for compressing and decompressing when sending or
|
||||
// receiving messages.
|
||||
type Compressor interface {
|
||||
// Compress writes the data written to wc to w after compressing it. If an
|
||||
// error occurs while initializing the compressor, that error is returned
|
||||
// instead.
|
||||
Compress(w io.Writer) (io.WriteCloser, error)
|
||||
// Decompress reads data from r, decompresses it, and provides the
|
||||
// uncompressed data via the returned io.Reader. If an error occurs while
|
||||
// initializing the decompressor, that error is returned instead.
|
||||
Decompress(r io.Reader) (io.Reader, error)
|
||||
// Name is the name of the compression codec and is used to set the content
|
||||
// coding header. The result must be static; the result cannot change
|
||||
// between calls.
|
||||
Name() string
|
||||
}
|
||||
|
||||
var registeredCompressor = make(map[string]Compressor)
|
||||
|
||||
// RegisterCompressor registers the compressor with gRPC by its name. It can
|
||||
// be activated when sending an RPC via grpc.UseCompressor(). It will be
|
||||
// automatically accessed when receiving a message based on the content coding
|
||||
// header. Servers also use it to send a response with the same encoding as
|
||||
// the request.
|
||||
//
|
||||
// NOTE: this function must only be called during initialization time (i.e. in
|
||||
// an init() function), and is not thread-safe. If multiple Compressors are
|
||||
// registered with the same name, the one registered last will take effect.
|
||||
func RegisterCompressor(c Compressor) {
|
||||
registeredCompressor[c.Name()] = c
|
||||
}
|
||||
|
||||
// GetCompressor returns Compressor for the given compressor name.
|
||||
func GetCompressor(name string) Compressor {
|
||||
return registeredCompressor[name]
|
||||
}
|
||||
|
||||
// Codec defines the interface gRPC uses to encode and decode messages. Note
|
||||
// that implementations of this interface must be thread safe; a Codec's
|
||||
// methods can be called from concurrent goroutines.
|
||||
type Codec interface {
|
||||
// Marshal returns the wire format of v.
|
||||
Marshal(v interface{}) ([]byte, error)
|
||||
// Unmarshal parses the wire format into v.
|
||||
Unmarshal(data []byte, v interface{}) error
|
||||
// Name returns the name of the Codec implementation. The returned string
|
||||
// will be used as part of content type in transmission. The result must be
|
||||
// static; the result cannot change between calls.
|
||||
Name() string
|
||||
}
|
||||
|
||||
var registeredCodecs = make(map[string]Codec)
|
||||
|
||||
// RegisterCodec registers the provided Codec for use with all gRPC clients and
|
||||
// servers.
|
||||
//
|
||||
// The Codec will be stored and looked up by result of its Name() method, which
|
||||
// should match the content-subtype of the encoding handled by the Codec. This
|
||||
// is case-insensitive, and is stored and looked up as lowercase. If the
|
||||
// result of calling Name() is an empty string, RegisterCodec will panic. See
|
||||
// Content-Type on
|
||||
// https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#requests for
|
||||
// more details.
|
||||
//
|
||||
// NOTE: this function must only be called during initialization time (i.e. in
|
||||
// an init() function), and is not thread-safe. If multiple Compressors are
|
||||
// registered with the same name, the one registered last will take effect.
|
||||
func RegisterCodec(codec Codec) {
|
||||
if codec == nil {
|
||||
panic("cannot register a nil Codec")
|
||||
}
|
||||
contentSubtype := strings.ToLower(codec.Name())
|
||||
if contentSubtype == "" {
|
||||
panic("cannot register Codec with empty string result for String()")
|
||||
}
|
||||
registeredCodecs[contentSubtype] = codec
|
||||
}
|
||||
|
||||
// GetCodec gets a registered Codec by content-subtype, or nil if no Codec is
|
||||
// registered for the content-subtype.
|
||||
//
|
||||
// The content-subtype is expected to be lowercase.
|
||||
func GetCodec(contentSubtype string) Codec {
|
||||
return registeredCodecs[contentSubtype]
|
||||
}
|
110
vendor/google.golang.org/grpc/encoding/proto/proto.go
generated
vendored
Normal file
@ -0,0 +1,110 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2018 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 proto defines the protobuf codec. Importing this package will
|
||||
// register the codec.
|
||||
package proto
|
||||
|
||||
import (
|
||||
"math"
|
||||
"sync"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"google.golang.org/grpc/encoding"
|
||||
)
|
||||
|
||||
// Name is the name registered for the proto compressor.
|
||||
const Name = "proto"
|
||||
|
||||
func init() {
|
||||
encoding.RegisterCodec(codec{})
|
||||
}
|
||||
|
||||
// codec is a Codec implementation with protobuf. It is the default codec for gRPC.
|
||||
type codec struct{}
|
||||
|
||||
type cachedProtoBuffer struct {
|
||||
lastMarshaledSize uint32
|
||||
proto.Buffer
|
||||
}
|
||||
|
||||
func capToMaxInt32(val int) uint32 {
|
||||
if val > math.MaxInt32 {
|
||||
return uint32(math.MaxInt32)
|
||||
}
|
||||
return uint32(val)
|
||||
}
|
||||
|
||||
func marshal(v interface{}, cb *cachedProtoBuffer) ([]byte, error) {
|
||||
protoMsg := v.(proto.Message)
|
||||
newSlice := make([]byte, 0, cb.lastMarshaledSize)
|
||||
|
||||
cb.SetBuf(newSlice)
|
||||
cb.Reset()
|
||||
if err := cb.Marshal(protoMsg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
out := cb.Bytes()
|
||||
cb.lastMarshaledSize = capToMaxInt32(len(out))
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (codec) Marshal(v interface{}) ([]byte, error) {
|
||||
if pm, ok := v.(proto.Marshaler); ok {
|
||||
// object can marshal itself, no need for buffer
|
||||
return pm.Marshal()
|
||||
}
|
||||
|
||||
cb := protoBufferPool.Get().(*cachedProtoBuffer)
|
||||
out, err := marshal(v, cb)
|
||||
|
||||
// put back buffer and lose the ref to the slice
|
||||
cb.SetBuf(nil)
|
||||
protoBufferPool.Put(cb)
|
||||
return out, err
|
||||
}
|
||||
|
||||
func (codec) Unmarshal(data []byte, v interface{}) error {
|
||||
protoMsg := v.(proto.Message)
|
||||
protoMsg.Reset()
|
||||
|
||||
if pu, ok := protoMsg.(proto.Unmarshaler); ok {
|
||||
// object can unmarshal itself, no need for buffer
|
||||
return pu.Unmarshal(data)
|
||||
}
|
||||
|
||||
cb := protoBufferPool.Get().(*cachedProtoBuffer)
|
||||
cb.SetBuf(data)
|
||||
err := cb.Unmarshal(protoMsg)
|
||||
cb.SetBuf(nil)
|
||||
protoBufferPool.Put(cb)
|
||||
return err
|
||||
}
|
||||
|
||||
func (codec) Name() string {
|
||||
return Name
|
||||
}
|
||||
|
||||
var protoBufferPool = &sync.Pool{
|
||||
New: func() interface{} {
|
||||
return &cachedProtoBuffer{
|
||||
Buffer: proto.Buffer{},
|
||||
lastMarshaledSize: 16,
|
||||
}
|
||||
},
|
||||
}
|
37
vendor/google.golang.org/grpc/envconfig.go
generated
vendored
Normal file
@ -0,0 +1,37 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2018 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 grpc
|
||||
|
||||
import (
|
||||
"os"
|
||||
"strings"
|
||||
)
|
||||
|
||||
const (
|
||||
envConfigPrefix = "GRPC_GO_"
|
||||
envConfigStickinessStr = envConfigPrefix + "STICKINESS"
|
||||
)
|
||||
|
||||
var (
|
||||
envConfigStickinessOn bool
|
||||
)
|
||||
|
||||
func init() {
|
||||
envConfigStickinessOn = strings.EqualFold(os.Getenv(envConfigStickinessStr), "on")
|
||||
}
|
70
vendor/google.golang.org/grpc/go16.go
generated
vendored
Normal file
@ -0,0 +1,70 @@
|
||||
// +build go1.6,!go1.7
|
||||
|
||||
/*
|
||||
*
|
||||
* Copyright 2016 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 grpc
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
"net/http"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
"google.golang.org/grpc/transport"
|
||||
)
|
||||
|
||||
// dialContext connects to the address on the named network.
|
||||
func dialContext(ctx context.Context, network, address string) (net.Conn, error) {
|
||||
return (&net.Dialer{Cancel: ctx.Done()}).Dial(network, address)
|
||||
}
|
||||
|
||||
func sendHTTPRequest(ctx context.Context, req *http.Request, conn net.Conn) error {
|
||||
req.Cancel = ctx.Done()
|
||||
if err := req.Write(conn); err != nil {
|
||||
return fmt.Errorf("failed to write the HTTP request: %v", err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// toRPCErr converts an error into an error from the status package.
|
||||
func toRPCErr(err error) error {
|
||||
if err == nil || err == io.EOF {
|
||||
return err
|
||||
}
|
||||
if _, ok := status.FromError(err); ok {
|
||||
return err
|
||||
}
|
||||
switch e := err.(type) {
|
||||
case transport.StreamError:
|
||||
return status.Error(e.Code, e.Desc)
|
||||
case transport.ConnectionError:
|
||||
return status.Error(codes.Unavailable, e.Desc)
|
||||
default:
|
||||
switch err {
|
||||
case context.DeadlineExceeded:
|
||||
return status.Error(codes.DeadlineExceeded, err.Error())
|
||||
case context.Canceled:
|
||||
return status.Error(codes.Canceled, err.Error())
|
||||
}
|
||||
}
|
||||
return status.Error(codes.Unknown, err.Error())
|
||||
}
|
71
vendor/google.golang.org/grpc/go17.go
generated
vendored
Normal file
@ -0,0 +1,71 @@
|
||||
// +build go1.7
|
||||
|
||||
/*
|
||||
*
|
||||
* Copyright 2016 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 grpc
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
"net/http"
|
||||
|
||||
netctx "golang.org/x/net/context"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
"google.golang.org/grpc/transport"
|
||||
)
|
||||
|
||||
// dialContext connects to the address on the named network.
|
||||
func dialContext(ctx context.Context, network, address string) (net.Conn, error) {
|
||||
return (&net.Dialer{}).DialContext(ctx, network, address)
|
||||
}
|
||||
|
||||
func sendHTTPRequest(ctx context.Context, req *http.Request, conn net.Conn) error {
|
||||
req = req.WithContext(ctx)
|
||||
if err := req.Write(conn); err != nil {
|
||||
return fmt.Errorf("failed to write the HTTP request: %v", err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// toRPCErr converts an error into an error from the status package.
|
||||
func toRPCErr(err error) error {
|
||||
if err == nil || err == io.EOF {
|
||||
return err
|
||||
}
|
||||
if _, ok := status.FromError(err); ok {
|
||||
return err
|
||||
}
|
||||
switch e := err.(type) {
|
||||
case transport.StreamError:
|
||||
return status.Error(e.Code, e.Desc)
|
||||
case transport.ConnectionError:
|
||||
return status.Error(codes.Unavailable, e.Desc)
|
||||
default:
|
||||
switch err {
|
||||
case context.DeadlineExceeded, netctx.DeadlineExceeded:
|
||||
return status.Error(codes.DeadlineExceeded, err.Error())
|
||||
case context.Canceled, netctx.Canceled:
|
||||
return status.Error(codes.Canceled, err.Error())
|
||||
}
|
||||
}
|
||||
return status.Error(codes.Unknown, err.Error())
|
||||
}
|
889
vendor/google.golang.org/grpc/grpclb.go
generated
vendored
@ -19,21 +19,32 @@
|
||||
package grpc
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"net"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc/codes"
|
||||
lbmpb "google.golang.org/grpc/grpclb/grpc_lb_v1/messages"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
lbpb "google.golang.org/grpc/grpclb/grpc_lb_v1/messages"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/metadata"
|
||||
"google.golang.org/grpc/naming"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
const (
|
||||
lbTokeyKey = "lb-token"
|
||||
defaultFallbackTimeout = 10 * time.Second
|
||||
grpclbName = "grpclb"
|
||||
)
|
||||
|
||||
func convertDuration(d *lbpb.Duration) time.Duration {
|
||||
if d == nil {
|
||||
return 0
|
||||
}
|
||||
return time.Duration(d.Seconds)*time.Second + time.Duration(d.Nanos)*time.Nanosecond
|
||||
}
|
||||
|
||||
// Client API for LoadBalancer service.
|
||||
// Mostly copied from generated pb.go file.
|
||||
// To avoid circular dependency.
|
||||
@ -47,7 +58,7 @@ func (c *loadBalancerClient) BalanceLoad(ctx context.Context, opts ...CallOption
|
||||
ServerStreams: true,
|
||||
ClientStreams: true,
|
||||
}
|
||||
stream, err := NewClientStream(ctx, desc, c.cc, "/grpc.lb.v1.LoadBalancer/BalanceLoad", opts...)
|
||||
stream, err := c.cc.NewStream(ctx, desc, "/grpc.lb.v1.LoadBalancer/BalanceLoad", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -59,646 +70,272 @@ type balanceLoadClientStream struct {
|
||||
ClientStream
|
||||
}
|
||||
|
||||
func (x *balanceLoadClientStream) Send(m *lbmpb.LoadBalanceRequest) error {
|
||||
func (x *balanceLoadClientStream) Send(m *lbpb.LoadBalanceRequest) error {
|
||||
return x.ClientStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *balanceLoadClientStream) Recv() (*lbmpb.LoadBalanceResponse, error) {
|
||||
m := new(lbmpb.LoadBalanceResponse)
|
||||
func (x *balanceLoadClientStream) Recv() (*lbpb.LoadBalanceResponse, error) {
|
||||
m := new(lbpb.LoadBalanceResponse)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
// NewGRPCLBBalancer creates a grpclb load balancer.
|
||||
func NewGRPCLBBalancer(r naming.Resolver) Balancer {
|
||||
return &grpclbBalancer{
|
||||
r: r,
|
||||
func init() {
|
||||
balancer.Register(newLBBuilder())
|
||||
}
|
||||
|
||||
// newLBBuilder creates a builder for grpclb.
|
||||
func newLBBuilder() balancer.Builder {
|
||||
return NewLBBuilderWithFallbackTimeout(defaultFallbackTimeout)
|
||||
}
|
||||
|
||||
// NewLBBuilderWithFallbackTimeout creates a grpclb builder with the given
|
||||
// fallbackTimeout. If no response is received from the remote balancer within
|
||||
// fallbackTimeout, the backend addresses from the resolved address list will be
|
||||
// used.
|
||||
//
|
||||
// Only call this function when a non-default fallback timeout is needed.
|
||||
func NewLBBuilderWithFallbackTimeout(fallbackTimeout time.Duration) balancer.Builder {
|
||||
return &lbBuilder{
|
||||
fallbackTimeout: fallbackTimeout,
|
||||
}
|
||||
}
|
||||
|
||||
type remoteBalancerInfo struct {
|
||||
addr string
|
||||
// the server name used for authentication with the remote LB server.
|
||||
name string
|
||||
type lbBuilder struct {
|
||||
fallbackTimeout time.Duration
|
||||
}
|
||||
|
||||
// grpclbAddrInfo consists of the information of a backend server.
|
||||
type grpclbAddrInfo struct {
|
||||
addr Address
|
||||
connected bool
|
||||
// dropForRateLimiting indicates whether this particular request should be
|
||||
// dropped by the client for rate limiting.
|
||||
dropForRateLimiting bool
|
||||
// dropForLoadBalancing indicates whether this particular request should be
|
||||
// dropped by the client for load balancing.
|
||||
dropForLoadBalancing bool
|
||||
func (b *lbBuilder) Name() string {
|
||||
return grpclbName
|
||||
}
|
||||
|
||||
type grpclbBalancer struct {
|
||||
r naming.Resolver
|
||||
target string
|
||||
mu sync.Mutex
|
||||
seq int // a sequence number to make sure addrCh does not get stale addresses.
|
||||
w naming.Watcher
|
||||
addrCh chan []Address
|
||||
rbs []remoteBalancerInfo
|
||||
addrs []*grpclbAddrInfo
|
||||
next int
|
||||
waitCh chan struct{}
|
||||
done bool
|
||||
rand *rand.Rand
|
||||
func (b *lbBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer {
|
||||
// This generates a manual resolver builder with a random scheme. This
|
||||
// scheme will be used to dial to remote LB, so we can send filtered address
|
||||
// updates to remote LB ClientConn using this manual resolver.
|
||||
scheme := "grpclb_internal_" + strconv.FormatInt(time.Now().UnixNano(), 36)
|
||||
r := &lbManualResolver{scheme: scheme, ccb: cc}
|
||||
|
||||
clientStats lbmpb.ClientStats
|
||||
}
|
||||
|
||||
func (b *grpclbBalancer) watchAddrUpdates(w naming.Watcher, ch chan []remoteBalancerInfo) error {
|
||||
updates, err := w.Next()
|
||||
if err != nil {
|
||||
grpclog.Warningf("grpclb: failed to get next addr update from watcher: %v", err)
|
||||
return err
|
||||
var target string
|
||||
targetSplitted := strings.Split(cc.Target(), ":///")
|
||||
if len(targetSplitted) < 2 {
|
||||
target = cc.Target()
|
||||
} else {
|
||||
target = targetSplitted[1]
|
||||
}
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if b.done {
|
||||
return ErrClientConnClosing
|
||||
|
||||
lb := &lbBalancer{
|
||||
cc: newLBCacheClientConn(cc),
|
||||
target: target,
|
||||
opt: opt,
|
||||
fallbackTimeout: b.fallbackTimeout,
|
||||
doneCh: make(chan struct{}),
|
||||
|
||||
manualResolver: r,
|
||||
csEvltr: &connectivityStateEvaluator{},
|
||||
subConns: make(map[resolver.Address]balancer.SubConn),
|
||||
scStates: make(map[balancer.SubConn]connectivity.State),
|
||||
picker: &errPicker{err: balancer.ErrNoSubConnAvailable},
|
||||
clientStats: &rpcStats{},
|
||||
}
|
||||
for _, update := range updates {
|
||||
switch update.Op {
|
||||
case naming.Add:
|
||||
var exist bool
|
||||
for _, v := range b.rbs {
|
||||
// TODO: Is the same addr with different server name a different balancer?
|
||||
if update.Addr == v.addr {
|
||||
exist = true
|
||||
break
|
||||
}
|
||||
|
||||
return lb
|
||||
}
|
||||
|
||||
type lbBalancer struct {
|
||||
cc *lbCacheClientConn
|
||||
target string
|
||||
opt balancer.BuildOptions
|
||||
fallbackTimeout time.Duration
|
||||
doneCh chan struct{}
|
||||
|
||||
// manualResolver is used in the remote LB ClientConn inside grpclb. When
|
||||
// resolved address updates are received by grpclb, filtered updates will be
|
||||
// send to remote LB ClientConn through this resolver.
|
||||
manualResolver *lbManualResolver
|
||||
// The ClientConn to talk to the remote balancer.
|
||||
ccRemoteLB *ClientConn
|
||||
|
||||
// Support client side load reporting. Each picker gets a reference to this,
|
||||
// and will update its content.
|
||||
clientStats *rpcStats
|
||||
|
||||
mu sync.Mutex // guards everything following.
|
||||
// The full server list including drops, used to check if the newly received
|
||||
// serverList contains anything new. Each generate picker will also have
|
||||
// reference to this list to do the first layer pick.
|
||||
fullServerList []*lbpb.Server
|
||||
// All backends addresses, with metadata set to nil. This list contains all
|
||||
// backend addresses in the same order and with the same duplicates as in
|
||||
// serverlist. When generating picker, a SubConn slice with the same order
|
||||
// but with only READY SCs will be gerenated.
|
||||
backendAddrs []resolver.Address
|
||||
// Roundrobin functionalities.
|
||||
csEvltr *connectivityStateEvaluator
|
||||
state connectivity.State
|
||||
subConns map[resolver.Address]balancer.SubConn // Used to new/remove SubConn.
|
||||
scStates map[balancer.SubConn]connectivity.State // Used to filter READY SubConns.
|
||||
picker balancer.Picker
|
||||
// Support fallback to resolved backend addresses if there's no response
|
||||
// from remote balancer within fallbackTimeout.
|
||||
fallbackTimerExpired bool
|
||||
serverListReceived bool
|
||||
// resolvedBackendAddrs is resolvedAddrs minus remote balancers. It's set
|
||||
// when resolved address updates are received, and read in the goroutine
|
||||
// handling fallback.
|
||||
resolvedBackendAddrs []resolver.Address
|
||||
}
|
||||
|
||||
// regeneratePicker takes a snapshot of the balancer, and generates a picker from
|
||||
// it. The picker
|
||||
// - always returns ErrTransientFailure if the balancer is in TransientFailure,
|
||||
// - does two layer roundrobin pick otherwise.
|
||||
// Caller must hold lb.mu.
|
||||
func (lb *lbBalancer) regeneratePicker() {
|
||||
if lb.state == connectivity.TransientFailure {
|
||||
lb.picker = &errPicker{err: balancer.ErrTransientFailure}
|
||||
return
|
||||
}
|
||||
var readySCs []balancer.SubConn
|
||||
for _, a := range lb.backendAddrs {
|
||||
if sc, ok := lb.subConns[a]; ok {
|
||||
if st, ok := lb.scStates[sc]; ok && st == connectivity.Ready {
|
||||
readySCs = append(readySCs, sc)
|
||||
}
|
||||
if exist {
|
||||
continue
|
||||
}
|
||||
md, ok := update.Metadata.(*naming.AddrMetadataGRPCLB)
|
||||
if !ok {
|
||||
// TODO: Revisit the handling here and may introduce some fallback mechanism.
|
||||
grpclog.Errorf("The name resolution contains unexpected metadata %v", update.Metadata)
|
||||
continue
|
||||
}
|
||||
switch md.AddrType {
|
||||
case naming.Backend:
|
||||
// TODO: Revisit the handling here and may introduce some fallback mechanism.
|
||||
grpclog.Errorf("The name resolution does not give grpclb addresses")
|
||||
continue
|
||||
case naming.GRPCLB:
|
||||
b.rbs = append(b.rbs, remoteBalancerInfo{
|
||||
addr: update.Addr,
|
||||
name: md.ServerName,
|
||||
})
|
||||
default:
|
||||
grpclog.Errorf("Received unknow address type %d", md.AddrType)
|
||||
continue
|
||||
}
|
||||
case naming.Delete:
|
||||
for i, v := range b.rbs {
|
||||
if update.Addr == v.addr {
|
||||
copy(b.rbs[i:], b.rbs[i+1:])
|
||||
b.rbs = b.rbs[:len(b.rbs)-1]
|
||||
break
|
||||
}
|
||||
}
|
||||
default:
|
||||
grpclog.Errorf("Unknown update.Op %v", update.Op)
|
||||
}
|
||||
}
|
||||
// TODO: Fall back to the basic round-robin load balancing if the resulting address is
|
||||
// not a load balancer.
|
||||
|
||||
if len(lb.fullServerList) <= 0 {
|
||||
if len(readySCs) <= 0 {
|
||||
lb.picker = &errPicker{err: balancer.ErrNoSubConnAvailable}
|
||||
return
|
||||
}
|
||||
lb.picker = &rrPicker{subConns: readySCs}
|
||||
return
|
||||
}
|
||||
lb.picker = &lbPicker{
|
||||
serverList: lb.fullServerList,
|
||||
subConns: readySCs,
|
||||
stats: lb.clientStats,
|
||||
}
|
||||
}
|
||||
|
||||
func (lb *lbBalancer) HandleSubConnStateChange(sc balancer.SubConn, s connectivity.State) {
|
||||
grpclog.Infof("lbBalancer: handle SubConn state change: %p, %v", sc, s)
|
||||
lb.mu.Lock()
|
||||
defer lb.mu.Unlock()
|
||||
|
||||
oldS, ok := lb.scStates[sc]
|
||||
if !ok {
|
||||
grpclog.Infof("lbBalancer: got state changes for an unknown SubConn: %p, %v", sc, s)
|
||||
return
|
||||
}
|
||||
lb.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(lb.scStates, sc)
|
||||
}
|
||||
|
||||
oldAggrState := lb.state
|
||||
lb.state = lb.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 (oldS == connectivity.Ready) != (s == connectivity.Ready) ||
|
||||
(lb.state == connectivity.TransientFailure) != (oldAggrState == connectivity.TransientFailure) {
|
||||
lb.regeneratePicker()
|
||||
}
|
||||
|
||||
lb.cc.UpdateBalancerState(lb.state, lb.picker)
|
||||
}
|
||||
|
||||
// fallbackToBackendsAfter blocks for fallbackTimeout and falls back to use
|
||||
// resolved backends (backends received from resolver, not from remote balancer)
|
||||
// if no connection to remote balancers was successful.
|
||||
func (lb *lbBalancer) fallbackToBackendsAfter(fallbackTimeout time.Duration) {
|
||||
timer := time.NewTimer(fallbackTimeout)
|
||||
defer timer.Stop()
|
||||
select {
|
||||
case <-ch:
|
||||
case <-timer.C:
|
||||
case <-lb.doneCh:
|
||||
return
|
||||
}
|
||||
lb.mu.Lock()
|
||||
if lb.serverListReceived {
|
||||
lb.mu.Unlock()
|
||||
return
|
||||
}
|
||||
lb.fallbackTimerExpired = true
|
||||
lb.refreshSubConns(lb.resolvedBackendAddrs)
|
||||
lb.mu.Unlock()
|
||||
}
|
||||
|
||||
// HandleResolvedAddrs sends the updated remoteLB addresses to remoteLB
|
||||
// clientConn. The remoteLB clientConn will handle creating/removing remoteLB
|
||||
// connections.
|
||||
func (lb *lbBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) {
|
||||
grpclog.Infof("lbBalancer: handleResolvedResult: %+v", addrs)
|
||||
if len(addrs) <= 0 {
|
||||
return
|
||||
}
|
||||
|
||||
var remoteBalancerAddrs, backendAddrs []resolver.Address
|
||||
for _, a := range addrs {
|
||||
if a.Type == resolver.GRPCLB {
|
||||
remoteBalancerAddrs = append(remoteBalancerAddrs, a)
|
||||
} else {
|
||||
backendAddrs = append(backendAddrs, a)
|
||||
}
|
||||
}
|
||||
|
||||
if lb.ccRemoteLB == nil {
|
||||
if len(remoteBalancerAddrs) <= 0 {
|
||||
grpclog.Errorf("grpclb: no remote balancer address is available, should never happen")
|
||||
return
|
||||
}
|
||||
// First time receiving resolved addresses, create a cc to remote
|
||||
// balancers.
|
||||
lb.dialRemoteLB(remoteBalancerAddrs[0].ServerName)
|
||||
// Start the fallback goroutine.
|
||||
go lb.fallbackToBackendsAfter(lb.fallbackTimeout)
|
||||
}
|
||||
|
||||
// cc to remote balancers uses lb.manualResolver. Send the updated remote
|
||||
// balancer addresses to it through manualResolver.
|
||||
lb.manualResolver.NewAddress(remoteBalancerAddrs)
|
||||
|
||||
lb.mu.Lock()
|
||||
lb.resolvedBackendAddrs = backendAddrs
|
||||
// If serverListReceived is true, connection to remote balancer was
|
||||
// successful and there's no need to do fallback anymore.
|
||||
// If fallbackTimerExpired is false, fallback hasn't happened yet.
|
||||
if !lb.serverListReceived && lb.fallbackTimerExpired {
|
||||
// This means we received a new list of resolved backends, and we are
|
||||
// still in fallback mode. Need to update the list of backends we are
|
||||
// using to the new list of backends.
|
||||
lb.refreshSubConns(lb.resolvedBackendAddrs)
|
||||
}
|
||||
lb.mu.Unlock()
|
||||
}
|
||||
|
||||
func (lb *lbBalancer) Close() {
|
||||
select {
|
||||
case <-lb.doneCh:
|
||||
return
|
||||
default:
|
||||
}
|
||||
ch <- b.rbs
|
||||
return nil
|
||||
}
|
||||
|
||||
func convertDuration(d *lbmpb.Duration) time.Duration {
|
||||
if d == nil {
|
||||
return 0
|
||||
}
|
||||
return time.Duration(d.Seconds)*time.Second + time.Duration(d.Nanos)*time.Nanosecond
|
||||
}
|
||||
|
||||
func (b *grpclbBalancer) processServerList(l *lbmpb.ServerList, seq int) {
|
||||
if l == nil {
|
||||
return
|
||||
}
|
||||
servers := l.GetServers()
|
||||
var (
|
||||
sl []*grpclbAddrInfo
|
||||
addrs []Address
|
||||
)
|
||||
for _, s := range servers {
|
||||
md := metadata.Pairs("lb-token", s.LoadBalanceToken)
|
||||
ip := net.IP(s.IpAddress)
|
||||
ipStr := ip.String()
|
||||
if ip.To4() == nil {
|
||||
// Add square brackets to ipv6 addresses, otherwise net.Dial() and
|
||||
// net.SplitHostPort() will return too many colons error.
|
||||
ipStr = fmt.Sprintf("[%s]", ipStr)
|
||||
}
|
||||
addr := Address{
|
||||
Addr: fmt.Sprintf("%s:%d", ipStr, s.Port),
|
||||
Metadata: &md,
|
||||
}
|
||||
sl = append(sl, &grpclbAddrInfo{
|
||||
addr: addr,
|
||||
dropForRateLimiting: s.DropForRateLimiting,
|
||||
dropForLoadBalancing: s.DropForLoadBalancing,
|
||||
})
|
||||
addrs = append(addrs, addr)
|
||||
}
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if b.done || seq < b.seq {
|
||||
return
|
||||
}
|
||||
if len(sl) > 0 {
|
||||
// reset b.next to 0 when replacing the server list.
|
||||
b.next = 0
|
||||
b.addrs = sl
|
||||
b.addrCh <- addrs
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (b *grpclbBalancer) sendLoadReport(s *balanceLoadClientStream, interval time.Duration, done <-chan struct{}) {
|
||||
ticker := time.NewTicker(interval)
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
case <-done:
|
||||
return
|
||||
}
|
||||
b.mu.Lock()
|
||||
stats := b.clientStats
|
||||
b.clientStats = lbmpb.ClientStats{} // Clear the stats.
|
||||
b.mu.Unlock()
|
||||
t := time.Now()
|
||||
stats.Timestamp = &lbmpb.Timestamp{
|
||||
Seconds: t.Unix(),
|
||||
Nanos: int32(t.Nanosecond()),
|
||||
}
|
||||
if err := s.Send(&lbmpb.LoadBalanceRequest{
|
||||
LoadBalanceRequestType: &lbmpb.LoadBalanceRequest_ClientStats{
|
||||
ClientStats: &stats,
|
||||
},
|
||||
}); err != nil {
|
||||
grpclog.Errorf("grpclb: failed to send load report: %v", err)
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (b *grpclbBalancer) callRemoteBalancer(lbc *loadBalancerClient, seq int) (retry bool) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
stream, err := lbc.BalanceLoad(ctx)
|
||||
if err != nil {
|
||||
grpclog.Errorf("grpclb: failed to perform RPC to the remote balancer %v", err)
|
||||
return
|
||||
}
|
||||
b.mu.Lock()
|
||||
if b.done {
|
||||
b.mu.Unlock()
|
||||
return
|
||||
}
|
||||
b.mu.Unlock()
|
||||
initReq := &lbmpb.LoadBalanceRequest{
|
||||
LoadBalanceRequestType: &lbmpb.LoadBalanceRequest_InitialRequest{
|
||||
InitialRequest: &lbmpb.InitialLoadBalanceRequest{
|
||||
Name: b.target,
|
||||
},
|
||||
},
|
||||
}
|
||||
if err := stream.Send(initReq); err != nil {
|
||||
grpclog.Errorf("grpclb: failed to send init request: %v", err)
|
||||
// TODO: backoff on retry?
|
||||
return true
|
||||
}
|
||||
reply, err := stream.Recv()
|
||||
if err != nil {
|
||||
grpclog.Errorf("grpclb: failed to recv init response: %v", err)
|
||||
// TODO: backoff on retry?
|
||||
return true
|
||||
}
|
||||
initResp := reply.GetInitialResponse()
|
||||
if initResp == nil {
|
||||
grpclog.Errorf("grpclb: reply from remote balancer did not include initial response.")
|
||||
return
|
||||
}
|
||||
// TODO: Support delegation.
|
||||
if initResp.LoadBalancerDelegate != "" {
|
||||
// delegation
|
||||
grpclog.Errorf("TODO: Delegation is not supported yet.")
|
||||
return
|
||||
}
|
||||
streamDone := make(chan struct{})
|
||||
defer close(streamDone)
|
||||
b.mu.Lock()
|
||||
b.clientStats = lbmpb.ClientStats{} // Clear client stats.
|
||||
b.mu.Unlock()
|
||||
if d := convertDuration(initResp.ClientStatsReportInterval); d > 0 {
|
||||
go b.sendLoadReport(stream, d, streamDone)
|
||||
}
|
||||
// Retrieve the server list.
|
||||
for {
|
||||
reply, err := stream.Recv()
|
||||
if err != nil {
|
||||
grpclog.Errorf("grpclb: failed to recv server list: %v", err)
|
||||
break
|
||||
}
|
||||
b.mu.Lock()
|
||||
if b.done || seq < b.seq {
|
||||
b.mu.Unlock()
|
||||
return
|
||||
}
|
||||
b.seq++ // tick when receiving a new list of servers.
|
||||
seq = b.seq
|
||||
b.mu.Unlock()
|
||||
if serverList := reply.GetServerList(); serverList != nil {
|
||||
b.processServerList(serverList, seq)
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func (b *grpclbBalancer) Start(target string, config BalancerConfig) error {
|
||||
b.rand = rand.New(rand.NewSource(time.Now().Unix()))
|
||||
// TODO: Fall back to the basic direct connection if there is no name resolver.
|
||||
if b.r == nil {
|
||||
return errors.New("there is no name resolver installed")
|
||||
}
|
||||
b.target = target
|
||||
b.mu.Lock()
|
||||
if b.done {
|
||||
b.mu.Unlock()
|
||||
return ErrClientConnClosing
|
||||
}
|
||||
b.addrCh = make(chan []Address)
|
||||
w, err := b.r.Resolve(target)
|
||||
if err != nil {
|
||||
b.mu.Unlock()
|
||||
grpclog.Errorf("grpclb: failed to resolve address: %v, err: %v", target, err)
|
||||
return err
|
||||
}
|
||||
b.w = w
|
||||
b.mu.Unlock()
|
||||
balancerAddrsCh := make(chan []remoteBalancerInfo, 1)
|
||||
// Spawn a goroutine to monitor the name resolution of remote load balancer.
|
||||
go func() {
|
||||
for {
|
||||
if err := b.watchAddrUpdates(w, balancerAddrsCh); err != nil {
|
||||
grpclog.Warningf("grpclb: the naming watcher stops working due to %v.\n", err)
|
||||
close(balancerAddrsCh)
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
// Spawn a goroutine to talk to the remote load balancer.
|
||||
go func() {
|
||||
var (
|
||||
cc *ClientConn
|
||||
// ccError is closed when there is an error in the current cc.
|
||||
// A new rb should be picked from rbs and connected.
|
||||
ccError chan struct{}
|
||||
rb *remoteBalancerInfo
|
||||
rbs []remoteBalancerInfo
|
||||
rbIdx int
|
||||
)
|
||||
|
||||
defer func() {
|
||||
if ccError != nil {
|
||||
select {
|
||||
case <-ccError:
|
||||
default:
|
||||
close(ccError)
|
||||
}
|
||||
}
|
||||
if cc != nil {
|
||||
cc.Close()
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
var ok bool
|
||||
select {
|
||||
case rbs, ok = <-balancerAddrsCh:
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
foundIdx := -1
|
||||
if rb != nil {
|
||||
for i, trb := range rbs {
|
||||
if trb == *rb {
|
||||
foundIdx = i
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if foundIdx >= 0 {
|
||||
if foundIdx >= 1 {
|
||||
// Move the address in use to the beginning of the list.
|
||||
b.rbs[0], b.rbs[foundIdx] = b.rbs[foundIdx], b.rbs[0]
|
||||
rbIdx = 0
|
||||
}
|
||||
continue // If found, don't dial new cc.
|
||||
} else if len(rbs) > 0 {
|
||||
// Pick a random one from the list, instead of always using the first one.
|
||||
if l := len(rbs); l > 1 && rb != nil {
|
||||
tmpIdx := b.rand.Intn(l - 1)
|
||||
b.rbs[0], b.rbs[tmpIdx] = b.rbs[tmpIdx], b.rbs[0]
|
||||
}
|
||||
rbIdx = 0
|
||||
rb = &rbs[0]
|
||||
} else {
|
||||
// foundIdx < 0 && len(rbs) <= 0.
|
||||
rb = nil
|
||||
}
|
||||
case <-ccError:
|
||||
ccError = nil
|
||||
if rbIdx < len(rbs)-1 {
|
||||
rbIdx++
|
||||
rb = &rbs[rbIdx]
|
||||
} else {
|
||||
rb = nil
|
||||
}
|
||||
}
|
||||
|
||||
if rb == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
if cc != nil {
|
||||
cc.Close()
|
||||
}
|
||||
// Talk to the remote load balancer to get the server list.
|
||||
var (
|
||||
err error
|
||||
dopts []DialOption
|
||||
)
|
||||
if creds := config.DialCreds; creds != nil {
|
||||
if rb.name != "" {
|
||||
if err := creds.OverrideServerName(rb.name); err != nil {
|
||||
grpclog.Warningf("grpclb: failed to override the server name in the credentials: %v", err)
|
||||
continue
|
||||
}
|
||||
}
|
||||
dopts = append(dopts, WithTransportCredentials(creds))
|
||||
} else {
|
||||
dopts = append(dopts, WithInsecure())
|
||||
}
|
||||
if dialer := config.Dialer; dialer != nil {
|
||||
// WithDialer takes a different type of function, so we instead use a special DialOption here.
|
||||
dopts = append(dopts, func(o *dialOptions) { o.copts.Dialer = dialer })
|
||||
}
|
||||
dopts = append(dopts, WithBlock())
|
||||
ccError = make(chan struct{})
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
|
||||
cc, err = DialContext(ctx, rb.addr, dopts...)
|
||||
cancel()
|
||||
if err != nil {
|
||||
grpclog.Warningf("grpclb: failed to setup a connection to the remote balancer %v: %v", rb.addr, err)
|
||||
close(ccError)
|
||||
continue
|
||||
}
|
||||
b.mu.Lock()
|
||||
b.seq++ // tick when getting a new balancer address
|
||||
seq := b.seq
|
||||
b.next = 0
|
||||
b.mu.Unlock()
|
||||
go func(cc *ClientConn, ccError chan struct{}) {
|
||||
lbc := &loadBalancerClient{cc}
|
||||
b.callRemoteBalancer(lbc, seq)
|
||||
cc.Close()
|
||||
select {
|
||||
case <-ccError:
|
||||
default:
|
||||
close(ccError)
|
||||
}
|
||||
}(cc, ccError)
|
||||
}
|
||||
}()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *grpclbBalancer) down(addr Address, err error) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
for _, a := range b.addrs {
|
||||
if addr == a.addr {
|
||||
a.connected = false
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (b *grpclbBalancer) Up(addr Address) func(error) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if b.done {
|
||||
return nil
|
||||
}
|
||||
var cnt int
|
||||
for _, a := range b.addrs {
|
||||
if a.addr == addr {
|
||||
if a.connected {
|
||||
return nil
|
||||
}
|
||||
a.connected = true
|
||||
}
|
||||
if a.connected && !a.dropForRateLimiting && !a.dropForLoadBalancing {
|
||||
cnt++
|
||||
}
|
||||
}
|
||||
// addr is the only one which is connected. Notify the Get() callers who are blocking.
|
||||
if cnt == 1 && b.waitCh != nil {
|
||||
close(b.waitCh)
|
||||
b.waitCh = nil
|
||||
}
|
||||
return func(err error) {
|
||||
b.down(addr, err)
|
||||
}
|
||||
}
|
||||
|
||||
func (b *grpclbBalancer) Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) {
|
||||
var ch chan struct{}
|
||||
b.mu.Lock()
|
||||
if b.done {
|
||||
b.mu.Unlock()
|
||||
err = ErrClientConnClosing
|
||||
return
|
||||
}
|
||||
seq := b.seq
|
||||
|
||||
defer func() {
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
put = func() {
|
||||
s, ok := rpcInfoFromContext(ctx)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if b.done || seq < b.seq {
|
||||
return
|
||||
}
|
||||
b.clientStats.NumCallsFinished++
|
||||
if !s.bytesSent {
|
||||
b.clientStats.NumCallsFinishedWithClientFailedToSend++
|
||||
} else if s.bytesReceived {
|
||||
b.clientStats.NumCallsFinishedKnownReceived++
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
b.clientStats.NumCallsStarted++
|
||||
if len(b.addrs) > 0 {
|
||||
if b.next >= len(b.addrs) {
|
||||
b.next = 0
|
||||
}
|
||||
next := b.next
|
||||
for {
|
||||
a := b.addrs[next]
|
||||
next = (next + 1) % len(b.addrs)
|
||||
if a.connected {
|
||||
if !a.dropForRateLimiting && !a.dropForLoadBalancing {
|
||||
addr = a.addr
|
||||
b.next = next
|
||||
b.mu.Unlock()
|
||||
return
|
||||
}
|
||||
if !opts.BlockingWait {
|
||||
b.next = next
|
||||
if a.dropForLoadBalancing {
|
||||
b.clientStats.NumCallsFinished++
|
||||
b.clientStats.NumCallsFinishedWithDropForLoadBalancing++
|
||||
} else if a.dropForRateLimiting {
|
||||
b.clientStats.NumCallsFinished++
|
||||
b.clientStats.NumCallsFinishedWithDropForRateLimiting++
|
||||
}
|
||||
b.mu.Unlock()
|
||||
err = Errorf(codes.Unavailable, "%s drops requests", a.addr.Addr)
|
||||
return
|
||||
}
|
||||
}
|
||||
if next == b.next {
|
||||
// Has iterated all the possible address but none is connected.
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if !opts.BlockingWait {
|
||||
b.clientStats.NumCallsFinished++
|
||||
b.clientStats.NumCallsFinishedWithClientFailedToSend++
|
||||
b.mu.Unlock()
|
||||
err = Errorf(codes.Unavailable, "there is no address available")
|
||||
return
|
||||
}
|
||||
// Wait on b.waitCh for non-failfast RPCs.
|
||||
if b.waitCh == nil {
|
||||
ch = make(chan struct{})
|
||||
b.waitCh = ch
|
||||
} else {
|
||||
ch = b.waitCh
|
||||
}
|
||||
b.mu.Unlock()
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
b.mu.Lock()
|
||||
b.clientStats.NumCallsFinished++
|
||||
b.clientStats.NumCallsFinishedWithClientFailedToSend++
|
||||
b.mu.Unlock()
|
||||
err = ctx.Err()
|
||||
return
|
||||
case <-ch:
|
||||
b.mu.Lock()
|
||||
if b.done {
|
||||
b.clientStats.NumCallsFinished++
|
||||
b.clientStats.NumCallsFinishedWithClientFailedToSend++
|
||||
b.mu.Unlock()
|
||||
err = ErrClientConnClosing
|
||||
return
|
||||
}
|
||||
|
||||
if len(b.addrs) > 0 {
|
||||
if b.next >= len(b.addrs) {
|
||||
b.next = 0
|
||||
}
|
||||
next := b.next
|
||||
for {
|
||||
a := b.addrs[next]
|
||||
next = (next + 1) % len(b.addrs)
|
||||
if a.connected {
|
||||
if !a.dropForRateLimiting && !a.dropForLoadBalancing {
|
||||
addr = a.addr
|
||||
b.next = next
|
||||
b.mu.Unlock()
|
||||
return
|
||||
}
|
||||
if !opts.BlockingWait {
|
||||
b.next = next
|
||||
if a.dropForLoadBalancing {
|
||||
b.clientStats.NumCallsFinished++
|
||||
b.clientStats.NumCallsFinishedWithDropForLoadBalancing++
|
||||
} else if a.dropForRateLimiting {
|
||||
b.clientStats.NumCallsFinished++
|
||||
b.clientStats.NumCallsFinishedWithDropForRateLimiting++
|
||||
}
|
||||
b.mu.Unlock()
|
||||
err = Errorf(codes.Unavailable, "drop requests for the addreess %s", a.addr.Addr)
|
||||
return
|
||||
}
|
||||
}
|
||||
if next == b.next {
|
||||
// Has iterated all the possible address but none is connected.
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
// The newly added addr got removed by Down() again.
|
||||
if b.waitCh == nil {
|
||||
ch = make(chan struct{})
|
||||
b.waitCh = ch
|
||||
} else {
|
||||
ch = b.waitCh
|
||||
}
|
||||
b.mu.Unlock()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (b *grpclbBalancer) Notify() <-chan []Address {
|
||||
return b.addrCh
|
||||
}
|
||||
|
||||
func (b *grpclbBalancer) Close() error {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if b.done {
|
||||
return errBalancerClosed
|
||||
}
|
||||
b.done = true
|
||||
if b.waitCh != nil {
|
||||
close(b.waitCh)
|
||||
}
|
||||
if b.addrCh != nil {
|
||||
close(b.addrCh)
|
||||
}
|
||||
if b.w != nil {
|
||||
b.w.Close()
|
||||
}
|
||||
return nil
|
||||
close(lb.doneCh)
|
||||
if lb.ccRemoteLB != nil {
|
||||
lb.ccRemoteLB.Close()
|
||||
}
|
||||
lb.cc.close()
|
||||
}
|
||||
|
410
vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/messages/messages.pb.go
generated
vendored
@ -1,24 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// source: grpc_lb_v1/messages/messages.proto
|
||||
|
||||
/*
|
||||
Package messages is a generated protocol buffer package.
|
||||
|
||||
It is generated from these files:
|
||||
grpc_lb_v1/messages/messages.proto
|
||||
|
||||
It has these top-level messages:
|
||||
Duration
|
||||
Timestamp
|
||||
LoadBalanceRequest
|
||||
InitialLoadBalanceRequest
|
||||
ClientStats
|
||||
LoadBalanceResponse
|
||||
InitialLoadBalanceResponse
|
||||
ServerList
|
||||
Server
|
||||
*/
|
||||
package messages
|
||||
package messages // import "google.golang.org/grpc/grpclb/grpc_lb_v1/messages"
|
||||
|
||||
import proto "github.com/golang/protobuf/proto"
|
||||
import fmt "fmt"
|
||||
@ -45,13 +28,35 @@ type Duration struct {
|
||||
// of one second or more, a non-zero value for the `nanos` field must be
|
||||
// of the same sign as the `seconds` field. Must be from -999,999,999
|
||||
// to +999,999,999 inclusive.
|
||||
Nanos int32 `protobuf:"varint,2,opt,name=nanos" json:"nanos,omitempty"`
|
||||
Nanos int32 `protobuf:"varint,2,opt,name=nanos" json:"nanos,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Duration) Reset() { *m = Duration{} }
|
||||
func (m *Duration) String() string { return proto.CompactTextString(m) }
|
||||
func (*Duration) ProtoMessage() {}
|
||||
func (*Duration) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
|
||||
func (m *Duration) Reset() { *m = Duration{} }
|
||||
func (m *Duration) String() string { return proto.CompactTextString(m) }
|
||||
func (*Duration) ProtoMessage() {}
|
||||
func (*Duration) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_messages_b81c731f0e83edbd, []int{0}
|
||||
}
|
||||
func (m *Duration) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Duration.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Duration) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Duration.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *Duration) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Duration.Merge(dst, src)
|
||||
}
|
||||
func (m *Duration) XXX_Size() int {
|
||||
return xxx_messageInfo_Duration.Size(m)
|
||||
}
|
||||
func (m *Duration) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Duration.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Duration proto.InternalMessageInfo
|
||||
|
||||
func (m *Duration) GetSeconds() int64 {
|
||||
if m != nil {
|
||||
@ -76,13 +81,35 @@ type Timestamp struct {
|
||||
// second values with fractions must still have non-negative nanos values
|
||||
// that count forward in time. Must be from 0 to 999,999,999
|
||||
// inclusive.
|
||||
Nanos int32 `protobuf:"varint,2,opt,name=nanos" json:"nanos,omitempty"`
|
||||
Nanos int32 `protobuf:"varint,2,opt,name=nanos" json:"nanos,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Timestamp) Reset() { *m = Timestamp{} }
|
||||
func (m *Timestamp) String() string { return proto.CompactTextString(m) }
|
||||
func (*Timestamp) ProtoMessage() {}
|
||||
func (*Timestamp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
|
||||
func (m *Timestamp) Reset() { *m = Timestamp{} }
|
||||
func (m *Timestamp) String() string { return proto.CompactTextString(m) }
|
||||
func (*Timestamp) ProtoMessage() {}
|
||||
func (*Timestamp) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_messages_b81c731f0e83edbd, []int{1}
|
||||
}
|
||||
func (m *Timestamp) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Timestamp.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Timestamp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Timestamp.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *Timestamp) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Timestamp.Merge(dst, src)
|
||||
}
|
||||
func (m *Timestamp) XXX_Size() int {
|
||||
return xxx_messageInfo_Timestamp.Size(m)
|
||||
}
|
||||
func (m *Timestamp) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Timestamp.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Timestamp proto.InternalMessageInfo
|
||||
|
||||
func (m *Timestamp) GetSeconds() int64 {
|
||||
if m != nil {
|
||||
@ -103,12 +130,34 @@ type LoadBalanceRequest struct {
|
||||
// *LoadBalanceRequest_InitialRequest
|
||||
// *LoadBalanceRequest_ClientStats
|
||||
LoadBalanceRequestType isLoadBalanceRequest_LoadBalanceRequestType `protobuf_oneof:"load_balance_request_type"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *LoadBalanceRequest) Reset() { *m = LoadBalanceRequest{} }
|
||||
func (m *LoadBalanceRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*LoadBalanceRequest) ProtoMessage() {}
|
||||
func (*LoadBalanceRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
|
||||
func (m *LoadBalanceRequest) Reset() { *m = LoadBalanceRequest{} }
|
||||
func (m *LoadBalanceRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*LoadBalanceRequest) ProtoMessage() {}
|
||||
func (*LoadBalanceRequest) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_messages_b81c731f0e83edbd, []int{2}
|
||||
}
|
||||
func (m *LoadBalanceRequest) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_LoadBalanceRequest.Unmarshal(m, b)
|
||||
}
|
||||
func (m *LoadBalanceRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_LoadBalanceRequest.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *LoadBalanceRequest) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_LoadBalanceRequest.Merge(dst, src)
|
||||
}
|
||||
func (m *LoadBalanceRequest) XXX_Size() int {
|
||||
return xxx_messageInfo_LoadBalanceRequest.Size(m)
|
||||
}
|
||||
func (m *LoadBalanceRequest) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_LoadBalanceRequest.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_LoadBalanceRequest proto.InternalMessageInfo
|
||||
|
||||
type isLoadBalanceRequest_LoadBalanceRequestType interface {
|
||||
isLoadBalanceRequest_LoadBalanceRequestType()
|
||||
@ -204,12 +253,12 @@ func _LoadBalanceRequest_OneofSizer(msg proto.Message) (n int) {
|
||||
switch x := m.LoadBalanceRequestType.(type) {
|
||||
case *LoadBalanceRequest_InitialRequest:
|
||||
s := proto.Size(x.InitialRequest)
|
||||
n += proto.SizeVarint(1<<3 | proto.WireBytes)
|
||||
n += 1 // tag and wire
|
||||
n += proto.SizeVarint(uint64(s))
|
||||
n += s
|
||||
case *LoadBalanceRequest_ClientStats:
|
||||
s := proto.Size(x.ClientStats)
|
||||
n += proto.SizeVarint(2<<3 | proto.WireBytes)
|
||||
n += 1 // tag and wire
|
||||
n += proto.SizeVarint(uint64(s))
|
||||
n += s
|
||||
case nil:
|
||||
@ -222,13 +271,35 @@ func _LoadBalanceRequest_OneofSizer(msg proto.Message) (n int) {
|
||||
type InitialLoadBalanceRequest struct {
|
||||
// Name of load balanced service (IE, balancer.service.com)
|
||||
// length should be less than 256 bytes.
|
||||
Name string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
|
||||
Name string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *InitialLoadBalanceRequest) Reset() { *m = InitialLoadBalanceRequest{} }
|
||||
func (m *InitialLoadBalanceRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*InitialLoadBalanceRequest) ProtoMessage() {}
|
||||
func (*InitialLoadBalanceRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
|
||||
func (m *InitialLoadBalanceRequest) Reset() { *m = InitialLoadBalanceRequest{} }
|
||||
func (m *InitialLoadBalanceRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*InitialLoadBalanceRequest) ProtoMessage() {}
|
||||
func (*InitialLoadBalanceRequest) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_messages_b81c731f0e83edbd, []int{3}
|
||||
}
|
||||
func (m *InitialLoadBalanceRequest) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_InitialLoadBalanceRequest.Unmarshal(m, b)
|
||||
}
|
||||
func (m *InitialLoadBalanceRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_InitialLoadBalanceRequest.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *InitialLoadBalanceRequest) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_InitialLoadBalanceRequest.Merge(dst, src)
|
||||
}
|
||||
func (m *InitialLoadBalanceRequest) XXX_Size() int {
|
||||
return xxx_messageInfo_InitialLoadBalanceRequest.Size(m)
|
||||
}
|
||||
func (m *InitialLoadBalanceRequest) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_InitialLoadBalanceRequest.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_InitialLoadBalanceRequest proto.InternalMessageInfo
|
||||
|
||||
func (m *InitialLoadBalanceRequest) GetName() string {
|
||||
if m != nil {
|
||||
@ -256,13 +327,35 @@ type ClientStats struct {
|
||||
NumCallsFinishedWithClientFailedToSend int64 `protobuf:"varint,6,opt,name=num_calls_finished_with_client_failed_to_send,json=numCallsFinishedWithClientFailedToSend" json:"num_calls_finished_with_client_failed_to_send,omitempty"`
|
||||
// The total number of RPCs that finished and are known to have been received
|
||||
// by a server.
|
||||
NumCallsFinishedKnownReceived int64 `protobuf:"varint,7,opt,name=num_calls_finished_known_received,json=numCallsFinishedKnownReceived" json:"num_calls_finished_known_received,omitempty"`
|
||||
NumCallsFinishedKnownReceived int64 `protobuf:"varint,7,opt,name=num_calls_finished_known_received,json=numCallsFinishedKnownReceived" json:"num_calls_finished_known_received,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *ClientStats) Reset() { *m = ClientStats{} }
|
||||
func (m *ClientStats) String() string { return proto.CompactTextString(m) }
|
||||
func (*ClientStats) ProtoMessage() {}
|
||||
func (*ClientStats) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} }
|
||||
func (m *ClientStats) Reset() { *m = ClientStats{} }
|
||||
func (m *ClientStats) String() string { return proto.CompactTextString(m) }
|
||||
func (*ClientStats) ProtoMessage() {}
|
||||
func (*ClientStats) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_messages_b81c731f0e83edbd, []int{4}
|
||||
}
|
||||
func (m *ClientStats) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_ClientStats.Unmarshal(m, b)
|
||||
}
|
||||
func (m *ClientStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_ClientStats.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *ClientStats) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_ClientStats.Merge(dst, src)
|
||||
}
|
||||
func (m *ClientStats) XXX_Size() int {
|
||||
return xxx_messageInfo_ClientStats.Size(m)
|
||||
}
|
||||
func (m *ClientStats) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_ClientStats.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_ClientStats proto.InternalMessageInfo
|
||||
|
||||
func (m *ClientStats) GetTimestamp() *Timestamp {
|
||||
if m != nil {
|
||||
@ -318,12 +411,34 @@ type LoadBalanceResponse struct {
|
||||
// *LoadBalanceResponse_InitialResponse
|
||||
// *LoadBalanceResponse_ServerList
|
||||
LoadBalanceResponseType isLoadBalanceResponse_LoadBalanceResponseType `protobuf_oneof:"load_balance_response_type"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *LoadBalanceResponse) Reset() { *m = LoadBalanceResponse{} }
|
||||
func (m *LoadBalanceResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*LoadBalanceResponse) ProtoMessage() {}
|
||||
func (*LoadBalanceResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} }
|
||||
func (m *LoadBalanceResponse) Reset() { *m = LoadBalanceResponse{} }
|
||||
func (m *LoadBalanceResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*LoadBalanceResponse) ProtoMessage() {}
|
||||
func (*LoadBalanceResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_messages_b81c731f0e83edbd, []int{5}
|
||||
}
|
||||
func (m *LoadBalanceResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_LoadBalanceResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *LoadBalanceResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_LoadBalanceResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *LoadBalanceResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_LoadBalanceResponse.Merge(dst, src)
|
||||
}
|
||||
func (m *LoadBalanceResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_LoadBalanceResponse.Size(m)
|
||||
}
|
||||
func (m *LoadBalanceResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_LoadBalanceResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_LoadBalanceResponse proto.InternalMessageInfo
|
||||
|
||||
type isLoadBalanceResponse_LoadBalanceResponseType interface {
|
||||
isLoadBalanceResponse_LoadBalanceResponseType()
|
||||
@ -419,12 +534,12 @@ func _LoadBalanceResponse_OneofSizer(msg proto.Message) (n int) {
|
||||
switch x := m.LoadBalanceResponseType.(type) {
|
||||
case *LoadBalanceResponse_InitialResponse:
|
||||
s := proto.Size(x.InitialResponse)
|
||||
n += proto.SizeVarint(1<<3 | proto.WireBytes)
|
||||
n += 1 // tag and wire
|
||||
n += proto.SizeVarint(uint64(s))
|
||||
n += s
|
||||
case *LoadBalanceResponse_ServerList:
|
||||
s := proto.Size(x.ServerList)
|
||||
n += proto.SizeVarint(2<<3 | proto.WireBytes)
|
||||
n += 1 // tag and wire
|
||||
n += proto.SizeVarint(uint64(s))
|
||||
n += s
|
||||
case nil:
|
||||
@ -445,12 +560,34 @@ type InitialLoadBalanceResponse struct {
|
||||
// to the load balancer. Stats should only be reported when the duration is
|
||||
// positive.
|
||||
ClientStatsReportInterval *Duration `protobuf:"bytes,2,opt,name=client_stats_report_interval,json=clientStatsReportInterval" json:"client_stats_report_interval,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *InitialLoadBalanceResponse) Reset() { *m = InitialLoadBalanceResponse{} }
|
||||
func (m *InitialLoadBalanceResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*InitialLoadBalanceResponse) ProtoMessage() {}
|
||||
func (*InitialLoadBalanceResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} }
|
||||
func (m *InitialLoadBalanceResponse) Reset() { *m = InitialLoadBalanceResponse{} }
|
||||
func (m *InitialLoadBalanceResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*InitialLoadBalanceResponse) ProtoMessage() {}
|
||||
func (*InitialLoadBalanceResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_messages_b81c731f0e83edbd, []int{6}
|
||||
}
|
||||
func (m *InitialLoadBalanceResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_InitialLoadBalanceResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *InitialLoadBalanceResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_InitialLoadBalanceResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *InitialLoadBalanceResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_InitialLoadBalanceResponse.Merge(dst, src)
|
||||
}
|
||||
func (m *InitialLoadBalanceResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_InitialLoadBalanceResponse.Size(m)
|
||||
}
|
||||
func (m *InitialLoadBalanceResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_InitialLoadBalanceResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_InitialLoadBalanceResponse proto.InternalMessageInfo
|
||||
|
||||
func (m *InitialLoadBalanceResponse) GetLoadBalancerDelegate() string {
|
||||
if m != nil {
|
||||
@ -471,13 +608,35 @@ type ServerList struct {
|
||||
// be updated when server resolutions change or as needed to balance load
|
||||
// across more servers. The client should consume the server list in order
|
||||
// unless instructed otherwise via the client_config.
|
||||
Servers []*Server `protobuf:"bytes,1,rep,name=servers" json:"servers,omitempty"`
|
||||
Servers []*Server `protobuf:"bytes,1,rep,name=servers" json:"servers,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *ServerList) Reset() { *m = ServerList{} }
|
||||
func (m *ServerList) String() string { return proto.CompactTextString(m) }
|
||||
func (*ServerList) ProtoMessage() {}
|
||||
func (*ServerList) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} }
|
||||
func (m *ServerList) Reset() { *m = ServerList{} }
|
||||
func (m *ServerList) String() string { return proto.CompactTextString(m) }
|
||||
func (*ServerList) ProtoMessage() {}
|
||||
func (*ServerList) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_messages_b81c731f0e83edbd, []int{7}
|
||||
}
|
||||
func (m *ServerList) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_ServerList.Unmarshal(m, b)
|
||||
}
|
||||
func (m *ServerList) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_ServerList.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *ServerList) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_ServerList.Merge(dst, src)
|
||||
}
|
||||
func (m *ServerList) XXX_Size() int {
|
||||
return xxx_messageInfo_ServerList.Size(m)
|
||||
}
|
||||
func (m *ServerList) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_ServerList.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_ServerList proto.InternalMessageInfo
|
||||
|
||||
func (m *ServerList) GetServers() []*Server {
|
||||
if m != nil {
|
||||
@ -508,13 +667,35 @@ type Server struct {
|
||||
DropForRateLimiting bool `protobuf:"varint,4,opt,name=drop_for_rate_limiting,json=dropForRateLimiting" json:"drop_for_rate_limiting,omitempty"`
|
||||
// Indicates whether this particular request should be dropped by the client
|
||||
// for load balancing.
|
||||
DropForLoadBalancing bool `protobuf:"varint,5,opt,name=drop_for_load_balancing,json=dropForLoadBalancing" json:"drop_for_load_balancing,omitempty"`
|
||||
DropForLoadBalancing bool `protobuf:"varint,5,opt,name=drop_for_load_balancing,json=dropForLoadBalancing" json:"drop_for_load_balancing,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Server) Reset() { *m = Server{} }
|
||||
func (m *Server) String() string { return proto.CompactTextString(m) }
|
||||
func (*Server) ProtoMessage() {}
|
||||
func (*Server) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} }
|
||||
func (m *Server) Reset() { *m = Server{} }
|
||||
func (m *Server) String() string { return proto.CompactTextString(m) }
|
||||
func (*Server) ProtoMessage() {}
|
||||
func (*Server) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_messages_b81c731f0e83edbd, []int{8}
|
||||
}
|
||||
func (m *Server) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Server.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Server) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Server.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *Server) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Server.Merge(dst, src)
|
||||
}
|
||||
func (m *Server) XXX_Size() int {
|
||||
return xxx_messageInfo_Server.Size(m)
|
||||
}
|
||||
func (m *Server) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Server.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Server proto.InternalMessageInfo
|
||||
|
||||
func (m *Server) GetIpAddress() []byte {
|
||||
if m != nil {
|
||||
@ -563,53 +744,56 @@ func init() {
|
||||
proto.RegisterType((*Server)(nil), "grpc.lb.v1.Server")
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("grpc_lb_v1/messages/messages.proto", fileDescriptor0) }
|
||||
|
||||
var fileDescriptor0 = []byte{
|
||||
// 709 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xdd, 0x4e, 0x1b, 0x3b,
|
||||
0x10, 0x26, 0x27, 0x01, 0x92, 0x09, 0x3a, 0xe4, 0x98, 0x1c, 0x08, 0x14, 0x24, 0xba, 0x52, 0x69,
|
||||
0x54, 0xd1, 0x20, 0xa0, 0xbd, 0xe8, 0xcf, 0x45, 0x1b, 0x10, 0x0a, 0x2d, 0x17, 0x95, 0x43, 0x55,
|
||||
0xa9, 0x52, 0x65, 0x39, 0xd9, 0x21, 0x58, 0x6c, 0xec, 0xad, 0xed, 0x04, 0xf5, 0x11, 0xfa, 0x28,
|
||||
0x7d, 0x8c, 0xaa, 0xcf, 0xd0, 0xf7, 0xa9, 0xd6, 0xbb, 0x9b, 0x5d, 0x20, 0x80, 0x7a, 0x67, 0x8f,
|
||||
0xbf, 0xf9, 0xbe, 0xf1, 0xac, 0xbf, 0x59, 0xf0, 0x06, 0x3a, 0xec, 0xb3, 0xa0, 0xc7, 0xc6, 0xbb,
|
||||
0x3b, 0x43, 0x34, 0x86, 0x0f, 0xd0, 0x4c, 0x16, 0xad, 0x50, 0x2b, 0xab, 0x08, 0x44, 0x98, 0x56,
|
||||
0xd0, 0x6b, 0x8d, 0x77, 0xbd, 0x97, 0x50, 0x3e, 0x1c, 0x69, 0x6e, 0x85, 0x92, 0xa4, 0x01, 0xf3,
|
||||
0x06, 0xfb, 0x4a, 0xfa, 0xa6, 0x51, 0xd8, 0x2c, 0x34, 0x8b, 0x34, 0xdd, 0x92, 0x3a, 0xcc, 0x4a,
|
||||
0x2e, 0x95, 0x69, 0xfc, 0xb3, 0x59, 0x68, 0xce, 0xd2, 0x78, 0xe3, 0xbd, 0x82, 0xca, 0xa9, 0x18,
|
||||
0xa2, 0xb1, 0x7c, 0x18, 0xfe, 0x75, 0xf2, 0xcf, 0x02, 0x90, 0x13, 0xc5, 0xfd, 0x36, 0x0f, 0xb8,
|
||||
0xec, 0x23, 0xc5, 0xaf, 0x23, 0x34, 0x96, 0x7c, 0x80, 0x45, 0x21, 0x85, 0x15, 0x3c, 0x60, 0x3a,
|
||||
0x0e, 0x39, 0xba, 0xea, 0xde, 0xa3, 0x56, 0x56, 0x75, 0xeb, 0x38, 0x86, 0xdc, 0xcc, 0xef, 0xcc,
|
||||
0xd0, 0x7f, 0x93, 0xfc, 0x94, 0xf1, 0x35, 0x2c, 0xf4, 0x03, 0x81, 0xd2, 0x32, 0x63, 0xb9, 0x8d,
|
||||
0xab, 0xa8, 0xee, 0xad, 0xe4, 0xe9, 0x0e, 0xdc, 0x79, 0x37, 0x3a, 0xee, 0xcc, 0xd0, 0x6a, 0x3f,
|
||||
0xdb, 0xb6, 0x1f, 0xc0, 0x6a, 0xa0, 0xb8, 0xcf, 0x7a, 0xb1, 0x4c, 0x5a, 0x14, 0xb3, 0xdf, 0x42,
|
||||
0xf4, 0x76, 0x60, 0xf5, 0xd6, 0x4a, 0x08, 0x81, 0x92, 0xe4, 0x43, 0x74, 0xe5, 0x57, 0xa8, 0x5b,
|
||||
0x7b, 0xdf, 0x4b, 0x50, 0xcd, 0x89, 0x91, 0x7d, 0xa8, 0xd8, 0xb4, 0x83, 0xc9, 0x3d, 0xff, 0xcf,
|
||||
0x17, 0x36, 0x69, 0x2f, 0xcd, 0x70, 0xe4, 0x09, 0xfc, 0x27, 0x47, 0x43, 0xd6, 0xe7, 0x41, 0x60,
|
||||
0xa2, 0x3b, 0x69, 0x8b, 0xbe, 0xbb, 0x55, 0x91, 0x2e, 0xca, 0xd1, 0xf0, 0x20, 0x8a, 0x77, 0xe3,
|
||||
0x30, 0xd9, 0x06, 0x92, 0x61, 0xcf, 0x84, 0x14, 0xe6, 0x1c, 0xfd, 0x46, 0xd1, 0x81, 0x6b, 0x29,
|
||||
0xf8, 0x28, 0x89, 0x13, 0x06, 0xad, 0x9b, 0x68, 0x76, 0x29, 0xec, 0x39, 0xf3, 0xb5, 0x0a, 0xd9,
|
||||
0x99, 0xd2, 0x4c, 0x73, 0x8b, 0x2c, 0x10, 0x43, 0x61, 0x85, 0x1c, 0x34, 0x4a, 0x8e, 0xe9, 0xf1,
|
||||
0x75, 0xa6, 0x4f, 0xc2, 0x9e, 0x1f, 0x6a, 0x15, 0x1e, 0x29, 0x4d, 0xb9, 0xc5, 0x93, 0x04, 0x4e,
|
||||
0x38, 0xec, 0xdc, 0x2b, 0x90, 0x6b, 0x77, 0xa4, 0x30, 0xeb, 0x14, 0x9a, 0x77, 0x28, 0x64, 0xbd,
|
||||
0x8f, 0x24, 0xbe, 0xc0, 0xd3, 0xdb, 0x24, 0x92, 0x67, 0x70, 0xc6, 0x45, 0x80, 0x3e, 0xb3, 0x8a,
|
||||
0x19, 0x94, 0x7e, 0x63, 0xce, 0x09, 0x6c, 0x4d, 0x13, 0x88, 0x3f, 0xd5, 0x91, 0xc3, 0x9f, 0xaa,
|
||||
0x2e, 0x4a, 0x9f, 0x74, 0xe0, 0xe1, 0x14, 0xfa, 0x0b, 0xa9, 0x2e, 0x25, 0xd3, 0xd8, 0x47, 0x31,
|
||||
0x46, 0xbf, 0x31, 0xef, 0x28, 0x37, 0xae, 0x53, 0xbe, 0x8f, 0x50, 0x34, 0x01, 0x79, 0xbf, 0x0a,
|
||||
0xb0, 0x74, 0xe5, 0xd9, 0x98, 0x50, 0x49, 0x83, 0xa4, 0x0b, 0xb5, 0xcc, 0x01, 0x71, 0x2c, 0x79,
|
||||
0x1a, 0x5b, 0xf7, 0x59, 0x20, 0x46, 0x77, 0x66, 0xe8, 0xe2, 0xc4, 0x03, 0x09, 0xe9, 0x0b, 0xa8,
|
||||
0x1a, 0xd4, 0x63, 0xd4, 0x2c, 0x10, 0xc6, 0x26, 0x1e, 0x58, 0xce, 0xf3, 0x75, 0xdd, 0xf1, 0x89,
|
||||
0x70, 0x1e, 0x02, 0x33, 0xd9, 0xb5, 0xd7, 0x61, 0xed, 0x9a, 0x03, 0x62, 0xce, 0xd8, 0x02, 0x3f,
|
||||
0x0a, 0xb0, 0x76, 0x7b, 0x29, 0xe4, 0x19, 0x2c, 0xe7, 0x93, 0x35, 0xf3, 0x31, 0xc0, 0x01, 0xb7,
|
||||
0xa9, 0x2d, 0xea, 0x41, 0x96, 0xa4, 0x0f, 0x93, 0x33, 0xf2, 0x11, 0xd6, 0xf3, 0x96, 0x65, 0x1a,
|
||||
0x43, 0xa5, 0x2d, 0x13, 0xd2, 0xa2, 0x1e, 0xf3, 0x20, 0x29, 0xbf, 0x9e, 0x2f, 0x3f, 0x1d, 0x62,
|
||||
0x74, 0x35, 0xe7, 0x5e, 0xea, 0xf2, 0x8e, 0x93, 0x34, 0xef, 0x0d, 0x40, 0x76, 0x4b, 0xb2, 0x1d,
|
||||
0x0d, 0xac, 0x68, 0x17, 0x0d, 0xac, 0x62, 0xb3, 0xba, 0x47, 0x6e, 0xb6, 0x83, 0xa6, 0x90, 0x77,
|
||||
0xa5, 0x72, 0xb1, 0x56, 0xf2, 0x7e, 0x17, 0x60, 0x2e, 0x3e, 0x21, 0x1b, 0x00, 0x22, 0x64, 0xdc,
|
||||
0xf7, 0x35, 0x9a, 0x78, 0xe4, 0x2d, 0xd0, 0x8a, 0x08, 0xdf, 0xc6, 0x81, 0xc8, 0xfd, 0x91, 0x76,
|
||||
0x32, 0xf3, 0xdc, 0x3a, 0x32, 0xe3, 0x95, 0x4e, 0x5a, 0x75, 0x81, 0xd2, 0x99, 0xb1, 0x42, 0x6b,
|
||||
0xb9, 0x46, 0x9c, 0x46, 0x71, 0xb2, 0x0f, 0xcb, 0x77, 0x98, 0xae, 0x4c, 0x97, 0xfc, 0x29, 0x06,
|
||||
0x7b, 0x0e, 0x2b, 0x77, 0x19, 0xa9, 0x4c, 0xeb, 0xfe, 0x14, 0xd3, 0xb4, 0xe1, 0x73, 0x39, 0xfd,
|
||||
0x47, 0xf4, 0xe6, 0xdc, 0x4f, 0x62, 0xff, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xa3, 0x36, 0x86,
|
||||
0xa6, 0x4a, 0x06, 0x00, 0x00,
|
||||
func init() {
|
||||
proto.RegisterFile("grpc_lb_v1/messages/messages.proto", fileDescriptor_messages_b81c731f0e83edbd)
|
||||
}
|
||||
|
||||
var fileDescriptor_messages_b81c731f0e83edbd = []byte{
|
||||
// 731 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xdd, 0x4e, 0x1b, 0x39,
|
||||
0x14, 0x26, 0x9b, 0x00, 0xc9, 0x09, 0x5a, 0xb2, 0x26, 0x0b, 0x81, 0x05, 0x89, 0x1d, 0x69, 0xd9,
|
||||
0x68, 0xc5, 0x4e, 0x04, 0xd9, 0xbd, 0xe8, 0xcf, 0x45, 0x1b, 0x10, 0x0a, 0x2d, 0x17, 0x95, 0x43,
|
||||
0x55, 0xa9, 0x52, 0x65, 0x39, 0x19, 0x33, 0x58, 0x38, 0xf6, 0xd4, 0x76, 0x82, 0xfa, 0x08, 0x7d,
|
||||
0x94, 0x3e, 0x46, 0xd5, 0x67, 0xe8, 0xfb, 0x54, 0xe3, 0x99, 0xc9, 0x0c, 0x10, 0x40, 0xbd, 0x89,
|
||||
0xec, 0xe3, 0xef, 0x7c, 0xdf, 0xf1, 0x89, 0xbf, 0x33, 0xe0, 0x85, 0x3a, 0x1a, 0x11, 0x31, 0x24,
|
||||
0xd3, 0x83, 0xce, 0x98, 0x19, 0x43, 0x43, 0x66, 0x66, 0x0b, 0x3f, 0xd2, 0xca, 0x2a, 0x04, 0x31,
|
||||
0xc6, 0x17, 0x43, 0x7f, 0x7a, 0xe0, 0x3d, 0x85, 0xea, 0xf1, 0x44, 0x53, 0xcb, 0x95, 0x44, 0x2d,
|
||||
0x58, 0x36, 0x6c, 0xa4, 0x64, 0x60, 0x5a, 0xa5, 0xdd, 0x52, 0xbb, 0x8c, 0xb3, 0x2d, 0x6a, 0xc2,
|
||||
0xa2, 0xa4, 0x52, 0x99, 0xd6, 0x2f, 0xbb, 0xa5, 0xf6, 0x22, 0x4e, 0x36, 0xde, 0x33, 0xa8, 0x9d,
|
||||
0xf3, 0x31, 0x33, 0x96, 0x8e, 0xa3, 0x9f, 0x4e, 0xfe, 0x5a, 0x02, 0x74, 0xa6, 0x68, 0xd0, 0xa3,
|
||||
0x82, 0xca, 0x11, 0xc3, 0xec, 0xe3, 0x84, 0x19, 0x8b, 0xde, 0xc0, 0x2a, 0x97, 0xdc, 0x72, 0x2a,
|
||||
0x88, 0x4e, 0x42, 0x8e, 0xae, 0x7e, 0xf8, 0x97, 0x9f, 0x57, 0xed, 0x9f, 0x26, 0x90, 0xbb, 0xf9,
|
||||
0xfd, 0x05, 0xfc, 0x6b, 0x9a, 0x9f, 0x31, 0x3e, 0x87, 0x95, 0x91, 0xe0, 0x4c, 0x5a, 0x62, 0x2c,
|
||||
0xb5, 0x49, 0x15, 0xf5, 0xc3, 0x8d, 0x22, 0xdd, 0x91, 0x3b, 0x1f, 0xc4, 0xc7, 0xfd, 0x05, 0x5c,
|
||||
0x1f, 0xe5, 0xdb, 0xde, 0x1f, 0xb0, 0x29, 0x14, 0x0d, 0xc8, 0x30, 0x91, 0xc9, 0x8a, 0x22, 0xf6,
|
||||
0x53, 0xc4, 0xbc, 0x0e, 0x6c, 0xde, 0x5b, 0x09, 0x42, 0x50, 0x91, 0x74, 0xcc, 0x5c, 0xf9, 0x35,
|
||||
0xec, 0xd6, 0xde, 0xe7, 0x0a, 0xd4, 0x0b, 0x62, 0xa8, 0x0b, 0x35, 0x9b, 0x75, 0x30, 0xbd, 0xe7,
|
||||
0xef, 0xc5, 0xc2, 0x66, 0xed, 0xc5, 0x39, 0x0e, 0xfd, 0x03, 0xbf, 0xc9, 0xc9, 0x98, 0x8c, 0xa8,
|
||||
0x10, 0x26, 0xbe, 0x93, 0xb6, 0x2c, 0x70, 0xb7, 0x2a, 0xe3, 0x55, 0x39, 0x19, 0x1f, 0xc5, 0xf1,
|
||||
0x41, 0x12, 0x46, 0xfb, 0x80, 0x72, 0xec, 0x05, 0x97, 0xdc, 0x5c, 0xb2, 0xa0, 0x55, 0x76, 0xe0,
|
||||
0x46, 0x06, 0x3e, 0x49, 0xe3, 0x88, 0x80, 0x7f, 0x17, 0x4d, 0xae, 0xb9, 0xbd, 0x24, 0x81, 0x56,
|
||||
0x11, 0xb9, 0x50, 0x9a, 0x68, 0x6a, 0x19, 0x11, 0x7c, 0xcc, 0x2d, 0x97, 0x61, 0xab, 0xe2, 0x98,
|
||||
0xfe, 0xbe, 0xcd, 0xf4, 0x8e, 0xdb, 0xcb, 0x63, 0xad, 0xa2, 0x13, 0xa5, 0x31, 0xb5, 0xec, 0x2c,
|
||||
0x85, 0x23, 0x0a, 0x9d, 0x47, 0x05, 0x0a, 0xed, 0x8e, 0x15, 0x16, 0x9d, 0x42, 0xfb, 0x01, 0x85,
|
||||
0xbc, 0xf7, 0xb1, 0xc4, 0x07, 0xf8, 0xf7, 0x3e, 0x89, 0xf4, 0x19, 0x5c, 0x50, 0x2e, 0x58, 0x40,
|
||||
0xac, 0x22, 0x86, 0xc9, 0xa0, 0xb5, 0xe4, 0x04, 0xf6, 0xe6, 0x09, 0x24, 0x7f, 0xd5, 0x89, 0xc3,
|
||||
0x9f, 0xab, 0x01, 0x93, 0x01, 0xea, 0xc3, 0x9f, 0x73, 0xe8, 0xaf, 0xa4, 0xba, 0x96, 0x44, 0xb3,
|
||||
0x11, 0xe3, 0x53, 0x16, 0xb4, 0x96, 0x1d, 0xe5, 0xce, 0x6d, 0xca, 0xd7, 0x31, 0x0a, 0xa7, 0x20,
|
||||
0xef, 0x5b, 0x09, 0xd6, 0x6e, 0x3c, 0x1b, 0x13, 0x29, 0x69, 0x18, 0x1a, 0x40, 0x23, 0x77, 0x40,
|
||||
0x12, 0x4b, 0x9f, 0xc6, 0xde, 0x63, 0x16, 0x48, 0xd0, 0xfd, 0x05, 0xbc, 0x3a, 0xf3, 0x40, 0x4a,
|
||||
0xfa, 0x04, 0xea, 0x86, 0xe9, 0x29, 0xd3, 0x44, 0x70, 0x63, 0x53, 0x0f, 0xac, 0x17, 0xf9, 0x06,
|
||||
0xee, 0xf8, 0x8c, 0x3b, 0x0f, 0x81, 0x99, 0xed, 0x7a, 0xdb, 0xb0, 0x75, 0xcb, 0x01, 0x09, 0x67,
|
||||
0x62, 0x81, 0x2f, 0x25, 0xd8, 0xba, 0xbf, 0x14, 0xf4, 0x1f, 0xac, 0x17, 0x93, 0x35, 0x09, 0x98,
|
||||
0x60, 0x21, 0xb5, 0x99, 0x2d, 0x9a, 0x22, 0x4f, 0xd2, 0xc7, 0xe9, 0x19, 0x7a, 0x0b, 0xdb, 0x45,
|
||||
0xcb, 0x12, 0xcd, 0x22, 0xa5, 0x2d, 0xe1, 0xd2, 0x32, 0x3d, 0xa5, 0x22, 0x2d, 0xbf, 0x59, 0x2c,
|
||||
0x3f, 0x1b, 0x62, 0x78, 0xb3, 0xe0, 0x5e, 0xec, 0xf2, 0x4e, 0xd3, 0x34, 0xef, 0x05, 0x40, 0x7e,
|
||||
0x4b, 0xb4, 0x1f, 0x0f, 0xac, 0x78, 0x17, 0x0f, 0xac, 0x72, 0xbb, 0x7e, 0x88, 0xee, 0xb6, 0x03,
|
||||
0x67, 0x90, 0x57, 0x95, 0x6a, 0xb9, 0x51, 0xf1, 0xbe, 0x97, 0x60, 0x29, 0x39, 0x41, 0x3b, 0x00,
|
||||
0x3c, 0x22, 0x34, 0x08, 0x34, 0x33, 0xc9, 0xc8, 0x5b, 0xc1, 0x35, 0x1e, 0xbd, 0x4c, 0x02, 0xb1,
|
||||
0xfb, 0x63, 0xed, 0x74, 0xe6, 0xb9, 0x75, 0x6c, 0xc6, 0x1b, 0x9d, 0xb4, 0xea, 0x8a, 0x49, 0x67,
|
||||
0xc6, 0x1a, 0x6e, 0x14, 0x1a, 0x71, 0x1e, 0xc7, 0x51, 0x17, 0xd6, 0x1f, 0x30, 0x5d, 0x15, 0xaf,
|
||||
0x05, 0x73, 0x0c, 0xf6, 0x3f, 0x6c, 0x3c, 0x64, 0xa4, 0x2a, 0x6e, 0x06, 0x73, 0x4c, 0xd3, 0xeb,
|
||||
0xbe, 0x3f, 0x08, 0x95, 0x0a, 0x05, 0xf3, 0x43, 0x25, 0xa8, 0x0c, 0x7d, 0xa5, 0xc3, 0x4e, 0xdc,
|
||||
0x0d, 0xf7, 0x23, 0x86, 0x9d, 0x39, 0x5f, 0x95, 0xe1, 0x92, 0xfb, 0x9a, 0x74, 0x7f, 0x04, 0x00,
|
||||
0x00, 0xff, 0xff, 0x8e, 0xd0, 0x70, 0xb7, 0x73, 0x06, 0x00, 0x00,
|
||||
}
|
||||
|
159
vendor/google.golang.org/grpc/grpclb_picker.go
generated
vendored
Normal file
@ -0,0 +1,159 @@
|
||||
/*
|
||||
*
|
||||
* 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 grpc
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/codes"
|
||||
lbpb "google.golang.org/grpc/grpclb/grpc_lb_v1/messages"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
type rpcStats struct {
|
||||
NumCallsStarted int64
|
||||
NumCallsFinished int64
|
||||
NumCallsFinishedWithDropForRateLimiting int64
|
||||
NumCallsFinishedWithDropForLoadBalancing int64
|
||||
NumCallsFinishedWithClientFailedToSend int64
|
||||
NumCallsFinishedKnownReceived int64
|
||||
}
|
||||
|
||||
// toClientStats converts rpcStats to lbpb.ClientStats, and clears rpcStats.
|
||||
func (s *rpcStats) toClientStats() *lbpb.ClientStats {
|
||||
stats := &lbpb.ClientStats{
|
||||
NumCallsStarted: atomic.SwapInt64(&s.NumCallsStarted, 0),
|
||||
NumCallsFinished: atomic.SwapInt64(&s.NumCallsFinished, 0),
|
||||
NumCallsFinishedWithDropForRateLimiting: atomic.SwapInt64(&s.NumCallsFinishedWithDropForRateLimiting, 0),
|
||||
NumCallsFinishedWithDropForLoadBalancing: atomic.SwapInt64(&s.NumCallsFinishedWithDropForLoadBalancing, 0),
|
||||
NumCallsFinishedWithClientFailedToSend: atomic.SwapInt64(&s.NumCallsFinishedWithClientFailedToSend, 0),
|
||||
NumCallsFinishedKnownReceived: atomic.SwapInt64(&s.NumCallsFinishedKnownReceived, 0),
|
||||
}
|
||||
return stats
|
||||
}
|
||||
|
||||
func (s *rpcStats) dropForRateLimiting() {
|
||||
atomic.AddInt64(&s.NumCallsStarted, 1)
|
||||
atomic.AddInt64(&s.NumCallsFinishedWithDropForRateLimiting, 1)
|
||||
atomic.AddInt64(&s.NumCallsFinished, 1)
|
||||
}
|
||||
|
||||
func (s *rpcStats) dropForLoadBalancing() {
|
||||
atomic.AddInt64(&s.NumCallsStarted, 1)
|
||||
atomic.AddInt64(&s.NumCallsFinishedWithDropForLoadBalancing, 1)
|
||||
atomic.AddInt64(&s.NumCallsFinished, 1)
|
||||
}
|
||||
|
||||
func (s *rpcStats) failedToSend() {
|
||||
atomic.AddInt64(&s.NumCallsStarted, 1)
|
||||
atomic.AddInt64(&s.NumCallsFinishedWithClientFailedToSend, 1)
|
||||
atomic.AddInt64(&s.NumCallsFinished, 1)
|
||||
}
|
||||
|
||||
func (s *rpcStats) knownReceived() {
|
||||
atomic.AddInt64(&s.NumCallsStarted, 1)
|
||||
atomic.AddInt64(&s.NumCallsFinishedKnownReceived, 1)
|
||||
atomic.AddInt64(&s.NumCallsFinished, 1)
|
||||
}
|
||||
|
||||
type errPicker struct {
|
||||
// Pick always returns this err.
|
||||
err error
|
||||
}
|
||||
|
||||
func (p *errPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
||||
return nil, nil, p.err
|
||||
}
|
||||
|
||||
// rrPicker does roundrobin on subConns. It's typically used when there's no
|
||||
// response from remote balancer, and grpclb falls back to the resolved
|
||||
// backends.
|
||||
//
|
||||
// It guaranteed that len(subConns) > 0.
|
||||
type rrPicker struct {
|
||||
mu sync.Mutex
|
||||
subConns []balancer.SubConn // The subConns that were READY when taking the snapshot.
|
||||
subConnsNext int
|
||||
}
|
||||
|
||||
func (p *rrPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
||||
p.mu.Lock()
|
||||
defer p.mu.Unlock()
|
||||
sc := p.subConns[p.subConnsNext]
|
||||
p.subConnsNext = (p.subConnsNext + 1) % len(p.subConns)
|
||||
return sc, nil, nil
|
||||
}
|
||||
|
||||
// lbPicker does two layers of picks:
|
||||
//
|
||||
// First layer: roundrobin on all servers in serverList, including drops and backends.
|
||||
// - If it picks a drop, the RPC will fail as being dropped.
|
||||
// - If it picks a backend, do a second layer pick to pick the real backend.
|
||||
//
|
||||
// Second layer: roundrobin on all READY backends.
|
||||
//
|
||||
// It's guaranteed that len(serverList) > 0.
|
||||
type lbPicker struct {
|
||||
mu sync.Mutex
|
||||
serverList []*lbpb.Server
|
||||
serverListNext int
|
||||
subConns []balancer.SubConn // The subConns that were READY when taking the snapshot.
|
||||
subConnsNext int
|
||||
|
||||
stats *rpcStats
|
||||
}
|
||||
|
||||
func (p *lbPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
||||
p.mu.Lock()
|
||||
defer p.mu.Unlock()
|
||||
|
||||
// Layer one roundrobin on serverList.
|
||||
s := p.serverList[p.serverListNext]
|
||||
p.serverListNext = (p.serverListNext + 1) % len(p.serverList)
|
||||
|
||||
// If it's a drop, return an error and fail the RPC.
|
||||
if s.DropForRateLimiting {
|
||||
p.stats.dropForRateLimiting()
|
||||
return nil, nil, status.Errorf(codes.Unavailable, "request dropped by grpclb")
|
||||
}
|
||||
if s.DropForLoadBalancing {
|
||||
p.stats.dropForLoadBalancing()
|
||||
return nil, nil, status.Errorf(codes.Unavailable, "request dropped by grpclb")
|
||||
}
|
||||
|
||||
// If not a drop but there's no ready subConns.
|
||||
if len(p.subConns) <= 0 {
|
||||
return nil, nil, balancer.ErrNoSubConnAvailable
|
||||
}
|
||||
|
||||
// Return the next ready subConn in the list, also collect rpc stats.
|
||||
sc := p.subConns[p.subConnsNext]
|
||||
p.subConnsNext = (p.subConnsNext + 1) % len(p.subConns)
|
||||
done := func(info balancer.DoneInfo) {
|
||||
if !info.BytesSent {
|
||||
p.stats.failedToSend()
|
||||
} else if info.BytesReceived {
|
||||
p.stats.knownReceived()
|
||||
}
|
||||
}
|
||||
return sc, done, nil
|
||||
}
|
266
vendor/google.golang.org/grpc/grpclb_remote_balancer.go
generated
vendored
Normal file
@ -0,0 +1,266 @@
|
||||
/*
|
||||
*
|
||||
* 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 grpc
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net"
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/channelz"
|
||||
|
||||
"google.golang.org/grpc/connectivity"
|
||||
lbpb "google.golang.org/grpc/grpclb/grpc_lb_v1/messages"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/metadata"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
// processServerList updates balaner's internal state, create/remove SubConns
|
||||
// and regenerates picker using the received serverList.
|
||||
func (lb *lbBalancer) processServerList(l *lbpb.ServerList) {
|
||||
grpclog.Infof("lbBalancer: processing server list: %+v", l)
|
||||
lb.mu.Lock()
|
||||
defer lb.mu.Unlock()
|
||||
|
||||
// Set serverListReceived to true so fallback will not take effect if it has
|
||||
// not hit timeout.
|
||||
lb.serverListReceived = true
|
||||
|
||||
// If the new server list == old server list, do nothing.
|
||||
if reflect.DeepEqual(lb.fullServerList, l.Servers) {
|
||||
grpclog.Infof("lbBalancer: new serverlist same as the previous one, ignoring")
|
||||
return
|
||||
}
|
||||
lb.fullServerList = l.Servers
|
||||
|
||||
var backendAddrs []resolver.Address
|
||||
for _, s := range l.Servers {
|
||||
if s.DropForLoadBalancing || s.DropForRateLimiting {
|
||||
continue
|
||||
}
|
||||
|
||||
md := metadata.Pairs(lbTokeyKey, s.LoadBalanceToken)
|
||||
ip := net.IP(s.IpAddress)
|
||||
ipStr := ip.String()
|
||||
if ip.To4() == nil {
|
||||
// Add square brackets to ipv6 addresses, otherwise net.Dial() and
|
||||
// net.SplitHostPort() will return too many colons error.
|
||||
ipStr = fmt.Sprintf("[%s]", ipStr)
|
||||
}
|
||||
addr := resolver.Address{
|
||||
Addr: fmt.Sprintf("%s:%d", ipStr, s.Port),
|
||||
Metadata: &md,
|
||||
}
|
||||
|
||||
backendAddrs = append(backendAddrs, addr)
|
||||
}
|
||||
|
||||
// Call refreshSubConns to create/remove SubConns.
|
||||
lb.refreshSubConns(backendAddrs)
|
||||
// Regenerate and update picker no matter if there's update on backends (if
|
||||
// any SubConn will be newed/removed). Because since the full serverList was
|
||||
// different, there might be updates in drops or pick weights(different
|
||||
// number of duplicates). We need to update picker with the fulllist.
|
||||
//
|
||||
// Now with cache, even if SubConn was newed/removed, there might be no
|
||||
// state changes.
|
||||
lb.regeneratePicker()
|
||||
lb.cc.UpdateBalancerState(lb.state, lb.picker)
|
||||
}
|
||||
|
||||
// refreshSubConns creates/removes SubConns with backendAddrs. It returns a bool
|
||||
// indicating whether the backendAddrs are different from the cached
|
||||
// backendAddrs (whether any SubConn was newed/removed).
|
||||
// Caller must hold lb.mu.
|
||||
func (lb *lbBalancer) refreshSubConns(backendAddrs []resolver.Address) bool {
|
||||
lb.backendAddrs = nil
|
||||
var backendsUpdated bool
|
||||
// addrsSet is the set converted from backendAddrs, it's used to quick
|
||||
// lookup for an address.
|
||||
addrsSet := make(map[resolver.Address]struct{})
|
||||
// Create new SubConns.
|
||||
for _, addr := range backendAddrs {
|
||||
addrWithoutMD := addr
|
||||
addrWithoutMD.Metadata = nil
|
||||
addrsSet[addrWithoutMD] = struct{}{}
|
||||
lb.backendAddrs = append(lb.backendAddrs, addrWithoutMD)
|
||||
|
||||
if _, ok := lb.subConns[addrWithoutMD]; !ok {
|
||||
backendsUpdated = true
|
||||
|
||||
// Use addrWithMD to create the SubConn.
|
||||
sc, err := lb.cc.NewSubConn([]resolver.Address{addr}, balancer.NewSubConnOptions{})
|
||||
if err != nil {
|
||||
grpclog.Warningf("roundrobinBalancer: failed to create new SubConn: %v", err)
|
||||
continue
|
||||
}
|
||||
lb.subConns[addrWithoutMD] = sc // Use the addr without MD as key for the map.
|
||||
if _, ok := lb.scStates[sc]; !ok {
|
||||
// Only set state of new sc to IDLE. The state could already be
|
||||
// READY for cached SubConns.
|
||||
lb.scStates[sc] = connectivity.Idle
|
||||
}
|
||||
sc.Connect()
|
||||
}
|
||||
}
|
||||
|
||||
for a, sc := range lb.subConns {
|
||||
// a was removed by resolver.
|
||||
if _, ok := addrsSet[a]; !ok {
|
||||
backendsUpdated = true
|
||||
|
||||
lb.cc.RemoveSubConn(sc)
|
||||
delete(lb.subConns, a)
|
||||
// Keep the state of this sc in b.scStates until sc's state becomes Shutdown.
|
||||
// The entry will be deleted in HandleSubConnStateChange.
|
||||
}
|
||||
}
|
||||
|
||||
return backendsUpdated
|
||||
}
|
||||
|
||||
func (lb *lbBalancer) readServerList(s *balanceLoadClientStream) error {
|
||||
for {
|
||||
reply, err := s.Recv()
|
||||
if err != nil {
|
||||
return fmt.Errorf("grpclb: failed to recv server list: %v", err)
|
||||
}
|
||||
if serverList := reply.GetServerList(); serverList != nil {
|
||||
lb.processServerList(serverList)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (lb *lbBalancer) sendLoadReport(s *balanceLoadClientStream, interval time.Duration) {
|
||||
ticker := time.NewTicker(interval)
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
case <-s.Context().Done():
|
||||
return
|
||||
}
|
||||
stats := lb.clientStats.toClientStats()
|
||||
t := time.Now()
|
||||
stats.Timestamp = &lbpb.Timestamp{
|
||||
Seconds: t.Unix(),
|
||||
Nanos: int32(t.Nanosecond()),
|
||||
}
|
||||
if err := s.Send(&lbpb.LoadBalanceRequest{
|
||||
LoadBalanceRequestType: &lbpb.LoadBalanceRequest_ClientStats{
|
||||
ClientStats: stats,
|
||||
},
|
||||
}); err != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (lb *lbBalancer) callRemoteBalancer() error {
|
||||
lbClient := &loadBalancerClient{cc: lb.ccRemoteLB}
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
stream, err := lbClient.BalanceLoad(ctx, FailFast(false))
|
||||
if err != nil {
|
||||
return fmt.Errorf("grpclb: failed to perform RPC to the remote balancer %v", err)
|
||||
}
|
||||
|
||||
// grpclb handshake on the stream.
|
||||
initReq := &lbpb.LoadBalanceRequest{
|
||||
LoadBalanceRequestType: &lbpb.LoadBalanceRequest_InitialRequest{
|
||||
InitialRequest: &lbpb.InitialLoadBalanceRequest{
|
||||
Name: lb.target,
|
||||
},
|
||||
},
|
||||
}
|
||||
if err := stream.Send(initReq); err != nil {
|
||||
return fmt.Errorf("grpclb: failed to send init request: %v", err)
|
||||
}
|
||||
reply, err := stream.Recv()
|
||||
if err != nil {
|
||||
return fmt.Errorf("grpclb: failed to recv init response: %v", err)
|
||||
}
|
||||
initResp := reply.GetInitialResponse()
|
||||
if initResp == nil {
|
||||
return fmt.Errorf("grpclb: reply from remote balancer did not include initial response")
|
||||
}
|
||||
if initResp.LoadBalancerDelegate != "" {
|
||||
return fmt.Errorf("grpclb: Delegation is not supported")
|
||||
}
|
||||
|
||||
go func() {
|
||||
if d := convertDuration(initResp.ClientStatsReportInterval); d > 0 {
|
||||
lb.sendLoadReport(stream, d)
|
||||
}
|
||||
}()
|
||||
return lb.readServerList(stream)
|
||||
}
|
||||
|
||||
func (lb *lbBalancer) watchRemoteBalancer() {
|
||||
for {
|
||||
err := lb.callRemoteBalancer()
|
||||
select {
|
||||
case <-lb.doneCh:
|
||||
return
|
||||
default:
|
||||
if err != nil {
|
||||
grpclog.Error(err)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
func (lb *lbBalancer) dialRemoteLB(remoteLBName string) {
|
||||
var dopts []DialOption
|
||||
if creds := lb.opt.DialCreds; creds != nil {
|
||||
if err := creds.OverrideServerName(remoteLBName); err == nil {
|
||||
dopts = append(dopts, WithTransportCredentials(creds))
|
||||
} else {
|
||||
grpclog.Warningf("grpclb: failed to override the server name in the credentials: %v, using Insecure", err)
|
||||
dopts = append(dopts, WithInsecure())
|
||||
}
|
||||
} else {
|
||||
dopts = append(dopts, WithInsecure())
|
||||
}
|
||||
if lb.opt.Dialer != nil {
|
||||
// WithDialer takes a different type of function, so we instead use a
|
||||
// special DialOption here.
|
||||
dopts = append(dopts, withContextDialer(lb.opt.Dialer))
|
||||
}
|
||||
// Explicitly set pickfirst as the balancer.
|
||||
dopts = append(dopts, WithBalancerName(PickFirstBalancerName))
|
||||
dopts = append(dopts, withResolverBuilder(lb.manualResolver))
|
||||
if channelz.IsOn() {
|
||||
dopts = append(dopts, WithChannelzParentID(lb.opt.ChannelzParentID))
|
||||
}
|
||||
|
||||
// DialContext using manualResolver.Scheme, which is a random scheme generated
|
||||
// when init grpclb. The target name is not important.
|
||||
cc, err := DialContext(context.Background(), "grpclb:///grpclb.server", dopts...)
|
||||
if err != nil {
|
||||
grpclog.Fatalf("failed to dial: %v", err)
|
||||
}
|
||||
lb.ccRemoteLB = cc
|
||||
go lb.watchRemoteBalancer()
|
||||
}
|
214
vendor/google.golang.org/grpc/grpclb_util.go
generated
vendored
Normal file
@ -0,0 +1,214 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2016 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 grpc
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
// The parent ClientConn should re-resolve when grpclb loses connection to the
|
||||
// remote balancer. When the ClientConn inside grpclb gets a TransientFailure,
|
||||
// it calls lbManualResolver.ResolveNow(), which calls parent ClientConn's
|
||||
// ResolveNow, and eventually results in re-resolve happening in parent
|
||||
// ClientConn's resolver (DNS for example).
|
||||
//
|
||||
// parent
|
||||
// ClientConn
|
||||
// +-----------------------------------------------------------------+
|
||||
// | parent +---------------------------------+ |
|
||||
// | DNS ClientConn | grpclb | |
|
||||
// | resolver balancerWrapper | | |
|
||||
// | + + | grpclb grpclb | |
|
||||
// | | | | ManualResolver ClientConn | |
|
||||
// | | | | + + | |
|
||||
// | | | | | | Transient | |
|
||||
// | | | | | | Failure | |
|
||||
// | | | | | <--------- | | |
|
||||
// | | | <--------------- | ResolveNow | | |
|
||||
// | | <--------- | ResolveNow | | | | |
|
||||
// | | ResolveNow | | | | | |
|
||||
// | | | | | | | |
|
||||
// | + + | + + | |
|
||||
// | +---------------------------------+ |
|
||||
// +-----------------------------------------------------------------+
|
||||
|
||||
// lbManualResolver is used by the ClientConn inside grpclb. It's a manual
|
||||
// resolver with a special ResolveNow() function.
|
||||
//
|
||||
// When ResolveNow() is called, it calls ResolveNow() on the parent ClientConn,
|
||||
// so when grpclb client lose contact with remote balancers, the parent
|
||||
// ClientConn's resolver will re-resolve.
|
||||
type lbManualResolver struct {
|
||||
scheme string
|
||||
ccr resolver.ClientConn
|
||||
|
||||
ccb balancer.ClientConn
|
||||
}
|
||||
|
||||
func (r *lbManualResolver) Build(_ resolver.Target, cc resolver.ClientConn, _ resolver.BuildOption) (resolver.Resolver, error) {
|
||||
r.ccr = cc
|
||||
return r, nil
|
||||
}
|
||||
|
||||
func (r *lbManualResolver) Scheme() string {
|
||||
return r.scheme
|
||||
}
|
||||
|
||||
// ResolveNow calls resolveNow on the parent ClientConn.
|
||||
func (r *lbManualResolver) ResolveNow(o resolver.ResolveNowOption) {
|
||||
r.ccb.ResolveNow(o)
|
||||
}
|
||||
|
||||
// Close is a noop for Resolver.
|
||||
func (*lbManualResolver) Close() {}
|
||||
|
||||
// NewAddress calls cc.NewAddress.
|
||||
func (r *lbManualResolver) NewAddress(addrs []resolver.Address) {
|
||||
r.ccr.NewAddress(addrs)
|
||||
}
|
||||
|
||||
// NewServiceConfig calls cc.NewServiceConfig.
|
||||
func (r *lbManualResolver) NewServiceConfig(sc string) {
|
||||
r.ccr.NewServiceConfig(sc)
|
||||
}
|
||||
|
||||
const subConnCacheTime = time.Second * 10
|
||||
|
||||
// lbCacheClientConn is a wrapper balancer.ClientConn with a SubConn cache.
|
||||
// SubConns will be kept in cache for subConnCacheTime before being removed.
|
||||
//
|
||||
// Its new and remove methods are updated to do cache first.
|
||||
type lbCacheClientConn struct {
|
||||
cc balancer.ClientConn
|
||||
timeout time.Duration
|
||||
|
||||
mu sync.Mutex
|
||||
// subConnCache only keeps subConns that are being deleted.
|
||||
subConnCache map[resolver.Address]*subConnCacheEntry
|
||||
subConnToAddr map[balancer.SubConn]resolver.Address
|
||||
}
|
||||
|
||||
type subConnCacheEntry struct {
|
||||
sc balancer.SubConn
|
||||
|
||||
cancel func()
|
||||
abortDeleting bool
|
||||
}
|
||||
|
||||
func newLBCacheClientConn(cc balancer.ClientConn) *lbCacheClientConn {
|
||||
return &lbCacheClientConn{
|
||||
cc: cc,
|
||||
timeout: subConnCacheTime,
|
||||
subConnCache: make(map[resolver.Address]*subConnCacheEntry),
|
||||
subConnToAddr: make(map[balancer.SubConn]resolver.Address),
|
||||
}
|
||||
}
|
||||
|
||||
func (ccc *lbCacheClientConn) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) {
|
||||
if len(addrs) != 1 {
|
||||
return nil, fmt.Errorf("grpclb calling NewSubConn with addrs of length %v", len(addrs))
|
||||
}
|
||||
addrWithoutMD := addrs[0]
|
||||
addrWithoutMD.Metadata = nil
|
||||
|
||||
ccc.mu.Lock()
|
||||
defer ccc.mu.Unlock()
|
||||
if entry, ok := ccc.subConnCache[addrWithoutMD]; ok {
|
||||
// If entry is in subConnCache, the SubConn was being deleted.
|
||||
// cancel function will never be nil.
|
||||
entry.cancel()
|
||||
delete(ccc.subConnCache, addrWithoutMD)
|
||||
return entry.sc, nil
|
||||
}
|
||||
|
||||
scNew, err := ccc.cc.NewSubConn(addrs, opts)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ccc.subConnToAddr[scNew] = addrWithoutMD
|
||||
return scNew, nil
|
||||
}
|
||||
|
||||
func (ccc *lbCacheClientConn) RemoveSubConn(sc balancer.SubConn) {
|
||||
ccc.mu.Lock()
|
||||
defer ccc.mu.Unlock()
|
||||
addr, ok := ccc.subConnToAddr[sc]
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
if entry, ok := ccc.subConnCache[addr]; ok {
|
||||
if entry.sc != sc {
|
||||
// This could happen if NewSubConn was called multiple times for the
|
||||
// same address, and those SubConns are all removed. We remove sc
|
||||
// immediately here.
|
||||
delete(ccc.subConnToAddr, sc)
|
||||
ccc.cc.RemoveSubConn(sc)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
entry := &subConnCacheEntry{
|
||||
sc: sc,
|
||||
}
|
||||
ccc.subConnCache[addr] = entry
|
||||
|
||||
timer := time.AfterFunc(ccc.timeout, func() {
|
||||
ccc.mu.Lock()
|
||||
if entry.abortDeleting {
|
||||
return
|
||||
}
|
||||
ccc.cc.RemoveSubConn(sc)
|
||||
delete(ccc.subConnToAddr, sc)
|
||||
delete(ccc.subConnCache, addr)
|
||||
ccc.mu.Unlock()
|
||||
})
|
||||
entry.cancel = func() {
|
||||
if !timer.Stop() {
|
||||
// If stop was not successful, the timer has fired (this can only
|
||||
// happen in a race). But the deleting function is blocked on ccc.mu
|
||||
// because the mutex was held by the caller of this function.
|
||||
//
|
||||
// Set abortDeleting to true to abort the deleting function. When
|
||||
// the lock is released, the deleting function will acquire the
|
||||
// lock, check the value of abortDeleting and return.
|
||||
entry.abortDeleting = true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ccc *lbCacheClientConn) UpdateBalancerState(s connectivity.State, p balancer.Picker) {
|
||||
ccc.cc.UpdateBalancerState(s, p)
|
||||
}
|
||||
|
||||
func (ccc *lbCacheClientConn) close() {
|
||||
ccc.mu.Lock()
|
||||
// Only cancel all existing timers. There's no need to remove SubConns.
|
||||
for _, entry := range ccc.subConnCache {
|
||||
entry.cancel()
|
||||
}
|
||||
ccc.mu.Unlock()
|
||||
}
|
3
vendor/google.golang.org/grpc/grpclog/grpclog.go
generated
vendored
@ -105,18 +105,21 @@ func Fatalln(args ...interface{}) {
|
||||
}
|
||||
|
||||
// Print prints to the logger. Arguments are handled in the manner of fmt.Print.
|
||||
//
|
||||
// Deprecated: use Info.
|
||||
func Print(args ...interface{}) {
|
||||
logger.Info(args...)
|
||||
}
|
||||
|
||||
// Printf prints to the logger. Arguments are handled in the manner of fmt.Printf.
|
||||
//
|
||||
// Deprecated: use Infof.
|
||||
func Printf(format string, args ...interface{}) {
|
||||
logger.Infof(format, args...)
|
||||
}
|
||||
|
||||
// Println prints to the logger. Arguments are handled in the manner of fmt.Println.
|
||||
//
|
||||
// Deprecated: use Infoln.
|
||||
func Println(args ...interface{}) {
|
||||
logger.Infoln(args...)
|
||||
|
2
vendor/google.golang.org/grpc/grpclog/logger.go
generated
vendored
@ -19,6 +19,7 @@
|
||||
package grpclog
|
||||
|
||||
// Logger mimics golang's standard Logger as an interface.
|
||||
//
|
||||
// Deprecated: use LoggerV2.
|
||||
type Logger interface {
|
||||
Fatal(args ...interface{})
|
||||
@ -31,6 +32,7 @@ type Logger interface {
|
||||
|
||||
// SetLogger sets the logger that is used in grpc. Call only from
|
||||
// init() functions.
|
||||
//
|
||||
// Deprecated: use SetLoggerV2.
|
||||
func SetLogger(l Logger) {
|
||||
logger = &loggerWrapper{Logger: l}
|
||||
|
91
vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go
generated
vendored
@ -1,17 +1,7 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// source: grpc_health_v1/health.proto
|
||||
|
||||
/*
|
||||
Package grpc_health_v1 is a generated protocol buffer package.
|
||||
|
||||
It is generated from these files:
|
||||
grpc_health_v1/health.proto
|
||||
|
||||
It has these top-level messages:
|
||||
HealthCheckRequest
|
||||
HealthCheckResponse
|
||||
*/
|
||||
package grpc_health_v1
|
||||
package grpc_health_v1 // import "google.golang.org/grpc/health/grpc_health_v1"
|
||||
|
||||
import proto "github.com/golang/protobuf/proto"
|
||||
import fmt "fmt"
|
||||
@ -56,17 +46,39 @@ func (x HealthCheckResponse_ServingStatus) String() string {
|
||||
return proto.EnumName(HealthCheckResponse_ServingStatus_name, int32(x))
|
||||
}
|
||||
func (HealthCheckResponse_ServingStatus) EnumDescriptor() ([]byte, []int) {
|
||||
return fileDescriptor0, []int{1, 0}
|
||||
return fileDescriptor_health_8e5b8a3074428511, []int{1, 0}
|
||||
}
|
||||
|
||||
type HealthCheckRequest struct {
|
||||
Service string `protobuf:"bytes,1,opt,name=service" json:"service,omitempty"`
|
||||
Service string `protobuf:"bytes,1,opt,name=service" json:"service,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *HealthCheckRequest) Reset() { *m = HealthCheckRequest{} }
|
||||
func (m *HealthCheckRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*HealthCheckRequest) ProtoMessage() {}
|
||||
func (*HealthCheckRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
|
||||
func (m *HealthCheckRequest) Reset() { *m = HealthCheckRequest{} }
|
||||
func (m *HealthCheckRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*HealthCheckRequest) ProtoMessage() {}
|
||||
func (*HealthCheckRequest) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_health_8e5b8a3074428511, []int{0}
|
||||
}
|
||||
func (m *HealthCheckRequest) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_HealthCheckRequest.Unmarshal(m, b)
|
||||
}
|
||||
func (m *HealthCheckRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_HealthCheckRequest.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *HealthCheckRequest) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_HealthCheckRequest.Merge(dst, src)
|
||||
}
|
||||
func (m *HealthCheckRequest) XXX_Size() int {
|
||||
return xxx_messageInfo_HealthCheckRequest.Size(m)
|
||||
}
|
||||
func (m *HealthCheckRequest) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_HealthCheckRequest.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_HealthCheckRequest proto.InternalMessageInfo
|
||||
|
||||
func (m *HealthCheckRequest) GetService() string {
|
||||
if m != nil {
|
||||
@ -76,13 +88,35 @@ func (m *HealthCheckRequest) GetService() string {
|
||||
}
|
||||
|
||||
type HealthCheckResponse struct {
|
||||
Status HealthCheckResponse_ServingStatus `protobuf:"varint,1,opt,name=status,enum=grpc.health.v1.HealthCheckResponse_ServingStatus" json:"status,omitempty"`
|
||||
Status HealthCheckResponse_ServingStatus `protobuf:"varint,1,opt,name=status,enum=grpc.health.v1.HealthCheckResponse_ServingStatus" json:"status,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *HealthCheckResponse) Reset() { *m = HealthCheckResponse{} }
|
||||
func (m *HealthCheckResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*HealthCheckResponse) ProtoMessage() {}
|
||||
func (*HealthCheckResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
|
||||
func (m *HealthCheckResponse) Reset() { *m = HealthCheckResponse{} }
|
||||
func (m *HealthCheckResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*HealthCheckResponse) ProtoMessage() {}
|
||||
func (*HealthCheckResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_health_8e5b8a3074428511, []int{1}
|
||||
}
|
||||
func (m *HealthCheckResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_HealthCheckResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *HealthCheckResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_HealthCheckResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (dst *HealthCheckResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_HealthCheckResponse.Merge(dst, src)
|
||||
}
|
||||
func (m *HealthCheckResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_HealthCheckResponse.Size(m)
|
||||
}
|
||||
func (m *HealthCheckResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_HealthCheckResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_HealthCheckResponse proto.InternalMessageInfo
|
||||
|
||||
func (m *HealthCheckResponse) GetStatus() HealthCheckResponse_ServingStatus {
|
||||
if m != nil {
|
||||
@ -169,10 +203,10 @@ var _Health_serviceDesc = grpc.ServiceDesc{
|
||||
Metadata: "grpc_health_v1/health.proto",
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("grpc_health_v1/health.proto", fileDescriptor0) }
|
||||
func init() { proto.RegisterFile("grpc_health_v1/health.proto", fileDescriptor_health_8e5b8a3074428511) }
|
||||
|
||||
var fileDescriptor0 = []byte{
|
||||
// 213 bytes of a gzipped FileDescriptorProto
|
||||
var fileDescriptor_health_8e5b8a3074428511 = []byte{
|
||||
// 269 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4e, 0x2f, 0x2a, 0x48,
|
||||
0x8e, 0xcf, 0x48, 0x4d, 0xcc, 0x29, 0xc9, 0x88, 0x2f, 0x33, 0xd4, 0x87, 0xb0, 0xf4, 0x0a, 0x8a,
|
||||
0xf2, 0x4b, 0xf2, 0x85, 0xf8, 0x40, 0x92, 0x7a, 0x50, 0xa1, 0x32, 0x43, 0x25, 0x3d, 0x2e, 0x21,
|
||||
@ -185,6 +219,9 @@ var fileDescriptor0 = []byte{
|
||||
0x0f, 0xf7, 0x13, 0x60, 0x00, 0x71, 0x82, 0x5d, 0x83, 0xc2, 0x3c, 0xfd, 0xdc, 0x05, 0x18, 0x85,
|
||||
0xf8, 0xb9, 0xb8, 0xfd, 0xfc, 0x43, 0xe2, 0x61, 0x02, 0x4c, 0x46, 0x51, 0x5c, 0x6c, 0x10, 0x8b,
|
||||
0x84, 0x02, 0xb8, 0x58, 0xc1, 0x96, 0x09, 0x29, 0xe1, 0x75, 0x09, 0xd8, 0xbf, 0x52, 0xca, 0x44,
|
||||
0xb8, 0x36, 0x89, 0x0d, 0x1c, 0x82, 0xc6, 0x80, 0x00, 0x00, 0x00, 0xff, 0xff, 0x53, 0x2b, 0x65,
|
||||
0x20, 0x60, 0x01, 0x00, 0x00,
|
||||
0xb8, 0xd6, 0x29, 0x91, 0x4b, 0x30, 0x33, 0x1f, 0x4d, 0xa1, 0x13, 0x37, 0x44, 0x65, 0x00, 0x28,
|
||||
0x70, 0x03, 0x18, 0xa3, 0x74, 0xd2, 0xf3, 0xf3, 0xd3, 0x73, 0x52, 0xf5, 0xd2, 0xf3, 0x73, 0x12,
|
||||
0xf3, 0xd2, 0xf5, 0xf2, 0x8b, 0xd2, 0xf5, 0x41, 0x1a, 0xa0, 0x71, 0xa0, 0x8f, 0x1a, 0x33, 0xab,
|
||||
0x98, 0xf8, 0xdc, 0x41, 0xa6, 0x41, 0x8c, 0xd0, 0x0b, 0x33, 0x4c, 0x62, 0x03, 0x47, 0x92, 0x31,
|
||||
0x20, 0x00, 0x00, 0xff, 0xff, 0xb7, 0x70, 0xc4, 0xa7, 0xc3, 0x01, 0x00, 0x00,
|
||||
}
|
||||
|
6
vendor/google.golang.org/grpc/health/health.go
generated
vendored
@ -16,7 +16,7 @@
|
||||
*
|
||||
*/
|
||||
|
||||
//go:generate protoc --go_out=plugins=grpc:. grpc_health_v1/health.proto
|
||||
//go:generate protoc --go_out=plugins=grpc,paths=source_relative:. grpc_health_v1/health.proto
|
||||
|
||||
// Package health provides some utility functions to health-check a server. The implementation
|
||||
// is based on protobuf. Users need to write their own implementations if other IDLs are used.
|
||||
@ -26,9 +26,9 @@ import (
|
||||
"sync"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
healthpb "google.golang.org/grpc/health/grpc_health_v1"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
// Server implements `service Health`.
|
||||
@ -60,7 +60,7 @@ func (s *Server) Check(ctx context.Context, in *healthpb.HealthCheckRequest) (*h
|
||||
Status: status,
|
||||
}, nil
|
||||
}
|
||||
return nil, grpc.Errorf(codes.NotFound, "unknown service")
|
||||
return nil, status.Error(codes.NotFound, "unknown service")
|
||||
}
|
||||
|
||||
// SetServingStatus is called when need to reset the serving status of a service
|
||||
|
4
vendor/google.golang.org/grpc/interceptor.go
generated
vendored
@ -48,7 +48,9 @@ type UnaryServerInfo struct {
|
||||
}
|
||||
|
||||
// UnaryHandler defines the handler invoked by UnaryServerInterceptor to complete the normal
|
||||
// execution of a unary RPC.
|
||||
// execution of a unary RPC. If a UnaryHandler returns an error, it should be produced by the
|
||||
// status package, or else gRPC will use codes.Unknown as the status code and err.Error() as
|
||||
// the status message of the RPC.
|
||||
type UnaryHandler func(ctx context.Context, req interface{}) (interface{}, error)
|
||||
|
||||
// UnaryServerInterceptor provides a hook to intercept the execution of a unary RPC on the server. info
|
||||
|
7
vendor/google.golang.org/grpc/internal/internal.go
generated
vendored
@ -19,13 +19,6 @@
|
||||
// the godoc of the top-level grpc package.
|
||||
package internal
|
||||
|
||||
// TestingCloseConns closes all existing transports but keeps
|
||||
// grpcServer.lis accepting new connections.
|
||||
//
|
||||
// The provided grpcServer must be of type *grpc.Server. It is untyped
|
||||
// for circular dependency reasons.
|
||||
var TestingCloseConns func(grpcServer interface{})
|
||||
|
||||
// TestingUseHandlerImpl enables the http.Handler-based server implementation.
|
||||
// It must be called before Serve and requires TLS credentials.
|
||||
//
|
||||
|
89
vendor/google.golang.org/grpc/metadata/metadata.go
generated
vendored
@ -17,7 +17,8 @@
|
||||
*/
|
||||
|
||||
// Package metadata define the structure of the metadata supported by gRPC library.
|
||||
// Please refer to https://grpc.io/docs/guides/wire.html for more information about custom-metadata.
|
||||
// Please refer to https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md
|
||||
// for more information about custom-metadata.
|
||||
package metadata // import "google.golang.org/grpc/metadata"
|
||||
|
||||
import (
|
||||
@ -27,7 +28,9 @@ import (
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
// DecodeKeyValue returns k, v, nil. It is deprecated and should not be used.
|
||||
// DecodeKeyValue returns k, v, nil.
|
||||
//
|
||||
// Deprecated: use k and v directly instead.
|
||||
func DecodeKeyValue(k, v string) (string, string, error) {
|
||||
return k, v, nil
|
||||
}
|
||||
@ -94,6 +97,30 @@ func (md MD) Copy() MD {
|
||||
return Join(md)
|
||||
}
|
||||
|
||||
// Get obtains the values for a given key.
|
||||
func (md MD) Get(k string) []string {
|
||||
k = strings.ToLower(k)
|
||||
return md[k]
|
||||
}
|
||||
|
||||
// Set sets the value of a given key with a slice of values.
|
||||
func (md MD) Set(k string, vals ...string) {
|
||||
if len(vals) == 0 {
|
||||
return
|
||||
}
|
||||
k = strings.ToLower(k)
|
||||
md[k] = vals
|
||||
}
|
||||
|
||||
// Append adds the values to key k, not overwriting what was already stored at that key.
|
||||
func (md MD) Append(k string, vals ...string) {
|
||||
if len(vals) == 0 {
|
||||
return
|
||||
}
|
||||
k = strings.ToLower(k)
|
||||
md[k] = append(md[k], vals...)
|
||||
}
|
||||
|
||||
// Join joins any number of mds into a single MD.
|
||||
// The order of values for each key is determined by the order in which
|
||||
// the mds containing those values are presented to Join.
|
||||
@ -115,9 +142,26 @@ func NewIncomingContext(ctx context.Context, md MD) context.Context {
|
||||
return context.WithValue(ctx, mdIncomingKey{}, md)
|
||||
}
|
||||
|
||||
// NewOutgoingContext creates a new context with outgoing md attached.
|
||||
// NewOutgoingContext creates a new context with outgoing md attached. If used
|
||||
// in conjunction with AppendToOutgoingContext, NewOutgoingContext will
|
||||
// overwrite any previously-appended metadata.
|
||||
func NewOutgoingContext(ctx context.Context, md MD) context.Context {
|
||||
return context.WithValue(ctx, mdOutgoingKey{}, md)
|
||||
return context.WithValue(ctx, mdOutgoingKey{}, rawMD{md: md})
|
||||
}
|
||||
|
||||
// AppendToOutgoingContext returns a new context with the provided kv merged
|
||||
// with any existing metadata in the context. Please refer to the
|
||||
// documentation of Pairs for a description of kv.
|
||||
func AppendToOutgoingContext(ctx context.Context, kv ...string) context.Context {
|
||||
if len(kv)%2 == 1 {
|
||||
panic(fmt.Sprintf("metadata: AppendToOutgoingContext got an odd number of input pairs for metadata: %d", len(kv)))
|
||||
}
|
||||
md, _ := ctx.Value(mdOutgoingKey{}).(rawMD)
|
||||
added := make([][]string, len(md.added)+1)
|
||||
copy(added, md.added)
|
||||
added[len(added)-1] = make([]string, len(kv))
|
||||
copy(added[len(added)-1], kv)
|
||||
return context.WithValue(ctx, mdOutgoingKey{}, rawMD{md: md.md, added: added})
|
||||
}
|
||||
|
||||
// FromIncomingContext returns the incoming metadata in ctx if it exists. The
|
||||
@ -128,10 +172,39 @@ func FromIncomingContext(ctx context.Context) (md MD, ok bool) {
|
||||
return
|
||||
}
|
||||
|
||||
// FromOutgoingContextRaw returns the un-merged, intermediary contents
|
||||
// of rawMD. Remember to perform strings.ToLower on the keys. The returned
|
||||
// MD should not be modified. Writing to it may cause races. Modification
|
||||
// should be made to copies of the returned MD.
|
||||
//
|
||||
// This is intended for gRPC-internal use ONLY.
|
||||
func FromOutgoingContextRaw(ctx context.Context) (MD, [][]string, bool) {
|
||||
raw, ok := ctx.Value(mdOutgoingKey{}).(rawMD)
|
||||
if !ok {
|
||||
return nil, nil, false
|
||||
}
|
||||
|
||||
return raw.md, raw.added, true
|
||||
}
|
||||
|
||||
// FromOutgoingContext returns the outgoing metadata in ctx if it exists. The
|
||||
// returned MD should not be modified. Writing to it may cause races.
|
||||
// Modification should be made to the copies of the returned MD.
|
||||
func FromOutgoingContext(ctx context.Context) (md MD, ok bool) {
|
||||
md, ok = ctx.Value(mdOutgoingKey{}).(MD)
|
||||
return
|
||||
// Modification should be made to copies of the returned MD.
|
||||
func FromOutgoingContext(ctx context.Context) (MD, bool) {
|
||||
raw, ok := ctx.Value(mdOutgoingKey{}).(rawMD)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
|
||||
mds := make([]MD, 0, len(raw.added)+1)
|
||||
mds = append(mds, raw.md)
|
||||
for _, vv := range raw.added {
|
||||
mds = append(mds, Pairs(vv...))
|
||||
}
|
||||
return Join(mds...), ok
|
||||
}
|
||||
|
||||
type rawMD struct {
|
||||
md MD
|
||||
added [][]string
|
||||
}
|
||||
|
6
vendor/google.golang.org/grpc/naming/dns_resolver.go
generated
vendored
@ -153,10 +153,10 @@ type ipWatcher struct {
|
||||
updateChan chan *Update
|
||||
}
|
||||
|
||||
// Next returns the adrress resolution Update for the target. For IP address,
|
||||
// the resolution is itself, thus polling name server is unncessary. Therefore,
|
||||
// Next returns the address resolution Update for the target. For IP address,
|
||||
// the resolution is itself, thus polling name server is unnecessary. Therefore,
|
||||
// Next() will return an Update the first time it is called, and will be blocked
|
||||
// for all following calls as no Update exisits until watcher is closed.
|
||||
// for all following calls as no Update exists until watcher is closed.
|
||||
func (i *ipWatcher) Next() ([]*Update, error) {
|
||||
u, ok := <-i.updateChan
|
||||
if !ok {
|
||||
|
2
vendor/google.golang.org/grpc/naming/go17.go
generated
vendored
@ -1,4 +1,4 @@
|
||||
// +build go1.7, !go1.8
|
||||
// +build go1.6,!go1.8
|
||||
|
||||
/*
|
||||
*
|
||||
|
12
vendor/google.golang.org/grpc/naming/naming.go
generated
vendored
@ -18,20 +18,26 @@
|
||||
|
||||
// Package naming defines the naming API and related data structures for gRPC.
|
||||
// The interface is EXPERIMENTAL and may be suject to change.
|
||||
//
|
||||
// Deprecated: please use package resolver.
|
||||
package naming
|
||||
|
||||
// Operation defines the corresponding operations for a name resolution change.
|
||||
//
|
||||
// Deprecated: please use package resolver.
|
||||
type Operation uint8
|
||||
|
||||
const (
|
||||
// Add indicates a new address is added.
|
||||
Add Operation = iota
|
||||
// Delete indicates an exisiting address is deleted.
|
||||
// Delete indicates an existing address is deleted.
|
||||
Delete
|
||||
)
|
||||
|
||||
// Update defines a name resolution update. Notice that it is not valid having both
|
||||
// empty string Addr and nil Metadata in an Update.
|
||||
//
|
||||
// Deprecated: please use package resolver.
|
||||
type Update struct {
|
||||
// Op indicates the operation of the update.
|
||||
Op Operation
|
||||
@ -43,12 +49,16 @@ type Update struct {
|
||||
}
|
||||
|
||||
// Resolver creates a Watcher for a target to track its resolution changes.
|
||||
//
|
||||
// Deprecated: please use package resolver.
|
||||
type Resolver interface {
|
||||
// Resolve creates a Watcher for target.
|
||||
Resolve(target string) (Watcher, error)
|
||||
}
|
||||
|
||||
// Watcher watches for the updates on the specified target.
|
||||
//
|
||||
// Deprecated: please use package resolver.
|
||||
type Watcher interface {
|
||||
// Next blocks until an update or error happens. It may return one or more
|
||||
// updates. The first call should get the full set of the results. It should
|
||||
|