mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #8840 from gyuho/health-balancer
*: refactor clientv3 balancer, upgrade gRPC to v1.7.2
This commit is contained in:
commit
b64c1bfce6
@ -1,439 +0,0 @@
|
||||
// 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.
|
||||
|
||||
package clientv3
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net/url"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
// ErrNoAddrAvilable is returned by Get() when the balancer does not have
|
||||
// any active connection to endpoints at the time.
|
||||
// This error is returned only when opts.BlockingWait is true.
|
||||
var ErrNoAddrAvilable = status.Error(codes.Unavailable, "there is no address available")
|
||||
|
||||
type notifyMsg int
|
||||
|
||||
const (
|
||||
notifyReset notifyMsg = iota
|
||||
notifyNext
|
||||
)
|
||||
|
||||
// simpleBalancer does the bare minimum to expose multiple eps
|
||||
// to the grpc reconnection code path
|
||||
type simpleBalancer struct {
|
||||
// addrs are the client's endpoint addresses for grpc
|
||||
addrs []grpc.Address
|
||||
|
||||
// eps holds the raw endpoints from the client
|
||||
eps []string
|
||||
|
||||
// notifyCh notifies grpc of the set of addresses for connecting
|
||||
notifyCh chan []grpc.Address
|
||||
|
||||
// readyc closes once the first connection is up
|
||||
readyc chan struct{}
|
||||
readyOnce sync.Once
|
||||
|
||||
// mu protects all fields below.
|
||||
mu sync.RWMutex
|
||||
|
||||
// upc closes when pinAddr transitions from empty to non-empty or the balancer closes.
|
||||
upc chan struct{}
|
||||
|
||||
// downc closes when grpc calls down() on pinAddr
|
||||
downc chan struct{}
|
||||
|
||||
// stopc is closed to signal updateNotifyLoop should stop.
|
||||
stopc chan struct{}
|
||||
|
||||
// donec closes when all goroutines are exited
|
||||
donec chan struct{}
|
||||
|
||||
// updateAddrsC notifies updateNotifyLoop to update addrs.
|
||||
updateAddrsC chan notifyMsg
|
||||
|
||||
// grpc issues TLS cert checks using the string passed into dial so
|
||||
// that string must be the host. To recover the full scheme://host URL,
|
||||
// have a map from hosts to the original endpoint.
|
||||
hostPort2ep map[string]string
|
||||
|
||||
// pinAddr is the currently pinned address; set to the empty string on
|
||||
// initialization and shutdown.
|
||||
pinAddr string
|
||||
|
||||
closed bool
|
||||
}
|
||||
|
||||
func newSimpleBalancer(eps []string) *simpleBalancer {
|
||||
notifyCh := make(chan []grpc.Address)
|
||||
addrs := eps2addrs(eps)
|
||||
sb := &simpleBalancer{
|
||||
addrs: addrs,
|
||||
eps: eps,
|
||||
notifyCh: notifyCh,
|
||||
readyc: make(chan struct{}),
|
||||
upc: make(chan struct{}),
|
||||
stopc: make(chan struct{}),
|
||||
downc: make(chan struct{}),
|
||||
donec: make(chan struct{}),
|
||||
updateAddrsC: make(chan notifyMsg),
|
||||
hostPort2ep: getHostPort2ep(eps),
|
||||
}
|
||||
close(sb.downc)
|
||||
go sb.updateNotifyLoop()
|
||||
return sb
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) Start(target string, config grpc.BalancerConfig) error { return nil }
|
||||
|
||||
func (b *simpleBalancer) ConnectNotify() <-chan struct{} {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
return b.upc
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) ready() <-chan struct{} { return b.readyc }
|
||||
|
||||
func (b *simpleBalancer) endpoint(hostPort string) string {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
return b.hostPort2ep[hostPort]
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) endpoints() []string {
|
||||
b.mu.RLock()
|
||||
defer b.mu.RUnlock()
|
||||
return b.eps
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) pinned() string {
|
||||
b.mu.RLock()
|
||||
defer b.mu.RUnlock()
|
||||
return b.pinAddr
|
||||
}
|
||||
|
||||
func getHostPort2ep(eps []string) map[string]string {
|
||||
hm := make(map[string]string, len(eps))
|
||||
for i := range eps {
|
||||
_, host, _ := parseEndpoint(eps[i])
|
||||
hm[host] = eps[i]
|
||||
}
|
||||
return hm
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) updateAddrs(eps ...string) {
|
||||
np := getHostPort2ep(eps)
|
||||
|
||||
b.mu.Lock()
|
||||
|
||||
match := len(np) == len(b.hostPort2ep)
|
||||
for k, v := range np {
|
||||
if b.hostPort2ep[k] != v {
|
||||
match = false
|
||||
break
|
||||
}
|
||||
}
|
||||
if match {
|
||||
// same endpoints, so no need to update address
|
||||
b.mu.Unlock()
|
||||
return
|
||||
}
|
||||
|
||||
b.hostPort2ep = np
|
||||
b.addrs, b.eps = eps2addrs(eps), eps
|
||||
|
||||
// updating notifyCh can trigger new connections,
|
||||
// only update addrs if all connections are down
|
||||
// or addrs does not include pinAddr.
|
||||
update := !hasAddr(b.addrs, b.pinAddr)
|
||||
b.mu.Unlock()
|
||||
|
||||
if update {
|
||||
select {
|
||||
case b.updateAddrsC <- notifyNext:
|
||||
case <-b.stopc:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) next() {
|
||||
b.mu.RLock()
|
||||
downc := b.downc
|
||||
b.mu.RUnlock()
|
||||
select {
|
||||
case b.updateAddrsC <- notifyNext:
|
||||
case <-b.stopc:
|
||||
}
|
||||
// wait until disconnect so new RPCs are not issued on old connection
|
||||
select {
|
||||
case <-downc:
|
||||
case <-b.stopc:
|
||||
}
|
||||
}
|
||||
|
||||
func hasAddr(addrs []grpc.Address, targetAddr string) bool {
|
||||
for _, addr := range addrs {
|
||||
if targetAddr == addr.Addr {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) updateNotifyLoop() {
|
||||
defer close(b.donec)
|
||||
|
||||
for {
|
||||
b.mu.RLock()
|
||||
upc, downc, addr := b.upc, b.downc, b.pinAddr
|
||||
b.mu.RUnlock()
|
||||
// downc or upc should be closed
|
||||
select {
|
||||
case <-downc:
|
||||
downc = nil
|
||||
default:
|
||||
}
|
||||
select {
|
||||
case <-upc:
|
||||
upc = nil
|
||||
default:
|
||||
}
|
||||
switch {
|
||||
case downc == nil && upc == nil:
|
||||
// stale
|
||||
select {
|
||||
case <-b.stopc:
|
||||
return
|
||||
default:
|
||||
}
|
||||
case downc == nil:
|
||||
b.notifyAddrs(notifyReset)
|
||||
select {
|
||||
case <-upc:
|
||||
case msg := <-b.updateAddrsC:
|
||||
b.notifyAddrs(msg)
|
||||
case <-b.stopc:
|
||||
return
|
||||
}
|
||||
case upc == nil:
|
||||
select {
|
||||
// close connections that are not the pinned address
|
||||
case b.notifyCh <- []grpc.Address{{Addr: addr}}:
|
||||
case <-downc:
|
||||
case <-b.stopc:
|
||||
return
|
||||
}
|
||||
select {
|
||||
case <-downc:
|
||||
b.notifyAddrs(notifyReset)
|
||||
case msg := <-b.updateAddrsC:
|
||||
b.notifyAddrs(msg)
|
||||
case <-b.stopc:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) notifyAddrs(msg notifyMsg) {
|
||||
if msg == notifyNext {
|
||||
select {
|
||||
case b.notifyCh <- []grpc.Address{}:
|
||||
case <-b.stopc:
|
||||
return
|
||||
}
|
||||
}
|
||||
b.mu.RLock()
|
||||
addrs := b.addrs
|
||||
pinAddr := b.pinAddr
|
||||
downc := b.downc
|
||||
b.mu.RUnlock()
|
||||
|
||||
var waitDown bool
|
||||
if pinAddr != "" {
|
||||
waitDown = true
|
||||
for _, a := range addrs {
|
||||
if a.Addr == pinAddr {
|
||||
waitDown = false
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
select {
|
||||
case b.notifyCh <- addrs:
|
||||
if waitDown {
|
||||
select {
|
||||
case <-downc:
|
||||
case <-b.stopc:
|
||||
}
|
||||
}
|
||||
case <-b.stopc:
|
||||
}
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) Up(addr grpc.Address) func(error) {
|
||||
f, _ := b.up(addr)
|
||||
return f
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) up(addr grpc.Address) (func(error), bool) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
|
||||
// gRPC might call Up after it called Close. We add this check
|
||||
// to "fix" it up at application layer. Otherwise, will panic
|
||||
// if b.upc is already closed.
|
||||
if b.closed {
|
||||
return func(err error) {}, false
|
||||
}
|
||||
// gRPC might call Up on a stale address.
|
||||
// Prevent updating pinAddr with a stale address.
|
||||
if !hasAddr(b.addrs, addr.Addr) {
|
||||
return func(err error) {}, false
|
||||
}
|
||||
if b.pinAddr != "" {
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/balancer: %q is up but not pinned (already pinned %q)", addr.Addr, b.pinAddr)
|
||||
}
|
||||
return func(err error) {}, false
|
||||
}
|
||||
// notify waiting Get()s and pin first connected address
|
||||
close(b.upc)
|
||||
b.downc = make(chan struct{})
|
||||
b.pinAddr = addr.Addr
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/balancer: pin %q", addr.Addr)
|
||||
}
|
||||
// notify client that a connection is up
|
||||
b.readyOnce.Do(func() { close(b.readyc) })
|
||||
return func(err error) {
|
||||
b.mu.Lock()
|
||||
b.upc = make(chan struct{})
|
||||
close(b.downc)
|
||||
b.pinAddr = ""
|
||||
b.mu.Unlock()
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/balancer: unpin %q (%q)", addr.Addr, err.Error())
|
||||
}
|
||||
}, true
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) Get(ctx context.Context, opts grpc.BalancerGetOptions) (grpc.Address, func(), error) {
|
||||
var (
|
||||
addr string
|
||||
closed bool
|
||||
)
|
||||
|
||||
// If opts.BlockingWait is false (for fail-fast RPCs), it should return
|
||||
// an address it has notified via Notify immediately instead of blocking.
|
||||
if !opts.BlockingWait {
|
||||
b.mu.RLock()
|
||||
closed = b.closed
|
||||
addr = b.pinAddr
|
||||
b.mu.RUnlock()
|
||||
if closed {
|
||||
return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
|
||||
}
|
||||
if addr == "" {
|
||||
return grpc.Address{Addr: ""}, nil, ErrNoAddrAvilable
|
||||
}
|
||||
return grpc.Address{Addr: addr}, func() {}, nil
|
||||
}
|
||||
|
||||
for {
|
||||
b.mu.RLock()
|
||||
ch := b.upc
|
||||
b.mu.RUnlock()
|
||||
select {
|
||||
case <-ch:
|
||||
case <-b.donec:
|
||||
return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
|
||||
case <-ctx.Done():
|
||||
return grpc.Address{Addr: ""}, nil, ctx.Err()
|
||||
}
|
||||
b.mu.RLock()
|
||||
closed = b.closed
|
||||
addr = b.pinAddr
|
||||
b.mu.RUnlock()
|
||||
// Close() which sets b.closed = true can be called before Get(), Get() must exit if balancer is closed.
|
||||
if closed {
|
||||
return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
|
||||
}
|
||||
if addr != "" {
|
||||
break
|
||||
}
|
||||
}
|
||||
return grpc.Address{Addr: addr}, func() {}, nil
|
||||
}
|
||||
|
||||
func (b *simpleBalancer) Notify() <-chan []grpc.Address { return b.notifyCh }
|
||||
|
||||
func (b *simpleBalancer) Close() error {
|
||||
b.mu.Lock()
|
||||
// In case gRPC calls close twice. TODO: remove the checking
|
||||
// when we are sure that gRPC wont call close twice.
|
||||
if b.closed {
|
||||
b.mu.Unlock()
|
||||
<-b.donec
|
||||
return nil
|
||||
}
|
||||
b.closed = true
|
||||
close(b.stopc)
|
||||
b.pinAddr = ""
|
||||
|
||||
// In the case of following scenario:
|
||||
// 1. upc is not closed; no pinned address
|
||||
// 2. client issues an RPC, calling invoke(), which calls Get(), enters for loop, blocks
|
||||
// 3. client.conn.Close() calls balancer.Close(); closed = true
|
||||
// 4. for loop in Get() never exits since ctx is the context passed in by the client and may not be canceled
|
||||
// we must close upc so Get() exits from blocking on upc
|
||||
select {
|
||||
case <-b.upc:
|
||||
default:
|
||||
// terminate all waiting Get()s
|
||||
close(b.upc)
|
||||
}
|
||||
|
||||
b.mu.Unlock()
|
||||
|
||||
// wait for updateNotifyLoop to finish
|
||||
<-b.donec
|
||||
close(b.notifyCh)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func getHost(ep string) string {
|
||||
url, uerr := url.Parse(ep)
|
||||
if uerr != nil || !strings.Contains(ep, "://") {
|
||||
return ep
|
||||
}
|
||||
return url.Host
|
||||
}
|
||||
|
||||
func eps2addrs(eps []string) []grpc.Address {
|
||||
addrs := make([]grpc.Address, len(eps))
|
||||
for i := range eps {
|
||||
addrs[i].Addr = getHost(eps[i])
|
||||
}
|
||||
return addrs
|
||||
}
|
@ -121,6 +121,19 @@ func (c *Client) SetEndpoints(eps ...string) {
|
||||
c.cfg.Endpoints = eps
|
||||
c.mu.Unlock()
|
||||
c.balancer.updateAddrs(eps...)
|
||||
|
||||
// updating notifyCh can trigger new connections,
|
||||
// need update addrs if all connections are down
|
||||
// or addrs does not include pinAddr.
|
||||
c.balancer.mu.RLock()
|
||||
update := !hasAddr(c.balancer.addrs, c.balancer.pinAddr)
|
||||
c.balancer.mu.RUnlock()
|
||||
if update {
|
||||
select {
|
||||
case c.balancer.updateAddrsC <- notifyNext:
|
||||
case <-c.balancer.stopc:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Sync synchronizes client's endpoints with the known endpoints from the etcd membership.
|
||||
@ -378,9 +391,9 @@ func newClient(cfg *Config) (*Client, error) {
|
||||
client.Password = cfg.Password
|
||||
}
|
||||
|
||||
sb := newSimpleBalancer(cfg.Endpoints)
|
||||
hc := func(ep string) (bool, error) { return grpcHealthCheck(client, ep) }
|
||||
client.balancer = newHealthBalancer(sb, cfg.DialTimeout, hc)
|
||||
client.balancer = newHealthBalancer(cfg.Endpoints, cfg.DialTimeout, func(ep string) (bool, error) {
|
||||
return grpcHealthCheck(client, ep)
|
||||
})
|
||||
|
||||
// use Endpoints[0] so that for https:// without any tls config given, then
|
||||
// grpc will assume the certificate server name is the endpoint host.
|
||||
|
@ -16,6 +16,9 @@ package clientv3
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"net/url"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -25,207 +28,549 @@ import (
|
||||
"google.golang.org/grpc/status"
|
||||
)
|
||||
|
||||
const minHealthRetryDuration = 3 * time.Second
|
||||
const unknownService = "unknown service grpc.health.v1.Health"
|
||||
const (
|
||||
minHealthRetryDuration = 3 * time.Second
|
||||
unknownService = "unknown service grpc.health.v1.Health"
|
||||
)
|
||||
|
||||
// ErrNoAddrAvilable is returned by Get() when the balancer does not have
|
||||
// any active connection to endpoints at the time.
|
||||
// This error is returned only when opts.BlockingWait is true.
|
||||
var ErrNoAddrAvilable = status.Error(codes.Unavailable, "there is no address available")
|
||||
|
||||
type healthCheckFunc func(ep string) (bool, error)
|
||||
|
||||
// healthBalancer wraps a balancer so that it uses health checking
|
||||
// to choose its endpoints.
|
||||
type notifyMsg int
|
||||
|
||||
const (
|
||||
notifyReset notifyMsg = iota
|
||||
notifyNext
|
||||
)
|
||||
|
||||
// healthBalancer does the bare minimum to expose multiple eps
|
||||
// to the grpc reconnection code path
|
||||
type healthBalancer struct {
|
||||
*simpleBalancer
|
||||
// addrs are the client's endpoint addresses for grpc
|
||||
addrs []grpc.Address
|
||||
|
||||
// eps holds the raw endpoints from the client
|
||||
eps []string
|
||||
|
||||
// notifyCh notifies grpc of the set of addresses for connecting
|
||||
notifyCh chan []grpc.Address
|
||||
|
||||
// readyc closes once the first connection is up
|
||||
readyc chan struct{}
|
||||
readyOnce sync.Once
|
||||
|
||||
// healthCheck checks an endpoint's health.
|
||||
healthCheck healthCheckFunc
|
||||
healthCheckTimeout time.Duration
|
||||
|
||||
// mu protects addrs, eps, unhealthy map, and stopc.
|
||||
unhealthyMu sync.RWMutex
|
||||
unhealthyHostPorts map[string]time.Time
|
||||
|
||||
// mu protects all fields below.
|
||||
mu sync.RWMutex
|
||||
|
||||
// addrs stores all grpc addresses associated with the balancer.
|
||||
addrs []grpc.Address
|
||||
// upc closes when pinAddr transitions from empty to non-empty or the balancer closes.
|
||||
upc chan struct{}
|
||||
|
||||
// eps stores all client endpoints
|
||||
eps []string
|
||||
|
||||
// unhealthy tracks the last unhealthy time of endpoints.
|
||||
unhealthy map[string]time.Time
|
||||
// downc closes when grpc calls down() on pinAddr
|
||||
downc chan struct{}
|
||||
|
||||
// stopc is closed to signal updateNotifyLoop should stop.
|
||||
stopc chan struct{}
|
||||
stopOnce sync.Once
|
||||
wg sync.WaitGroup
|
||||
|
||||
// donec closes when all goroutines are exited
|
||||
donec chan struct{}
|
||||
|
||||
// updateAddrsC notifies updateNotifyLoop to update addrs.
|
||||
updateAddrsC chan notifyMsg
|
||||
|
||||
// grpc issues TLS cert checks using the string passed into dial so
|
||||
// that string must be the host. To recover the full scheme://host URL,
|
||||
// have a map from hosts to the original endpoint.
|
||||
hostPort2ep map[string]string
|
||||
|
||||
wg sync.WaitGroup
|
||||
// pinAddr is the currently pinned address; set to the empty string on
|
||||
// initialization and shutdown.
|
||||
pinAddr string
|
||||
|
||||
closed bool
|
||||
}
|
||||
|
||||
func newHealthBalancer(b *simpleBalancer, timeout time.Duration, hc healthCheckFunc) *healthBalancer {
|
||||
func newHealthBalancer(eps []string, timeout time.Duration, hc healthCheckFunc) *healthBalancer {
|
||||
notifyCh := make(chan []grpc.Address)
|
||||
addrs := eps2addrs(eps)
|
||||
hb := &healthBalancer{
|
||||
simpleBalancer: b,
|
||||
healthCheck: hc,
|
||||
eps: b.endpoints(),
|
||||
addrs: eps2addrs(b.endpoints()),
|
||||
hostPort2ep: getHostPort2ep(b.endpoints()),
|
||||
unhealthy: make(map[string]time.Time),
|
||||
stopc: make(chan struct{}),
|
||||
addrs: addrs,
|
||||
eps: eps,
|
||||
notifyCh: notifyCh,
|
||||
readyc: make(chan struct{}),
|
||||
healthCheck: hc,
|
||||
unhealthyHostPorts: make(map[string]time.Time),
|
||||
upc: make(chan struct{}),
|
||||
stopc: make(chan struct{}),
|
||||
downc: make(chan struct{}),
|
||||
donec: make(chan struct{}),
|
||||
updateAddrsC: make(chan notifyMsg),
|
||||
hostPort2ep: getHostPort2ep(eps),
|
||||
}
|
||||
if timeout < minHealthRetryDuration {
|
||||
timeout = minHealthRetryDuration
|
||||
}
|
||||
hb.healthCheckTimeout = timeout
|
||||
|
||||
close(hb.downc)
|
||||
go hb.updateNotifyLoop()
|
||||
hb.wg.Add(1)
|
||||
go func() {
|
||||
defer hb.wg.Done()
|
||||
hb.updateUnhealthy(timeout)
|
||||
hb.updateUnhealthy()
|
||||
}()
|
||||
|
||||
return hb
|
||||
}
|
||||
|
||||
func (hb *healthBalancer) Up(addr grpc.Address) func(error) {
|
||||
f, used := hb.up(addr)
|
||||
if !used {
|
||||
return f
|
||||
func (b *healthBalancer) Start(target string, config grpc.BalancerConfig) error { return nil }
|
||||
|
||||
func (b *healthBalancer) ConnectNotify() <-chan struct{} {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
return b.upc
|
||||
}
|
||||
|
||||
func (b *healthBalancer) ready() <-chan struct{} { return b.readyc }
|
||||
|
||||
func (b *healthBalancer) endpoint(hostPort string) string {
|
||||
b.mu.RLock()
|
||||
defer b.mu.RUnlock()
|
||||
return b.hostPort2ep[hostPort]
|
||||
}
|
||||
|
||||
func (b *healthBalancer) pinned() string {
|
||||
b.mu.RLock()
|
||||
defer b.mu.RUnlock()
|
||||
return b.pinAddr
|
||||
}
|
||||
|
||||
func (b *healthBalancer) hostPortError(hostPort string, err error) {
|
||||
if b.endpoint(hostPort) == "" {
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/balancer: %q is stale (skip marking as unhealthy on %q)", hostPort, err.Error())
|
||||
}
|
||||
return
|
||||
}
|
||||
return func(err error) {
|
||||
// If connected to a black hole endpoint or a killed server, the gRPC ping
|
||||
// timeout will induce a network I/O error, and retrying until success;
|
||||
// finding healthy endpoint on retry could take several timeouts and redials.
|
||||
// To avoid wasting retries, gray-list unhealthy endpoints.
|
||||
hb.hostPortError(addr.Addr, err)
|
||||
f(err)
|
||||
|
||||
b.unhealthyMu.Lock()
|
||||
b.unhealthyHostPorts[hostPort] = time.Now()
|
||||
b.unhealthyMu.Unlock()
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/balancer: %q is marked unhealthy (%q)", hostPort, err.Error())
|
||||
}
|
||||
}
|
||||
|
||||
func (hb *healthBalancer) up(addr grpc.Address) (func(error), bool) {
|
||||
if !hb.mayPin(addr) {
|
||||
return func(err error) {}, false
|
||||
func (b *healthBalancer) removeUnhealthy(hostPort, msg string) {
|
||||
if b.endpoint(hostPort) == "" {
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/balancer: %q was not in unhealthy (%q)", hostPort, msg)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
b.unhealthyMu.Lock()
|
||||
delete(b.unhealthyHostPorts, hostPort)
|
||||
b.unhealthyMu.Unlock()
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/balancer: %q is removed from unhealthy (%q)", hostPort, msg)
|
||||
}
|
||||
return hb.simpleBalancer.up(addr)
|
||||
}
|
||||
|
||||
func (hb *healthBalancer) Close() error {
|
||||
hb.stopOnce.Do(func() { close(hb.stopc) })
|
||||
hb.wg.Wait()
|
||||
return hb.simpleBalancer.Close()
|
||||
func (b *healthBalancer) countUnhealthy() (count int) {
|
||||
b.unhealthyMu.RLock()
|
||||
count = len(b.unhealthyHostPorts)
|
||||
b.unhealthyMu.RUnlock()
|
||||
return count
|
||||
}
|
||||
|
||||
func (hb *healthBalancer) updateAddrs(eps ...string) {
|
||||
addrs, hostPort2ep := eps2addrs(eps), getHostPort2ep(eps)
|
||||
hb.mu.Lock()
|
||||
hb.addrs, hb.eps, hb.hostPort2ep = addrs, eps, hostPort2ep
|
||||
hb.unhealthy = make(map[string]time.Time)
|
||||
hb.mu.Unlock()
|
||||
hb.simpleBalancer.updateAddrs(eps...)
|
||||
func (b *healthBalancer) isUnhealthy(hostPort string) (unhealthy bool) {
|
||||
b.unhealthyMu.RLock()
|
||||
_, unhealthy = b.unhealthyHostPorts[hostPort]
|
||||
b.unhealthyMu.RUnlock()
|
||||
return unhealthy
|
||||
}
|
||||
|
||||
func (hb *healthBalancer) endpoint(host string) string {
|
||||
hb.mu.RLock()
|
||||
defer hb.mu.RUnlock()
|
||||
return hb.hostPort2ep[host]
|
||||
func (b *healthBalancer) cleanupUnhealthy() {
|
||||
b.unhealthyMu.Lock()
|
||||
for k, v := range b.unhealthyHostPorts {
|
||||
if time.Since(v) > b.healthCheckTimeout {
|
||||
delete(b.unhealthyHostPorts, k)
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/balancer: removed %q from unhealthy after %v", k, b.healthCheckTimeout)
|
||||
}
|
||||
}
|
||||
}
|
||||
b.unhealthyMu.Unlock()
|
||||
}
|
||||
|
||||
func (hb *healthBalancer) endpoints() []string {
|
||||
hb.mu.RLock()
|
||||
defer hb.mu.RUnlock()
|
||||
return hb.eps
|
||||
func (b *healthBalancer) liveAddrs() ([]grpc.Address, map[string]struct{}) {
|
||||
unhealthyCnt := b.countUnhealthy()
|
||||
|
||||
b.mu.RLock()
|
||||
defer b.mu.RUnlock()
|
||||
|
||||
hbAddrs := b.addrs
|
||||
if len(b.addrs) == 1 || unhealthyCnt == 0 || unhealthyCnt == len(b.addrs) {
|
||||
liveHostPorts := make(map[string]struct{}, len(b.hostPort2ep))
|
||||
for k := range b.hostPort2ep {
|
||||
liveHostPorts[k] = struct{}{}
|
||||
}
|
||||
return hbAddrs, liveHostPorts
|
||||
}
|
||||
|
||||
addrs := make([]grpc.Address, 0, len(b.addrs)-unhealthyCnt)
|
||||
liveHostPorts := make(map[string]struct{}, len(addrs))
|
||||
for _, addr := range b.addrs {
|
||||
if !b.isUnhealthy(addr.Addr) {
|
||||
addrs = append(addrs, addr)
|
||||
liveHostPorts[addr.Addr] = struct{}{}
|
||||
}
|
||||
}
|
||||
return addrs, liveHostPorts
|
||||
}
|
||||
|
||||
func (hb *healthBalancer) updateUnhealthy(timeout time.Duration) {
|
||||
func (b *healthBalancer) updateUnhealthy() {
|
||||
for {
|
||||
select {
|
||||
case <-time.After(timeout):
|
||||
hb.mu.Lock()
|
||||
for k, v := range hb.unhealthy {
|
||||
if time.Since(v) > timeout {
|
||||
delete(hb.unhealthy, k)
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/health-balancer: removes %q from unhealthy after %v", k, timeout)
|
||||
}
|
||||
case <-time.After(b.healthCheckTimeout):
|
||||
b.cleanupUnhealthy()
|
||||
pinned := b.pinned()
|
||||
if pinned == "" || b.isUnhealthy(pinned) {
|
||||
select {
|
||||
case b.updateAddrsC <- notifyNext:
|
||||
case <-b.stopc:
|
||||
return
|
||||
}
|
||||
}
|
||||
hb.mu.Unlock()
|
||||
eps := []string{}
|
||||
for _, addr := range hb.liveAddrs() {
|
||||
eps = append(eps, hb.endpoint(addr.Addr))
|
||||
}
|
||||
hb.simpleBalancer.updateAddrs(eps...)
|
||||
case <-hb.stopc:
|
||||
case <-b.stopc:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (hb *healthBalancer) liveAddrs() []grpc.Address {
|
||||
hb.mu.RLock()
|
||||
defer hb.mu.RUnlock()
|
||||
hbAddrs := hb.addrs
|
||||
if len(hb.addrs) == 1 || len(hb.unhealthy) == 0 || len(hb.unhealthy) == len(hb.addrs) {
|
||||
return hbAddrs
|
||||
}
|
||||
addrs := make([]grpc.Address, 0, len(hb.addrs)-len(hb.unhealthy))
|
||||
for _, addr := range hb.addrs {
|
||||
if _, unhealthy := hb.unhealthy[addr.Addr]; !unhealthy {
|
||||
addrs = append(addrs, addr)
|
||||
func (b *healthBalancer) updateAddrs(eps ...string) {
|
||||
np := getHostPort2ep(eps)
|
||||
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
|
||||
match := len(np) == len(b.hostPort2ep)
|
||||
if match {
|
||||
for k, v := range np {
|
||||
if b.hostPort2ep[k] != v {
|
||||
match = false
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
return addrs
|
||||
if match {
|
||||
// same endpoints, so no need to update address
|
||||
return
|
||||
}
|
||||
|
||||
b.hostPort2ep = np
|
||||
b.addrs, b.eps = eps2addrs(eps), eps
|
||||
|
||||
b.unhealthyMu.Lock()
|
||||
b.unhealthyHostPorts = make(map[string]time.Time)
|
||||
b.unhealthyMu.Unlock()
|
||||
}
|
||||
|
||||
func (hb *healthBalancer) hostPortError(hostPort string, err error) {
|
||||
hb.mu.Lock()
|
||||
if _, ok := hb.hostPort2ep[hostPort]; ok {
|
||||
hb.unhealthy[hostPort] = time.Now()
|
||||
func (b *healthBalancer) next() {
|
||||
b.mu.RLock()
|
||||
downc := b.downc
|
||||
b.mu.RUnlock()
|
||||
select {
|
||||
case b.updateAddrsC <- notifyNext:
|
||||
case <-b.stopc:
|
||||
}
|
||||
// wait until disconnect so new RPCs are not issued on old connection
|
||||
select {
|
||||
case <-downc:
|
||||
case <-b.stopc:
|
||||
}
|
||||
}
|
||||
|
||||
func (b *healthBalancer) updateNotifyLoop() {
|
||||
defer close(b.donec)
|
||||
|
||||
for {
|
||||
b.mu.RLock()
|
||||
upc, downc, addr := b.upc, b.downc, b.pinAddr
|
||||
b.mu.RUnlock()
|
||||
// downc or upc should be closed
|
||||
select {
|
||||
case <-downc:
|
||||
downc = nil
|
||||
default:
|
||||
}
|
||||
select {
|
||||
case <-upc:
|
||||
upc = nil
|
||||
default:
|
||||
}
|
||||
switch {
|
||||
case downc == nil && upc == nil:
|
||||
// stale
|
||||
select {
|
||||
case <-b.stopc:
|
||||
return
|
||||
default:
|
||||
}
|
||||
case downc == nil:
|
||||
b.notifyAddrs(notifyReset)
|
||||
select {
|
||||
case <-upc:
|
||||
case msg := <-b.updateAddrsC:
|
||||
b.notifyAddrs(msg)
|
||||
case <-b.stopc:
|
||||
return
|
||||
}
|
||||
case upc == nil:
|
||||
select {
|
||||
// close connections that are not the pinned address
|
||||
case b.notifyCh <- []grpc.Address{{Addr: addr}}:
|
||||
case <-downc:
|
||||
case <-b.stopc:
|
||||
return
|
||||
}
|
||||
select {
|
||||
case <-downc:
|
||||
b.notifyAddrs(notifyReset)
|
||||
case msg := <-b.updateAddrsC:
|
||||
b.notifyAddrs(msg)
|
||||
case <-b.stopc:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (b *healthBalancer) notifyAddrs(msg notifyMsg) {
|
||||
if msg == notifyNext {
|
||||
select {
|
||||
case b.notifyCh <- []grpc.Address{}:
|
||||
case <-b.stopc:
|
||||
return
|
||||
}
|
||||
}
|
||||
b.mu.RLock()
|
||||
pinAddr := b.pinAddr
|
||||
downc := b.downc
|
||||
b.mu.RUnlock()
|
||||
addrs, hostPorts := b.liveAddrs()
|
||||
|
||||
var waitDown bool
|
||||
if pinAddr != "" {
|
||||
_, ok := hostPorts[pinAddr]
|
||||
waitDown = !ok
|
||||
}
|
||||
|
||||
select {
|
||||
case b.notifyCh <- addrs:
|
||||
if waitDown {
|
||||
select {
|
||||
case <-downc:
|
||||
case <-b.stopc:
|
||||
}
|
||||
}
|
||||
case <-b.stopc:
|
||||
}
|
||||
}
|
||||
|
||||
func (b *healthBalancer) Up(addr grpc.Address) func(error) {
|
||||
if !b.mayPin(addr) {
|
||||
return func(err error) {}
|
||||
}
|
||||
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
|
||||
// gRPC might call Up after it called Close. We add this check
|
||||
// to "fix" it up at application layer. Otherwise, will panic
|
||||
// if b.upc is already closed.
|
||||
if b.closed {
|
||||
return func(err error) {}
|
||||
}
|
||||
|
||||
// gRPC might call Up on a stale address.
|
||||
// Prevent updating pinAddr with a stale address.
|
||||
if !hasAddr(b.addrs, addr.Addr) {
|
||||
return func(err error) {}
|
||||
}
|
||||
|
||||
if b.pinAddr != "" {
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/health-balancer: marking %q as unhealthy (%q)", hostPort, err.Error())
|
||||
logger.Infof("clientv3/balancer: %q is up but not pinned (already pinned %q)", addr.Addr, b.pinAddr)
|
||||
}
|
||||
return func(err error) {}
|
||||
}
|
||||
|
||||
// notify waiting Get()s and pin first connected address
|
||||
close(b.upc)
|
||||
b.downc = make(chan struct{})
|
||||
b.pinAddr = addr.Addr
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/balancer: pin %q", addr.Addr)
|
||||
}
|
||||
|
||||
// notify client that a connection is up
|
||||
b.readyOnce.Do(func() { close(b.readyc) })
|
||||
|
||||
return func(err error) {
|
||||
// If connected to a black hole endpoint or a killed server, the gRPC ping
|
||||
// timeout will induce a network I/O error, and retrying until success;
|
||||
// finding healthy endpoint on retry could take several timeouts and redials.
|
||||
// To avoid wasting retries, gray-list unhealthy endpoints.
|
||||
b.hostPortError(addr.Addr, err)
|
||||
|
||||
b.mu.Lock()
|
||||
b.upc = make(chan struct{})
|
||||
close(b.downc)
|
||||
b.pinAddr = ""
|
||||
b.mu.Unlock()
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/balancer: unpin %q (%q)", addr.Addr, err.Error())
|
||||
}
|
||||
}
|
||||
hb.mu.Unlock()
|
||||
}
|
||||
|
||||
func (hb *healthBalancer) mayPin(addr grpc.Address) bool {
|
||||
hb.mu.RLock()
|
||||
if _, ok := hb.hostPort2ep[addr.Addr]; !ok { // stale host:port
|
||||
hb.mu.RUnlock()
|
||||
func (b *healthBalancer) mayPin(addr grpc.Address) bool {
|
||||
if b.endpoint(addr.Addr) == "" { // stale host:port
|
||||
return false
|
||||
}
|
||||
skip := len(hb.addrs) == 1 || len(hb.unhealthy) == 0 || len(hb.addrs) == len(hb.unhealthy)
|
||||
failedTime, bad := hb.unhealthy[addr.Addr]
|
||||
dur := hb.healthCheckTimeout
|
||||
hb.mu.RUnlock()
|
||||
|
||||
b.unhealthyMu.RLock()
|
||||
unhealthyCnt := len(b.unhealthyHostPorts)
|
||||
failedTime, bad := b.unhealthyHostPorts[addr.Addr]
|
||||
b.unhealthyMu.RUnlock()
|
||||
|
||||
b.mu.RLock()
|
||||
skip := len(b.addrs) == 1 || unhealthyCnt == 0 || len(b.addrs) == unhealthyCnt
|
||||
b.mu.RUnlock()
|
||||
if skip || !bad {
|
||||
return true
|
||||
}
|
||||
|
||||
// prevent isolated member's endpoint from being infinitely retried, as follows:
|
||||
// 1. keepalive pings detects GoAway with http2.ErrCodeEnhanceYourCalm
|
||||
// 2. balancer 'Up' unpins with grpc: failed with network I/O error
|
||||
// 3. grpc-healthcheck still SERVING, thus retry to pin
|
||||
// instead, return before grpc-healthcheck if failed within healthcheck timeout
|
||||
if elapsed := time.Since(failedTime); elapsed < dur {
|
||||
if elapsed := time.Since(failedTime); elapsed < b.healthCheckTimeout {
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/health-balancer: %q is up but not pinned (failed %v ago, require minimum %v after failure)", addr.Addr, elapsed, dur)
|
||||
logger.Infof("clientv3/balancer: %q is up but not pinned (failed %v ago, require minimum %v after failure)", addr.Addr, elapsed, b.healthCheckTimeout)
|
||||
}
|
||||
return false
|
||||
}
|
||||
if ok, _ := hb.healthCheck(addr.Addr); ok {
|
||||
hb.mu.Lock()
|
||||
delete(hb.unhealthy, addr.Addr)
|
||||
hb.mu.Unlock()
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/health-balancer: %q is healthy (health check success)", addr.Addr)
|
||||
}
|
||||
|
||||
if ok, _ := b.healthCheck(addr.Addr); ok {
|
||||
b.removeUnhealthy(addr.Addr, "health check success")
|
||||
return true
|
||||
}
|
||||
hb.mu.Lock()
|
||||
hb.unhealthy[addr.Addr] = time.Now()
|
||||
hb.mu.Unlock()
|
||||
if logger.V(4) {
|
||||
logger.Infof("clientv3/health-balancer: %q becomes unhealthy (health check failed)", addr.Addr)
|
||||
}
|
||||
|
||||
b.hostPortError(addr.Addr, errors.New("health check failed"))
|
||||
return false
|
||||
}
|
||||
|
||||
func (b *healthBalancer) Get(ctx context.Context, opts grpc.BalancerGetOptions) (grpc.Address, func(), error) {
|
||||
var (
|
||||
addr string
|
||||
closed bool
|
||||
)
|
||||
|
||||
// If opts.BlockingWait is false (for fail-fast RPCs), it should return
|
||||
// an address it has notified via Notify immediately instead of blocking.
|
||||
if !opts.BlockingWait {
|
||||
b.mu.RLock()
|
||||
closed = b.closed
|
||||
addr = b.pinAddr
|
||||
b.mu.RUnlock()
|
||||
if closed {
|
||||
return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
|
||||
}
|
||||
if addr == "" {
|
||||
return grpc.Address{Addr: ""}, nil, ErrNoAddrAvilable
|
||||
}
|
||||
return grpc.Address{Addr: addr}, func() {}, nil
|
||||
}
|
||||
|
||||
for {
|
||||
b.mu.RLock()
|
||||
ch := b.upc
|
||||
b.mu.RUnlock()
|
||||
select {
|
||||
case <-ch:
|
||||
case <-b.donec:
|
||||
return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
|
||||
case <-ctx.Done():
|
||||
return grpc.Address{Addr: ""}, nil, ctx.Err()
|
||||
}
|
||||
b.mu.RLock()
|
||||
closed = b.closed
|
||||
addr = b.pinAddr
|
||||
b.mu.RUnlock()
|
||||
// Close() which sets b.closed = true can be called before Get(), Get() must exit if balancer is closed.
|
||||
if closed {
|
||||
return grpc.Address{Addr: ""}, nil, grpc.ErrClientConnClosing
|
||||
}
|
||||
if addr != "" {
|
||||
break
|
||||
}
|
||||
}
|
||||
return grpc.Address{Addr: addr}, func() {}, nil
|
||||
}
|
||||
|
||||
func (b *healthBalancer) Notify() <-chan []grpc.Address { return b.notifyCh }
|
||||
|
||||
func (b *healthBalancer) Close() error {
|
||||
b.mu.Lock()
|
||||
// In case gRPC calls close twice. TODO: remove the checking
|
||||
// when we are sure that gRPC wont call close twice.
|
||||
if b.closed {
|
||||
b.mu.Unlock()
|
||||
<-b.donec
|
||||
return nil
|
||||
}
|
||||
b.closed = true
|
||||
b.stopOnce.Do(func() { close(b.stopc) })
|
||||
b.pinAddr = ""
|
||||
|
||||
// In the case of following scenario:
|
||||
// 1. upc is not closed; no pinned address
|
||||
// 2. client issues an RPC, calling invoke(), which calls Get(), enters for loop, blocks
|
||||
// 3. client.conn.Close() calls balancer.Close(); closed = true
|
||||
// 4. for loop in Get() never exits since ctx is the context passed in by the client and may not be canceled
|
||||
// we must close upc so Get() exits from blocking on upc
|
||||
select {
|
||||
case <-b.upc:
|
||||
default:
|
||||
// terminate all waiting Get()s
|
||||
close(b.upc)
|
||||
}
|
||||
|
||||
b.mu.Unlock()
|
||||
b.wg.Wait()
|
||||
|
||||
// wait for updateNotifyLoop to finish
|
||||
<-b.donec
|
||||
close(b.notifyCh)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func grpcHealthCheck(client *Client, ep string) (bool, error) {
|
||||
conn, err := client.dial(ep)
|
||||
if err != nil {
|
||||
@ -238,8 +583,7 @@ func grpcHealthCheck(client *Client, ep string) (bool, error) {
|
||||
cancel()
|
||||
if err != nil {
|
||||
if s, ok := status.FromError(err); ok && s.Code() == codes.Unavailable {
|
||||
if s.Message() == unknownService {
|
||||
// etcd < v3.3.0
|
||||
if s.Message() == unknownService { // etcd < v3.3.0
|
||||
return true, nil
|
||||
}
|
||||
}
|
||||
@ -247,3 +591,37 @@ func grpcHealthCheck(client *Client, ep string) (bool, error) {
|
||||
}
|
||||
return resp.Status == healthpb.HealthCheckResponse_SERVING, nil
|
||||
}
|
||||
|
||||
func hasAddr(addrs []grpc.Address, targetAddr string) bool {
|
||||
for _, addr := range addrs {
|
||||
if targetAddr == addr.Addr {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func getHost(ep string) string {
|
||||
url, uerr := url.Parse(ep)
|
||||
if uerr != nil || !strings.Contains(ep, "://") {
|
||||
return ep
|
||||
}
|
||||
return url.Host
|
||||
}
|
||||
|
||||
func eps2addrs(eps []string) []grpc.Address {
|
||||
addrs := make([]grpc.Address, len(eps))
|
||||
for i := range eps {
|
||||
addrs[i].Addr = getHost(eps[i])
|
||||
}
|
||||
return addrs
|
||||
}
|
||||
|
||||
func getHostPort2ep(eps []string) map[string]string {
|
||||
hm := make(map[string]string, len(eps))
|
||||
for i := range eps {
|
||||
_, host, _ := parseEndpoint(eps[i])
|
||||
hm[host] = eps[i]
|
||||
}
|
||||
return hm
|
||||
}
|
||||
|
@ -1,4 +1,4 @@
|
||||
// Copyright 2016 The etcd Authors
|
||||
// Copyright 2017 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.
|
||||
@ -28,29 +28,27 @@ import (
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
var (
|
||||
endpoints = []string{"localhost:2379", "localhost:22379", "localhost:32379"}
|
||||
)
|
||||
var endpoints = []string{"localhost:2379", "localhost:22379", "localhost:32379"}
|
||||
|
||||
func TestBalancerGetUnblocking(t *testing.T) {
|
||||
sb := newSimpleBalancer(endpoints)
|
||||
defer sb.Close()
|
||||
if addrs := <-sb.Notify(); len(addrs) != len(endpoints) {
|
||||
t.Errorf("Initialize newSimpleBalancer should have triggered Notify() chan, but it didn't")
|
||||
hb := newHealthBalancer(endpoints, minHealthRetryDuration, func(string) (bool, error) { return true, nil })
|
||||
defer hb.Close()
|
||||
if addrs := <-hb.Notify(); len(addrs) != len(endpoints) {
|
||||
t.Errorf("Initialize newHealthBalancer should have triggered Notify() chan, but it didn't")
|
||||
}
|
||||
unblockingOpts := grpc.BalancerGetOptions{BlockingWait: false}
|
||||
|
||||
_, _, err := sb.Get(context.Background(), unblockingOpts)
|
||||
_, _, err := hb.Get(context.Background(), unblockingOpts)
|
||||
if err != ErrNoAddrAvilable {
|
||||
t.Errorf("Get() with no up endpoints should return ErrNoAddrAvailable, got: %v", err)
|
||||
}
|
||||
|
||||
down1 := sb.Up(grpc.Address{Addr: endpoints[1]})
|
||||
if addrs := <-sb.Notify(); len(addrs) != 1 {
|
||||
down1 := hb.Up(grpc.Address{Addr: endpoints[1]})
|
||||
if addrs := <-hb.Notify(); len(addrs) != 1 {
|
||||
t.Errorf("first Up() should have triggered balancer to send the first connected address via Notify chan so that other connections can be closed")
|
||||
}
|
||||
down2 := sb.Up(grpc.Address{Addr: endpoints[2]})
|
||||
addrFirst, putFun, err := sb.Get(context.Background(), unblockingOpts)
|
||||
down2 := hb.Up(grpc.Address{Addr: endpoints[2]})
|
||||
addrFirst, putFun, err := hb.Get(context.Background(), unblockingOpts)
|
||||
if err != nil {
|
||||
t.Errorf("Get() with up endpoints should success, got %v", err)
|
||||
}
|
||||
@ -60,32 +58,32 @@ func TestBalancerGetUnblocking(t *testing.T) {
|
||||
if putFun == nil {
|
||||
t.Errorf("Get() returned unexpected nil put function")
|
||||
}
|
||||
addrSecond, _, _ := sb.Get(context.Background(), unblockingOpts)
|
||||
addrSecond, _, _ := hb.Get(context.Background(), unblockingOpts)
|
||||
if addrFirst.Addr != addrSecond.Addr {
|
||||
t.Errorf("Get() didn't return the same address as previous call, got %v and %v", addrFirst, addrSecond)
|
||||
}
|
||||
|
||||
down1(errors.New("error"))
|
||||
if addrs := <-sb.Notify(); len(addrs) != len(endpoints) {
|
||||
t.Errorf("closing the only connection should triggered balancer to send the all endpoints via Notify chan so that we can establish a connection")
|
||||
if addrs := <-hb.Notify(); len(addrs) != len(endpoints)-1 { // we call down on one endpoint
|
||||
t.Errorf("closing the only connection should triggered balancer to send the %d endpoints via Notify chan so that we can establish a connection", len(endpoints)-1)
|
||||
}
|
||||
down2(errors.New("error"))
|
||||
_, _, err = sb.Get(context.Background(), unblockingOpts)
|
||||
_, _, err = hb.Get(context.Background(), unblockingOpts)
|
||||
if err != ErrNoAddrAvilable {
|
||||
t.Errorf("Get() with no up endpoints should return ErrNoAddrAvailable, got: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestBalancerGetBlocking(t *testing.T) {
|
||||
sb := newSimpleBalancer(endpoints)
|
||||
defer sb.Close()
|
||||
if addrs := <-sb.Notify(); len(addrs) != len(endpoints) {
|
||||
t.Errorf("Initialize newSimpleBalancer should have triggered Notify() chan, but it didn't")
|
||||
hb := newHealthBalancer(endpoints, minHealthRetryDuration, func(string) (bool, error) { return true, nil })
|
||||
defer hb.Close()
|
||||
if addrs := <-hb.Notify(); len(addrs) != len(endpoints) {
|
||||
t.Errorf("Initialize newHealthBalancer should have triggered Notify() chan, but it didn't")
|
||||
}
|
||||
blockingOpts := grpc.BalancerGetOptions{BlockingWait: true}
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100)
|
||||
_, _, err := sb.Get(ctx, blockingOpts)
|
||||
_, _, err := hb.Get(ctx, blockingOpts)
|
||||
cancel()
|
||||
if err != context.DeadlineExceeded {
|
||||
t.Errorf("Get() with no up endpoints should timeout, got %v", err)
|
||||
@ -94,15 +92,15 @@ func TestBalancerGetBlocking(t *testing.T) {
|
||||
downC := make(chan func(error), 1)
|
||||
|
||||
go func() {
|
||||
// ensure sb.Up() will be called after sb.Get() to see if Up() releases blocking Get()
|
||||
// ensure hb.Up() will be called after hb.Get() to see if Up() releases blocking Get()
|
||||
time.Sleep(time.Millisecond * 100)
|
||||
f := sb.Up(grpc.Address{Addr: endpoints[1]})
|
||||
if addrs := <-sb.Notify(); len(addrs) != 1 {
|
||||
f := hb.Up(grpc.Address{Addr: endpoints[1]})
|
||||
if addrs := <-hb.Notify(); len(addrs) != 1 {
|
||||
t.Errorf("first Up() should have triggered balancer to send the first connected address via Notify chan so that other connections can be closed")
|
||||
}
|
||||
downC <- f
|
||||
}()
|
||||
addrFirst, putFun, err := sb.Get(context.Background(), blockingOpts)
|
||||
addrFirst, putFun, err := hb.Get(context.Background(), blockingOpts)
|
||||
if err != nil {
|
||||
t.Errorf("Get() with up endpoints should success, got %v", err)
|
||||
}
|
||||
@ -114,19 +112,19 @@ func TestBalancerGetBlocking(t *testing.T) {
|
||||
}
|
||||
down1 := <-downC
|
||||
|
||||
down2 := sb.Up(grpc.Address{Addr: endpoints[2]})
|
||||
addrSecond, _, _ := sb.Get(context.Background(), blockingOpts)
|
||||
down2 := hb.Up(grpc.Address{Addr: endpoints[2]})
|
||||
addrSecond, _, _ := hb.Get(context.Background(), blockingOpts)
|
||||
if addrFirst.Addr != addrSecond.Addr {
|
||||
t.Errorf("Get() didn't return the same address as previous call, got %v and %v", addrFirst, addrSecond)
|
||||
}
|
||||
|
||||
down1(errors.New("error"))
|
||||
if addrs := <-sb.Notify(); len(addrs) != len(endpoints) {
|
||||
t.Errorf("closing the only connection should triggered balancer to send the all endpoints via Notify chan so that we can establish a connection")
|
||||
if addrs := <-hb.Notify(); len(addrs) != len(endpoints)-1 { // we call down on one endpoint
|
||||
t.Errorf("closing the only connection should triggered balancer to send the %d endpoints via Notify chan so that we can establish a connection", len(endpoints)-1)
|
||||
}
|
||||
down2(errors.New("error"))
|
||||
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
|
||||
_, _, err = sb.Get(ctx, blockingOpts)
|
||||
_, _, err = hb.Get(ctx, blockingOpts)
|
||||
cancel()
|
||||
if err != context.DeadlineExceeded {
|
||||
t.Errorf("Get() with no up endpoints should timeout, got %v", err)
|
||||
@ -168,9 +166,8 @@ func TestHealthBalancerGraylist(t *testing.T) {
|
||||
}()
|
||||
}
|
||||
|
||||
sb := newSimpleBalancer(eps)
|
||||
tf := func(s string) (bool, error) { return false, nil }
|
||||
hb := newHealthBalancer(sb, 5*time.Second, tf)
|
||||
hb := newHealthBalancer(eps, 5*time.Second, tf)
|
||||
|
||||
conn, err := grpc.Dial("", grpc.WithInsecure(), grpc.WithBalancer(hb))
|
||||
testutil.AssertNil(t, err)
|
||||
@ -203,13 +200,13 @@ func TestBalancerDoNotBlockOnClose(t *testing.T) {
|
||||
defer kcl.close()
|
||||
|
||||
for i := 0; i < 5; i++ {
|
||||
sb := newSimpleBalancer(kcl.endpoints())
|
||||
conn, err := grpc.Dial("", grpc.WithInsecure(), grpc.WithBalancer(sb))
|
||||
hb := newHealthBalancer(kcl.endpoints(), minHealthRetryDuration, func(string) (bool, error) { return true, nil })
|
||||
conn, err := grpc.Dial("", grpc.WithInsecure(), grpc.WithBalancer(hb))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
kvc := pb.NewKVClient(conn)
|
||||
<-sb.readyc
|
||||
<-hb.readyc
|
||||
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(100)
|
||||
@ -225,7 +222,7 @@ func TestBalancerDoNotBlockOnClose(t *testing.T) {
|
||||
bclosec, cclosec := make(chan struct{}), make(chan struct{})
|
||||
go func() {
|
||||
defer close(bclosec)
|
||||
sb.Close()
|
||||
hb.Close()
|
||||
}()
|
||||
go func() {
|
||||
defer close(cclosec)
|
@ -472,8 +472,9 @@ func TestKVNewAfterClose(t *testing.T) {
|
||||
|
||||
donec := make(chan struct{})
|
||||
go func() {
|
||||
if _, err := cli.Get(context.TODO(), "foo"); err != context.Canceled {
|
||||
t.Fatalf("expected %v, got %v", context.Canceled, err)
|
||||
_, err := cli.Get(context.TODO(), "foo")
|
||||
if err != context.Canceled && err != grpc.ErrClientConnClosing {
|
||||
t.Fatalf("expected %v or %v, got %v", context.Canceled, grpc.ErrClientConnClosing, err)
|
||||
}
|
||||
close(donec)
|
||||
}()
|
||||
|
@ -45,7 +45,8 @@ func isNonRepeatableStopError(err error) bool {
|
||||
if ev.Code() != codes.Unavailable {
|
||||
return true
|
||||
}
|
||||
return rpctypes.ErrorDesc(err) != "there is no address available"
|
||||
desc := rpctypes.ErrorDesc(err)
|
||||
return desc != "there is no address available" && desc != "there is no connection available"
|
||||
}
|
||||
|
||||
func (c *Client) newRetryWrapper(isStop retryStopErrFunc) retryRPCFunc {
|
||||
|
2
cmd/vendor/google.golang.org/grpc/balancer.go
generated
vendored
2
cmd/vendor/google.golang.org/grpc/balancer.go
generated
vendored
@ -403,6 +403,6 @@ type pickFirst struct {
|
||||
*roundRobin
|
||||
}
|
||||
|
||||
func pickFirstBalancer(r naming.Resolver) Balancer {
|
||||
func pickFirstBalancerV1(r naming.Resolver) Balancer {
|
||||
return &pickFirst{&roundRobin{r: r}}
|
||||
}
|
||||
|
206
cmd/vendor/google.golang.org/grpc/balancer/balancer.go
generated
vendored
Normal file
206
cmd/vendor/google.golang.org/grpc/balancer/balancer.go
generated
vendored
Normal file
@ -0,0 +1,206 @@
|
||||
/*
|
||||
*
|
||||
* 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 balancer defines APIs for load balancing in gRPC.
|
||||
// All APIs in this package are experimental.
|
||||
package balancer
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"net"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
"google.golang.org/grpc/credentials"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
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.
|
||||
func Register(b Builder) {
|
||||
m[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.
|
||||
func Get(name string) Builder {
|
||||
if b, ok := m[name]; ok {
|
||||
return b
|
||||
}
|
||||
return defaultBuilder
|
||||
}
|
||||
|
||||
// SubConn represents a gRPC sub connection.
|
||||
// Each sub connection contains a list of addresses. gRPC will
|
||||
// try to connect to them (in sequence), and stop trying the
|
||||
// remainder once one connection is successful.
|
||||
//
|
||||
// The reconnect backoff will be applied on the list, not a single address.
|
||||
// For example, try_on_all_addresses -> backoff -> try_on_all_addresses.
|
||||
//
|
||||
// All SubConns start in IDLE, and will not try to connect. To trigger
|
||||
// the connecting, Balancers must call Connect.
|
||||
// When the connection encounters an error, it will reconnect immediately.
|
||||
// When the connection becomes IDLE, it will not reconnect unless Connect is
|
||||
// called.
|
||||
type SubConn interface {
|
||||
// UpdateAddresses updates the addresses used in this SubConn.
|
||||
// gRPC checks if currently-connected address is still in the new list.
|
||||
// If it's in the list, the connection will be kept.
|
||||
// If it's not in the list, the connection will gracefully closed, and
|
||||
// a new connection will be created.
|
||||
//
|
||||
// This will trigger a state transition for the SubConn.
|
||||
UpdateAddresses([]resolver.Address)
|
||||
// Connect starts the connecting for this SubConn.
|
||||
Connect()
|
||||
}
|
||||
|
||||
// NewSubConnOptions contains options to create new SubConn.
|
||||
type NewSubConnOptions struct{}
|
||||
|
||||
// ClientConn represents a gRPC ClientConn.
|
||||
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.
|
||||
// Behaviors of the SubConn can be controlled by options.
|
||||
NewSubConn([]resolver.Address, NewSubConnOptions) (SubConn, error)
|
||||
// RemoveSubConn removes the SubConn from ClientConn.
|
||||
// The SubConn will be shutdown.
|
||||
RemoveSubConn(SubConn)
|
||||
|
||||
// UpdateBalancerState is called by balancer to nofity gRPC that some internal
|
||||
// state in balancer has changed.
|
||||
//
|
||||
// gRPC will update the connectivity state of the ClientConn, and will call pick
|
||||
// on the new picker to pick new SubConn.
|
||||
UpdateBalancerState(s connectivity.State, p Picker)
|
||||
|
||||
// Target returns the dial target for this ClientConn.
|
||||
Target() string
|
||||
}
|
||||
|
||||
// BuildOptions contains additional information for Build.
|
||||
type BuildOptions struct {
|
||||
// DialCreds is the transport credential the Balancer implementation can
|
||||
// use to dial to a remote load balancer server. The Balancer implementations
|
||||
// can ignore this if it does not need to talk to another party securely.
|
||||
DialCreds credentials.TransportCredentials
|
||||
// Dialer is the custom dialer the Balancer implementation can use to dial
|
||||
// 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)
|
||||
}
|
||||
|
||||
// Builder creates a balancer.
|
||||
type Builder interface {
|
||||
// Build creates a new balancer with the ClientConn.
|
||||
Build(cc ClientConn, opts BuildOptions) Balancer
|
||||
// Name returns the name of balancers built by this builder.
|
||||
// It will be used to pick balancers (for example in service config).
|
||||
Name() string
|
||||
}
|
||||
|
||||
// PickOptions contains addition information for the Pick operation.
|
||||
type PickOptions struct{}
|
||||
|
||||
// DoneInfo contains additional information for done.
|
||||
type DoneInfo struct {
|
||||
// Err is the rpc error the RPC finished with. It could be nil.
|
||||
Err error
|
||||
}
|
||||
|
||||
var (
|
||||
// ErrNoSubConnAvailable indicates no SubConn is available for pick().
|
||||
// gRPC will block the RPC until a new picker is available via UpdateBalancerState().
|
||||
ErrNoSubConnAvailable = errors.New("no SubConn is available")
|
||||
// ErrTransientFailure indicates all SubConns are in TransientFailure.
|
||||
// WaitForReady RPCs will block, non-WaitForReady RPCs will fail.
|
||||
ErrTransientFailure = errors.New("all SubConns are in TransientFailure")
|
||||
)
|
||||
|
||||
// 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
|
||||
// internal state has changed.
|
||||
//
|
||||
// The pickers used by gRPC can be updated by ClientConn.UpdateBalancerState().
|
||||
type Picker interface {
|
||||
// Pick returns the SubConn to be used to send the RPC.
|
||||
// The returned SubConn must be one returned by NewSubConn().
|
||||
//
|
||||
// This functions is expected to return:
|
||||
// - a SubConn that is known to be READY;
|
||||
// - ErrNoSubConnAvailable if no SubConn is available, but progress is being
|
||||
// made (for example, some SubConn is in CONNECTING mode);
|
||||
// - other errors if no active connecting is happening (for example, all SubConn
|
||||
// are in TRANSIENT_FAILURE mode).
|
||||
//
|
||||
// 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.
|
||||
//
|
||||
// If the returned error is not nil:
|
||||
// - If the error is ErrNoSubConnAvailable, gRPC will block until UpdateBalancerState()
|
||||
// - If the error is ErrTransientFailure:
|
||||
// - If the RPC is wait-for-ready, gRPC will block until UpdateBalancerState()
|
||||
// is called to pick again;
|
||||
// - Otherwise, RPC will fail with unavailable error.
|
||||
// - Else (error is other non-nil error):
|
||||
// - The RPC will fail with unavailable error.
|
||||
//
|
||||
// The returned done() function will be called once the rpc has finished, with the
|
||||
// final status of that RPC.
|
||||
// done may be nil if balancer doesn't care about the RPC status.
|
||||
Pick(ctx context.Context, opts PickOptions) (conn SubConn, done func(DoneInfo), err error)
|
||||
}
|
||||
|
||||
// Balancer takes input from gRPC, manages SubConns, and collects and aggregates
|
||||
// the connectivity states.
|
||||
//
|
||||
// It also generates and updates the Picker used by gRPC to pick SubConns for RPCs.
|
||||
//
|
||||
// HandleSubConnectionStateChange, HandleResolvedAddrs and Close are guaranteed
|
||||
// to be called synchronously from the same goroutine.
|
||||
// There's no guarantee on picker.Pick, it may be called anytime.
|
||||
type Balancer interface {
|
||||
// HandleSubConnStateChange is called by gRPC when the connectivity state
|
||||
// of sc has changed.
|
||||
// Balancer is expected to aggregate all the state of SubConn and report
|
||||
// that back to gRPC.
|
||||
// Balancer should also generate and update Pickers when its internal state has
|
||||
// been changed by the new state.
|
||||
HandleSubConnStateChange(sc SubConn, state connectivity.State)
|
||||
// HandleResolvedAddrs is called by gRPC to send updated resolved addresses to
|
||||
// balancers.
|
||||
// Balancer can create new SubConn or remove SubConn with the addresses.
|
||||
// An empty address slice and a non-nil error will be passed if the resolver returns
|
||||
// non-nil error to gRPC.
|
||||
HandleResolvedAddrs([]resolver.Address, error)
|
||||
// Close closes the balancer. The balancer is not required to call
|
||||
// ClientConn.RemoveSubConn for its existing SubConns.
|
||||
Close()
|
||||
}
|
248
cmd/vendor/google.golang.org/grpc/balancer_conn_wrappers.go
generated
vendored
Normal file
248
cmd/vendor/google.golang.org/grpc/balancer_conn_wrappers.go
generated
vendored
Normal file
@ -0,0 +1,248 @@
|
||||
/*
|
||||
*
|
||||
* 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"
|
||||
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
// scStateUpdate contains the subConn and the new state it changed to.
|
||||
type scStateUpdate struct {
|
||||
sc balancer.SubConn
|
||||
state connectivity.State
|
||||
}
|
||||
|
||||
// scStateUpdateBuffer is an unbounded channel for scStateChangeTuple.
|
||||
// TODO make a general purpose buffer that uses interface{}.
|
||||
type scStateUpdateBuffer struct {
|
||||
c chan *scStateUpdate
|
||||
mu sync.Mutex
|
||||
backlog []*scStateUpdate
|
||||
}
|
||||
|
||||
func newSCStateUpdateBuffer() *scStateUpdateBuffer {
|
||||
return &scStateUpdateBuffer{
|
||||
c: make(chan *scStateUpdate, 1),
|
||||
}
|
||||
}
|
||||
|
||||
func (b *scStateUpdateBuffer) put(t *scStateUpdate) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if len(b.backlog) == 0 {
|
||||
select {
|
||||
case b.c <- t:
|
||||
return
|
||||
default:
|
||||
}
|
||||
}
|
||||
b.backlog = append(b.backlog, t)
|
||||
}
|
||||
|
||||
func (b *scStateUpdateBuffer) load() {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if len(b.backlog) > 0 {
|
||||
select {
|
||||
case b.c <- b.backlog[0]:
|
||||
b.backlog[0] = nil
|
||||
b.backlog = b.backlog[1:]
|
||||
default:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// get returns the channel that receives a recvMsg in the buffer.
|
||||
//
|
||||
// Upon receiving, the caller should call load to send another
|
||||
// scStateChangeTuple onto the channel if there is any.
|
||||
func (b *scStateUpdateBuffer) get() <-chan *scStateUpdate {
|
||||
return b.c
|
||||
}
|
||||
|
||||
// resolverUpdate contains the new resolved addresses or error if there's
|
||||
// any.
|
||||
type resolverUpdate struct {
|
||||
addrs []resolver.Address
|
||||
err error
|
||||
}
|
||||
|
||||
// ccBalancerWrapper is a wrapper on top of cc for balancers.
|
||||
// It implements balancer.ClientConn interface.
|
||||
type ccBalancerWrapper struct {
|
||||
cc *ClientConn
|
||||
balancer balancer.Balancer
|
||||
stateChangeQueue *scStateUpdateBuffer
|
||||
resolverUpdateCh chan *resolverUpdate
|
||||
done chan struct{}
|
||||
}
|
||||
|
||||
func newCCBalancerWrapper(cc *ClientConn, b balancer.Builder, bopts balancer.BuildOptions) *ccBalancerWrapper {
|
||||
ccb := &ccBalancerWrapper{
|
||||
cc: cc,
|
||||
stateChangeQueue: newSCStateUpdateBuffer(),
|
||||
resolverUpdateCh: make(chan *resolverUpdate, 1),
|
||||
done: make(chan struct{}),
|
||||
}
|
||||
go ccb.watcher()
|
||||
ccb.balancer = b.Build(ccb, bopts)
|
||||
return ccb
|
||||
}
|
||||
|
||||
// watcher balancer functions sequencially, so the balancer can be implemeneted
|
||||
// lock-free.
|
||||
func (ccb *ccBalancerWrapper) watcher() {
|
||||
for {
|
||||
select {
|
||||
case t := <-ccb.stateChangeQueue.get():
|
||||
ccb.stateChangeQueue.load()
|
||||
ccb.balancer.HandleSubConnStateChange(t.sc, t.state)
|
||||
case t := <-ccb.resolverUpdateCh:
|
||||
ccb.balancer.HandleResolvedAddrs(t.addrs, t.err)
|
||||
case <-ccb.done:
|
||||
}
|
||||
|
||||
select {
|
||||
case <-ccb.done:
|
||||
ccb.balancer.Close()
|
||||
return
|
||||
default:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) close() {
|
||||
close(ccb.done)
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) handleSubConnStateChange(sc balancer.SubConn, s connectivity.State) {
|
||||
// When updating addresses for a SubConn, if the address in use is not in
|
||||
// the new addresses, the old ac will be tearDown() and a new ac will be
|
||||
// created. tearDown() generates a state change with Shutdown state, we
|
||||
// don't want the balancer to receive this state change. So before
|
||||
// tearDown() on the old ac, ac.acbw (acWrapper) will be set to nil, and
|
||||
// this function will be called with (nil, Shutdown). We don't need to call
|
||||
// balancer method in this case.
|
||||
if sc == nil {
|
||||
return
|
||||
}
|
||||
ccb.stateChangeQueue.put(&scStateUpdate{
|
||||
sc: sc,
|
||||
state: s,
|
||||
})
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) handleResolvedAddrs(addrs []resolver.Address, err error) {
|
||||
select {
|
||||
case <-ccb.resolverUpdateCh:
|
||||
default:
|
||||
}
|
||||
ccb.resolverUpdateCh <- &resolverUpdate{
|
||||
addrs: addrs,
|
||||
err: err,
|
||||
}
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) {
|
||||
grpclog.Infof("ccBalancerWrapper: new subconn: %v", addrs)
|
||||
ac, err := ccb.cc.newAddrConn(addrs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
acbw := &acBalancerWrapper{ac: ac}
|
||||
ac.acbw = acbw
|
||||
return acbw, nil
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) RemoveSubConn(sc balancer.SubConn) {
|
||||
grpclog.Infof("ccBalancerWrapper: removing subconn")
|
||||
acbw, ok := sc.(*acBalancerWrapper)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
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.cc.csMgr.updateState(s)
|
||||
ccb.cc.blockingpicker.updatePicker(p)
|
||||
}
|
||||
|
||||
func (ccb *ccBalancerWrapper) Target() string {
|
||||
return ccb.cc.target
|
||||
}
|
||||
|
||||
// acBalancerWrapper is a wrapper on top of ac for balancers.
|
||||
// It implements balancer.SubConn interface.
|
||||
type acBalancerWrapper struct {
|
||||
mu sync.Mutex
|
||||
ac *addrConn
|
||||
}
|
||||
|
||||
func (acbw *acBalancerWrapper) UpdateAddresses(addrs []resolver.Address) {
|
||||
grpclog.Infof("acBalancerWrapper: UpdateAddresses called with %v", addrs)
|
||||
acbw.mu.Lock()
|
||||
defer acbw.mu.Unlock()
|
||||
if !acbw.ac.tryUpdateAddrs(addrs) {
|
||||
cc := acbw.ac.cc
|
||||
acbw.ac.mu.Lock()
|
||||
// Set old ac.acbw to nil so the Shutdown state update will be ignored
|
||||
// by balancer.
|
||||
//
|
||||
// TODO(bar) the state transition could be wrong when tearDown() old ac
|
||||
// and creating new ac, fix the transition.
|
||||
acbw.ac.acbw = nil
|
||||
acbw.ac.mu.Unlock()
|
||||
acState := acbw.ac.getState()
|
||||
acbw.ac.tearDown(errConnDrain)
|
||||
|
||||
if acState == connectivity.Shutdown {
|
||||
return
|
||||
}
|
||||
|
||||
ac, err := cc.newAddrConn(addrs)
|
||||
if err != nil {
|
||||
grpclog.Warningf("acBalancerWrapper: UpdateAddresses: failed to newAddrConn: %v", err)
|
||||
return
|
||||
}
|
||||
acbw.ac = ac
|
||||
ac.acbw = acbw
|
||||
if acState != connectivity.Idle {
|
||||
ac.connect(false)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (acbw *acBalancerWrapper) Connect() {
|
||||
acbw.mu.Lock()
|
||||
defer acbw.mu.Unlock()
|
||||
acbw.ac.connect(false)
|
||||
}
|
||||
|
||||
func (acbw *acBalancerWrapper) getAddrConn() *addrConn {
|
||||
acbw.mu.Lock()
|
||||
defer acbw.mu.Unlock()
|
||||
return acbw.ac
|
||||
}
|
367
cmd/vendor/google.golang.org/grpc/balancer_v1_wrapper.go
generated
vendored
Normal file
367
cmd/vendor/google.golang.org/grpc/balancer_v1_wrapper.go
generated
vendored
Normal file
@ -0,0 +1,367 @@
|
||||
/*
|
||||
*
|
||||
* 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"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
type balancerWrapperBuilder struct {
|
||||
b Balancer // The v1 balancer.
|
||||
}
|
||||
|
||||
func (bwb *balancerWrapperBuilder) Build(cc balancer.ClientConn, opts balancer.BuildOptions) balancer.Balancer {
|
||||
bwb.b.Start(cc.Target(), 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,
|
||||
}
|
||||
cc.UpdateBalancerState(connectivity.Idle, bw)
|
||||
go bw.lbWatcher()
|
||||
return bw
|
||||
}
|
||||
|
||||
func (bwb *balancerWrapperBuilder) Name() string {
|
||||
return "wrapper"
|
||||
}
|
||||
|
||||
type scState struct {
|
||||
addr Address // The v1 address type.
|
||||
s connectivity.State
|
||||
down func(error)
|
||||
}
|
||||
|
||||
type balancerWrapper struct {
|
||||
balancer Balancer // The v1 balancer.
|
||||
pickfirst bool
|
||||
|
||||
cc balancer.ClientConn
|
||||
|
||||
// To aggregate the connectivity state.
|
||||
csEvltr *connectivityStateEvaluator
|
||||
state connectivity.State
|
||||
|
||||
mu sync.Mutex
|
||||
conns map[resolver.Address]balancer.SubConn
|
||||
connSt map[balancer.SubConn]*scState
|
||||
// This channel is closed when handling the first resolver result.
|
||||
// lbWatcher blocks until this is closed, to avoid race between
|
||||
// - NewSubConn is created, cc wants to notify balancer of state changes;
|
||||
// - Build hasn't return, cc doesn't have access to balancer.
|
||||
startCh chan struct{}
|
||||
}
|
||||
|
||||
// lbWatcher watches the Notify channel of the balancer and manages
|
||||
// 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(),
|
||||
Type: resolver.Backend,
|
||||
}
|
||||
sc, err := bw.cc.NewSubConn([]resolver.Address{a}, balancer.NewSubConnOptions{})
|
||||
if err != nil {
|
||||
grpclog.Warningf("Error creating connection to %v. Err: %v", a, err)
|
||||
} else {
|
||||
bw.mu.Lock()
|
||||
bw.conns[a] = sc
|
||||
bw.connSt[sc] = &scState{
|
||||
addr: Address{Addr: bw.cc.Target()},
|
||||
s: connectivity.Idle,
|
||||
}
|
||||
bw.mu.Unlock()
|
||||
sc.Connect()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
for addrs := range notifyCh {
|
||||
grpclog.Infof("balancerWrapper: got update addr from Notify: %v\n", addrs)
|
||||
if bw.pickfirst {
|
||||
var (
|
||||
oldA resolver.Address
|
||||
oldSC balancer.SubConn
|
||||
)
|
||||
bw.mu.Lock()
|
||||
for oldA, oldSC = range bw.conns {
|
||||
break
|
||||
}
|
||||
bw.mu.Unlock()
|
||||
if len(addrs) <= 0 {
|
||||
if oldSC != nil {
|
||||
// Teardown old sc.
|
||||
bw.mu.Lock()
|
||||
delete(bw.conns, oldA)
|
||||
delete(bw.connSt, oldSC)
|
||||
bw.mu.Unlock()
|
||||
bw.cc.RemoveSubConn(oldSC)
|
||||
}
|
||||
continue
|
||||
}
|
||||
|
||||
var newAddrs []resolver.Address
|
||||
for _, a := range addrs {
|
||||
newAddr := resolver.Address{
|
||||
Addr: a.Addr,
|
||||
Type: resolver.Backend, // All addresses from balancer are all backends.
|
||||
ServerName: "",
|
||||
Metadata: a.Metadata,
|
||||
}
|
||||
newAddrs = append(newAddrs, newAddr)
|
||||
}
|
||||
if oldSC == nil {
|
||||
// Create new sc.
|
||||
sc, err := bw.cc.NewSubConn(newAddrs, balancer.NewSubConnOptions{})
|
||||
if err != nil {
|
||||
grpclog.Warningf("Error creating connection to %v. Err: %v", newAddrs, err)
|
||||
} else {
|
||||
bw.mu.Lock()
|
||||
// For pickfirst, there should be only one SubConn, so the
|
||||
// address doesn't matter. All states updating (up and down)
|
||||
// and picking should all happen on that only SubConn.
|
||||
bw.conns[resolver.Address{}] = sc
|
||||
bw.connSt[sc] = &scState{
|
||||
addr: addrs[0], // Use the first address.
|
||||
s: connectivity.Idle,
|
||||
}
|
||||
bw.mu.Unlock()
|
||||
sc.Connect()
|
||||
}
|
||||
} else {
|
||||
oldSC.UpdateAddresses(newAddrs)
|
||||
bw.mu.Lock()
|
||||
bw.connSt[oldSC].addr = addrs[0]
|
||||
bw.mu.Unlock()
|
||||
}
|
||||
} else {
|
||||
var (
|
||||
add []resolver.Address // Addresses need to setup connections.
|
||||
del []balancer.SubConn // Connections need to tear down.
|
||||
)
|
||||
resAddrs := make(map[resolver.Address]Address)
|
||||
for _, a := range addrs {
|
||||
resAddrs[resolver.Address{
|
||||
Addr: a.Addr,
|
||||
Type: resolver.Backend, // All addresses from balancer are all backends.
|
||||
ServerName: "",
|
||||
Metadata: a.Metadata,
|
||||
}] = a
|
||||
}
|
||||
bw.mu.Lock()
|
||||
for a := range resAddrs {
|
||||
if _, ok := bw.conns[a]; !ok {
|
||||
add = append(add, a)
|
||||
}
|
||||
}
|
||||
for a, c := range bw.conns {
|
||||
if _, ok := resAddrs[a]; !ok {
|
||||
del = append(del, c)
|
||||
delete(bw.conns, a)
|
||||
// Keep the state of this sc in bw.connSt until its state becomes Shutdown.
|
||||
}
|
||||
}
|
||||
bw.mu.Unlock()
|
||||
for _, a := range add {
|
||||
sc, err := bw.cc.NewSubConn([]resolver.Address{a}, balancer.NewSubConnOptions{})
|
||||
if err != nil {
|
||||
grpclog.Warningf("Error creating connection to %v. Err: %v", a, err)
|
||||
} else {
|
||||
bw.mu.Lock()
|
||||
bw.conns[a] = sc
|
||||
bw.connSt[sc] = &scState{
|
||||
addr: resAddrs[a],
|
||||
s: connectivity.Idle,
|
||||
}
|
||||
bw.mu.Unlock()
|
||||
sc.Connect()
|
||||
}
|
||||
}
|
||||
for _, c := range del {
|
||||
bw.cc.RemoveSubConn(c)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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]
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
if s == connectivity.Idle {
|
||||
sc.Connect()
|
||||
}
|
||||
oldS := scSt.s
|
||||
scSt.s = s
|
||||
if oldS != connectivity.Ready && s == connectivity.Ready {
|
||||
scSt.down = bw.balancer.Up(scSt.addr)
|
||||
} else if oldS == connectivity.Ready && s != connectivity.Ready {
|
||||
if scSt.down != nil {
|
||||
scSt.down(errConnClosing)
|
||||
}
|
||||
}
|
||||
sa := bw.csEvltr.recordTransition(oldS, s)
|
||||
if bw.state != sa {
|
||||
bw.state = sa
|
||||
}
|
||||
bw.cc.UpdateBalancerState(bw.state, bw)
|
||||
if s == connectivity.Shutdown {
|
||||
// Remove state for this sc.
|
||||
delete(bw.connSt, sc)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (bw *balancerWrapper) HandleResolvedAddrs([]resolver.Address, error) {
|
||||
bw.mu.Lock()
|
||||
defer bw.mu.Unlock()
|
||||
select {
|
||||
case <-bw.startCh:
|
||||
default:
|
||||
close(bw.startCh)
|
||||
}
|
||||
// There should be a resolver inside the balancer.
|
||||
// All updates here, if any, are ignored.
|
||||
return
|
||||
}
|
||||
|
||||
func (bw *balancerWrapper) Close() {
|
||||
bw.mu.Lock()
|
||||
defer bw.mu.Unlock()
|
||||
select {
|
||||
case <-bw.startCh:
|
||||
default:
|
||||
close(bw.startCh)
|
||||
}
|
||||
bw.balancer.Close()
|
||||
return
|
||||
}
|
||||
|
||||
// The picker is the balancerWrapper itself.
|
||||
// Pick should never return ErrNoSubConnAvailable.
|
||||
// It either blocks or returns error, consistent with v1 balancer Get().
|
||||
func (bw *balancerWrapper) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
||||
failfast := true // Default failfast is true.
|
||||
if ss, ok := rpcInfoFromContext(ctx); ok {
|
||||
failfast = ss.failfast
|
||||
}
|
||||
a, p, err := bw.balancer.Get(ctx, BalancerGetOptions{BlockingWait: !failfast})
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
var done func(balancer.DoneInfo)
|
||||
if p != nil {
|
||||
done = func(i balancer.DoneInfo) { p() }
|
||||
}
|
||||
var sc balancer.SubConn
|
||||
bw.mu.Lock()
|
||||
defer bw.mu.Unlock()
|
||||
if bw.pickfirst {
|
||||
// Get the first sc in conns.
|
||||
for _, sc = range bw.conns {
|
||||
break
|
||||
}
|
||||
} else {
|
||||
var ok bool
|
||||
sc, ok = bw.conns[resolver.Address{
|
||||
Addr: a.Addr,
|
||||
Type: resolver.Backend,
|
||||
ServerName: "",
|
||||
Metadata: a.Metadata,
|
||||
}]
|
||||
if !ok && failfast {
|
||||
return nil, nil, 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 sc, done, nil
|
||||
}
|
||||
|
||||
// connectivityStateEvaluator gets updated by addrConns when their
|
||||
// states transition, based on which it evaluates the state of
|
||||
// ClientConn.
|
||||
type connectivityStateEvaluator struct {
|
||||
mu sync.Mutex
|
||||
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.
|
||||
// TODO Note that in later releases, a ClientConn with no activity will be put into an Idle state.
|
||||
func (cse *connectivityStateEvaluator) recordTransition(oldState, newState connectivity.State) connectivity.State {
|
||||
cse.mu.Lock()
|
||||
defer cse.mu.Unlock()
|
||||
|
||||
// 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
|
||||
}
|
38
cmd/vendor/google.golang.org/grpc/call.go
generated
vendored
38
cmd/vendor/google.golang.org/grpc/call.go
generated
vendored
@ -25,6 +25,7 @@ import (
|
||||
|
||||
"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"
|
||||
@ -135,7 +136,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
}
|
||||
|
||||
func invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) (e error) {
|
||||
c := defaultCallInfo
|
||||
c := defaultCallInfo()
|
||||
mc := cc.GetMethodConfig(method)
|
||||
if mc.WaitForReady != nil {
|
||||
c.failFast = !*mc.WaitForReady
|
||||
@ -149,13 +150,13 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
|
||||
opts = append(cc.dopts.callOptions, opts...)
|
||||
for _, o := range opts {
|
||||
if err := o.before(&c); err != nil {
|
||||
if err := o.before(c); err != nil {
|
||||
return toRPCErr(err)
|
||||
}
|
||||
}
|
||||
defer func() {
|
||||
for _, o := range opts {
|
||||
o.after(&c)
|
||||
o.after(c)
|
||||
}
|
||||
}()
|
||||
|
||||
@ -178,7 +179,7 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
}
|
||||
}()
|
||||
}
|
||||
ctx = newContextWithRPCInfo(ctx)
|
||||
ctx = newContextWithRPCInfo(ctx, c.failFast)
|
||||
sh := cc.dopts.copts.StatsHandler
|
||||
if sh != nil {
|
||||
ctx = sh.TagRPC(ctx, &stats.RPCTagInfo{FullMethodName: method, FailFast: c.failFast})
|
||||
@ -206,9 +207,9 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
err error
|
||||
t transport.ClientTransport
|
||||
stream *transport.Stream
|
||||
// Record the put handler from Balancer.Get(...). It is called once the
|
||||
// Record the done handler from Balancer.Get(...). It is called once the
|
||||
// RPC has completed or failed.
|
||||
put func()
|
||||
done func(balancer.DoneInfo)
|
||||
)
|
||||
// TODO(zhaoq): Need a formal spec of fail-fast.
|
||||
callHdr := &transport.CallHdr{
|
||||
@ -222,10 +223,7 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
callHdr.Creds = c.creds
|
||||
}
|
||||
|
||||
gopts := BalancerGetOptions{
|
||||
BlockingWait: !c.failFast,
|
||||
}
|
||||
t, put, err = cc.getTransport(ctx, gopts)
|
||||
t, done, err = cc.getTransport(ctx, c.failFast)
|
||||
if err != nil {
|
||||
// TODO(zhaoq): Probably revisit the error handling.
|
||||
if _, ok := status.FromError(err); ok {
|
||||
@ -245,14 +243,14 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
}
|
||||
stream, err = t.NewStream(ctx, callHdr)
|
||||
if err != nil {
|
||||
if put != 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})
|
||||
}
|
||||
put()
|
||||
done(balancer.DoneInfo{Err: err})
|
||||
}
|
||||
if _, ok := err.(transport.ConnectionError); (ok || err == transport.ErrStreamDrain) && !c.failFast {
|
||||
continue
|
||||
@ -262,14 +260,14 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
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)
|
||||
err = sendRequest(ctx, cc.dopts, cc.dopts.cp, c, callHdr, stream, t, args, topts)
|
||||
if err != nil {
|
||||
if put != nil {
|
||||
if done != nil {
|
||||
updateRPCInfoInContext(ctx, rpcInfo{
|
||||
bytesSent: stream.BytesSent(),
|
||||
bytesReceived: stream.BytesReceived(),
|
||||
})
|
||||
put()
|
||||
done(balancer.DoneInfo{Err: err})
|
||||
}
|
||||
// Retry a non-failfast RPC when
|
||||
// i) there is a connection error; or
|
||||
@ -279,14 +277,14 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
}
|
||||
return toRPCErr(err)
|
||||
}
|
||||
err = recvResponse(ctx, cc.dopts, t, &c, stream, reply)
|
||||
err = recvResponse(ctx, cc.dopts, t, c, stream, reply)
|
||||
if err != nil {
|
||||
if put != nil {
|
||||
if done != nil {
|
||||
updateRPCInfoInContext(ctx, rpcInfo{
|
||||
bytesSent: stream.BytesSent(),
|
||||
bytesReceived: stream.BytesReceived(),
|
||||
})
|
||||
put()
|
||||
done(balancer.DoneInfo{Err: err})
|
||||
}
|
||||
if _, ok := err.(transport.ConnectionError); (ok || err == transport.ErrStreamDrain) && !c.failFast {
|
||||
continue
|
||||
@ -297,12 +295,12 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli
|
||||
c.traceInfo.tr.LazyLog(&payload{sent: false, msg: reply}, true)
|
||||
}
|
||||
t.CloseStream(stream, nil)
|
||||
if put != nil {
|
||||
if done != nil {
|
||||
updateRPCInfoInContext(ctx, rpcInfo{
|
||||
bytesSent: stream.BytesSent(),
|
||||
bytesReceived: stream.BytesReceived(),
|
||||
})
|
||||
put()
|
||||
done(balancer.DoneInfo{Err: err})
|
||||
}
|
||||
return stream.Status().Err()
|
||||
}
|
||||
|
766
cmd/vendor/google.golang.org/grpc/clientconn.go
generated
vendored
766
cmd/vendor/google.golang.org/grpc/clientconn.go
generated
vendored
File diff suppressed because it is too large
Load Diff
98
cmd/vendor/google.golang.org/grpc/go16.go
generated
vendored
98
cmd/vendor/google.golang.org/grpc/go16.go
generated
vendored
@ -1,98 +0,0 @@
|
||||
// +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"
|
||||
"os"
|
||||
|
||||
"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 _, 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())
|
||||
case ErrClientConnClosing:
|
||||
return status.Error(codes.FailedPrecondition, err.Error())
|
||||
}
|
||||
}
|
||||
return status.Error(codes.Unknown, err.Error())
|
||||
}
|
||||
|
||||
// convertCode converts a standard Go error into its canonical code. Note that
|
||||
// this is only used to translate the error returned by the server applications.
|
||||
func convertCode(err error) codes.Code {
|
||||
switch err {
|
||||
case nil:
|
||||
return codes.OK
|
||||
case io.EOF:
|
||||
return codes.OutOfRange
|
||||
case io.ErrClosedPipe, io.ErrNoProgress, io.ErrShortBuffer, io.ErrShortWrite, io.ErrUnexpectedEOF:
|
||||
return codes.FailedPrecondition
|
||||
case os.ErrInvalid:
|
||||
return codes.InvalidArgument
|
||||
case context.Canceled:
|
||||
return codes.Canceled
|
||||
case context.DeadlineExceeded:
|
||||
return codes.DeadlineExceeded
|
||||
}
|
||||
switch {
|
||||
case os.IsExist(err):
|
||||
return codes.AlreadyExists
|
||||
case os.IsNotExist(err):
|
||||
return codes.NotFound
|
||||
case os.IsPermission(err):
|
||||
return codes.PermissionDenied
|
||||
}
|
||||
return codes.Unknown
|
||||
}
|
98
cmd/vendor/google.golang.org/grpc/go17.go
generated
vendored
98
cmd/vendor/google.golang.org/grpc/go17.go
generated
vendored
@ -1,98 +0,0 @@
|
||||
// +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"
|
||||
"io"
|
||||
"net"
|
||||
"net/http"
|
||||
"os"
|
||||
|
||||
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 err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// toRPCErr converts an error into an error from the status package.
|
||||
func toRPCErr(err error) error {
|
||||
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())
|
||||
case ErrClientConnClosing:
|
||||
return status.Error(codes.FailedPrecondition, err.Error())
|
||||
}
|
||||
}
|
||||
return status.Error(codes.Unknown, err.Error())
|
||||
}
|
||||
|
||||
// convertCode converts a standard Go error into its canonical code. Note that
|
||||
// this is only used to translate the error returned by the server applications.
|
||||
func convertCode(err error) codes.Code {
|
||||
switch err {
|
||||
case nil:
|
||||
return codes.OK
|
||||
case io.EOF:
|
||||
return codes.OutOfRange
|
||||
case io.ErrClosedPipe, io.ErrNoProgress, io.ErrShortBuffer, io.ErrShortWrite, io.ErrUnexpectedEOF:
|
||||
return codes.FailedPrecondition
|
||||
case os.ErrInvalid:
|
||||
return codes.InvalidArgument
|
||||
case context.Canceled, netctx.Canceled:
|
||||
return codes.Canceled
|
||||
case context.DeadlineExceeded, netctx.DeadlineExceeded:
|
||||
return codes.DeadlineExceeded
|
||||
}
|
||||
switch {
|
||||
case os.IsExist(err):
|
||||
return codes.AlreadyExists
|
||||
case os.IsNotExist(err):
|
||||
return codes.NotFound
|
||||
case os.IsPermission(err):
|
||||
return codes.PermissionDenied
|
||||
}
|
||||
return codes.Unknown
|
||||
}
|
42
cmd/vendor/google.golang.org/grpc/grpclb.go
generated
vendored
42
cmd/vendor/google.golang.org/grpc/grpclb.go
generated
vendored
@ -73,7 +73,7 @@ func (x *balanceLoadClientStream) Recv() (*lbmpb.LoadBalanceResponse, error) {
|
||||
|
||||
// NewGRPCLBBalancer creates a grpclb load balancer.
|
||||
func NewGRPCLBBalancer(r naming.Resolver) Balancer {
|
||||
return &balancer{
|
||||
return &grpclbBalancer{
|
||||
r: r,
|
||||
}
|
||||
}
|
||||
@ -96,7 +96,7 @@ type grpclbAddrInfo struct {
|
||||
dropForLoadBalancing bool
|
||||
}
|
||||
|
||||
type balancer struct {
|
||||
type grpclbBalancer struct {
|
||||
r naming.Resolver
|
||||
target string
|
||||
mu sync.Mutex
|
||||
@ -113,7 +113,7 @@ type balancer struct {
|
||||
clientStats lbmpb.ClientStats
|
||||
}
|
||||
|
||||
func (b *balancer) watchAddrUpdates(w naming.Watcher, ch chan []remoteBalancerInfo) error {
|
||||
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)
|
||||
@ -187,7 +187,7 @@ func convertDuration(d *lbmpb.Duration) time.Duration {
|
||||
return time.Duration(d.Seconds)*time.Second + time.Duration(d.Nanos)*time.Nanosecond
|
||||
}
|
||||
|
||||
func (b *balancer) processServerList(l *lbmpb.ServerList, seq int) {
|
||||
func (b *grpclbBalancer) processServerList(l *lbmpb.ServerList, seq int) {
|
||||
if l == nil {
|
||||
return
|
||||
}
|
||||
@ -230,7 +230,7 @@ func (b *balancer) processServerList(l *lbmpb.ServerList, seq int) {
|
||||
return
|
||||
}
|
||||
|
||||
func (b *balancer) sendLoadReport(s *balanceLoadClientStream, interval time.Duration, done <-chan struct{}) {
|
||||
func (b *grpclbBalancer) sendLoadReport(s *balanceLoadClientStream, interval time.Duration, done <-chan struct{}) {
|
||||
ticker := time.NewTicker(interval)
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
@ -259,7 +259,7 @@ func (b *balancer) sendLoadReport(s *balanceLoadClientStream, interval time.Dura
|
||||
}
|
||||
}
|
||||
|
||||
func (b *balancer) callRemoteBalancer(lbc *loadBalancerClient, seq int) (retry bool) {
|
||||
func (b *grpclbBalancer) callRemoteBalancer(lbc *loadBalancerClient, seq int) (retry bool) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
stream, err := lbc.BalanceLoad(ctx)
|
||||
@ -332,7 +332,7 @@ func (b *balancer) callRemoteBalancer(lbc *loadBalancerClient, seq int) (retry b
|
||||
return true
|
||||
}
|
||||
|
||||
func (b *balancer) Start(target string, config BalancerConfig) error {
|
||||
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 {
|
||||
@ -461,8 +461,11 @@ func (b *balancer) Start(target string, config BalancerConfig) error {
|
||||
// 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{})
|
||||
cc, err = Dial(rb.addr, dopts...)
|
||||
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)
|
||||
@ -488,7 +491,7 @@ func (b *balancer) Start(target string, config BalancerConfig) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *balancer) down(addr Address, err error) {
|
||||
func (b *grpclbBalancer) down(addr Address, err error) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
for _, a := range b.addrs {
|
||||
@ -499,7 +502,7 @@ func (b *balancer) down(addr Address, err error) {
|
||||
}
|
||||
}
|
||||
|
||||
func (b *balancer) Up(addr Address) func(error) {
|
||||
func (b *grpclbBalancer) Up(addr Address) func(error) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if b.done {
|
||||
@ -527,7 +530,7 @@ func (b *balancer) Up(addr Address) func(error) {
|
||||
}
|
||||
}
|
||||
|
||||
func (b *balancer) Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) {
|
||||
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 {
|
||||
@ -597,17 +600,10 @@ func (b *balancer) Get(ctx context.Context, opts BalancerGetOptions) (addr Addre
|
||||
}
|
||||
}
|
||||
if !opts.BlockingWait {
|
||||
if len(b.addrs) == 0 {
|
||||
b.clientStats.NumCallsFinished++
|
||||
b.clientStats.NumCallsFinishedWithClientFailedToSend++
|
||||
b.mu.Unlock()
|
||||
err = Errorf(codes.Unavailable, "there is no address available")
|
||||
return
|
||||
}
|
||||
// Returns the next addr on b.addrs for a failfast RPC.
|
||||
addr = b.addrs[b.next].addr
|
||||
b.next++
|
||||
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.
|
||||
@ -684,11 +680,11 @@ func (b *balancer) Get(ctx context.Context, opts BalancerGetOptions) (addr Addre
|
||||
}
|
||||
}
|
||||
|
||||
func (b *balancer) Notify() <-chan []Address {
|
||||
func (b *grpclbBalancer) Notify() <-chan []Address {
|
||||
return b.addrCh
|
||||
}
|
||||
|
||||
func (b *balancer) Close() error {
|
||||
func (b *grpclbBalancer) Close() error {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if b.done {
|
||||
|
2
cmd/vendor/google.golang.org/grpc/naming/go17.go
generated
vendored
2
cmd/vendor/google.golang.org/grpc/naming/go17.go
generated
vendored
@ -1,4 +1,4 @@
|
||||
// +build go1.6, !go1.8
|
||||
// +build go1.7, !go1.8
|
||||
|
||||
/*
|
||||
*
|
||||
|
141
cmd/vendor/google.golang.org/grpc/picker_wrapper.go
generated
vendored
Normal file
141
cmd/vendor/google.golang.org/grpc/picker_wrapper.go
generated
vendored
Normal file
@ -0,0 +1,141 @@
|
||||
/*
|
||||
*
|
||||
* 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"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/status"
|
||||
"google.golang.org/grpc/transport"
|
||||
)
|
||||
|
||||
// pickerWrapper is a wrapper of balancer.Picker. It blocks on certain pick
|
||||
// actions and unblock when there's a picker update.
|
||||
type pickerWrapper struct {
|
||||
mu sync.Mutex
|
||||
done bool
|
||||
blockingCh chan struct{}
|
||||
picker balancer.Picker
|
||||
}
|
||||
|
||||
func newPickerWrapper() *pickerWrapper {
|
||||
bp := &pickerWrapper{blockingCh: make(chan struct{})}
|
||||
return bp
|
||||
}
|
||||
|
||||
// updatePicker is called by UpdateBalancerState. It unblocks all blocked pick.
|
||||
func (bp *pickerWrapper) updatePicker(p balancer.Picker) {
|
||||
bp.mu.Lock()
|
||||
if bp.done {
|
||||
bp.mu.Unlock()
|
||||
return
|
||||
}
|
||||
bp.picker = p
|
||||
// bp.blockingCh should never be nil.
|
||||
close(bp.blockingCh)
|
||||
bp.blockingCh = make(chan struct{})
|
||||
bp.mu.Unlock()
|
||||
}
|
||||
|
||||
// pick returns the transport that will be used for the RPC.
|
||||
// It may block in the following cases:
|
||||
// - there's no picker
|
||||
// - the current picker returns ErrNoSubConnAvailable
|
||||
// - the current picker returns other errors and failfast is false.
|
||||
// - the subConn returned by the current picker is not READY
|
||||
// When one of these situations happens, pick blocks until the picker gets updated.
|
||||
func (bp *pickerWrapper) pick(ctx context.Context, failfast bool, opts balancer.PickOptions) (transport.ClientTransport, func(balancer.DoneInfo), error) {
|
||||
var (
|
||||
p balancer.Picker
|
||||
ch chan struct{}
|
||||
)
|
||||
|
||||
for {
|
||||
bp.mu.Lock()
|
||||
if bp.done {
|
||||
bp.mu.Unlock()
|
||||
return nil, nil, ErrClientConnClosing
|
||||
}
|
||||
|
||||
if bp.picker == nil {
|
||||
ch = bp.blockingCh
|
||||
}
|
||||
if ch == bp.blockingCh {
|
||||
// This could happen when either:
|
||||
// - bp.picker is nil (the previous if condition), or
|
||||
// - has called pick on the current picker.
|
||||
bp.mu.Unlock()
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, nil, ctx.Err()
|
||||
case <-ch:
|
||||
}
|
||||
continue
|
||||
}
|
||||
|
||||
ch = bp.blockingCh
|
||||
p = bp.picker
|
||||
bp.mu.Unlock()
|
||||
|
||||
subConn, put, err := p.Pick(ctx, opts)
|
||||
|
||||
if err != nil {
|
||||
switch err {
|
||||
case balancer.ErrNoSubConnAvailable:
|
||||
continue
|
||||
case balancer.ErrTransientFailure:
|
||||
if !failfast {
|
||||
continue
|
||||
}
|
||||
return nil, nil, status.Errorf(codes.Unavailable, "%v", err)
|
||||
default:
|
||||
// err is some other error.
|
||||
return nil, nil, toRPCErr(err)
|
||||
}
|
||||
}
|
||||
|
||||
acw, ok := subConn.(*acBalancerWrapper)
|
||||
if !ok {
|
||||
grpclog.Infof("subconn returned from pick is not *acBalancerWrapper")
|
||||
continue
|
||||
}
|
||||
if t, ok := acw.getAddrConn().getReadyTransport(); ok {
|
||||
return t, put, nil
|
||||
}
|
||||
grpclog.Infof("blockingPicker: the picked transport is not ready, loop back to repick")
|
||||
// If ok == false, ac.state is not READY.
|
||||
// A valid picker always returns READY subConn. This means the state of ac
|
||||
// just changed, and picker will be updated shortly.
|
||||
// continue back to the beginning of the for loop to repick.
|
||||
}
|
||||
}
|
||||
|
||||
func (bp *pickerWrapper) close() {
|
||||
bp.mu.Lock()
|
||||
defer bp.mu.Unlock()
|
||||
if bp.done {
|
||||
return
|
||||
}
|
||||
bp.done = true
|
||||
close(bp.blockingCh)
|
||||
}
|
95
cmd/vendor/google.golang.org/grpc/pickfirst.go
generated
vendored
Normal file
95
cmd/vendor/google.golang.org/grpc/pickfirst.go
generated
vendored
Normal file
@ -0,0 +1,95 @@
|
||||
/*
|
||||
*
|
||||
* 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 (
|
||||
"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"
|
||||
)
|
||||
|
||||
func newPickfirstBuilder() balancer.Builder {
|
||||
return &pickfirstBuilder{}
|
||||
}
|
||||
|
||||
type pickfirstBuilder struct{}
|
||||
|
||||
func (*pickfirstBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer {
|
||||
return &pickfirstBalancer{cc: cc}
|
||||
}
|
||||
|
||||
func (*pickfirstBuilder) Name() string {
|
||||
return "pickfirst"
|
||||
}
|
||||
|
||||
type pickfirstBalancer struct {
|
||||
cc balancer.ClientConn
|
||||
sc balancer.SubConn
|
||||
}
|
||||
|
||||
func (b *pickfirstBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) {
|
||||
if err != nil {
|
||||
grpclog.Infof("pickfirstBalancer: HandleResolvedAddrs called with error %v", err)
|
||||
return
|
||||
}
|
||||
if b.sc == nil {
|
||||
b.sc, err = b.cc.NewSubConn(addrs, balancer.NewSubConnOptions{})
|
||||
if err != nil {
|
||||
grpclog.Errorf("pickfirstBalancer: failed to NewSubConn: %v", err)
|
||||
return
|
||||
}
|
||||
b.cc.UpdateBalancerState(connectivity.Idle, &picker{sc: b.sc})
|
||||
} else {
|
||||
b.sc.UpdateAddresses(addrs)
|
||||
}
|
||||
}
|
||||
|
||||
func (b *pickfirstBalancer) HandleSubConnStateChange(sc balancer.SubConn, s connectivity.State) {
|
||||
grpclog.Infof("pickfirstBalancer: HandleSubConnStateChange: %p, %v", sc, s)
|
||||
if b.sc != sc || s == connectivity.Shutdown {
|
||||
b.sc = nil
|
||||
return
|
||||
}
|
||||
|
||||
switch s {
|
||||
case connectivity.Ready, connectivity.Idle:
|
||||
b.cc.UpdateBalancerState(s, &picker{sc: sc})
|
||||
case connectivity.Connecting:
|
||||
b.cc.UpdateBalancerState(s, &picker{err: balancer.ErrNoSubConnAvailable})
|
||||
case connectivity.TransientFailure:
|
||||
b.cc.UpdateBalancerState(s, &picker{err: balancer.ErrTransientFailure})
|
||||
}
|
||||
}
|
||||
|
||||
func (b *pickfirstBalancer) Close() {
|
||||
}
|
||||
|
||||
type picker struct {
|
||||
err error
|
||||
sc balancer.SubConn
|
||||
}
|
||||
|
||||
func (p *picker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) {
|
||||
if p.err != nil {
|
||||
return nil, nil, p.err
|
||||
}
|
||||
return p.sc, nil, nil
|
||||
}
|
3
cmd/vendor/google.golang.org/grpc/proxy.go
generated
vendored
3
cmd/vendor/google.golang.org/grpc/proxy.go
generated
vendored
@ -82,7 +82,8 @@ func doHTTPConnectHandshake(ctx context.Context, conn net.Conn, addr string) (_
|
||||
Header: map[string][]string{"User-Agent": {grpcUA}},
|
||||
})
|
||||
|
||||
if err := sendHTTPRequest(ctx, req, conn); err != nil {
|
||||
req = req.WithContext(ctx)
|
||||
if err := req.Write(conn); err != nil {
|
||||
return nil, fmt.Errorf("failed to write the HTTP request: %v", err)
|
||||
}
|
||||
|
||||
|
143
cmd/vendor/google.golang.org/grpc/resolver/resolver.go
generated
vendored
Normal file
143
cmd/vendor/google.golang.org/grpc/resolver/resolver.go
generated
vendored
Normal file
@ -0,0 +1,143 @@
|
||||
/*
|
||||
*
|
||||
* 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 resolver defines APIs for name resolution in gRPC.
|
||||
// All APIs in this package are experimental.
|
||||
package resolver
|
||||
|
||||
var (
|
||||
// m is a map from scheme to resolver builder.
|
||||
m = make(map[string]Builder)
|
||||
// defaultScheme is the default scheme to use.
|
||||
defaultScheme string
|
||||
)
|
||||
|
||||
// TODO(bar) install dns resolver in init(){}.
|
||||
|
||||
// Register registers the resolver builder to the resolver map.
|
||||
// b.Scheme will be used as the scheme registered with this builder.
|
||||
func Register(b Builder) {
|
||||
m[b.Scheme()] = b
|
||||
}
|
||||
|
||||
// Get returns the resolver builder registered with the given scheme.
|
||||
// If no builder is register with the scheme, the default scheme will
|
||||
// be used.
|
||||
// If the default scheme is not modified, "dns" will be the default
|
||||
// scheme, and the preinstalled dns resolver will be used.
|
||||
// If the default scheme is modified, and a resolver is registered with
|
||||
// the scheme, that resolver will be returned.
|
||||
// If the default scheme is modified, and no resolver is registered with
|
||||
// the scheme, nil will be returned.
|
||||
func Get(scheme string) Builder {
|
||||
if b, ok := m[scheme]; ok {
|
||||
return b
|
||||
}
|
||||
if b, ok := m[defaultScheme]; ok {
|
||||
return b
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetDefaultScheme sets the default scheme that will be used.
|
||||
// The default default scheme is "dns".
|
||||
func SetDefaultScheme(scheme string) {
|
||||
defaultScheme = scheme
|
||||
}
|
||||
|
||||
// AddressType indicates the address type returned by name resolution.
|
||||
type AddressType uint8
|
||||
|
||||
const (
|
||||
// Backend indicates the address is for a backend server.
|
||||
Backend AddressType = iota
|
||||
// GRPCLB indicates the address is for a grpclb load balancer.
|
||||
GRPCLB
|
||||
)
|
||||
|
||||
// Address represents a server the client connects to.
|
||||
// This is the EXPERIMENTAL API and may be changed or extended in the future.
|
||||
type Address struct {
|
||||
// Addr is the server address on which a connection will be established.
|
||||
Addr string
|
||||
// Type is the type of this address.
|
||||
Type AddressType
|
||||
// ServerName is the name of this address.
|
||||
// It's the name of the grpc load balancer, which will be used for authentication.
|
||||
ServerName string
|
||||
// Metadata is the information associated with Addr, which may be used
|
||||
// to make load balancing decision.
|
||||
Metadata interface{}
|
||||
}
|
||||
|
||||
// BuildOption includes additional information for the builder to create
|
||||
// the resolver.
|
||||
type BuildOption struct {
|
||||
}
|
||||
|
||||
// ClientConn contains the callbacks for resolver to notify any updates
|
||||
// to the gRPC ClientConn.
|
||||
type ClientConn interface {
|
||||
// NewAddress is called by resolver to notify ClientConn a new list
|
||||
// of resolved addresses.
|
||||
// The address list should be the complete list of resolved addresses.
|
||||
NewAddress(addresses []Address)
|
||||
// NewServiceConfig is called by resolver to notify ClientConn a new
|
||||
// service config. The service config should be provided as a json string.
|
||||
NewServiceConfig(serviceConfig string)
|
||||
}
|
||||
|
||||
// Target represents a target for gRPC, as specified in:
|
||||
// https://github.com/grpc/grpc/blob/master/doc/naming.md.
|
||||
type Target struct {
|
||||
Scheme string
|
||||
Authority string
|
||||
Endpoint string
|
||||
}
|
||||
|
||||
// Builder creates a resolver that will be used to watch name resolution updates.
|
||||
type Builder interface {
|
||||
// Build creates a new resolver for the given target.
|
||||
//
|
||||
// gRPC dial calls Build synchronously, and fails if the returned error is
|
||||
// not nil.
|
||||
Build(target Target, cc ClientConn, opts BuildOption) (Resolver, error)
|
||||
// Scheme returns the scheme supported by this resolver.
|
||||
// Scheme is defined at https://github.com/grpc/grpc/blob/master/doc/naming.md.
|
||||
Scheme() string
|
||||
}
|
||||
|
||||
// ResolveNowOption includes additional information for ResolveNow.
|
||||
type ResolveNowOption struct{}
|
||||
|
||||
// Resolver watches for the updates on the specified target.
|
||||
// Updates include address updates and service config updates.
|
||||
type Resolver interface {
|
||||
// ResolveNow will be called by gRPC to try to resolve the target name again.
|
||||
// It's just a hint, resolver can ignore this if it's not necessary.
|
||||
ResolveNow(ResolveNowOption)
|
||||
// Close closes the resolver.
|
||||
Close()
|
||||
}
|
||||
|
||||
// UnregisterForTesting removes the resolver builder with the given scheme from the
|
||||
// resolver map.
|
||||
// This function is for testing only.
|
||||
func UnregisterForTesting(scheme string) {
|
||||
delete(m, scheme)
|
||||
}
|
139
cmd/vendor/google.golang.org/grpc/resolver_conn_wrapper.go
generated
vendored
Normal file
139
cmd/vendor/google.golang.org/grpc/resolver_conn_wrapper.go
generated
vendored
Normal file
@ -0,0 +1,139 @@
|
||||
/*
|
||||
*
|
||||
* 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 (
|
||||
"strings"
|
||||
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/resolver"
|
||||
)
|
||||
|
||||
// ccResolverWrapper is a wrapper on top of cc for resolvers.
|
||||
// It implements resolver.ClientConnection interface.
|
||||
type ccResolverWrapper struct {
|
||||
cc *ClientConn
|
||||
resolver resolver.Resolver
|
||||
addrCh chan []resolver.Address
|
||||
scCh chan string
|
||||
done chan struct{}
|
||||
}
|
||||
|
||||
// split2 returns the values from strings.SplitN(s, sep, 2).
|
||||
// If sep is not found, it returns "", s instead.
|
||||
func split2(s, sep string) (string, string) {
|
||||
spl := strings.SplitN(s, sep, 2)
|
||||
if len(spl) < 2 {
|
||||
return "", s
|
||||
}
|
||||
return spl[0], spl[1]
|
||||
}
|
||||
|
||||
// parseTarget splits target into a struct containing scheme, authority and
|
||||
// endpoint.
|
||||
func parseTarget(target string) (ret resolver.Target) {
|
||||
ret.Scheme, ret.Endpoint = split2(target, "://")
|
||||
ret.Authority, ret.Endpoint = split2(ret.Endpoint, "/")
|
||||
return ret
|
||||
}
|
||||
|
||||
// newCCResolverWrapper parses cc.target for scheme and gets the resolver
|
||||
// builder for this scheme. It then builds the resolver and starts the
|
||||
// monitoring goroutine for it.
|
||||
//
|
||||
// This function could return nil, nil, in tests for old behaviors.
|
||||
// TODO(bar) never return nil, nil when DNS becomes the default resolver.
|
||||
func newCCResolverWrapper(cc *ClientConn) (*ccResolverWrapper, error) {
|
||||
target := parseTarget(cc.target)
|
||||
grpclog.Infof("dialing to target with scheme: %q", target.Scheme)
|
||||
|
||||
rb := resolver.Get(target.Scheme)
|
||||
if rb == nil {
|
||||
// TODO(bar) return error when DNS becomes the default (implemented and
|
||||
// registered by DNS package).
|
||||
grpclog.Infof("could not get resolver for scheme: %q", target.Scheme)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
ccr := &ccResolverWrapper{
|
||||
cc: cc,
|
||||
addrCh: make(chan []resolver.Address, 1),
|
||||
scCh: make(chan string, 1),
|
||||
done: make(chan struct{}),
|
||||
}
|
||||
|
||||
var err error
|
||||
ccr.resolver, err = rb.Build(target, ccr, resolver.BuildOption{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
go ccr.watcher()
|
||||
return ccr, nil
|
||||
}
|
||||
|
||||
// watcher processes address updates and service config updates sequencially.
|
||||
// Otherwise, we need to resolve possible races between address and service
|
||||
// config (e.g. they specify different balancer types).
|
||||
func (ccr *ccResolverWrapper) watcher() {
|
||||
for {
|
||||
select {
|
||||
case <-ccr.done:
|
||||
return
|
||||
default:
|
||||
}
|
||||
|
||||
select {
|
||||
case addrs := <-ccr.addrCh:
|
||||
grpclog.Infof("ccResolverWrapper: sending new addresses to balancer wrapper: %v", addrs)
|
||||
// TODO(bar switching) this should never be nil. Pickfirst should be default.
|
||||
if ccr.cc.balancerWrapper != nil {
|
||||
// TODO(bar switching) create balancer if it's nil?
|
||||
ccr.cc.balancerWrapper.handleResolvedAddrs(addrs, nil)
|
||||
}
|
||||
case sc := <-ccr.scCh:
|
||||
grpclog.Infof("ccResolverWrapper: got new service config: %v", sc)
|
||||
case <-ccr.done:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ccr *ccResolverWrapper) close() {
|
||||
ccr.resolver.Close()
|
||||
close(ccr.done)
|
||||
}
|
||||
|
||||
// NewAddress is called by the resolver implemenetion to send addresses to gRPC.
|
||||
func (ccr *ccResolverWrapper) NewAddress(addrs []resolver.Address) {
|
||||
select {
|
||||
case <-ccr.addrCh:
|
||||
default:
|
||||
}
|
||||
ccr.addrCh <- addrs
|
||||
}
|
||||
|
||||
// NewServiceConfig is called by the resolver implemenetion to send service
|
||||
// configs to gPRC.
|
||||
func (ccr *ccResolverWrapper) NewServiceConfig(sc string) {
|
||||
select {
|
||||
case <-ccr.scCh:
|
||||
default:
|
||||
}
|
||||
ccr.scCh <- sc
|
||||
}
|
71
cmd/vendor/google.golang.org/grpc/rpc_util.go
generated
vendored
71
cmd/vendor/google.golang.org/grpc/rpc_util.go
generated
vendored
@ -21,10 +21,12 @@ package grpc
|
||||
import (
|
||||
"bytes"
|
||||
"compress/gzip"
|
||||
stdctx "context"
|
||||
"encoding/binary"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"math"
|
||||
"os"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -132,7 +134,9 @@ type callInfo struct {
|
||||
creds credentials.PerRPCCredentials
|
||||
}
|
||||
|
||||
var defaultCallInfo = callInfo{failFast: true}
|
||||
func defaultCallInfo() *callInfo {
|
||||
return &callInfo{failFast: true}
|
||||
}
|
||||
|
||||
// CallOption configures a Call before it starts or extracts information from
|
||||
// a Call after it completes.
|
||||
@ -384,14 +388,15 @@ func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{
|
||||
}
|
||||
|
||||
type rpcInfo struct {
|
||||
failfast bool
|
||||
bytesSent bool
|
||||
bytesReceived bool
|
||||
}
|
||||
|
||||
type rpcInfoContextKey struct{}
|
||||
|
||||
func newContextWithRPCInfo(ctx context.Context) context.Context {
|
||||
return context.WithValue(ctx, rpcInfoContextKey{}, &rpcInfo{})
|
||||
func newContextWithRPCInfo(ctx context.Context, failfast bool) context.Context {
|
||||
return context.WithValue(ctx, rpcInfoContextKey{}, &rpcInfo{failfast: failfast})
|
||||
}
|
||||
|
||||
func rpcInfoFromContext(ctx context.Context) (s *rpcInfo, ok bool) {
|
||||
@ -401,11 +406,63 @@ func rpcInfoFromContext(ctx context.Context) (s *rpcInfo, ok bool) {
|
||||
|
||||
func updateRPCInfoInContext(ctx context.Context, s rpcInfo) {
|
||||
if ss, ok := rpcInfoFromContext(ctx); ok {
|
||||
*ss = s
|
||||
ss.bytesReceived = s.bytesReceived
|
||||
ss.bytesSent = s.bytesSent
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// toRPCErr converts an error into an error from the status package.
|
||||
func toRPCErr(err error) error {
|
||||
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, stdctx.DeadlineExceeded:
|
||||
return status.Error(codes.DeadlineExceeded, err.Error())
|
||||
case context.Canceled, stdctx.Canceled:
|
||||
return status.Error(codes.Canceled, err.Error())
|
||||
case ErrClientConnClosing:
|
||||
return status.Error(codes.FailedPrecondition, err.Error())
|
||||
}
|
||||
}
|
||||
return status.Error(codes.Unknown, err.Error())
|
||||
}
|
||||
|
||||
// convertCode converts a standard Go error into its canonical code. Note that
|
||||
// this is only used to translate the error returned by the server applications.
|
||||
func convertCode(err error) codes.Code {
|
||||
switch err {
|
||||
case nil:
|
||||
return codes.OK
|
||||
case io.EOF:
|
||||
return codes.OutOfRange
|
||||
case io.ErrClosedPipe, io.ErrNoProgress, io.ErrShortBuffer, io.ErrShortWrite, io.ErrUnexpectedEOF:
|
||||
return codes.FailedPrecondition
|
||||
case os.ErrInvalid:
|
||||
return codes.InvalidArgument
|
||||
case context.Canceled, stdctx.Canceled:
|
||||
return codes.Canceled
|
||||
case context.DeadlineExceeded, stdctx.DeadlineExceeded:
|
||||
return codes.DeadlineExceeded
|
||||
}
|
||||
switch {
|
||||
case os.IsExist(err):
|
||||
return codes.AlreadyExists
|
||||
case os.IsNotExist(err):
|
||||
return codes.NotFound
|
||||
case os.IsPermission(err):
|
||||
return codes.PermissionDenied
|
||||
}
|
||||
return codes.Unknown
|
||||
}
|
||||
|
||||
// Code returns the error code for err if it was produced by the rpc system.
|
||||
// Otherwise, it returns codes.Unknown.
|
||||
//
|
||||
@ -452,7 +509,7 @@ type MethodConfig struct {
|
||||
// MaxReqSize is the maximum allowed payload size for an individual request in a
|
||||
// stream (client->server) in bytes. The size which is measured is the serialized
|
||||
// payload after per-message compression (but before stream compression) in bytes.
|
||||
// The actual value used is the minumum of the value specified here and the value set
|
||||
// The actual value used is the minimum of the value specified here and the value set
|
||||
// by the application via the gRPC client API. If either one is not set, then the other
|
||||
// will be used. If neither is set, then the built-in default is used.
|
||||
MaxReqSize *int
|
||||
@ -497,7 +554,7 @@ func getMaxSize(mcMax, doptMax *int, defaultVal int) *int {
|
||||
|
||||
// SupportPackageIsVersion3 is referenced from generated protocol buffer files.
|
||||
// The latest support package version is 4.
|
||||
// SupportPackageIsVersion3 is kept for compability. It will be removed in the
|
||||
// SupportPackageIsVersion3 is kept for compatibility. It will be removed in the
|
||||
// next support package version update.
|
||||
const SupportPackageIsVersion3 = true
|
||||
|
||||
@ -510,6 +567,6 @@ const SupportPackageIsVersion3 = true
|
||||
const SupportPackageIsVersion4 = true
|
||||
|
||||
// Version is the current grpc version.
|
||||
const Version = "1.6.0"
|
||||
const Version = "1.7.2"
|
||||
|
||||
const grpcUA = "grpc-go/" + Version
|
||||
|
25
cmd/vendor/google.golang.org/grpc/server.go
generated
vendored
25
cmd/vendor/google.golang.org/grpc/server.go
generated
vendored
@ -116,6 +116,8 @@ type options struct {
|
||||
keepalivePolicy keepalive.EnforcementPolicy
|
||||
initialWindowSize int32
|
||||
initialConnWindowSize int32
|
||||
writeBufferSize int
|
||||
readBufferSize int
|
||||
}
|
||||
|
||||
var defaultServerOptions = options{
|
||||
@ -126,6 +128,22 @@ var defaultServerOptions = options{
|
||||
// A ServerOption sets options such as credentials, codec and keepalive parameters, etc.
|
||||
type ServerOption func(*options)
|
||||
|
||||
// WriteBufferSize lets you set the size of write buffer, this determines how much data can be batched
|
||||
// before doing a write on the wire.
|
||||
func WriteBufferSize(s int) ServerOption {
|
||||
return func(o *options) {
|
||||
o.writeBufferSize = s
|
||||
}
|
||||
}
|
||||
|
||||
// ReadBufferSize lets you set the size of read buffer, this determines how much data can be read at most
|
||||
// for one read syscall.
|
||||
func ReadBufferSize(s int) ServerOption {
|
||||
return func(o *options) {
|
||||
o.readBufferSize = s
|
||||
}
|
||||
}
|
||||
|
||||
// InitialWindowSize returns a ServerOption that sets window size for stream.
|
||||
// The lower bound for window size is 64K and any value smaller than that will be ignored.
|
||||
func InitialWindowSize(s int32) ServerOption {
|
||||
@ -260,7 +278,7 @@ func StatsHandler(h stats.Handler) ServerOption {
|
||||
// handler that will be invoked instead of returning the "unimplemented" gRPC
|
||||
// error whenever a request is received for an unregistered service or method.
|
||||
// The handling function has full access to the Context of the request and the
|
||||
// stream, and the invocation passes through interceptors.
|
||||
// stream, and the invocation bypasses interceptors.
|
||||
func UnknownServiceHandler(streamHandler StreamHandler) ServerOption {
|
||||
return func(o *options) {
|
||||
o.unknownStreamDesc = &StreamDesc{
|
||||
@ -524,6 +542,8 @@ func (s *Server) serveHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo)
|
||||
KeepalivePolicy: s.opts.keepalivePolicy,
|
||||
InitialWindowSize: s.opts.initialWindowSize,
|
||||
InitialConnWindowSize: s.opts.initialConnWindowSize,
|
||||
WriteBufferSize: s.opts.writeBufferSize,
|
||||
ReadBufferSize: s.opts.readBufferSize,
|
||||
}
|
||||
st, err := transport.NewServerTransport("http2", c, config)
|
||||
if err != nil {
|
||||
@ -891,9 +911,6 @@ func (s *Server) processStreamingRPC(t transport.ServerTransport, stream *transp
|
||||
trInfo: trInfo,
|
||||
statsHandler: sh,
|
||||
}
|
||||
if ss.cp != nil {
|
||||
ss.cbuf = new(bytes.Buffer)
|
||||
}
|
||||
if trInfo != nil {
|
||||
trInfo.tr.LazyLog(&trInfo.firstLine, false)
|
||||
defer func() {
|
||||
|
10
cmd/vendor/google.golang.org/grpc/stats/stats.go
generated
vendored
10
cmd/vendor/google.golang.org/grpc/stats/stats.go
generated
vendored
@ -135,8 +135,6 @@ func (s *OutPayload) isRPCStats() {}
|
||||
type OutHeader struct {
|
||||
// Client is true if this OutHeader is from client side.
|
||||
Client bool
|
||||
// WireLength is the wire length of header.
|
||||
WireLength int
|
||||
|
||||
// The following fields are valid only if Client is true.
|
||||
// FullMethod is the full RPC method string, i.e., /package.service/method.
|
||||
@ -220,7 +218,7 @@ type outgoingTagsKey struct{}
|
||||
// the outgoing RPC with the header grpc-tags-bin. Subsequent calls to
|
||||
// SetTags will overwrite the values from earlier calls.
|
||||
//
|
||||
// NOTE: this is provided only for backward compatibilty with existing clients
|
||||
// NOTE: this is provided only for backward compatibility with existing clients
|
||||
// and will likely be removed in an upcoming release. New uses should transmit
|
||||
// this type of data using metadata with a different, non-reserved (i.e. does
|
||||
// not begin with "grpc-") header name.
|
||||
@ -230,7 +228,7 @@ func SetTags(ctx context.Context, b []byte) context.Context {
|
||||
|
||||
// Tags returns the tags from the context for the inbound RPC.
|
||||
//
|
||||
// NOTE: this is provided only for backward compatibilty with existing clients
|
||||
// NOTE: this is provided only for backward compatibility with existing clients
|
||||
// and will likely be removed in an upcoming release. New uses should transmit
|
||||
// this type of data using metadata with a different, non-reserved (i.e. does
|
||||
// not begin with "grpc-") header name.
|
||||
@ -262,7 +260,7 @@ type outgoingTraceKey struct{}
|
||||
// the outgoing RPC with the header grpc-trace-bin. Subsequent calls to
|
||||
// SetTrace will overwrite the values from earlier calls.
|
||||
//
|
||||
// NOTE: this is provided only for backward compatibilty with existing clients
|
||||
// NOTE: this is provided only for backward compatibility with existing clients
|
||||
// and will likely be removed in an upcoming release. New uses should transmit
|
||||
// this type of data using metadata with a different, non-reserved (i.e. does
|
||||
// not begin with "grpc-") header name.
|
||||
@ -272,7 +270,7 @@ func SetTrace(ctx context.Context, b []byte) context.Context {
|
||||
|
||||
// Trace returns the trace from the context for the inbound RPC.
|
||||
//
|
||||
// NOTE: this is provided only for backward compatibilty with existing clients
|
||||
// NOTE: this is provided only for backward compatibility with existing clients
|
||||
// and will likely be removed in an upcoming release. New uses should transmit
|
||||
// this type of data using metadata with a different, non-reserved (i.e. does
|
||||
// not begin with "grpc-") header name.
|
||||
|
61
cmd/vendor/google.golang.org/grpc/stream.go
generated
vendored
61
cmd/vendor/google.golang.org/grpc/stream.go
generated
vendored
@ -27,6 +27,7 @@ import (
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"golang.org/x/net/trace"
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/metadata"
|
||||
"google.golang.org/grpc/peer"
|
||||
@ -106,10 +107,10 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
var (
|
||||
t transport.ClientTransport
|
||||
s *transport.Stream
|
||||
put func()
|
||||
done func(balancer.DoneInfo)
|
||||
cancel context.CancelFunc
|
||||
)
|
||||
c := defaultCallInfo
|
||||
c := defaultCallInfo()
|
||||
mc := cc.GetMethodConfig(method)
|
||||
if mc.WaitForReady != nil {
|
||||
c.failFast = !*mc.WaitForReady
|
||||
@ -126,7 +127,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
|
||||
opts = append(cc.dopts.callOptions, opts...)
|
||||
for _, o := range opts {
|
||||
if err := o.before(&c); err != nil {
|
||||
if err := o.before(c); err != nil {
|
||||
return nil, toRPCErr(err)
|
||||
}
|
||||
}
|
||||
@ -167,7 +168,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
}
|
||||
}()
|
||||
}
|
||||
ctx = newContextWithRPCInfo(ctx)
|
||||
ctx = newContextWithRPCInfo(ctx, c.failFast)
|
||||
sh := cc.dopts.copts.StatsHandler
|
||||
if sh != nil {
|
||||
ctx = sh.TagRPC(ctx, &stats.RPCTagInfo{FullMethodName: method, FailFast: c.failFast})
|
||||
@ -188,11 +189,8 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
}
|
||||
}()
|
||||
}
|
||||
gopts := BalancerGetOptions{
|
||||
BlockingWait: !c.failFast,
|
||||
}
|
||||
for {
|
||||
t, put, err = cc.getTransport(ctx, gopts)
|
||||
t, done, err = cc.getTransport(ctx, c.failFast)
|
||||
if err != nil {
|
||||
// TODO(zhaoq): Probably revisit the error handling.
|
||||
if _, ok := status.FromError(err); ok {
|
||||
@ -210,15 +208,15 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
|
||||
s, err = t.NewStream(ctx, callHdr)
|
||||
if err != nil {
|
||||
if _, ok := err.(transport.ConnectionError); ok && put != nil {
|
||||
if _, ok := err.(transport.ConnectionError); ok && done != nil {
|
||||
// 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})
|
||||
}
|
||||
if put != nil {
|
||||
put()
|
||||
put = nil
|
||||
if done != nil {
|
||||
done(balancer.DoneInfo{Err: err})
|
||||
done = nil
|
||||
}
|
||||
if _, ok := err.(transport.ConnectionError); (ok || err == transport.ErrStreamDrain) && !c.failFast {
|
||||
continue
|
||||
@ -240,10 +238,10 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
dc: cc.dopts.dc,
|
||||
cancel: cancel,
|
||||
|
||||
put: put,
|
||||
t: t,
|
||||
s: s,
|
||||
p: &parser{r: s},
|
||||
done: done,
|
||||
t: t,
|
||||
s: s,
|
||||
p: &parser{r: s},
|
||||
|
||||
tracing: EnableTracing,
|
||||
trInfo: trInfo,
|
||||
@ -251,9 +249,6 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
statsCtx: ctx,
|
||||
statsHandler: cc.dopts.copts.StatsHandler,
|
||||
}
|
||||
if cc.dopts.cp != nil {
|
||||
cs.cbuf = new(bytes.Buffer)
|
||||
}
|
||||
// Listen on ctx.Done() to detect cancellation and s.Done() to detect normal termination
|
||||
// when there is no pending I/O operations on this stream.
|
||||
go func() {
|
||||
@ -283,21 +278,20 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
|
||||
// clientStream implements a client side Stream.
|
||||
type clientStream struct {
|
||||
opts []CallOption
|
||||
c callInfo
|
||||
c *callInfo
|
||||
t transport.ClientTransport
|
||||
s *transport.Stream
|
||||
p *parser
|
||||
desc *StreamDesc
|
||||
codec Codec
|
||||
cp Compressor
|
||||
cbuf *bytes.Buffer
|
||||
dc Decompressor
|
||||
cancel context.CancelFunc
|
||||
|
||||
tracing bool // set to EnableTracing when the clientStream is created.
|
||||
|
||||
mu sync.Mutex
|
||||
put func()
|
||||
done func(balancer.DoneInfo)
|
||||
closed bool
|
||||
finished bool
|
||||
// trInfo.tr is set when the clientStream is created (if EnableTracing is true),
|
||||
@ -367,12 +361,7 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) {
|
||||
Client: true,
|
||||
}
|
||||
}
|
||||
hdr, data, err := encode(cs.codec, m, cs.cp, cs.cbuf, outPayload)
|
||||
defer func() {
|
||||
if cs.cbuf != nil {
|
||||
cs.cbuf.Reset()
|
||||
}
|
||||
}()
|
||||
hdr, data, err := encode(cs.codec, m, cs.cp, bytes.NewBuffer([]byte{}), outPayload)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -494,15 +483,15 @@ func (cs *clientStream) finish(err error) {
|
||||
}
|
||||
}()
|
||||
for _, o := range cs.opts {
|
||||
o.after(&cs.c)
|
||||
o.after(cs.c)
|
||||
}
|
||||
if cs.put != nil {
|
||||
if cs.done != nil {
|
||||
updateRPCInfoInContext(cs.s.Context(), rpcInfo{
|
||||
bytesSent: cs.s.BytesSent(),
|
||||
bytesReceived: cs.s.BytesReceived(),
|
||||
})
|
||||
cs.put()
|
||||
cs.put = nil
|
||||
cs.done(balancer.DoneInfo{Err: err})
|
||||
cs.done = nil
|
||||
}
|
||||
if cs.statsHandler != nil {
|
||||
end := &stats.End{
|
||||
@ -557,7 +546,6 @@ type serverStream struct {
|
||||
codec Codec
|
||||
cp Compressor
|
||||
dc Decompressor
|
||||
cbuf *bytes.Buffer
|
||||
maxReceiveMessageSize int
|
||||
maxSendMessageSize int
|
||||
trInfo *traceInfo
|
||||
@ -613,12 +601,7 @@ func (ss *serverStream) SendMsg(m interface{}) (err error) {
|
||||
if ss.statsHandler != nil {
|
||||
outPayload = &stats.OutPayload{}
|
||||
}
|
||||
hdr, data, err := encode(ss.codec, m, ss.cp, ss.cbuf, outPayload)
|
||||
defer func() {
|
||||
if ss.cbuf != nil {
|
||||
ss.cbuf.Reset()
|
||||
}
|
||||
}()
|
||||
hdr, data, err := encode(ss.codec, m, ss.cp, bytes.NewBuffer([]byte{}), outPayload)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
15
cmd/vendor/google.golang.org/grpc/trace.go
generated
vendored
15
cmd/vendor/google.golang.org/grpc/trace.go
generated
vendored
@ -31,7 +31,7 @@ import (
|
||||
|
||||
// EnableTracing controls whether to trace RPCs using the golang.org/x/net/trace package.
|
||||
// This should only be set before any RPCs are sent or received by this program.
|
||||
var EnableTracing = true
|
||||
var EnableTracing bool
|
||||
|
||||
// methodFamily returns the trace family for the given method.
|
||||
// It turns "/pkg.Service/GetFoo" into "pkg.Service".
|
||||
@ -76,6 +76,15 @@ func (f *firstLine) String() string {
|
||||
return line.String()
|
||||
}
|
||||
|
||||
const truncateSize = 100
|
||||
|
||||
func truncate(x string, l int) string {
|
||||
if l > len(x) {
|
||||
return x
|
||||
}
|
||||
return x[:l]
|
||||
}
|
||||
|
||||
// payload represents an RPC request or response payload.
|
||||
type payload struct {
|
||||
sent bool // whether this is an outgoing payload
|
||||
@ -85,9 +94,9 @@ type payload struct {
|
||||
|
||||
func (p payload) String() string {
|
||||
if p.sent {
|
||||
return fmt.Sprintf("sent: %v", p.msg)
|
||||
return truncate(fmt.Sprintf("sent: %v", p.msg), truncateSize)
|
||||
}
|
||||
return fmt.Sprintf("recv: %v", p.msg)
|
||||
return truncate(fmt.Sprintf("recv: %v", p.msg), truncateSize)
|
||||
}
|
||||
|
||||
type fmtStringer struct {
|
||||
|
6
cmd/vendor/google.golang.org/grpc/transport/bdp_estimator.go
generated
vendored
6
cmd/vendor/google.golang.org/grpc/transport/bdp_estimator.go
generated
vendored
@ -59,7 +59,7 @@ type bdpEstimator struct {
|
||||
sample uint32
|
||||
// bwMax is the maximum bandwidth noted so far (bytes/sec).
|
||||
bwMax float64
|
||||
// bool to keep track of the begining of a new measurement cycle.
|
||||
// bool to keep track of the beginning of a new measurement cycle.
|
||||
isSent bool
|
||||
// Callback to update the window sizes.
|
||||
updateFlowControl func(n uint32)
|
||||
@ -70,7 +70,7 @@ type bdpEstimator struct {
|
||||
}
|
||||
|
||||
// timesnap registers the time bdp ping was sent out so that
|
||||
// network rtt can be calculated when its ack is recieved.
|
||||
// network rtt can be calculated when its ack is received.
|
||||
// It is called (by controller) when the bdpPing is
|
||||
// being written on the wire.
|
||||
func (b *bdpEstimator) timesnap(d [8]byte) {
|
||||
@ -119,7 +119,7 @@ func (b *bdpEstimator) calculate(d [8]byte) {
|
||||
b.rtt += (rttSample - b.rtt) * float64(alpha)
|
||||
}
|
||||
b.isSent = false
|
||||
// The number of bytes accumalated so far in the sample is smaller
|
||||
// The number of bytes accumulated so far in the sample is smaller
|
||||
// than or equal to 1.5 times the real BDP on a saturated connection.
|
||||
bwCurrent := float64(b.sample) / (b.rtt * float64(1.5))
|
||||
if bwCurrent > b.bwMax {
|
||||
|
71
cmd/vendor/google.golang.org/grpc/transport/control.go
generated
vendored
71
cmd/vendor/google.golang.org/grpc/transport/control.go
generated
vendored
@ -22,9 +22,11 @@ import (
|
||||
"fmt"
|
||||
"math"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/http2"
|
||||
"golang.org/x/net/http2/hpack"
|
||||
)
|
||||
|
||||
const (
|
||||
@ -44,15 +46,44 @@ const (
|
||||
defaultKeepalivePolicyMinTime = time.Duration(5 * time.Minute)
|
||||
// max window limit set by HTTP2 Specs.
|
||||
maxWindowSize = math.MaxInt32
|
||||
// defaultLocalSendQuota sets is default value for number of data
|
||||
// bytes that each stream can schedule before some of it being
|
||||
// flushed out.
|
||||
defaultLocalSendQuota = 64 * 1024
|
||||
)
|
||||
|
||||
// The following defines various control items which could flow through
|
||||
// the control buffer of transport. They represent different aspects of
|
||||
// control tasks, e.g., flow control, settings, streaming resetting, etc.
|
||||
|
||||
type headerFrame struct {
|
||||
streamID uint32
|
||||
hf []hpack.HeaderField
|
||||
endStream bool
|
||||
}
|
||||
|
||||
func (*headerFrame) item() {}
|
||||
|
||||
type continuationFrame struct {
|
||||
streamID uint32
|
||||
endHeaders bool
|
||||
headerBlockFragment []byte
|
||||
}
|
||||
|
||||
type dataFrame struct {
|
||||
streamID uint32
|
||||
endStream bool
|
||||
d []byte
|
||||
f func()
|
||||
}
|
||||
|
||||
func (*dataFrame) item() {}
|
||||
|
||||
func (*continuationFrame) item() {}
|
||||
|
||||
type windowUpdate struct {
|
||||
streamID uint32
|
||||
increment uint32
|
||||
flush bool
|
||||
}
|
||||
|
||||
func (*windowUpdate) item() {}
|
||||
@ -97,8 +128,9 @@ func (*ping) item() {}
|
||||
type quotaPool struct {
|
||||
c chan int
|
||||
|
||||
mu sync.Mutex
|
||||
quota int
|
||||
mu sync.Mutex
|
||||
version uint32
|
||||
quota int
|
||||
}
|
||||
|
||||
// newQuotaPool creates a quotaPool which has quota q available to consume.
|
||||
@ -119,6 +151,10 @@ func newQuotaPool(q int) *quotaPool {
|
||||
func (qb *quotaPool) add(v int) {
|
||||
qb.mu.Lock()
|
||||
defer qb.mu.Unlock()
|
||||
qb.lockedAdd(v)
|
||||
}
|
||||
|
||||
func (qb *quotaPool) lockedAdd(v int) {
|
||||
select {
|
||||
case n := <-qb.c:
|
||||
qb.quota += n
|
||||
@ -139,6 +175,35 @@ func (qb *quotaPool) add(v int) {
|
||||
}
|
||||
}
|
||||
|
||||
func (qb *quotaPool) addAndUpdate(v int) {
|
||||
qb.mu.Lock()
|
||||
defer qb.mu.Unlock()
|
||||
qb.lockedAdd(v)
|
||||
// Update the version only after having added to the quota
|
||||
// so that if acquireWithVesrion sees the new vesrion it is
|
||||
// guaranteed to have seen the updated quota.
|
||||
// Also, still keep this inside of the lock, so that when
|
||||
// compareAndExecute is processing, this function doesn't
|
||||
// get executed partially (quota gets updated but the version
|
||||
// doesn't).
|
||||
atomic.AddUint32(&(qb.version), 1)
|
||||
}
|
||||
|
||||
func (qb *quotaPool) acquireWithVersion() (<-chan int, uint32) {
|
||||
return qb.c, atomic.LoadUint32(&(qb.version))
|
||||
}
|
||||
|
||||
func (qb *quotaPool) compareAndExecute(version uint32, success, failure func()) bool {
|
||||
qb.mu.Lock()
|
||||
defer qb.mu.Unlock()
|
||||
if version == atomic.LoadUint32(&(qb.version)) {
|
||||
success()
|
||||
return true
|
||||
}
|
||||
failure()
|
||||
return false
|
||||
}
|
||||
|
||||
// acquire returns the channel on which available quota amounts are sent.
|
||||
func (qb *quotaPool) acquire() <-chan int {
|
||||
return qb.c
|
||||
|
45
cmd/vendor/google.golang.org/grpc/transport/go16.go
generated
vendored
45
cmd/vendor/google.golang.org/grpc/transport/go16.go
generated
vendored
@ -1,45 +0,0 @@
|
||||
// +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 transport
|
||||
|
||||
import (
|
||||
"net"
|
||||
|
||||
"google.golang.org/grpc/codes"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
// 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)
|
||||
}
|
||||
|
||||
// ContextErr converts the error from context package into a StreamError.
|
||||
func ContextErr(err error) StreamError {
|
||||
switch err {
|
||||
case context.DeadlineExceeded:
|
||||
return streamErrorf(codes.DeadlineExceeded, "%v", err)
|
||||
case context.Canceled:
|
||||
return streamErrorf(codes.Canceled, "%v", err)
|
||||
}
|
||||
return streamErrorf(codes.Internal, "Unexpected error from context packet: %v", err)
|
||||
}
|
46
cmd/vendor/google.golang.org/grpc/transport/go17.go
generated
vendored
46
cmd/vendor/google.golang.org/grpc/transport/go17.go
generated
vendored
@ -1,46 +0,0 @@
|
||||
// +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 transport
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"google.golang.org/grpc/codes"
|
||||
|
||||
netctx "golang.org/x/net/context"
|
||||
)
|
||||
|
||||
// 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)
|
||||
}
|
||||
|
||||
// ContextErr converts the error from context package into a StreamError.
|
||||
func ContextErr(err error) StreamError {
|
||||
switch err {
|
||||
case context.DeadlineExceeded, netctx.DeadlineExceeded:
|
||||
return streamErrorf(codes.DeadlineExceeded, "%v", err)
|
||||
case context.Canceled, netctx.Canceled:
|
||||
return streamErrorf(codes.Canceled, "%v", err)
|
||||
}
|
||||
return streamErrorf(codes.Internal, "Unexpected error from context packet: %v", err)
|
||||
}
|
5
cmd/vendor/google.golang.org/grpc/transport/handler_server.go
generated
vendored
5
cmd/vendor/google.golang.org/grpc/transport/handler_server.go
generated
vendored
@ -173,7 +173,6 @@ func (ht *serverHandlerTransport) do(fn func()) error {
|
||||
case <-ht.closedCh:
|
||||
return ErrConnClosing
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@ -183,6 +182,7 @@ func (ht *serverHandlerTransport) WriteStatus(s *Stream, st *status.Status) erro
|
||||
ht.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
ht.streamDone = true
|
||||
ht.mu.Unlock()
|
||||
err := ht.do(func() {
|
||||
ht.writeCommonHeaders(s)
|
||||
@ -223,9 +223,6 @@ func (ht *serverHandlerTransport) WriteStatus(s *Stream, st *status.Status) erro
|
||||
}
|
||||
})
|
||||
close(ht.writes)
|
||||
ht.mu.Lock()
|
||||
ht.streamDone = true
|
||||
ht.mu.Unlock()
|
||||
return err
|
||||
}
|
||||
|
||||
|
606
cmd/vendor/google.golang.org/grpc/transport/http2_client.go
generated
vendored
606
cmd/vendor/google.golang.org/grpc/transport/http2_client.go
generated
vendored
@ -43,6 +43,7 @@ import (
|
||||
// http2Client implements the ClientTransport interface with HTTP2.
|
||||
type http2Client struct {
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
target string // server name/addr
|
||||
userAgent string
|
||||
md interface{}
|
||||
@ -52,17 +53,6 @@ type http2Client struct {
|
||||
authInfo credentials.AuthInfo // auth info about the connection
|
||||
nextID uint32 // the next stream ID to be used
|
||||
|
||||
// writableChan synchronizes write access to the transport.
|
||||
// A writer acquires the write lock by sending a value on writableChan
|
||||
// and releases it by receiving from writableChan.
|
||||
writableChan chan int
|
||||
// shutdownChan is closed when Close is called.
|
||||
// Blocking operations should select on shutdownChan to avoid
|
||||
// blocking forever after Close.
|
||||
// TODO(zhaoq): Maybe have a channel context?
|
||||
shutdownChan chan struct{}
|
||||
// errorChan is closed to notify the I/O error to the caller.
|
||||
errorChan chan struct{}
|
||||
// goAway is closed to notify the upper layer (i.e., addrConn.transportMonitor)
|
||||
// that the server sent GoAway on this transport.
|
||||
goAway chan struct{}
|
||||
@ -119,7 +109,7 @@ func dial(ctx context.Context, fn func(context.Context, string) (net.Conn, error
|
||||
if fn != nil {
|
||||
return fn(ctx, addr)
|
||||
}
|
||||
return dialContext(ctx, "tcp", addr)
|
||||
return (&net.Dialer{}).DialContext(ctx, "tcp", addr)
|
||||
}
|
||||
|
||||
func isTemporary(err error) bool {
|
||||
@ -153,9 +143,18 @@ func isTemporary(err error) bool {
|
||||
// newHTTP2Client constructs a connected ClientTransport to addr based on HTTP2
|
||||
// and starts to receive messages on it. Non-nil error returns if construction
|
||||
// fails.
|
||||
func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (_ ClientTransport, err error) {
|
||||
func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions, timeout time.Duration) (_ ClientTransport, err error) {
|
||||
scheme := "http"
|
||||
conn, err := dial(ctx, opts.Dialer, addr.Addr)
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
connectCtx, connectCancel := context.WithTimeout(ctx, timeout)
|
||||
defer func() {
|
||||
connectCancel()
|
||||
if err != nil {
|
||||
cancel()
|
||||
}
|
||||
}()
|
||||
|
||||
conn, err := dial(connectCtx, opts.Dialer, addr.Addr)
|
||||
if err != nil {
|
||||
if opts.FailOnNonTempDialError {
|
||||
return nil, connectionErrorf(isTemporary(err), err, "transport: error while dialing: %v", err)
|
||||
@ -174,7 +173,7 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (
|
||||
)
|
||||
if creds := opts.TransportCredentials; creds != nil {
|
||||
scheme = "https"
|
||||
conn, authInfo, err = creds.ClientHandshake(ctx, addr.Addr, conn)
|
||||
conn, authInfo, err = creds.ClientHandshake(connectCtx, addr.Addr, conn)
|
||||
if err != nil {
|
||||
// Credentials handshake errors are typically considered permanent
|
||||
// to avoid retrying on e.g. bad certificates.
|
||||
@ -198,8 +197,17 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (
|
||||
dynamicWindow = false
|
||||
}
|
||||
var buf bytes.Buffer
|
||||
writeBufSize := defaultWriteBufSize
|
||||
if opts.WriteBufferSize > 0 {
|
||||
writeBufSize = opts.WriteBufferSize
|
||||
}
|
||||
readBufSize := defaultReadBufSize
|
||||
if opts.ReadBufferSize > 0 {
|
||||
readBufSize = opts.ReadBufferSize
|
||||
}
|
||||
t := &http2Client{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
target: addr.Addr,
|
||||
userAgent: opts.UserAgent,
|
||||
md: addr.Metadata,
|
||||
@ -209,14 +217,11 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (
|
||||
authInfo: authInfo,
|
||||
// The client initiated stream id is odd starting from 1.
|
||||
nextID: 1,
|
||||
writableChan: make(chan int, 1),
|
||||
shutdownChan: make(chan struct{}),
|
||||
errorChan: make(chan struct{}),
|
||||
goAway: make(chan struct{}),
|
||||
awakenKeepalive: make(chan struct{}, 1),
|
||||
framer: newFramer(conn),
|
||||
hBuf: &buf,
|
||||
hEnc: hpack.NewEncoder(&buf),
|
||||
framer: newFramer(conn, writeBufSize, readBufSize),
|
||||
controlBuf: newControlBuffer(),
|
||||
fc: &inFlow{limit: uint32(icwz)},
|
||||
sendQuotaPool: newQuotaPool(defaultWindowSize),
|
||||
@ -270,12 +275,12 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (
|
||||
return nil, connectionErrorf(true, err, "transport: preface mismatch, wrote %d bytes; want %d", n, len(clientPreface))
|
||||
}
|
||||
if t.initialWindowSize != defaultWindowSize {
|
||||
err = t.framer.writeSettings(true, http2.Setting{
|
||||
err = t.framer.fr.WriteSettings(http2.Setting{
|
||||
ID: http2.SettingInitialWindowSize,
|
||||
Val: uint32(t.initialWindowSize),
|
||||
})
|
||||
} else {
|
||||
err = t.framer.writeSettings(true)
|
||||
err = t.framer.fr.WriteSettings()
|
||||
}
|
||||
if err != nil {
|
||||
t.Close()
|
||||
@ -283,31 +288,35 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (
|
||||
}
|
||||
// Adjust the connection flow control window if needed.
|
||||
if delta := uint32(icwz - defaultWindowSize); delta > 0 {
|
||||
if err := t.framer.writeWindowUpdate(true, 0, delta); err != nil {
|
||||
if err := t.framer.fr.WriteWindowUpdate(0, delta); err != nil {
|
||||
t.Close()
|
||||
return nil, connectionErrorf(true, err, "transport: failed to write window update: %v", err)
|
||||
}
|
||||
}
|
||||
go t.controller()
|
||||
t.framer.writer.Flush()
|
||||
go func() {
|
||||
loopyWriter(t.ctx, t.controlBuf, t.itemHandler)
|
||||
t.Close()
|
||||
}()
|
||||
if t.kp.Time != infinity {
|
||||
go t.keepalive()
|
||||
}
|
||||
t.writableChan <- 0
|
||||
return t, nil
|
||||
}
|
||||
|
||||
func (t *http2Client) newStream(ctx context.Context, callHdr *CallHdr) *Stream {
|
||||
// TODO(zhaoq): Handle uint32 overflow of Stream.id.
|
||||
s := &Stream{
|
||||
id: t.nextID,
|
||||
done: make(chan struct{}),
|
||||
goAway: make(chan struct{}),
|
||||
method: callHdr.Method,
|
||||
sendCompress: callHdr.SendCompress,
|
||||
buf: newRecvBuffer(),
|
||||
fc: &inFlow{limit: uint32(t.initialWindowSize)},
|
||||
sendQuotaPool: newQuotaPool(int(t.streamSendQuota)),
|
||||
headerChan: make(chan struct{}),
|
||||
id: t.nextID,
|
||||
done: make(chan struct{}),
|
||||
goAway: make(chan struct{}),
|
||||
method: callHdr.Method,
|
||||
sendCompress: callHdr.SendCompress,
|
||||
buf: newRecvBuffer(),
|
||||
fc: &inFlow{limit: uint32(t.initialWindowSize)},
|
||||
sendQuotaPool: newQuotaPool(int(t.streamSendQuota)),
|
||||
localSendQuota: newQuotaPool(defaultLocalSendQuota),
|
||||
headerChan: make(chan struct{}),
|
||||
}
|
||||
t.nextID += 2
|
||||
s.requestRead = func(n int) {
|
||||
@ -368,13 +377,13 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
authData[k] = v
|
||||
}
|
||||
}
|
||||
callAuthData := make(map[string]string)
|
||||
callAuthData := map[string]string{}
|
||||
// Check if credentials.PerRPCCredentials were provided via call options.
|
||||
// Note: if these credentials are provided both via dial options and call
|
||||
// options, then both sets of credentials will be applied.
|
||||
if callCreds := callHdr.Creds; callCreds != nil {
|
||||
if !t.isSecure && callCreds.RequireTransportSecurity() {
|
||||
return nil, streamErrorf(codes.Unauthenticated, "transport: cannot send secure credentials on an insecure conneciton")
|
||||
return nil, streamErrorf(codes.Unauthenticated, "transport: cannot send secure credentials on an insecure connection")
|
||||
}
|
||||
data, err := callCreds.GetRequestMetadata(ctx, audience)
|
||||
if err != nil {
|
||||
@ -400,7 +409,7 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
return nil, ErrConnClosing
|
||||
}
|
||||
t.mu.Unlock()
|
||||
sq, err := wait(ctx, nil, nil, t.shutdownChan, t.streamsQuota.acquire())
|
||||
sq, err := wait(ctx, t.ctx, nil, nil, t.streamsQuota.acquire())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -408,19 +417,66 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
if sq > 1 {
|
||||
t.streamsQuota.add(sq - 1)
|
||||
}
|
||||
if _, err := wait(ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil {
|
||||
// Return the quota back now because there is no stream returned to the caller.
|
||||
if _, ok := err.(StreamError); ok {
|
||||
t.streamsQuota.add(1)
|
||||
// TODO(mmukhi): Benchmark if the perfomance gets better if count the metadata and other header fields
|
||||
// first and create a slice of that exact size.
|
||||
// Make the slice of certain predictable size to reduce allocations made by append.
|
||||
hfLen := 7 // :method, :scheme, :path, :authority, content-type, user-agent, te
|
||||
hfLen += len(authData) + len(callAuthData)
|
||||
headerFields := make([]hpack.HeaderField, 0, hfLen)
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: ":method", Value: "POST"})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: ":scheme", Value: t.scheme})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: ":path", Value: callHdr.Method})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: ":authority", Value: callHdr.Host})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "content-type", Value: "application/grpc"})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "user-agent", Value: t.userAgent})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "te", Value: "trailers"})
|
||||
|
||||
if callHdr.SendCompress != "" {
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-encoding", Value: callHdr.SendCompress})
|
||||
}
|
||||
if dl, ok := ctx.Deadline(); ok {
|
||||
// Send out timeout regardless its value. The server can detect timeout context by itself.
|
||||
// TODO(mmukhi): Perhaps this field should be updated when actually writing out to the wire.
|
||||
timeout := dl.Sub(time.Now())
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-timeout", Value: encodeTimeout(timeout)})
|
||||
}
|
||||
for k, v := range authData {
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
}
|
||||
for k, v := range callAuthData {
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
}
|
||||
if b := stats.OutgoingTags(ctx); b != nil {
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-tags-bin", Value: encodeBinHeader(b)})
|
||||
}
|
||||
if b := stats.OutgoingTrace(ctx); b != nil {
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-trace-bin", Value: encodeBinHeader(b)})
|
||||
}
|
||||
if md, ok := metadata.FromOutgoingContext(ctx); ok {
|
||||
for k, vv := range md {
|
||||
// HTTP doesn't allow you to set pseudoheaders after non pseudoheaders were set.
|
||||
if isReservedHeader(k) {
|
||||
continue
|
||||
}
|
||||
for _, v := range vv {
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
}
|
||||
}
|
||||
}
|
||||
if md, ok := t.md.(*metadata.MD); ok {
|
||||
for k, vv := range *md {
|
||||
if isReservedHeader(k) {
|
||||
continue
|
||||
}
|
||||
for _, v := range vv {
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
}
|
||||
}
|
||||
return nil, err
|
||||
}
|
||||
t.mu.Lock()
|
||||
if t.state == draining {
|
||||
t.mu.Unlock()
|
||||
t.streamsQuota.add(1)
|
||||
// Need to make t writable again so that the rpc in flight can still proceed.
|
||||
t.writableChan <- 0
|
||||
return nil, ErrStreamDrain
|
||||
}
|
||||
if t.state != reachable {
|
||||
@ -434,7 +490,7 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
if len(t.activeStreams) == 1 {
|
||||
select {
|
||||
case t.awakenKeepalive <- struct{}{}:
|
||||
t.framer.writePing(false, false, [8]byte{})
|
||||
t.controlBuf.put(&ping{data: [8]byte{}})
|
||||
// Fill the awakenKeepalive channel again as this channel must be
|
||||
// kept non-writable except at the point that the keepalive()
|
||||
// goroutine is waiting either to be awaken or shutdown.
|
||||
@ -442,102 +498,13 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
t.controlBuf.put(&headerFrame{
|
||||
streamID: s.id,
|
||||
hf: headerFields,
|
||||
endStream: false,
|
||||
})
|
||||
t.mu.Unlock()
|
||||
|
||||
// HPACK encodes various headers. Note that once WriteField(...) is
|
||||
// called, the corresponding headers/continuation frame has to be sent
|
||||
// because hpack.Encoder is stateful.
|
||||
t.hBuf.Reset()
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: ":method", Value: "POST"})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: ":scheme", Value: t.scheme})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: ":path", Value: callHdr.Method})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: ":authority", Value: callHdr.Host})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "content-type", Value: "application/grpc"})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "user-agent", Value: t.userAgent})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "te", Value: "trailers"})
|
||||
|
||||
if callHdr.SendCompress != "" {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-encoding", Value: callHdr.SendCompress})
|
||||
}
|
||||
if dl, ok := ctx.Deadline(); ok {
|
||||
// Send out timeout regardless its value. The server can detect timeout context by itself.
|
||||
timeout := dl.Sub(time.Now())
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-timeout", Value: encodeTimeout(timeout)})
|
||||
}
|
||||
|
||||
for k, v := range authData {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
}
|
||||
for k, v := range callAuthData {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
}
|
||||
var (
|
||||
endHeaders bool
|
||||
)
|
||||
if b := stats.OutgoingTags(ctx); b != nil {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-tags-bin", Value: encodeBinHeader(b)})
|
||||
}
|
||||
if b := stats.OutgoingTrace(ctx); b != nil {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-trace-bin", Value: encodeBinHeader(b)})
|
||||
}
|
||||
if md, ok := metadata.FromOutgoingContext(ctx); ok {
|
||||
for k, vv := range md {
|
||||
// HTTP doesn't allow you to set pseudoheaders after non pseudoheaders were set.
|
||||
if isReservedHeader(k) {
|
||||
continue
|
||||
}
|
||||
for _, v := range vv {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
}
|
||||
}
|
||||
}
|
||||
if md, ok := t.md.(*metadata.MD); ok {
|
||||
for k, vv := range *md {
|
||||
if isReservedHeader(k) {
|
||||
continue
|
||||
}
|
||||
for _, v := range vv {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
}
|
||||
}
|
||||
}
|
||||
first := true
|
||||
bufLen := t.hBuf.Len()
|
||||
// Sends the headers in a single batch even when they span multiple frames.
|
||||
for !endHeaders {
|
||||
size := t.hBuf.Len()
|
||||
if size > http2MaxFrameLen {
|
||||
size = http2MaxFrameLen
|
||||
} else {
|
||||
endHeaders = true
|
||||
}
|
||||
var flush bool
|
||||
if callHdr.Flush && endHeaders {
|
||||
flush = true
|
||||
}
|
||||
if first {
|
||||
// Sends a HeadersFrame to server to start a new stream.
|
||||
p := http2.HeadersFrameParam{
|
||||
StreamID: s.id,
|
||||
BlockFragment: t.hBuf.Next(size),
|
||||
EndStream: false,
|
||||
EndHeaders: endHeaders,
|
||||
}
|
||||
// Do a force flush for the buffered frames iff it is the last headers frame
|
||||
// and there is header metadata to be sent. Otherwise, there is flushing until
|
||||
// the corresponding data frame is written.
|
||||
err = t.framer.writeHeaders(flush, p)
|
||||
first = false
|
||||
} else {
|
||||
// Sends Continuation frames for the leftover headers.
|
||||
err = t.framer.writeContinuation(flush, s.id, endHeaders, t.hBuf.Next(size))
|
||||
}
|
||||
if err != nil {
|
||||
t.notifyError(err)
|
||||
return nil, connectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
}
|
||||
s.mu.Lock()
|
||||
s.bytesSent = true
|
||||
s.mu.Unlock()
|
||||
@ -545,7 +512,6 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
if t.statsHandler != nil {
|
||||
outHeader := &stats.OutHeader{
|
||||
Client: true,
|
||||
WireLength: bufLen,
|
||||
FullMethod: callHdr.Method,
|
||||
RemoteAddr: t.remoteAddr,
|
||||
LocalAddr: t.localAddr,
|
||||
@ -553,7 +519,6 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
||||
}
|
||||
t.statsHandler.HandleRPC(s.ctx, outHeader)
|
||||
}
|
||||
t.writableChan <- 0
|
||||
return s, nil
|
||||
}
|
||||
|
||||
@ -623,12 +588,9 @@ func (t *http2Client) Close() (err error) {
|
||||
t.mu.Unlock()
|
||||
return
|
||||
}
|
||||
if t.state == reachable || t.state == draining {
|
||||
close(t.errorChan)
|
||||
}
|
||||
t.state = closing
|
||||
t.mu.Unlock()
|
||||
close(t.shutdownChan)
|
||||
t.cancel()
|
||||
err = t.conn.Close()
|
||||
t.mu.Lock()
|
||||
streams := t.activeStreams
|
||||
@ -650,23 +612,18 @@ func (t *http2Client) Close() (err error) {
|
||||
}
|
||||
t.statsHandler.HandleConn(t.ctx, connEnd)
|
||||
}
|
||||
return
|
||||
return err
|
||||
}
|
||||
|
||||
// GracefulClose sets the state to draining, which prevents new streams from
|
||||
// being created and causes the transport to be closed when the last active
|
||||
// stream is closed. If there are no active streams, the transport is closed
|
||||
// immediately. This does nothing if the transport is already draining or
|
||||
// closing.
|
||||
func (t *http2Client) GracefulClose() error {
|
||||
t.mu.Lock()
|
||||
switch t.state {
|
||||
case unreachable:
|
||||
// The server may close the connection concurrently. t is not available for
|
||||
// any streams. Close it now.
|
||||
t.mu.Unlock()
|
||||
t.Close()
|
||||
return nil
|
||||
case closing:
|
||||
t.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
if t.state == draining {
|
||||
case closing, draining:
|
||||
t.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
@ -681,32 +638,38 @@ func (t *http2Client) GracefulClose() error {
|
||||
|
||||
// Write formats the data into HTTP2 data frame(s) and sends it out. The caller
|
||||
// should proceed only if Write returns nil.
|
||||
// TODO(zhaoq): opts.Delay is ignored in this implementation. Support it later
|
||||
// if it improves the performance.
|
||||
func (t *http2Client) Write(s *Stream, hdr []byte, data []byte, opts *Options) error {
|
||||
secondStart := http2MaxFrameLen - len(hdr)%http2MaxFrameLen
|
||||
if len(data) < secondStart {
|
||||
secondStart = len(data)
|
||||
select {
|
||||
case <-s.ctx.Done():
|
||||
return ContextErr(s.ctx.Err())
|
||||
case <-t.ctx.Done():
|
||||
return ErrConnClosing
|
||||
default:
|
||||
}
|
||||
hdr = append(hdr, data[:secondStart]...)
|
||||
data = data[secondStart:]
|
||||
isLastSlice := (len(data) == 0)
|
||||
r := bytes.NewBuffer(hdr)
|
||||
var (
|
||||
p []byte
|
||||
oqv uint32
|
||||
)
|
||||
for {
|
||||
oqv = atomic.LoadUint32(&t.outQuotaVersion)
|
||||
if r.Len() > 0 || p != nil {
|
||||
|
||||
if hdr == nil && data == nil && opts.Last {
|
||||
// stream.CloseSend uses this to send an empty frame with endStream=True
|
||||
t.controlBuf.put(&dataFrame{streamID: s.id, endStream: true, f: func() {}})
|
||||
return nil
|
||||
}
|
||||
// Add data to header frame so that we can equally distribute data across frames.
|
||||
emptyLen := http2MaxFrameLen - len(hdr)
|
||||
if emptyLen > len(data) {
|
||||
emptyLen = len(data)
|
||||
}
|
||||
hdr = append(hdr, data[:emptyLen]...)
|
||||
data = data[emptyLen:]
|
||||
for idx, r := range [][]byte{hdr, data} {
|
||||
for len(r) > 0 {
|
||||
size := http2MaxFrameLen
|
||||
// Wait until the stream has some quota to send the data.
|
||||
sq, err := wait(s.ctx, s.done, s.goAway, t.shutdownChan, s.sendQuotaPool.acquire())
|
||||
quotaChan, quotaVer := s.sendQuotaPool.acquireWithVersion()
|
||||
sq, err := wait(s.ctx, t.ctx, s.done, s.goAway, quotaChan)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// Wait until the transport has some quota to send the data.
|
||||
tq, err := wait(s.ctx, s.done, s.goAway, t.shutdownChan, t.sendQuotaPool.acquire())
|
||||
tq, err := wait(s.ctx, t.ctx, s.done, s.goAway, t.sendQuotaPool.acquire())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -716,93 +679,51 @@ func (t *http2Client) Write(s *Stream, hdr []byte, data []byte, opts *Options) e
|
||||
if tq < size {
|
||||
size = tq
|
||||
}
|
||||
if p == nil {
|
||||
p = r.Next(size)
|
||||
if size > len(r) {
|
||||
size = len(r)
|
||||
}
|
||||
p := r[:size]
|
||||
ps := len(p)
|
||||
if ps < sq {
|
||||
// Overbooked stream quota. Return it back.
|
||||
s.sendQuotaPool.add(sq - ps)
|
||||
}
|
||||
if ps < tq {
|
||||
// Overbooked transport quota. Return it back.
|
||||
t.sendQuotaPool.add(tq - ps)
|
||||
}
|
||||
}
|
||||
var (
|
||||
endStream bool
|
||||
forceFlush bool
|
||||
)
|
||||
// Indicate there is a writer who is about to write a data frame.
|
||||
t.framer.adjustNumWriters(1)
|
||||
// Got some quota. Try to acquire writing privilege on the transport.
|
||||
if _, err := wait(s.ctx, s.done, s.goAway, t.shutdownChan, t.writableChan); err != nil {
|
||||
if _, ok := err.(StreamError); ok || err == io.EOF {
|
||||
// Return the connection quota back.
|
||||
t.sendQuotaPool.add(len(p))
|
||||
}
|
||||
if t.framer.adjustNumWriters(-1) == 0 {
|
||||
// This writer is the last one in this batch and has the
|
||||
// responsibility to flush the buffered frames. It queues
|
||||
// a flush request to controlBuf instead of flushing directly
|
||||
// in order to avoid the race with other writing or flushing.
|
||||
t.controlBuf.put(&flushIO{})
|
||||
}
|
||||
return err
|
||||
}
|
||||
select {
|
||||
case <-s.ctx.Done():
|
||||
t.sendQuotaPool.add(len(p))
|
||||
if t.framer.adjustNumWriters(-1) == 0 {
|
||||
t.controlBuf.put(&flushIO{})
|
||||
}
|
||||
t.writableChan <- 0
|
||||
return ContextErr(s.ctx.Err())
|
||||
default:
|
||||
}
|
||||
if oqv != atomic.LoadUint32(&t.outQuotaVersion) {
|
||||
// InitialWindowSize settings frame must have been received after we
|
||||
// acquired send quota but before we got the writable channel.
|
||||
// We must forsake this write.
|
||||
t.sendQuotaPool.add(len(p))
|
||||
s.sendQuotaPool.add(len(p))
|
||||
if t.framer.adjustNumWriters(-1) == 0 {
|
||||
t.controlBuf.put(&flushIO{})
|
||||
}
|
||||
t.writableChan <- 0
|
||||
continue
|
||||
}
|
||||
if r.Len() == 0 {
|
||||
if isLastSlice {
|
||||
if opts.Last {
|
||||
endStream = true
|
||||
// Acquire local send quota to be able to write to the controlBuf.
|
||||
ltq, err := wait(s.ctx, t.ctx, s.done, s.goAway, s.localSendQuota.acquire())
|
||||
if err != nil {
|
||||
if _, ok := err.(ConnectionError); !ok {
|
||||
t.sendQuotaPool.add(ps)
|
||||
}
|
||||
if t.framer.adjustNumWriters(0) == 1 {
|
||||
// Do a force flush iff this is last frame for the entire gRPC message
|
||||
// and the caller is the only writer at this moment.
|
||||
forceFlush = true
|
||||
}
|
||||
} else {
|
||||
isLastSlice = true
|
||||
if len(data) != 0 {
|
||||
r = bytes.NewBuffer(data)
|
||||
return err
|
||||
}
|
||||
s.localSendQuota.add(ltq - ps) // It's ok if we make it negative.
|
||||
var endStream bool
|
||||
// See if this is the last frame to be written.
|
||||
if opts.Last {
|
||||
if len(r)-size == 0 { // No more data in r after this iteration.
|
||||
if idx == 0 { // We're writing data header.
|
||||
if len(data) == 0 { // There's no data to follow.
|
||||
endStream = true
|
||||
}
|
||||
} else { // We're writing data.
|
||||
endStream = true
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
// If WriteData fails, all the pending streams will be handled
|
||||
// by http2Client.Close(). No explicit CloseStream() needs to be
|
||||
// invoked.
|
||||
if err := t.framer.writeData(forceFlush, s.id, endStream, p); err != nil {
|
||||
t.notifyError(err)
|
||||
return connectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
p = nil
|
||||
if t.framer.adjustNumWriters(-1) == 0 {
|
||||
t.framer.flushWrite()
|
||||
}
|
||||
t.writableChan <- 0
|
||||
if r.Len() == 0 {
|
||||
break
|
||||
success := func() {
|
||||
t.controlBuf.put(&dataFrame{streamID: s.id, endStream: endStream, d: p, f: func() { s.localSendQuota.add(ps) }})
|
||||
if ps < sq {
|
||||
s.sendQuotaPool.lockedAdd(sq - ps)
|
||||
}
|
||||
r = r[ps:]
|
||||
}
|
||||
failure := func() {
|
||||
s.sendQuotaPool.lockedAdd(sq)
|
||||
}
|
||||
if !s.sendQuotaPool.compareAndExecute(quotaVer, success, failure) {
|
||||
t.sendQuotaPool.add(ps)
|
||||
s.localSendQuota.add(ps)
|
||||
}
|
||||
}
|
||||
}
|
||||
if !opts.Last {
|
||||
@ -833,11 +754,11 @@ func (t *http2Client) adjustWindow(s *Stream, n uint32) {
|
||||
return
|
||||
}
|
||||
if w := s.fc.maybeAdjust(n); w > 0 {
|
||||
// Piggyback conneciton's window update along.
|
||||
// Piggyback connection's window update along.
|
||||
if cw := t.fc.resetPendingUpdate(); cw > 0 {
|
||||
t.controlBuf.put(&windowUpdate{0, cw, false})
|
||||
t.controlBuf.put(&windowUpdate{0, cw})
|
||||
}
|
||||
t.controlBuf.put(&windowUpdate{s.id, w, true})
|
||||
t.controlBuf.put(&windowUpdate{s.id, w})
|
||||
}
|
||||
}
|
||||
|
||||
@ -852,9 +773,9 @@ func (t *http2Client) updateWindow(s *Stream, n uint32) {
|
||||
}
|
||||
if w := s.fc.onRead(n); w > 0 {
|
||||
if cw := t.fc.resetPendingUpdate(); cw > 0 {
|
||||
t.controlBuf.put(&windowUpdate{0, cw, false})
|
||||
t.controlBuf.put(&windowUpdate{0, cw})
|
||||
}
|
||||
t.controlBuf.put(&windowUpdate{s.id, w, true})
|
||||
t.controlBuf.put(&windowUpdate{s.id, w})
|
||||
}
|
||||
}
|
||||
|
||||
@ -868,7 +789,7 @@ func (t *http2Client) updateFlowControl(n uint32) {
|
||||
}
|
||||
t.initialWindowSize = int32(n)
|
||||
t.mu.Unlock()
|
||||
t.controlBuf.put(&windowUpdate{0, t.fc.newLimit(n), false})
|
||||
t.controlBuf.put(&windowUpdate{0, t.fc.newLimit(n)})
|
||||
t.controlBuf.put(&settings{
|
||||
ack: false,
|
||||
ss: []http2.Setting{
|
||||
@ -898,15 +819,17 @@ func (t *http2Client) handleData(f *http2.DataFrame) {
|
||||
// Furthermore, if a bdpPing is being sent out we can piggyback
|
||||
// connection's window update for the bytes we just received.
|
||||
if sendBDPPing {
|
||||
t.controlBuf.put(&windowUpdate{0, uint32(size), false})
|
||||
if size != 0 { // Could've been an empty data frame.
|
||||
t.controlBuf.put(&windowUpdate{0, uint32(size)})
|
||||
}
|
||||
t.controlBuf.put(bdpPing)
|
||||
} else {
|
||||
if err := t.fc.onData(uint32(size)); err != nil {
|
||||
t.notifyError(connectionErrorf(true, err, "%v", err))
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
if w := t.fc.onRead(uint32(size)); w > 0 {
|
||||
t.controlBuf.put(&windowUpdate{0, w, true})
|
||||
t.controlBuf.put(&windowUpdate{0, w})
|
||||
}
|
||||
}
|
||||
// Select the right stream to dispatch.
|
||||
@ -930,7 +853,7 @@ func (t *http2Client) handleData(f *http2.DataFrame) {
|
||||
}
|
||||
if f.Header().Flags.Has(http2.FlagDataPadded) {
|
||||
if w := s.fc.onRead(uint32(size) - uint32(len(f.Data()))); w > 0 {
|
||||
t.controlBuf.put(&windowUpdate{s.id, w, true})
|
||||
t.controlBuf.put(&windowUpdate{s.id, w})
|
||||
}
|
||||
}
|
||||
s.mu.Unlock()
|
||||
@ -1019,10 +942,10 @@ func (t *http2Client) handleGoAway(f *http2.GoAwayFrame) {
|
||||
id := f.LastStreamID
|
||||
if id > 0 && id%2 != 1 {
|
||||
t.mu.Unlock()
|
||||
t.notifyError(connectionErrorf(true, nil, "received illegal http2 GOAWAY frame: stream ID %d is even", f.LastStreamID))
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
// A client can recieve multiple GoAways from server (look at https://github.com/grpc/grpc-go/issues/1387).
|
||||
// A client can receive multiple GoAways from server (look at https://github.com/grpc/grpc-go/issues/1387).
|
||||
// The idea is that the first GoAway will be sent with an ID of MaxInt32 and the second GoAway will be sent after an RTT delay
|
||||
// with the ID of the last stream the server will process.
|
||||
// Therefore, when we get the first GoAway we don't really close any streams. While in case of second GoAway we
|
||||
@ -1033,7 +956,7 @@ func (t *http2Client) handleGoAway(f *http2.GoAwayFrame) {
|
||||
// If there are multiple GoAways the first one should always have an ID greater than the following ones.
|
||||
if id > t.prevGoAwayID {
|
||||
t.mu.Unlock()
|
||||
t.notifyError(connectionErrorf(true, nil, "received illegal http2 GOAWAY frame: previously recv GOAWAY frame with LastStramID %d, currently recv %d", id, f.LastStreamID))
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
default:
|
||||
@ -1177,22 +1100,22 @@ func handleMalformedHTTP2(s *Stream, err error) {
|
||||
// TODO(zhaoq): Check the validity of the incoming frame sequence.
|
||||
func (t *http2Client) reader() {
|
||||
// Check the validity of server preface.
|
||||
frame, err := t.framer.readFrame()
|
||||
frame, err := t.framer.fr.ReadFrame()
|
||||
if err != nil {
|
||||
t.notifyError(err)
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
atomic.CompareAndSwapUint32(&t.activity, 0, 1)
|
||||
sf, ok := frame.(*http2.SettingsFrame)
|
||||
if !ok {
|
||||
t.notifyError(err)
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
t.handleSettings(sf)
|
||||
|
||||
// loop to keep reading incoming messages on this transport.
|
||||
for {
|
||||
frame, err := t.framer.readFrame()
|
||||
frame, err := t.framer.fr.ReadFrame()
|
||||
atomic.CompareAndSwapUint32(&t.activity, 0, 1)
|
||||
if err != nil {
|
||||
// Abort an active stream if the http2.Framer returns a
|
||||
@ -1204,12 +1127,12 @@ func (t *http2Client) reader() {
|
||||
t.mu.Unlock()
|
||||
if s != nil {
|
||||
// use error detail to provide better err message
|
||||
handleMalformedHTTP2(s, streamErrorf(http2ErrConvTab[se.Code], "%v", t.framer.errorDetail()))
|
||||
handleMalformedHTTP2(s, streamErrorf(http2ErrConvTab[se.Code], "%v", t.framer.fr.ErrorDetail()))
|
||||
}
|
||||
continue
|
||||
} else {
|
||||
// Transport error.
|
||||
t.notifyError(err)
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
}
|
||||
@ -1253,61 +1176,86 @@ func (t *http2Client) applySettings(ss []http2.Setting) {
|
||||
t.mu.Lock()
|
||||
for _, stream := range t.activeStreams {
|
||||
// Adjust the sending quota for each stream.
|
||||
stream.sendQuotaPool.add(int(s.Val) - int(t.streamSendQuota))
|
||||
stream.sendQuotaPool.addAndUpdate(int(s.Val) - int(t.streamSendQuota))
|
||||
}
|
||||
t.streamSendQuota = s.Val
|
||||
t.mu.Unlock()
|
||||
atomic.AddUint32(&t.outQuotaVersion, 1)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// controller running in a separate goroutine takes charge of sending control
|
||||
// frames (e.g., window update, reset stream, setting, etc.) to the server.
|
||||
func (t *http2Client) controller() {
|
||||
for {
|
||||
select {
|
||||
case i := <-t.controlBuf.get():
|
||||
t.controlBuf.load()
|
||||
select {
|
||||
case <-t.writableChan:
|
||||
switch i := i.(type) {
|
||||
case *windowUpdate:
|
||||
t.framer.writeWindowUpdate(i.flush, i.streamID, i.increment)
|
||||
case *settings:
|
||||
if i.ack {
|
||||
t.framer.writeSettingsAck(true)
|
||||
t.applySettings(i.ss)
|
||||
} else {
|
||||
t.framer.writeSettings(true, i.ss...)
|
||||
}
|
||||
case *resetStream:
|
||||
// If the server needs to be to intimated about stream closing,
|
||||
// then we need to make sure the RST_STREAM frame is written to
|
||||
// the wire before the headers of the next stream waiting on
|
||||
// streamQuota. We ensure this by adding to the streamsQuota pool
|
||||
// only after having acquired the writableChan to send RST_STREAM.
|
||||
t.streamsQuota.add(1)
|
||||
t.framer.writeRSTStream(true, i.streamID, i.code)
|
||||
case *flushIO:
|
||||
t.framer.flushWrite()
|
||||
case *ping:
|
||||
if !i.ack {
|
||||
t.bdpEst.timesnap(i.data)
|
||||
}
|
||||
t.framer.writePing(true, i.ack, i.data)
|
||||
default:
|
||||
errorf("transport: http2Client.controller got unexpected item type %v\n", i)
|
||||
}
|
||||
t.writableChan <- 0
|
||||
continue
|
||||
case <-t.shutdownChan:
|
||||
return
|
||||
}
|
||||
case <-t.shutdownChan:
|
||||
return
|
||||
// TODO(mmukhi): A lot of this code(and code in other places in the tranpsort layer)
|
||||
// is duplicated between the client and the server.
|
||||
// The transport layer needs to be refactored to take care of this.
|
||||
func (t *http2Client) itemHandler(i item) error {
|
||||
var err error
|
||||
switch i := i.(type) {
|
||||
case *dataFrame:
|
||||
err = t.framer.fr.WriteData(i.streamID, i.endStream, i.d)
|
||||
if err == nil {
|
||||
i.f()
|
||||
}
|
||||
case *headerFrame:
|
||||
t.hBuf.Reset()
|
||||
for _, f := range i.hf {
|
||||
t.hEnc.WriteField(f)
|
||||
}
|
||||
endHeaders := false
|
||||
first := true
|
||||
for !endHeaders {
|
||||
size := t.hBuf.Len()
|
||||
if size > http2MaxFrameLen {
|
||||
size = http2MaxFrameLen
|
||||
} else {
|
||||
endHeaders = true
|
||||
}
|
||||
if first {
|
||||
first = false
|
||||
err = t.framer.fr.WriteHeaders(http2.HeadersFrameParam{
|
||||
StreamID: i.streamID,
|
||||
BlockFragment: t.hBuf.Next(size),
|
||||
EndStream: i.endStream,
|
||||
EndHeaders: endHeaders,
|
||||
})
|
||||
} else {
|
||||
err = t.framer.fr.WriteContinuation(
|
||||
i.streamID,
|
||||
endHeaders,
|
||||
t.hBuf.Next(size),
|
||||
)
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
case *windowUpdate:
|
||||
err = t.framer.fr.WriteWindowUpdate(i.streamID, i.increment)
|
||||
case *settings:
|
||||
if i.ack {
|
||||
t.applySettings(i.ss)
|
||||
err = t.framer.fr.WriteSettingsAck()
|
||||
} else {
|
||||
err = t.framer.fr.WriteSettings(i.ss...)
|
||||
}
|
||||
case *resetStream:
|
||||
// If the server needs to be to intimated about stream closing,
|
||||
// then we need to make sure the RST_STREAM frame is written to
|
||||
// the wire before the headers of the next stream waiting on
|
||||
// streamQuota. We ensure this by adding to the streamsQuota pool
|
||||
// only after having acquired the writableChan to send RST_STREAM.
|
||||
err = t.framer.fr.WriteRSTStream(i.streamID, i.code)
|
||||
t.streamsQuota.add(1)
|
||||
case *flushIO:
|
||||
err = t.framer.writer.Flush()
|
||||
case *ping:
|
||||
if !i.ack {
|
||||
t.bdpEst.timesnap(i.data)
|
||||
}
|
||||
err = t.framer.fr.WritePing(i.ack, i.data)
|
||||
default:
|
||||
errorf("transport: http2Client.controller got unexpected item type %v\n", i)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// keepalive running in a separate goroutune makes sure the connection is alive by sending pings.
|
||||
@ -1331,7 +1279,7 @@ func (t *http2Client) keepalive() {
|
||||
case <-t.awakenKeepalive:
|
||||
// If the control gets here a ping has been sent
|
||||
// need to reset the timer with keepalive.Timeout.
|
||||
case <-t.shutdownChan:
|
||||
case <-t.ctx.Done():
|
||||
return
|
||||
}
|
||||
} else {
|
||||
@ -1350,13 +1298,13 @@ func (t *http2Client) keepalive() {
|
||||
}
|
||||
t.Close()
|
||||
return
|
||||
case <-t.shutdownChan:
|
||||
case <-t.ctx.Done():
|
||||
if !timer.Stop() {
|
||||
<-timer.C
|
||||
}
|
||||
return
|
||||
}
|
||||
case <-t.shutdownChan:
|
||||
case <-t.ctx.Done():
|
||||
if !timer.Stop() {
|
||||
<-timer.C
|
||||
}
|
||||
@ -1366,25 +1314,9 @@ func (t *http2Client) keepalive() {
|
||||
}
|
||||
|
||||
func (t *http2Client) Error() <-chan struct{} {
|
||||
return t.errorChan
|
||||
return t.ctx.Done()
|
||||
}
|
||||
|
||||
func (t *http2Client) GoAway() <-chan struct{} {
|
||||
return t.goAway
|
||||
}
|
||||
|
||||
func (t *http2Client) notifyError(err error) {
|
||||
t.mu.Lock()
|
||||
// make sure t.errorChan is closed only once.
|
||||
if t.state == draining {
|
||||
t.mu.Unlock()
|
||||
t.Close()
|
||||
return
|
||||
}
|
||||
if t.state == reachable {
|
||||
t.state = unreachable
|
||||
close(t.errorChan)
|
||||
infof("transport: http2Client.notifyError got notified that the client transport was broken %v.", err)
|
||||
}
|
||||
t.mu.Unlock()
|
||||
}
|
||||
|
598
cmd/vendor/google.golang.org/grpc/transport/http2_server.go
generated
vendored
598
cmd/vendor/google.golang.org/grpc/transport/http2_server.go
generated
vendored
@ -21,6 +21,7 @@ package transport
|
||||
import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
"math/rand"
|
||||
@ -51,23 +52,16 @@ var ErrIllegalHeaderWrite = errors.New("transport: the stream is done or WriteHe
|
||||
// http2Server implements the ServerTransport interface with HTTP2.
|
||||
type http2Server struct {
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
conn net.Conn
|
||||
remoteAddr net.Addr
|
||||
localAddr net.Addr
|
||||
maxStreamID uint32 // max stream ID ever seen
|
||||
authInfo credentials.AuthInfo // auth info about the connection
|
||||
inTapHandle tap.ServerInHandle
|
||||
// writableChan synchronizes write access to the transport.
|
||||
// A writer acquires the write lock by receiving a value on writableChan
|
||||
// and releases it by sending on writableChan.
|
||||
writableChan chan int
|
||||
// shutdownChan is closed when Close is called.
|
||||
// Blocking operations should select on shutdownChan to avoid
|
||||
// blocking forever after Close.
|
||||
shutdownChan chan struct{}
|
||||
framer *framer
|
||||
hBuf *bytes.Buffer // the buffer for HPACK encoding
|
||||
hEnc *hpack.Encoder // HPACK encoder
|
||||
framer *framer
|
||||
hBuf *bytes.Buffer // the buffer for HPACK encoding
|
||||
hEnc *hpack.Encoder // HPACK encoder
|
||||
// The max number of concurrent streams.
|
||||
maxStreams uint32
|
||||
// controlBuf delivers all the control related tasks (e.g., window
|
||||
@ -96,8 +90,6 @@ type http2Server struct {
|
||||
initialWindowSize int32
|
||||
bdpEst *bdpEstimator
|
||||
|
||||
outQuotaVersion uint32
|
||||
|
||||
mu sync.Mutex // guard the following
|
||||
|
||||
// drainChan is initialized when drain(...) is called the first time.
|
||||
@ -112,7 +104,7 @@ type http2Server struct {
|
||||
// the per-stream outbound flow control window size set by the peer.
|
||||
streamSendQuota uint32
|
||||
// idle is the time instant when the connection went idle.
|
||||
// This is either the begining of the connection or when the number of
|
||||
// This is either the beginning of the connection or when the number of
|
||||
// RPCs go down to 0.
|
||||
// When the connection is busy, this value is set to 0.
|
||||
idle time.Time
|
||||
@ -121,7 +113,15 @@ type http2Server struct {
|
||||
// newHTTP2Server constructs a ServerTransport based on HTTP2. ConnectionError is
|
||||
// returned if something goes wrong.
|
||||
func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err error) {
|
||||
framer := newFramer(conn)
|
||||
writeBufSize := defaultWriteBufSize
|
||||
if config.WriteBufferSize > 0 {
|
||||
writeBufSize = config.WriteBufferSize
|
||||
}
|
||||
readBufSize := defaultReadBufSize
|
||||
if config.ReadBufferSize > 0 {
|
||||
readBufSize = config.ReadBufferSize
|
||||
}
|
||||
framer := newFramer(conn, writeBufSize, readBufSize)
|
||||
// Send initial settings as connection preface to client.
|
||||
var isettings []http2.Setting
|
||||
// TODO(zhaoq): Have a better way to signal "no limit" because 0 is
|
||||
@ -151,12 +151,12 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
|
||||
ID: http2.SettingInitialWindowSize,
|
||||
Val: uint32(iwz)})
|
||||
}
|
||||
if err := framer.writeSettings(true, isettings...); err != nil {
|
||||
if err := framer.fr.WriteSettings(isettings...); err != nil {
|
||||
return nil, connectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
// Adjust the connection flow control window if needed.
|
||||
if delta := uint32(icwz - defaultWindowSize); delta > 0 {
|
||||
if err := framer.writeWindowUpdate(true, 0, delta); err != nil {
|
||||
if err := framer.fr.WriteWindowUpdate(0, delta); err != nil {
|
||||
return nil, connectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
}
|
||||
@ -183,8 +183,10 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
|
||||
kep.MinTime = defaultKeepalivePolicyMinTime
|
||||
}
|
||||
var buf bytes.Buffer
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
t := &http2Server{
|
||||
ctx: context.Background(),
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
conn: conn,
|
||||
remoteAddr: conn.RemoteAddr(),
|
||||
localAddr: conn.LocalAddr(),
|
||||
@ -198,8 +200,6 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
|
||||
fc: &inFlow{limit: uint32(icwz)},
|
||||
sendQuotaPool: newQuotaPool(defaultWindowSize),
|
||||
state: reachable,
|
||||
writableChan: make(chan int, 1),
|
||||
shutdownChan: make(chan struct{}),
|
||||
activeStreams: make(map[uint32]*Stream),
|
||||
streamSendQuota: defaultWindowSize,
|
||||
stats: config.StatsHandler,
|
||||
@ -222,9 +222,12 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
|
||||
connBegin := &stats.ConnBegin{}
|
||||
t.stats.HandleConn(t.ctx, connBegin)
|
||||
}
|
||||
go t.controller()
|
||||
t.framer.writer.Flush()
|
||||
go func() {
|
||||
loopyWriter(t.ctx, t.controlBuf, t.itemHandler)
|
||||
t.Close()
|
||||
}()
|
||||
go t.keepalive()
|
||||
t.writableChan <- 0
|
||||
return t, nil
|
||||
}
|
||||
|
||||
@ -313,6 +316,7 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(
|
||||
}
|
||||
t.maxStreamID = streamID
|
||||
s.sendQuotaPool = newQuotaPool(int(t.streamSendQuota))
|
||||
s.localSendQuota = newQuotaPool(defaultLocalSendQuota)
|
||||
t.activeStreams[streamID] = s
|
||||
if len(t.activeStreams) == 1 {
|
||||
t.idle = time.Time{}
|
||||
@ -366,7 +370,7 @@ func (t *http2Server) HandleStreams(handle func(*Stream), traceCtx func(context.
|
||||
return
|
||||
}
|
||||
|
||||
frame, err := t.framer.readFrame()
|
||||
frame, err := t.framer.fr.ReadFrame()
|
||||
if err == io.EOF || err == io.ErrUnexpectedEOF {
|
||||
t.Close()
|
||||
return
|
||||
@ -386,7 +390,7 @@ func (t *http2Server) HandleStreams(handle func(*Stream), traceCtx func(context.
|
||||
t.handleSettings(sf)
|
||||
|
||||
for {
|
||||
frame, err := t.framer.readFrame()
|
||||
frame, err := t.framer.fr.ReadFrame()
|
||||
atomic.StoreUint32(&t.activity, 1)
|
||||
if err != nil {
|
||||
if se, ok := err.(http2.StreamError); ok {
|
||||
@ -457,9 +461,9 @@ func (t *http2Server) adjustWindow(s *Stream, n uint32) {
|
||||
}
|
||||
if w := s.fc.maybeAdjust(n); w > 0 {
|
||||
if cw := t.fc.resetPendingUpdate(); cw > 0 {
|
||||
t.controlBuf.put(&windowUpdate{0, cw, false})
|
||||
t.controlBuf.put(&windowUpdate{0, cw})
|
||||
}
|
||||
t.controlBuf.put(&windowUpdate{s.id, w, true})
|
||||
t.controlBuf.put(&windowUpdate{s.id, w})
|
||||
}
|
||||
}
|
||||
|
||||
@ -474,9 +478,9 @@ func (t *http2Server) updateWindow(s *Stream, n uint32) {
|
||||
}
|
||||
if w := s.fc.onRead(n); w > 0 {
|
||||
if cw := t.fc.resetPendingUpdate(); cw > 0 {
|
||||
t.controlBuf.put(&windowUpdate{0, cw, false})
|
||||
t.controlBuf.put(&windowUpdate{0, cw})
|
||||
}
|
||||
t.controlBuf.put(&windowUpdate{s.id, w, true})
|
||||
t.controlBuf.put(&windowUpdate{s.id, w})
|
||||
}
|
||||
}
|
||||
|
||||
@ -490,7 +494,7 @@ func (t *http2Server) updateFlowControl(n uint32) {
|
||||
}
|
||||
t.initialWindowSize = int32(n)
|
||||
t.mu.Unlock()
|
||||
t.controlBuf.put(&windowUpdate{0, t.fc.newLimit(n), false})
|
||||
t.controlBuf.put(&windowUpdate{0, t.fc.newLimit(n)})
|
||||
t.controlBuf.put(&settings{
|
||||
ack: false,
|
||||
ss: []http2.Setting{
|
||||
@ -521,7 +525,9 @@ func (t *http2Server) handleData(f *http2.DataFrame) {
|
||||
// Furthermore, if a bdpPing is being sent out we can piggyback
|
||||
// connection's window update for the bytes we just received.
|
||||
if sendBDPPing {
|
||||
t.controlBuf.put(&windowUpdate{0, uint32(size), false})
|
||||
if size != 0 { // Could be an empty frame.
|
||||
t.controlBuf.put(&windowUpdate{0, uint32(size)})
|
||||
}
|
||||
t.controlBuf.put(bdpPing)
|
||||
} else {
|
||||
if err := t.fc.onData(uint32(size)); err != nil {
|
||||
@ -530,7 +536,7 @@ func (t *http2Server) handleData(f *http2.DataFrame) {
|
||||
return
|
||||
}
|
||||
if w := t.fc.onRead(uint32(size)); w > 0 {
|
||||
t.controlBuf.put(&windowUpdate{0, w, true})
|
||||
t.controlBuf.put(&windowUpdate{0, w})
|
||||
}
|
||||
}
|
||||
// Select the right stream to dispatch.
|
||||
@ -552,7 +558,7 @@ func (t *http2Server) handleData(f *http2.DataFrame) {
|
||||
}
|
||||
if f.Header().Flags.Has(http2.FlagDataPadded) {
|
||||
if w := s.fc.onRead(uint32(size) - uint32(len(f.Data()))); w > 0 {
|
||||
t.controlBuf.put(&windowUpdate{s.id, w, true})
|
||||
t.controlBuf.put(&windowUpdate{s.id, w})
|
||||
}
|
||||
}
|
||||
s.mu.Unlock()
|
||||
@ -593,10 +599,23 @@ func (t *http2Server) handleSettings(f *http2.SettingsFrame) {
|
||||
ss = append(ss, s)
|
||||
return nil
|
||||
})
|
||||
// The settings will be applied once the ack is sent.
|
||||
t.controlBuf.put(&settings{ack: true, ss: ss})
|
||||
}
|
||||
|
||||
func (t *http2Server) applySettings(ss []http2.Setting) {
|
||||
for _, s := range ss {
|
||||
if s.ID == http2.SettingInitialWindowSize {
|
||||
t.mu.Lock()
|
||||
for _, stream := range t.activeStreams {
|
||||
stream.sendQuotaPool.addAndUpdate(int(s.Val) - int(t.streamSendQuota))
|
||||
}
|
||||
t.streamSendQuota = s.Val
|
||||
t.mu.Unlock()
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
const (
|
||||
maxPingStrikes = 2
|
||||
defaultPingTimeout = 2 * time.Hour
|
||||
@ -634,7 +653,7 @@ func (t *http2Server) handlePing(f *http2.PingFrame) {
|
||||
t.mu.Unlock()
|
||||
if ns < 1 && !t.kep.PermitWithoutStream {
|
||||
// Keepalive shouldn't be active thus, this new ping should
|
||||
// have come after atleast defaultPingTimeout.
|
||||
// have come after at least defaultPingTimeout.
|
||||
if t.lastPingAt.Add(defaultPingTimeout).After(now) {
|
||||
t.pingStrikes++
|
||||
}
|
||||
@ -647,6 +666,7 @@ func (t *http2Server) handlePing(f *http2.PingFrame) {
|
||||
|
||||
if t.pingStrikes > maxPingStrikes {
|
||||
// Send goaway and close the connection.
|
||||
errorf("transport: Got to too many pings from the client, closing the connection.")
|
||||
t.controlBuf.put(&goAway{code: http2.ErrCodeEnhanceYourCalm, debugData: []byte("too_many_pings"), closeConn: true})
|
||||
}
|
||||
}
|
||||
@ -663,47 +683,16 @@ func (t *http2Server) handleWindowUpdate(f *http2.WindowUpdateFrame) {
|
||||
}
|
||||
}
|
||||
|
||||
func (t *http2Server) writeHeaders(s *Stream, b *bytes.Buffer, endStream bool) error {
|
||||
first := true
|
||||
endHeaders := false
|
||||
var err error
|
||||
defer func() {
|
||||
if err == nil {
|
||||
// Reset ping strikes when seding headers since that might cause the
|
||||
// peer to send ping.
|
||||
atomic.StoreUint32(&t.resetPingStrikes, 1)
|
||||
}
|
||||
}()
|
||||
// Sends the headers in a single batch.
|
||||
for !endHeaders {
|
||||
size := t.hBuf.Len()
|
||||
if size > http2MaxFrameLen {
|
||||
size = http2MaxFrameLen
|
||||
} else {
|
||||
endHeaders = true
|
||||
}
|
||||
if first {
|
||||
p := http2.HeadersFrameParam{
|
||||
StreamID: s.id,
|
||||
BlockFragment: b.Next(size),
|
||||
EndStream: endStream,
|
||||
EndHeaders: endHeaders,
|
||||
}
|
||||
err = t.framer.writeHeaders(endHeaders, p)
|
||||
first = false
|
||||
} else {
|
||||
err = t.framer.writeContinuation(endHeaders, s.id, endHeaders, b.Next(size))
|
||||
}
|
||||
if err != nil {
|
||||
t.Close()
|
||||
return connectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// WriteHeader sends the header metedata md back to the client.
|
||||
func (t *http2Server) WriteHeader(s *Stream, md metadata.MD) error {
|
||||
select {
|
||||
case <-s.ctx.Done():
|
||||
return ContextErr(s.ctx.Err())
|
||||
case <-t.ctx.Done():
|
||||
return ErrConnClosing
|
||||
default:
|
||||
}
|
||||
|
||||
s.mu.Lock()
|
||||
if s.headerOk || s.state == streamDone {
|
||||
s.mu.Unlock()
|
||||
@ -719,14 +708,13 @@ func (t *http2Server) WriteHeader(s *Stream, md metadata.MD) error {
|
||||
}
|
||||
md = s.header
|
||||
s.mu.Unlock()
|
||||
if _, err := wait(s.ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil {
|
||||
return err
|
||||
}
|
||||
t.hBuf.Reset()
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: ":status", Value: "200"})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "content-type", Value: "application/grpc"})
|
||||
// TODO(mmukhi): Benchmark if the perfomance gets better if count the metadata and other header fields
|
||||
// first and create a slice of that exact size.
|
||||
headerFields := make([]hpack.HeaderField, 0, 2) // at least :status, content-type will be there if none else.
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: ":status", Value: "200"})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "content-type", Value: "application/grpc"})
|
||||
if s.sendCompress != "" {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-encoding", Value: s.sendCompress})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-encoding", Value: s.sendCompress})
|
||||
}
|
||||
for k, vv := range md {
|
||||
if isReservedHeader(k) {
|
||||
@ -734,20 +722,20 @@ func (t *http2Server) WriteHeader(s *Stream, md metadata.MD) error {
|
||||
continue
|
||||
}
|
||||
for _, v := range vv {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
}
|
||||
}
|
||||
bufLen := t.hBuf.Len()
|
||||
if err := t.writeHeaders(s, t.hBuf, false); err != nil {
|
||||
return err
|
||||
}
|
||||
t.controlBuf.put(&headerFrame{
|
||||
streamID: s.id,
|
||||
hf: headerFields,
|
||||
endStream: false,
|
||||
})
|
||||
if t.stats != nil {
|
||||
outHeader := &stats.OutHeader{
|
||||
WireLength: bufLen,
|
||||
//WireLength: // TODO(mmukhi): Revisit this later, if needed.
|
||||
}
|
||||
t.stats.HandleRPC(s.Context(), outHeader)
|
||||
}
|
||||
t.writableChan <- 0
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -756,6 +744,12 @@ func (t *http2Server) WriteHeader(s *Stream, md metadata.MD) error {
|
||||
// TODO(zhaoq): Now it indicates the end of entire stream. Revisit if early
|
||||
// OK is adopted.
|
||||
func (t *http2Server) WriteStatus(s *Stream, st *status.Status) error {
|
||||
select {
|
||||
case <-t.ctx.Done():
|
||||
return ErrConnClosing
|
||||
default:
|
||||
}
|
||||
|
||||
var headersSent, hasHeader bool
|
||||
s.mu.Lock()
|
||||
if s.state == streamDone {
|
||||
@ -775,25 +769,15 @@ func (t *http2Server) WriteStatus(s *Stream, st *status.Status) error {
|
||||
headersSent = true
|
||||
}
|
||||
|
||||
// Always write a status regardless of context cancellation unless the stream
|
||||
// is terminated (e.g. by a RST_STREAM, GOAWAY, or transport error). The
|
||||
// server's application code is already done so it is fine to ignore s.ctx.
|
||||
select {
|
||||
case <-t.shutdownChan:
|
||||
return ErrConnClosing
|
||||
case <-t.writableChan:
|
||||
}
|
||||
t.hBuf.Reset()
|
||||
// TODO(mmukhi): Benchmark if the perfomance gets better if count the metadata and other header fields
|
||||
// first and create a slice of that exact size.
|
||||
headerFields := make([]hpack.HeaderField, 0, 2) // grpc-status and grpc-message will be there if none else.
|
||||
if !headersSent {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: ":status", Value: "200"})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "content-type", Value: "application/grpc"})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: ":status", Value: "200"})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "content-type", Value: "application/grpc"})
|
||||
}
|
||||
t.hEnc.WriteField(
|
||||
hpack.HeaderField{
|
||||
Name: "grpc-status",
|
||||
Value: strconv.Itoa(int(st.Code())),
|
||||
})
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-message", Value: encodeGrpcMessage(st.Message())})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-status", Value: strconv.Itoa(int(st.Code()))})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-message", Value: encodeGrpcMessage(st.Message())})
|
||||
|
||||
if p := st.Proto(); p != nil && len(p.Details) > 0 {
|
||||
stBytes, err := proto.Marshal(p)
|
||||
@ -802,7 +786,7 @@ func (t *http2Server) WriteStatus(s *Stream, st *status.Status) error {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: "grpc-status-details-bin", Value: encodeBinHeader(stBytes)})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-status-details-bin", Value: encodeBinHeader(stBytes)})
|
||||
}
|
||||
|
||||
// Attach the trailer metadata.
|
||||
@ -812,36 +796,32 @@ func (t *http2Server) WriteStatus(s *Stream, st *status.Status) error {
|
||||
continue
|
||||
}
|
||||
for _, v := range vv {
|
||||
t.hEnc.WriteField(hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
headerFields = append(headerFields, hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)})
|
||||
}
|
||||
}
|
||||
bufLen := t.hBuf.Len()
|
||||
if err := t.writeHeaders(s, t.hBuf, true); err != nil {
|
||||
t.Close()
|
||||
return err
|
||||
}
|
||||
t.controlBuf.put(&headerFrame{
|
||||
streamID: s.id,
|
||||
hf: headerFields,
|
||||
endStream: true,
|
||||
})
|
||||
if t.stats != nil {
|
||||
outTrailer := &stats.OutTrailer{
|
||||
WireLength: bufLen,
|
||||
}
|
||||
t.stats.HandleRPC(s.Context(), outTrailer)
|
||||
t.stats.HandleRPC(s.Context(), &stats.OutTrailer{})
|
||||
}
|
||||
t.closeStream(s)
|
||||
t.writableChan <- 0
|
||||
return nil
|
||||
}
|
||||
|
||||
// Write converts the data into HTTP2 data frame and sends it out. Non-nil error
|
||||
// is returns if it fails (e.g., framing error, transport error).
|
||||
func (t *http2Server) Write(s *Stream, hdr []byte, data []byte, opts *Options) (err error) {
|
||||
// TODO(zhaoq): Support multi-writers for a single stream.
|
||||
secondStart := http2MaxFrameLen - len(hdr)%http2MaxFrameLen
|
||||
if len(data) < secondStart {
|
||||
secondStart = len(data)
|
||||
select {
|
||||
case <-s.ctx.Done():
|
||||
return ContextErr(s.ctx.Err())
|
||||
case <-t.ctx.Done():
|
||||
return ErrConnClosing
|
||||
default:
|
||||
}
|
||||
hdr = append(hdr, data[:secondStart]...)
|
||||
data = data[secondStart:]
|
||||
isLastSlice := (len(data) == 0)
|
||||
|
||||
var writeHeaderFrame bool
|
||||
s.mu.Lock()
|
||||
if s.state == streamDone {
|
||||
@ -855,124 +835,74 @@ func (t *http2Server) Write(s *Stream, hdr []byte, data []byte, opts *Options) (
|
||||
if writeHeaderFrame {
|
||||
t.WriteHeader(s, nil)
|
||||
}
|
||||
r := bytes.NewBuffer(hdr)
|
||||
var (
|
||||
p []byte
|
||||
oqv uint32
|
||||
)
|
||||
for {
|
||||
if r.Len() == 0 && p == nil {
|
||||
return nil
|
||||
}
|
||||
oqv = atomic.LoadUint32(&t.outQuotaVersion)
|
||||
size := http2MaxFrameLen
|
||||
// Wait until the stream has some quota to send the data.
|
||||
sq, err := wait(s.ctx, nil, nil, t.shutdownChan, s.sendQuotaPool.acquire())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// Wait until the transport has some quota to send the data.
|
||||
tq, err := wait(s.ctx, nil, nil, t.shutdownChan, t.sendQuotaPool.acquire())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if sq < size {
|
||||
size = sq
|
||||
}
|
||||
if tq < size {
|
||||
size = tq
|
||||
}
|
||||
if p == nil {
|
||||
p = r.Next(size)
|
||||
}
|
||||
ps := len(p)
|
||||
if ps < sq {
|
||||
// Overbooked stream quota. Return it back.
|
||||
s.sendQuotaPool.add(sq - ps)
|
||||
}
|
||||
if ps < tq {
|
||||
// Overbooked transport quota. Return it back.
|
||||
t.sendQuotaPool.add(tq - ps)
|
||||
}
|
||||
t.framer.adjustNumWriters(1)
|
||||
// Got some quota. Try to acquire writing privilege on the
|
||||
// transport.
|
||||
if _, err := wait(s.ctx, nil, nil, t.shutdownChan, t.writableChan); err != nil {
|
||||
if _, ok := err.(StreamError); ok {
|
||||
// Return the connection quota back.
|
||||
t.sendQuotaPool.add(ps)
|
||||
// Add data to header frame so that we can equally distribute data across frames.
|
||||
emptyLen := http2MaxFrameLen - len(hdr)
|
||||
if emptyLen > len(data) {
|
||||
emptyLen = len(data)
|
||||
}
|
||||
hdr = append(hdr, data[:emptyLen]...)
|
||||
data = data[emptyLen:]
|
||||
for _, r := range [][]byte{hdr, data} {
|
||||
for len(r) > 0 {
|
||||
size := http2MaxFrameLen
|
||||
// Wait until the stream has some quota to send the data.
|
||||
quotaChan, quotaVer := s.sendQuotaPool.acquireWithVersion()
|
||||
sq, err := wait(s.ctx, t.ctx, nil, nil, quotaChan)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if t.framer.adjustNumWriters(-1) == 0 {
|
||||
// This writer is the last one in this batch and has the
|
||||
// responsibility to flush the buffered frames. It queues
|
||||
// a flush request to controlBuf instead of flushing directly
|
||||
// in order to avoid the race with other writing or flushing.
|
||||
t.controlBuf.put(&flushIO{})
|
||||
// Wait until the transport has some quota to send the data.
|
||||
tq, err := wait(s.ctx, t.ctx, nil, nil, t.sendQuotaPool.acquire())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return err
|
||||
}
|
||||
select {
|
||||
case <-s.ctx.Done():
|
||||
t.sendQuotaPool.add(ps)
|
||||
if t.framer.adjustNumWriters(-1) == 0 {
|
||||
t.controlBuf.put(&flushIO{})
|
||||
if sq < size {
|
||||
size = sq
|
||||
}
|
||||
t.writableChan <- 0
|
||||
return ContextErr(s.ctx.Err())
|
||||
default:
|
||||
}
|
||||
if oqv != atomic.LoadUint32(&t.outQuotaVersion) {
|
||||
// InitialWindowSize settings frame must have been received after we
|
||||
// acquired send quota but before we got the writable channel.
|
||||
// We must forsake this write.
|
||||
t.sendQuotaPool.add(ps)
|
||||
s.sendQuotaPool.add(ps)
|
||||
if t.framer.adjustNumWriters(-1) == 0 {
|
||||
t.controlBuf.put(&flushIO{})
|
||||
if tq < size {
|
||||
size = tq
|
||||
}
|
||||
t.writableChan <- 0
|
||||
continue
|
||||
}
|
||||
var forceFlush bool
|
||||
if r.Len() == 0 {
|
||||
if isLastSlice {
|
||||
if t.framer.adjustNumWriters(0) == 1 && !opts.Last {
|
||||
forceFlush = true
|
||||
if size > len(r) {
|
||||
size = len(r)
|
||||
}
|
||||
p := r[:size]
|
||||
ps := len(p)
|
||||
if ps < tq {
|
||||
// Overbooked transport quota. Return it back.
|
||||
t.sendQuotaPool.add(tq - ps)
|
||||
}
|
||||
// Acquire local send quota to be able to write to the controlBuf.
|
||||
ltq, err := wait(s.ctx, t.ctx, nil, nil, s.localSendQuota.acquire())
|
||||
if err != nil {
|
||||
if _, ok := err.(ConnectionError); !ok {
|
||||
t.sendQuotaPool.add(ps)
|
||||
}
|
||||
} else {
|
||||
r = bytes.NewBuffer(data)
|
||||
isLastSlice = true
|
||||
return err
|
||||
}
|
||||
s.localSendQuota.add(ltq - ps) // It's ok we make this negative.
|
||||
// Reset ping strikes when sending data since this might cause
|
||||
// the peer to send ping.
|
||||
atomic.StoreUint32(&t.resetPingStrikes, 1)
|
||||
success := func() {
|
||||
t.controlBuf.put(&dataFrame{streamID: s.id, endStream: false, d: p, f: func() {
|
||||
s.localSendQuota.add(ps)
|
||||
}})
|
||||
if ps < sq {
|
||||
// Overbooked stream quota. Return it back.
|
||||
s.sendQuotaPool.lockedAdd(sq - ps)
|
||||
}
|
||||
r = r[ps:]
|
||||
}
|
||||
failure := func() {
|
||||
s.sendQuotaPool.lockedAdd(sq)
|
||||
}
|
||||
if !s.sendQuotaPool.compareAndExecute(quotaVer, success, failure) {
|
||||
t.sendQuotaPool.add(ps)
|
||||
s.localSendQuota.add(ps)
|
||||
}
|
||||
}
|
||||
// Reset ping strikes when sending data since this might cause
|
||||
// the peer to send ping.
|
||||
atomic.StoreUint32(&t.resetPingStrikes, 1)
|
||||
if err := t.framer.writeData(forceFlush, s.id, false, p); err != nil {
|
||||
t.Close()
|
||||
return connectionErrorf(true, err, "transport: %v", err)
|
||||
}
|
||||
p = nil
|
||||
if t.framer.adjustNumWriters(-1) == 0 {
|
||||
t.framer.flushWrite()
|
||||
}
|
||||
t.writableChan <- 0
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func (t *http2Server) applySettings(ss []http2.Setting) {
|
||||
for _, s := range ss {
|
||||
if s.ID == http2.SettingInitialWindowSize {
|
||||
t.mu.Lock()
|
||||
defer t.mu.Unlock()
|
||||
for _, stream := range t.activeStreams {
|
||||
stream.sendQuotaPool.add(int(s.Val) - int(t.streamSendQuota))
|
||||
}
|
||||
t.streamSendQuota = s.Val
|
||||
atomic.AddUint32(&t.outQuotaVersion, 1)
|
||||
}
|
||||
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// keepalive running in a separate goroutine does the following:
|
||||
@ -988,7 +918,7 @@ func (t *http2Server) keepalive() {
|
||||
maxAge := time.NewTimer(t.kp.MaxConnectionAge)
|
||||
keepalive := time.NewTimer(t.kp.Time)
|
||||
// NOTE: All exit paths of this function should reset their
|
||||
// respecitve timers. A failure to do so will cause the
|
||||
// respective timers. A failure to do so will cause the
|
||||
// following clean-up to deadlock and eventually leak.
|
||||
defer func() {
|
||||
if !maxIdle.Stop() {
|
||||
@ -1031,7 +961,7 @@ func (t *http2Server) keepalive() {
|
||||
t.Close()
|
||||
// Reseting the timer so that the clean-up doesn't deadlock.
|
||||
maxAge.Reset(infinity)
|
||||
case <-t.shutdownChan:
|
||||
case <-t.ctx.Done():
|
||||
}
|
||||
return
|
||||
case <-keepalive.C:
|
||||
@ -1049,7 +979,7 @@ func (t *http2Server) keepalive() {
|
||||
pingSent = true
|
||||
t.controlBuf.put(p)
|
||||
keepalive.Reset(t.kp.Timeout)
|
||||
case <-t.shutdownChan:
|
||||
case <-t.ctx.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
@ -1057,93 +987,129 @@ func (t *http2Server) keepalive() {
|
||||
|
||||
var goAwayPing = &ping{data: [8]byte{1, 6, 1, 8, 0, 3, 3, 9}}
|
||||
|
||||
// controller running in a separate goroutine takes charge of sending control
|
||||
// frames (e.g., window update, reset stream, setting, etc.) to the server.
|
||||
func (t *http2Server) controller() {
|
||||
for {
|
||||
select {
|
||||
case i := <-t.controlBuf.get():
|
||||
t.controlBuf.load()
|
||||
// TODO(mmukhi): A lot of this code(and code in other places in the tranpsort layer)
|
||||
// is duplicated between the client and the server.
|
||||
// The transport layer needs to be refactored to take care of this.
|
||||
func (t *http2Server) itemHandler(i item) error {
|
||||
switch i := i.(type) {
|
||||
case *dataFrame:
|
||||
if err := t.framer.fr.WriteData(i.streamID, i.endStream, i.d); err != nil {
|
||||
return err
|
||||
}
|
||||
i.f()
|
||||
return nil
|
||||
case *headerFrame:
|
||||
t.hBuf.Reset()
|
||||
for _, f := range i.hf {
|
||||
t.hEnc.WriteField(f)
|
||||
}
|
||||
first := true
|
||||
endHeaders := false
|
||||
for !endHeaders {
|
||||
size := t.hBuf.Len()
|
||||
if size > http2MaxFrameLen {
|
||||
size = http2MaxFrameLen
|
||||
} else {
|
||||
endHeaders = true
|
||||
}
|
||||
var err error
|
||||
if first {
|
||||
first = false
|
||||
err = t.framer.fr.WriteHeaders(http2.HeadersFrameParam{
|
||||
StreamID: i.streamID,
|
||||
BlockFragment: t.hBuf.Next(size),
|
||||
EndStream: i.endStream,
|
||||
EndHeaders: endHeaders,
|
||||
})
|
||||
} else {
|
||||
err = t.framer.fr.WriteContinuation(
|
||||
i.streamID,
|
||||
endHeaders,
|
||||
t.hBuf.Next(size),
|
||||
)
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
atomic.StoreUint32(&t.resetPingStrikes, 1)
|
||||
return nil
|
||||
case *windowUpdate:
|
||||
return t.framer.fr.WriteWindowUpdate(i.streamID, i.increment)
|
||||
case *settings:
|
||||
if i.ack {
|
||||
t.applySettings(i.ss)
|
||||
return t.framer.fr.WriteSettingsAck()
|
||||
}
|
||||
return t.framer.fr.WriteSettings(i.ss...)
|
||||
case *resetStream:
|
||||
return t.framer.fr.WriteRSTStream(i.streamID, i.code)
|
||||
case *goAway:
|
||||
t.mu.Lock()
|
||||
if t.state == closing {
|
||||
t.mu.Unlock()
|
||||
// The transport is closing.
|
||||
return fmt.Errorf("transport: Connection closing")
|
||||
}
|
||||
sid := t.maxStreamID
|
||||
if !i.headsUp {
|
||||
// Stop accepting more streams now.
|
||||
t.state = draining
|
||||
t.mu.Unlock()
|
||||
if err := t.framer.fr.WriteGoAway(sid, i.code, i.debugData); err != nil {
|
||||
return err
|
||||
}
|
||||
if i.closeConn {
|
||||
// Abruptly close the connection following the GoAway (via
|
||||
// loopywriter). But flush out what's inside the buffer first.
|
||||
t.framer.writer.Flush()
|
||||
return fmt.Errorf("transport: Connection closing")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
t.mu.Unlock()
|
||||
// For a graceful close, send out a GoAway with stream ID of MaxUInt32,
|
||||
// Follow that with a ping and wait for the ack to come back or a timer
|
||||
// to expire. During this time accept new streams since they might have
|
||||
// originated before the GoAway reaches the client.
|
||||
// After getting the ack or timer expiration send out another GoAway this
|
||||
// time with an ID of the max stream server intends to process.
|
||||
if err := t.framer.fr.WriteGoAway(math.MaxUint32, http2.ErrCodeNo, []byte{}); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := t.framer.fr.WritePing(false, goAwayPing.data); err != nil {
|
||||
return err
|
||||
}
|
||||
go func() {
|
||||
timer := time.NewTimer(time.Minute)
|
||||
defer timer.Stop()
|
||||
select {
|
||||
case <-t.writableChan:
|
||||
switch i := i.(type) {
|
||||
case *windowUpdate:
|
||||
t.framer.writeWindowUpdate(i.flush, i.streamID, i.increment)
|
||||
case *settings:
|
||||
if i.ack {
|
||||
t.framer.writeSettingsAck(true)
|
||||
t.applySettings(i.ss)
|
||||
} else {
|
||||
t.framer.writeSettings(true, i.ss...)
|
||||
}
|
||||
case *resetStream:
|
||||
t.framer.writeRSTStream(true, i.streamID, i.code)
|
||||
case *goAway:
|
||||
t.mu.Lock()
|
||||
if t.state == closing {
|
||||
t.mu.Unlock()
|
||||
// The transport is closing.
|
||||
return
|
||||
}
|
||||
sid := t.maxStreamID
|
||||
if !i.headsUp {
|
||||
// Stop accepting more streams now.
|
||||
t.state = draining
|
||||
activeStreams := len(t.activeStreams)
|
||||
t.mu.Unlock()
|
||||
t.framer.writeGoAway(true, sid, i.code, i.debugData)
|
||||
if i.closeConn || activeStreams == 0 {
|
||||
// Abruptly close the connection following the GoAway.
|
||||
t.Close()
|
||||
}
|
||||
t.writableChan <- 0
|
||||
continue
|
||||
}
|
||||
t.mu.Unlock()
|
||||
// For a graceful close, send out a GoAway with stream ID of MaxUInt32,
|
||||
// Follow that with a ping and wait for the ack to come back or a timer
|
||||
// to expire. During this time accept new streams since they might have
|
||||
// originated before the GoAway reaches the client.
|
||||
// After getting the ack or timer expiration send out another GoAway this
|
||||
// time with an ID of the max stream server intends to process.
|
||||
t.framer.writeGoAway(true, math.MaxUint32, http2.ErrCodeNo, []byte{})
|
||||
t.framer.writePing(true, false, goAwayPing.data)
|
||||
go func() {
|
||||
timer := time.NewTimer(time.Minute)
|
||||
defer timer.Stop()
|
||||
select {
|
||||
case <-t.drainChan:
|
||||
case <-timer.C:
|
||||
case <-t.shutdownChan:
|
||||
return
|
||||
}
|
||||
t.controlBuf.put(&goAway{code: i.code, debugData: i.debugData})
|
||||
}()
|
||||
case *flushIO:
|
||||
t.framer.flushWrite()
|
||||
case *ping:
|
||||
if !i.ack {
|
||||
t.bdpEst.timesnap(i.data)
|
||||
}
|
||||
t.framer.writePing(true, i.ack, i.data)
|
||||
default:
|
||||
errorf("transport: http2Server.controller got unexpected item type %v\n", i)
|
||||
}
|
||||
t.writableChan <- 0
|
||||
continue
|
||||
case <-t.shutdownChan:
|
||||
case <-t.drainChan:
|
||||
case <-timer.C:
|
||||
case <-t.ctx.Done():
|
||||
return
|
||||
}
|
||||
case <-t.shutdownChan:
|
||||
return
|
||||
t.controlBuf.put(&goAway{code: i.code, debugData: i.debugData})
|
||||
}()
|
||||
return nil
|
||||
case *flushIO:
|
||||
return t.framer.writer.Flush()
|
||||
case *ping:
|
||||
if !i.ack {
|
||||
t.bdpEst.timesnap(i.data)
|
||||
}
|
||||
return t.framer.fr.WritePing(i.ack, i.data)
|
||||
default:
|
||||
err := status.Errorf(codes.Internal, "transport: http2Server.controller got unexpected item type %t", i)
|
||||
errorf("%v", err)
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Close starts shutting down the http2Server transport.
|
||||
// TODO(zhaoq): Now the destruction is not blocked on any pending streams. This
|
||||
// could cause some resource issue. Revisit this later.
|
||||
func (t *http2Server) Close() (err error) {
|
||||
func (t *http2Server) Close() error {
|
||||
t.mu.Lock()
|
||||
if t.state == closing {
|
||||
t.mu.Unlock()
|
||||
@ -1153,8 +1119,8 @@ func (t *http2Server) Close() (err error) {
|
||||
streams := t.activeStreams
|
||||
t.activeStreams = nil
|
||||
t.mu.Unlock()
|
||||
close(t.shutdownChan)
|
||||
err = t.conn.Close()
|
||||
t.cancel()
|
||||
err := t.conn.Close()
|
||||
// Cancel all active streams.
|
||||
for _, s := range streams {
|
||||
s.cancel()
|
||||
@ -1163,7 +1129,7 @@ func (t *http2Server) Close() (err error) {
|
||||
connEnd := &stats.ConnEnd{}
|
||||
t.stats.HandleConn(t.ctx, connEnd)
|
||||
}
|
||||
return
|
||||
return err
|
||||
}
|
||||
|
||||
// closeStream clears the footprint of a stream when the stream is not needed
|
||||
|
139
cmd/vendor/google.golang.org/grpc/transport/http_util.go
generated
vendored
139
cmd/vendor/google.golang.org/grpc/transport/http_util.go
generated
vendored
@ -28,7 +28,6 @@ import (
|
||||
"net/http"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
@ -45,7 +44,8 @@ const (
|
||||
// http://http2.github.io/http2-spec/#SettingValues
|
||||
http2InitHeaderTableSize = 4096
|
||||
// http2IOBufSize specifies the buffer size for sending frames.
|
||||
http2IOBufSize = 32 * 1024
|
||||
defaultWriteBufSize = 32 * 1024
|
||||
defaultReadBufSize = 32 * 1024
|
||||
)
|
||||
|
||||
var (
|
||||
@ -475,10 +475,10 @@ type framer struct {
|
||||
fr *http2.Framer
|
||||
}
|
||||
|
||||
func newFramer(conn net.Conn) *framer {
|
||||
func newFramer(conn net.Conn, writeBufferSize, readBufferSize int) *framer {
|
||||
f := &framer{
|
||||
reader: bufio.NewReaderSize(conn, http2IOBufSize),
|
||||
writer: bufio.NewWriterSize(conn, http2IOBufSize),
|
||||
reader: bufio.NewReaderSize(conn, readBufferSize),
|
||||
writer: bufio.NewWriterSize(conn, writeBufferSize),
|
||||
}
|
||||
f.fr = http2.NewFramer(f.writer, f.reader)
|
||||
// Opt-in to Frame reuse API on framer to reduce garbage.
|
||||
@ -487,132 +487,3 @@ func newFramer(conn net.Conn) *framer {
|
||||
f.fr.ReadMetaHeaders = hpack.NewDecoder(http2InitHeaderTableSize, nil)
|
||||
return f
|
||||
}
|
||||
|
||||
func (f *framer) adjustNumWriters(i int32) int32 {
|
||||
return atomic.AddInt32(&f.numWriters, i)
|
||||
}
|
||||
|
||||
// The following writeXXX functions can only be called when the caller gets
|
||||
// unblocked from writableChan channel (i.e., owns the privilege to write).
|
||||
|
||||
func (f *framer) writeContinuation(forceFlush bool, streamID uint32, endHeaders bool, headerBlockFragment []byte) error {
|
||||
if err := f.fr.WriteContinuation(streamID, endHeaders, headerBlockFragment); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) writeData(forceFlush bool, streamID uint32, endStream bool, data []byte) error {
|
||||
if err := f.fr.WriteData(streamID, endStream, data); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) writeGoAway(forceFlush bool, maxStreamID uint32, code http2.ErrCode, debugData []byte) error {
|
||||
if err := f.fr.WriteGoAway(maxStreamID, code, debugData); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) writeHeaders(forceFlush bool, p http2.HeadersFrameParam) error {
|
||||
if err := f.fr.WriteHeaders(p); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) writePing(forceFlush, ack bool, data [8]byte) error {
|
||||
if err := f.fr.WritePing(ack, data); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) writePriority(forceFlush bool, streamID uint32, p http2.PriorityParam) error {
|
||||
if err := f.fr.WritePriority(streamID, p); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) writePushPromise(forceFlush bool, p http2.PushPromiseParam) error {
|
||||
if err := f.fr.WritePushPromise(p); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) writeRSTStream(forceFlush bool, streamID uint32, code http2.ErrCode) error {
|
||||
if err := f.fr.WriteRSTStream(streamID, code); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) writeSettings(forceFlush bool, settings ...http2.Setting) error {
|
||||
if err := f.fr.WriteSettings(settings...); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) writeSettingsAck(forceFlush bool) error {
|
||||
if err := f.fr.WriteSettingsAck(); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) writeWindowUpdate(forceFlush bool, streamID, incr uint32) error {
|
||||
if err := f.fr.WriteWindowUpdate(streamID, incr); err != nil {
|
||||
return err
|
||||
}
|
||||
if forceFlush {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *framer) flushWrite() error {
|
||||
return f.writer.Flush()
|
||||
}
|
||||
|
||||
func (f *framer) readFrame() (http2.Frame, error) {
|
||||
return f.fr.ReadFrame()
|
||||
}
|
||||
|
||||
func (f *framer) errorDetail() error {
|
||||
return f.fr.ErrorDetail()
|
||||
}
|
||||
|
117
cmd/vendor/google.golang.org/grpc/transport/transport.go
generated
vendored
117
cmd/vendor/google.golang.org/grpc/transport/transport.go
generated
vendored
@ -21,10 +21,12 @@
|
||||
package transport // import "google.golang.org/grpc/transport"
|
||||
|
||||
import (
|
||||
stdctx "context"
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
"golang.org/x/net/http2"
|
||||
@ -67,20 +69,20 @@ func newRecvBuffer() *recvBuffer {
|
||||
|
||||
func (b *recvBuffer) put(r recvMsg) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if len(b.backlog) == 0 {
|
||||
select {
|
||||
case b.c <- r:
|
||||
b.mu.Unlock()
|
||||
return
|
||||
default:
|
||||
}
|
||||
}
|
||||
b.backlog = append(b.backlog, r)
|
||||
b.mu.Unlock()
|
||||
}
|
||||
|
||||
func (b *recvBuffer) load() {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if len(b.backlog) > 0 {
|
||||
select {
|
||||
case b.c <- b.backlog[0]:
|
||||
@ -89,6 +91,7 @@ func (b *recvBuffer) load() {
|
||||
default:
|
||||
}
|
||||
}
|
||||
b.mu.Unlock()
|
||||
}
|
||||
|
||||
// get returns the channel that receives a recvMsg in the buffer.
|
||||
@ -164,20 +167,20 @@ func newControlBuffer() *controlBuffer {
|
||||
|
||||
func (b *controlBuffer) put(r item) {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if len(b.backlog) == 0 {
|
||||
select {
|
||||
case b.c <- r:
|
||||
b.mu.Unlock()
|
||||
return
|
||||
default:
|
||||
}
|
||||
}
|
||||
b.backlog = append(b.backlog, r)
|
||||
b.mu.Unlock()
|
||||
}
|
||||
|
||||
func (b *controlBuffer) load() {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
if len(b.backlog) > 0 {
|
||||
select {
|
||||
case b.c <- b.backlog[0]:
|
||||
@ -186,6 +189,7 @@ func (b *controlBuffer) load() {
|
||||
default:
|
||||
}
|
||||
}
|
||||
b.mu.Unlock()
|
||||
}
|
||||
|
||||
// get returns the channel that receives an item in the buffer.
|
||||
@ -235,7 +239,8 @@ type Stream struct {
|
||||
// is used to adjust flow control, if need be.
|
||||
requestRead func(int)
|
||||
|
||||
sendQuotaPool *quotaPool
|
||||
sendQuotaPool *quotaPool
|
||||
localSendQuota *quotaPool
|
||||
// Close headerChan to indicate the end of reception of header metadata.
|
||||
headerChan chan struct{}
|
||||
// header caches the received header metadata.
|
||||
@ -313,8 +318,9 @@ func (s *Stream) Header() (metadata.MD, error) {
|
||||
// side only.
|
||||
func (s *Stream) Trailer() metadata.MD {
|
||||
s.mu.RLock()
|
||||
defer s.mu.RUnlock()
|
||||
return s.trailer.Copy()
|
||||
c := s.trailer.Copy()
|
||||
s.mu.RUnlock()
|
||||
return c
|
||||
}
|
||||
|
||||
// ServerTransport returns the underlying ServerTransport for the stream.
|
||||
@ -342,14 +348,16 @@ func (s *Stream) Status() *status.Status {
|
||||
// Server side only.
|
||||
func (s *Stream) SetHeader(md metadata.MD) error {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
if s.headerOk || s.state == streamDone {
|
||||
s.mu.Unlock()
|
||||
return ErrIllegalHeaderWrite
|
||||
}
|
||||
if md.Len() == 0 {
|
||||
s.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
s.header = metadata.Join(s.header, md)
|
||||
s.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -360,8 +368,8 @@ func (s *Stream) SetTrailer(md metadata.MD) error {
|
||||
return nil
|
||||
}
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
s.trailer = metadata.Join(s.trailer, md)
|
||||
s.mu.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -412,15 +420,17 @@ func (s *Stream) finish(st *status.Status) {
|
||||
// BytesSent indicates whether any bytes have been sent on this stream.
|
||||
func (s *Stream) BytesSent() bool {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
return s.bytesSent
|
||||
bs := s.bytesSent
|
||||
s.mu.Unlock()
|
||||
return bs
|
||||
}
|
||||
|
||||
// BytesReceived indicates whether any bytes have been received on this stream.
|
||||
func (s *Stream) BytesReceived() bool {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
return s.bytesReceived
|
||||
br := s.bytesReceived
|
||||
s.mu.Unlock()
|
||||
return br
|
||||
}
|
||||
|
||||
// GoString is implemented by Stream so context.String() won't
|
||||
@ -449,7 +459,6 @@ type transportState int
|
||||
|
||||
const (
|
||||
reachable transportState = iota
|
||||
unreachable
|
||||
closing
|
||||
draining
|
||||
)
|
||||
@ -464,6 +473,8 @@ type ServerConfig struct {
|
||||
KeepalivePolicy keepalive.EnforcementPolicy
|
||||
InitialWindowSize int32
|
||||
InitialConnWindowSize int32
|
||||
WriteBufferSize int
|
||||
ReadBufferSize int
|
||||
}
|
||||
|
||||
// NewServerTransport creates a ServerTransport with conn or non-nil error
|
||||
@ -491,10 +502,14 @@ type ConnectOptions struct {
|
||||
KeepaliveParams keepalive.ClientParameters
|
||||
// StatsHandler stores the handler for stats.
|
||||
StatsHandler stats.Handler
|
||||
// InitialWindowSize sets the intial window size for a stream.
|
||||
// InitialWindowSize sets the initial window size for a stream.
|
||||
InitialWindowSize int32
|
||||
// InitialConnWindowSize sets the intial window size for a connection.
|
||||
// InitialConnWindowSize sets the initial window size for a connection.
|
||||
InitialConnWindowSize int32
|
||||
// WriteBufferSize sets the size of write buffer which in turn determines how much data can be batched before it's written on the wire.
|
||||
WriteBufferSize int
|
||||
// ReadBufferSize sets the size of read buffer, which in turn determines how much data can be read at most for one read syscall.
|
||||
ReadBufferSize int
|
||||
}
|
||||
|
||||
// TargetInfo contains the information of the target such as network address and metadata.
|
||||
@ -505,8 +520,8 @@ type TargetInfo struct {
|
||||
|
||||
// NewClientTransport establishes the transport with the required ConnectOptions
|
||||
// and returns it to the caller.
|
||||
func NewClientTransport(ctx context.Context, target TargetInfo, opts ConnectOptions) (ClientTransport, error) {
|
||||
return newHTTP2Client(ctx, target, opts)
|
||||
func NewClientTransport(ctx context.Context, target TargetInfo, opts ConnectOptions, timeout time.Duration) (ClientTransport, error) {
|
||||
return newHTTP2Client(ctx, target, opts, timeout)
|
||||
}
|
||||
|
||||
// Options provides additional hints and information for message
|
||||
@ -518,7 +533,7 @@ type Options struct {
|
||||
|
||||
// Delay is a hint to the transport implementation for whether
|
||||
// the data could be buffered for a batching write. The
|
||||
// Transport implementation may ignore the hint.
|
||||
// transport implementation may ignore the hint.
|
||||
Delay bool
|
||||
}
|
||||
|
||||
@ -688,34 +703,33 @@ func (e StreamError) Error() string {
|
||||
return fmt.Sprintf("stream error: code = %s desc = %q", e.Code, e.Desc)
|
||||
}
|
||||
|
||||
// wait blocks until it can receive from ctx.Done, closing, or proceed.
|
||||
// If it receives from ctx.Done, it returns 0, the StreamError for ctx.Err.
|
||||
// If it receives from done, it returns 0, io.EOF if ctx is not done; otherwise
|
||||
// it return the StreamError for ctx.Err.
|
||||
// If it receives from goAway, it returns 0, ErrStreamDrain.
|
||||
// If it receives from closing, it returns 0, ErrConnClosing.
|
||||
// If it receives from proceed, it returns the received integer, nil.
|
||||
func wait(ctx context.Context, done, goAway, closing <-chan struct{}, proceed <-chan int) (int, error) {
|
||||
// wait blocks until it can receive from one of the provided contexts or channels
|
||||
func wait(ctx, tctx context.Context, done, goAway <-chan struct{}, proceed <-chan int) (int, error) {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return 0, ContextErr(ctx.Err())
|
||||
case <-done:
|
||||
// User cancellation has precedence.
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return 0, ContextErr(ctx.Err())
|
||||
default:
|
||||
}
|
||||
return 0, io.EOF
|
||||
case <-goAway:
|
||||
return 0, ErrStreamDrain
|
||||
case <-closing:
|
||||
case <-tctx.Done():
|
||||
return 0, ErrConnClosing
|
||||
case i := <-proceed:
|
||||
return i, nil
|
||||
}
|
||||
}
|
||||
|
||||
// ContextErr converts the error from context package into a StreamError.
|
||||
func ContextErr(err error) StreamError {
|
||||
switch err {
|
||||
case context.DeadlineExceeded, stdctx.DeadlineExceeded:
|
||||
return streamErrorf(codes.DeadlineExceeded, "%v", err)
|
||||
case context.Canceled, stdctx.Canceled:
|
||||
return streamErrorf(codes.Canceled, "%v", err)
|
||||
}
|
||||
return streamErrorf(codes.Internal, "Unexpected error from context packet: %v", err)
|
||||
}
|
||||
|
||||
// GoAwayReason contains the reason for the GoAway frame received.
|
||||
type GoAwayReason uint8
|
||||
|
||||
@ -725,6 +739,39 @@ const (
|
||||
// NoReason is the default value when GoAway frame is received.
|
||||
NoReason GoAwayReason = 1
|
||||
// TooManyPings indicates that a GoAway frame with ErrCodeEnhanceYourCalm
|
||||
// was recieved and that the debug data said "too_many_pings".
|
||||
// was received and that the debug data said "too_many_pings".
|
||||
TooManyPings GoAwayReason = 2
|
||||
)
|
||||
|
||||
// loopyWriter is run in a separate go routine. It is the single code path that will
|
||||
// write data on wire.
|
||||
func loopyWriter(ctx context.Context, cbuf *controlBuffer, handler func(item) error) {
|
||||
for {
|
||||
select {
|
||||
case i := <-cbuf.get():
|
||||
cbuf.load()
|
||||
if err := handler(i); err != nil {
|
||||
return
|
||||
}
|
||||
case <-ctx.Done():
|
||||
return
|
||||
}
|
||||
hasData:
|
||||
for {
|
||||
select {
|
||||
case i := <-cbuf.get():
|
||||
cbuf.load()
|
||||
if err := handler(i); err != nil {
|
||||
return
|
||||
}
|
||||
case <-ctx.Done():
|
||||
return
|
||||
default:
|
||||
if err := handler(&flushIO{}); err != nil {
|
||||
return
|
||||
}
|
||||
break hasData
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
8
glide.lock
generated
8
glide.lock
generated
@ -1,5 +1,5 @@
|
||||
hash: cff74aae5a6b8c11816c9994dedfdfdcd9f4137d61d8ed8ba0bf623f0ff21d50
|
||||
updated: 2017-11-10T09:46:28.3753-08:00
|
||||
hash: 8d556efcf8d917aba74445bd98eb9de3abcf981ca70adb53276026c4c3ecfcfd
|
||||
updated: 2017-11-10T12:07:01.187305-08:00
|
||||
imports:
|
||||
- name: github.com/beorn7/perks
|
||||
version: 4c0e84591b9aa9e6dcfdf3e020114cd81f89d5f9
|
||||
@ -158,8 +158,9 @@ imports:
|
||||
subpackages:
|
||||
- googleapis/rpc/status
|
||||
- name: google.golang.org/grpc
|
||||
version: f92cdcd7dcdc69e81b2d7b338479a19a8723cfa3
|
||||
version: 5ffe3083946d5603a0578721101dc8165b1d5b5f
|
||||
subpackages:
|
||||
- balancer
|
||||
- codes
|
||||
- connectivity
|
||||
- credentials
|
||||
@ -172,6 +173,7 @@ imports:
|
||||
- metadata
|
||||
- naming
|
||||
- peer
|
||||
- resolver
|
||||
- stats
|
||||
- status
|
||||
- tap
|
||||
|
@ -108,7 +108,7 @@ import:
|
||||
subpackages:
|
||||
- rate
|
||||
- package: google.golang.org/grpc
|
||||
version: v1.6.0
|
||||
version: v1.7.2
|
||||
subpackages:
|
||||
- codes
|
||||
- credentials
|
||||
|
Loading…
x
Reference in New Issue
Block a user