diff --git a/CHANGELOG-3.4.md b/CHANGELOG-3.4.md index 477cc33e103..1b5069deeb3 100644 --- a/CHANGELOG-3.4.md +++ b/CHANGELOG-3.4.md @@ -9,7 +9,7 @@ See [code changes](https://github.com/coreos/etcd/compare/v3.3.0...v3.4.0) and [ ### Improved -- Rewrite [client balancer](TODO) with [new gRPC balancer interface](TODO). +- Rewrite [client balancer](https://github.com/coreos/etcd/pull/9860) with [new gRPC balancer interface](https://github.com/coreos/etcd/issues/9106). - Add [backoff on watch retries on transient errors](https://github.com/coreos/etcd/pull/9840). - Add [jitter to watch progress notify](https://github.com/coreos/etcd/pull/9278) to prevent [spikes in `etcd_network_client_grpc_sent_bytes_total`](https://github.com/coreos/etcd/issues/9246). - Improve [slow request apply warning log](https://github.com/coreos/etcd/pull/9288). diff --git a/Gopkg.lock b/Gopkg.lock index 9aec626a676..19ab053bac6 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -100,6 +100,12 @@ packages = ["."] revision = "4201258b820c74ac8e6922fc9e6b52f71fe46f8d" +[[projects]] + name = "github.com/grpc-ecosystem/go-grpc-middleware" + packages = ["util/backoffutils"] + revision = "c250d6563d4d4c20252cd865923440e829844f4e" + version = "v1.0.0" + [[projects]] name = "github.com/grpc-ecosystem/go-grpc-prometheus" packages = ["."] @@ -335,9 +341,14 @@ packages = [ ".", "balancer", + "balancer/base", + "balancer/roundrobin", + "channelz", "codes", "connectivity", "credentials", + "encoding", + "encoding/proto", "grpclb/grpc_lb_v1/messages", "grpclog", "health", @@ -348,13 +359,15 @@ "naming", "peer", "resolver", + "resolver/dns", + "resolver/passthrough", "stats", "status", "tap", "transport" ] - revision = "5b3c4e850e90a4cf6a20ebd46c8b32a0a3afcb9e" - version = "v1.7.5" + revision = "7a6a684ca69eb4cae85ad0a484f2e531598c047b" + version = "v1.12.2" [[projects]] name = "gopkg.in/cheggaaa/pb.v1" @@ -371,6 +384,6 @@ [solve-meta] analyzer-name = "dep" analyzer-version = 1 - inputs-digest = "943bf7648c0129f59546321f569622e933f24a103b5d68525b82d5e47d52733f" + inputs-digest = "40c72e54abdb7d6c044f2f5fabb8fc7ac2eadcab047cc22b28d26b7b145b5e1b" solver-name = "gps-cdcl" solver-version = 1 diff --git a/bill-of-materials.json b/bill-of-materials.json index 42e73e61f12..c11992df879 100644 --- a/bill-of-materials.json +++ b/bill-of-materials.json @@ -134,6 +134,15 @@ } ] }, + { + "project": "github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils", + "licenses": [ + { + "type": "Apache License 2.0", + "confidence": 1 + } + ] + }, { "project": "github.com/grpc-ecosystem/go-grpc-prometheus", "licenses": [ diff --git a/clientv3/auth.go b/clientv3/auth.go index 0238920321c..bedbd132c18 100644 --- a/clientv3/auth.go +++ b/clientv3/auth.go @@ -21,7 +21,6 @@ import ( "github.com/coreos/etcd/auth/authpb" pb "github.com/coreos/etcd/etcdserver/etcdserverpb" - "google.golang.org/grpc" ) @@ -216,8 +215,8 @@ func (auth *authenticator) close() { auth.conn.Close() } -func newAuthenticator(endpoint string, opts []grpc.DialOption, c *Client) (*authenticator, error) { - conn, err := grpc.Dial(endpoint, opts...) +func newAuthenticator(ctx context.Context, target string, opts []grpc.DialOption, c *Client) (*authenticator, error) { + conn, err := grpc.DialContext(ctx, target, opts...) if err != nil { return nil, err } diff --git a/clientv3/balancer/balancer.go b/clientv3/balancer/balancer.go new file mode 100644 index 00000000000..6ecc5b5f828 --- /dev/null +++ b/clientv3/balancer/balancer.go @@ -0,0 +1,275 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package balancer + +import ( + "fmt" + "strconv" + "sync" + "time" + + "github.com/coreos/etcd/clientv3/balancer/picker" + + "go.uber.org/zap" + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/resolver" + _ "google.golang.org/grpc/resolver/dns" // register DNS resolver + _ "google.golang.org/grpc/resolver/passthrough" // register passthrough resolver +) + +// RegisterBuilder creates and registers a builder. Since this function calls balancer.Register, it +// must be invoked at initialization time. +func RegisterBuilder(cfg Config) { + bb := &builder{cfg} + balancer.Register(bb) + + bb.cfg.Logger.Info( + "registered balancer", + zap.String("policy", bb.cfg.Policy.String()), + zap.String("name", bb.cfg.Name), + ) +} + +type builder struct { + cfg Config +} + +// Build is called initially when creating "ccBalancerWrapper". +// "grpc.Dial" is called to this client connection. +// Then, resolved addresses will be handled via "HandleResolvedAddrs". +func (b *builder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer { + bb := &baseBalancer{ + id: strconv.FormatInt(time.Now().UnixNano(), 36), + policy: b.cfg.Policy, + name: b.cfg.Policy.String(), + lg: b.cfg.Logger, + + addrToSc: make(map[resolver.Address]balancer.SubConn), + scToAddr: make(map[balancer.SubConn]resolver.Address), + scToSt: make(map[balancer.SubConn]connectivity.State), + + currentConn: nil, + csEvltr: &connectivityStateEvaluator{}, + + // initialize picker always returns "ErrNoSubConnAvailable" + Picker: picker.NewErr(balancer.ErrNoSubConnAvailable), + } + if b.cfg.Name != "" { + bb.name = b.cfg.Name + } + if bb.lg == nil { + bb.lg = zap.NewNop() + } + + // TODO: support multiple connections + bb.mu.Lock() + bb.currentConn = cc + bb.mu.Unlock() + + bb.lg.Info( + "built balancer", + zap.String("balancer-id", bb.id), + zap.String("policy", bb.policy.String()), + zap.String("resolver-target", cc.Target()), + ) + return bb +} + +// Name implements "grpc/balancer.Builder" interface. +func (b *builder) Name() string { return b.cfg.Name } + +// Balancer defines client balancer interface. +type Balancer interface { + // Balancer is called on specified client connection. Client initiates gRPC + // connection with "grpc.Dial(addr, grpc.WithBalancerName)", and then those resolved + // addresses are passed to "grpc/balancer.Balancer.HandleResolvedAddrs". + // For each resolved address, balancer calls "balancer.ClientConn.NewSubConn". + // "grpc/balancer.Balancer.HandleSubConnStateChange" is called when connectivity state + // changes, thus requires failover logic in this method. + balancer.Balancer + + // Picker calls "Pick" for every client request. + picker.Picker +} + +type baseBalancer struct { + id string + policy picker.Policy + name string + lg *zap.Logger + + mu sync.RWMutex + + addrToSc map[resolver.Address]balancer.SubConn + scToAddr map[balancer.SubConn]resolver.Address + scToSt map[balancer.SubConn]connectivity.State + + currentConn balancer.ClientConn + currentState connectivity.State + csEvltr *connectivityStateEvaluator + + picker.Picker +} + +// HandleResolvedAddrs implements "grpc/balancer.Balancer" interface. +// gRPC sends initial or updated resolved addresses from "Build". +func (bb *baseBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) { + if err != nil { + bb.lg.Warn("HandleResolvedAddrs called with error", zap.String("balancer-id", bb.id), zap.Error(err)) + return + } + bb.lg.Info("resolved", zap.String("balancer-id", bb.id), zap.Strings("addresses", addrsToStrings(addrs))) + + bb.mu.Lock() + defer bb.mu.Unlock() + + resolved := make(map[resolver.Address]struct{}) + for _, addr := range addrs { + resolved[addr] = struct{}{} + if _, ok := bb.addrToSc[addr]; !ok { + sc, err := bb.currentConn.NewSubConn([]resolver.Address{addr}, balancer.NewSubConnOptions{}) + if err != nil { + bb.lg.Warn("NewSubConn failed", zap.String("balancer-id", bb.id), zap.Error(err), zap.String("address", addr.Addr)) + continue + } + bb.addrToSc[addr] = sc + bb.scToAddr[sc] = addr + bb.scToSt[sc] = connectivity.Idle + sc.Connect() + } + } + + for addr, sc := range bb.addrToSc { + if _, ok := resolved[addr]; !ok { + // was removed by resolver or failed to create subconn + bb.currentConn.RemoveSubConn(sc) + delete(bb.addrToSc, addr) + + bb.lg.Info( + "removed subconn", + zap.String("balancer-id", bb.id), + zap.String("address", addr.Addr), + zap.String("subconn", scToString(sc)), + ) + + // Keep the state of this sc in bb.scToSt until sc's state becomes Shutdown. + // The entry will be deleted in HandleSubConnStateChange. + // (DO NOT) delete(bb.scToAddr, sc) + // (DO NOT) delete(bb.scToSt, sc) + } + } +} + +// HandleSubConnStateChange implements "grpc/balancer.Balancer" interface. +func (bb *baseBalancer) HandleSubConnStateChange(sc balancer.SubConn, s connectivity.State) { + bb.mu.Lock() + defer bb.mu.Unlock() + + old, ok := bb.scToSt[sc] + if !ok { + bb.lg.Warn( + "state change for an unknown subconn", + zap.String("balancer-id", bb.id), + zap.String("subconn", scToString(sc)), + zap.String("state", s.String()), + ) + return + } + + bb.lg.Info( + "state changed", + zap.String("balancer-id", bb.id), + zap.Bool("connected", s == connectivity.Ready), + zap.String("subconn", scToString(sc)), + zap.String("address", bb.scToAddr[sc].Addr), + zap.String("old-state", old.String()), + zap.String("new-state", s.String()), + ) + + bb.scToSt[sc] = s + switch s { + case connectivity.Idle: + sc.Connect() + case connectivity.Shutdown: + // When an address was removed by resolver, b called RemoveSubConn but + // kept the sc's state in scToSt. Remove state for this sc here. + delete(bb.scToAddr, sc) + delete(bb.scToSt, sc) + } + + oldAggrState := bb.currentState + bb.currentState = bb.csEvltr.recordTransition(old, s) + + // Regenerate picker when one of the following happens: + // - this sc became ready from not-ready + // - this sc became not-ready from ready + // - the aggregated state of balancer became TransientFailure from non-TransientFailure + // - the aggregated state of balancer became non-TransientFailure from TransientFailure + if (s == connectivity.Ready) != (old == connectivity.Ready) || + (bb.currentState == connectivity.TransientFailure) != (oldAggrState == connectivity.TransientFailure) { + bb.regeneratePicker() + } + + bb.currentConn.UpdateBalancerState(bb.currentState, bb.Picker) + return +} + +func (bb *baseBalancer) regeneratePicker() { + if bb.currentState == connectivity.TransientFailure { + bb.lg.Info( + "generated transient error picker", + zap.String("balancer-id", bb.id), + zap.String("policy", bb.policy.String()), + ) + bb.Picker = picker.NewErr(balancer.ErrTransientFailure) + return + } + + // only pass ready subconns to picker + scs := make([]balancer.SubConn, 0) + addrToSc := make(map[resolver.Address]balancer.SubConn) + scToAddr := make(map[balancer.SubConn]resolver.Address) + for addr, sc := range bb.addrToSc { + if st, ok := bb.scToSt[sc]; ok && st == connectivity.Ready { + scs = append(scs, sc) + addrToSc[addr] = sc + scToAddr[sc] = addr + } + } + + switch bb.policy { + case picker.RoundrobinBalanced: + bb.Picker = picker.NewRoundrobinBalanced(bb.lg, scs, addrToSc, scToAddr) + + default: + panic(fmt.Errorf("invalid balancer picker policy (%d)", bb.policy)) + } + + bb.lg.Info( + "generated picker", + zap.String("balancer-id", bb.id), + zap.String("policy", bb.policy.String()), + zap.Strings("subconn-ready", scsToStrings(addrToSc)), + zap.Int("subconn-size", len(addrToSc)), + ) +} + +// Close implements "grpc/balancer.Balancer" interface. +// Close is a nop because base balancer doesn't have internal state to clean up, +// and it doesn't need to call RemoveSubConn for the SubConns. +func (bb *baseBalancer) Close() { + // TODO +} diff --git a/clientv3/balancer/balancer_test.go b/clientv3/balancer/balancer_test.go new file mode 100644 index 00000000000..be9d5ed24e0 --- /dev/null +++ b/clientv3/balancer/balancer_test.go @@ -0,0 +1,308 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package balancer + +import ( + "context" + "fmt" + "strings" + "testing" + "time" + + "github.com/coreos/etcd/clientv3/balancer/picker" + "github.com/coreos/etcd/clientv3/balancer/resolver/endpoint" + pb "github.com/coreos/etcd/etcdserver/etcdserverpb" + "github.com/coreos/etcd/pkg/mock/mockserver" + + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/peer" + "google.golang.org/grpc/status" +) + +// TestRoundRobinBalancedResolvableNoFailover ensures that +// requests to a resolvable endpoint can be balanced between +// multiple, if any, nodes. And there needs be no failover. +func TestRoundRobinBalancedResolvableNoFailover(t *testing.T) { + testCases := []struct { + name string + serverCount int + reqN int + network string + }{ + {name: "rrBalanced_1", serverCount: 1, reqN: 5, network: "tcp"}, + {name: "rrBalanced_1_unix_sockets", serverCount: 1, reqN: 5, network: "unix"}, + {name: "rrBalanced_3", serverCount: 3, reqN: 7, network: "tcp"}, + {name: "rrBalanced_5", serverCount: 5, reqN: 10, network: "tcp"}, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + ms, err := mockserver.StartMockServersOnNetwork(tc.serverCount, tc.network) + if err != nil { + t.Fatalf("failed to start mock servers: %v", err) + } + defer ms.Stop() + + var eps []string + for _, svr := range ms.Servers { + eps = append(eps, svr.ResolverAddress().Addr) + } + + rsv, err := endpoint.NewResolverGroup("nofailover") + if err != nil { + t.Fatal(err) + } + defer rsv.Close() + rsv.SetEndpoints(eps) + + name := genName() + cfg := Config{ + Policy: picker.RoundrobinBalanced, + Name: name, + Logger: zap.NewExample(), + } + RegisterBuilder(cfg) + conn, err := grpc.Dial(fmt.Sprintf("endpoint://nofailover/*"), grpc.WithInsecure(), grpc.WithBalancerName(name)) + if err != nil { + t.Fatalf("failed to dial mock server: %v", err) + } + defer conn.Close() + cli := pb.NewKVClient(conn) + + reqFunc := func(ctx context.Context) (picked string, err error) { + var p peer.Peer + _, err = cli.Range(ctx, &pb.RangeRequest{Key: []byte("/x")}, grpc.Peer(&p)) + if p.Addr != nil { + picked = p.Addr.String() + } + return picked, err + } + + prev, switches := "", 0 + for i := 0; i < tc.reqN; i++ { + picked, err := reqFunc(context.Background()) + if err != nil { + t.Fatalf("#%d: unexpected failure %v", i, err) + } + if prev == "" { + prev = picked + continue + } + if prev != picked { + switches++ + } + prev = picked + } + if tc.serverCount > 1 && switches < tc.reqN-3 { // -3 for initial resolutions + t.Fatalf("expected balanced loads for %d requests, got switches %d", tc.reqN, switches) + } + }) + } +} + +// TestRoundRobinBalancedResolvableFailoverFromServerFail ensures that +// loads be rebalanced while one server goes down and comes back. +func TestRoundRobinBalancedResolvableFailoverFromServerFail(t *testing.T) { + serverCount := 5 + ms, err := mockserver.StartMockServers(serverCount) + if err != nil { + t.Fatalf("failed to start mock servers: %s", err) + } + defer ms.Stop() + var eps []string + for _, svr := range ms.Servers { + eps = append(eps, svr.ResolverAddress().Addr) + } + + rsv, err := endpoint.NewResolverGroup("serverfail") + if err != nil { + t.Fatal(err) + } + defer rsv.Close() + rsv.SetEndpoints(eps) + + name := genName() + cfg := Config{ + Policy: picker.RoundrobinBalanced, + Name: name, + Logger: zap.NewExample(), + } + RegisterBuilder(cfg) + conn, err := grpc.Dial(fmt.Sprintf("endpoint://serverfail/mock.server"), grpc.WithInsecure(), grpc.WithBalancerName(name)) + if err != nil { + t.Fatalf("failed to dial mock server: %s", err) + } + defer conn.Close() + cli := pb.NewKVClient(conn) + + reqFunc := func(ctx context.Context) (picked string, err error) { + var p peer.Peer + _, err = cli.Range(ctx, &pb.RangeRequest{Key: []byte("/x")}, grpc.Peer(&p)) + if p.Addr != nil { + picked = p.Addr.String() + } + return picked, err + } + + // stop first server, loads should be redistributed + // stopped server should never be picked + ms.StopAt(0) + available := make(map[string]struct{}) + for i := 1; i < serverCount; i++ { + available[eps[i]] = struct{}{} + } + + reqN := 10 + prev, switches := "", 0 + for i := 0; i < reqN; i++ { + picked, err := reqFunc(context.Background()) + if err != nil && strings.Contains(err.Error(), "transport is closing") { + continue + } + if prev == "" { // first failover + if eps[0] == picked { + t.Fatalf("expected failover from %q, picked %q", eps[0], picked) + } + prev = picked + continue + } + if _, ok := available[picked]; !ok { + t.Fatalf("picked unavailable address %q (available %v)", picked, available) + } + if prev != picked { + switches++ + } + prev = picked + } + if switches < reqN-3 { // -3 for initial resolutions + failover + t.Fatalf("expected balanced loads for %d requests, got switches %d", reqN, switches) + } + + // now failed server comes back + ms.StartAt(0) + + // enough time for reconnecting to recovered server + time.Sleep(time.Second) + + prev, switches = "", 0 + recoveredAddr, recovered := eps[0], 0 + available[recoveredAddr] = struct{}{} + + for i := 0; i < 2*reqN; i++ { + picked, err := reqFunc(context.Background()) + if err != nil { + t.Fatalf("#%d: unexpected failure %v", i, err) + } + if prev == "" { + prev = picked + continue + } + if _, ok := available[picked]; !ok { + t.Fatalf("#%d: picked unavailable address %q (available %v)", i, picked, available) + } + if prev != picked { + switches++ + } + if picked == recoveredAddr { + recovered++ + } + prev = picked + } + if switches < reqN-3 { // -3 for initial resolutions + t.Fatalf("expected balanced loads for %d requests, got switches %d", reqN, switches) + } + if recovered < reqN/serverCount { + t.Fatalf("recovered server %q got only %d requests", recoveredAddr, recovered) + } +} + +// TestRoundRobinBalancedResolvableFailoverFromRequestFail ensures that +// loads be rebalanced while some requests are failed. +func TestRoundRobinBalancedResolvableFailoverFromRequestFail(t *testing.T) { + serverCount := 5 + ms, err := mockserver.StartMockServers(serverCount) + if err != nil { + t.Fatalf("failed to start mock servers: %s", err) + } + defer ms.Stop() + var eps []string + available := make(map[string]struct{}) + for _, svr := range ms.Servers { + eps = append(eps, svr.ResolverAddress().Addr) + available[svr.Address] = struct{}{} + } + rsv, err := endpoint.NewResolverGroup("requestfail") + if err != nil { + t.Fatal(err) + } + defer rsv.Close() + rsv.SetEndpoints(eps) + + name := genName() + cfg := Config{ + Policy: picker.RoundrobinBalanced, + Name: name, + Logger: zap.NewExample(), + } + RegisterBuilder(cfg) + conn, err := grpc.Dial(fmt.Sprintf("endpoint://requestfail/mock.server"), grpc.WithInsecure(), grpc.WithBalancerName(name)) + if err != nil { + t.Fatalf("failed to dial mock server: %s", err) + } + defer conn.Close() + cli := pb.NewKVClient(conn) + + reqFunc := func(ctx context.Context) (picked string, err error) { + var p peer.Peer + _, err = cli.Range(ctx, &pb.RangeRequest{Key: []byte("/x")}, grpc.Peer(&p)) + if p.Addr != nil { + picked = p.Addr.String() + } + return picked, err + } + + reqN := 20 + prev, switches := "", 0 + for i := 0; i < reqN; i++ { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + if i%2 == 0 { + cancel() + } + picked, err := reqFunc(ctx) + if i%2 == 0 { + if s, ok := status.FromError(err); ok && s.Code() != codes.Canceled || picked != "" { + t.Fatalf("#%d: expected %v, got %v", i, context.Canceled, err) + } + continue + } + if prev == "" && picked != "" { + prev = picked + continue + } + if _, ok := available[picked]; !ok { + t.Fatalf("#%d: picked unavailable address %q (available %v)", i, picked, available) + } + if prev != picked { + switches++ + } + prev = picked + } + if switches < reqN/2-3 { // -3 for initial resolutions + failover + t.Fatalf("expected balanced loads for %d requests, got switches %d", reqN, switches) + } +} diff --git a/clientv3/balancer/config.go b/clientv3/balancer/config.go new file mode 100644 index 00000000000..2156984df5b --- /dev/null +++ b/clientv3/balancer/config.go @@ -0,0 +1,36 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package balancer + +import ( + "github.com/coreos/etcd/clientv3/balancer/picker" + + "go.uber.org/zap" +) + +// Config defines balancer configurations. +type Config struct { + // Policy configures balancer policy. + Policy picker.Policy + + // Name defines an additional name for balancer. + // Useful for balancer testing to avoid register conflicts. + // If empty, defaults to policy name. + Name string + + // Logger configures balancer logging. + // If nil, logs are discarded. + Logger *zap.Logger +} diff --git a/clientv3/balancer/connectivity.go b/clientv3/balancer/connectivity.go new file mode 100644 index 00000000000..6cdeb3fa3a4 --- /dev/null +++ b/clientv3/balancer/connectivity.go @@ -0,0 +1,58 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package balancer + +import "google.golang.org/grpc/connectivity" + +// connectivityStateEvaluator gets updated by addrConns when their +// states transition, based on which it evaluates the state of +// ClientConn. +type connectivityStateEvaluator struct { + numReady uint64 // Number of addrConns in ready state. + numConnecting uint64 // Number of addrConns in connecting state. + numTransientFailure uint64 // Number of addrConns in transientFailure. +} + +// recordTransition records state change happening in every subConn and based on +// that it evaluates what aggregated state should be. +// It can only transition between Ready, Connecting and TransientFailure. Other states, +// Idle and Shutdown are transitioned into by ClientConn; in the beginning of the connection +// before any subConn is created ClientConn is in idle state. In the end when ClientConn +// closes it is in Shutdown state. +// +// recordTransition should only be called synchronously from the same goroutine. +func (cse *connectivityStateEvaluator) recordTransition(oldState, newState connectivity.State) connectivity.State { + // Update counters. + for idx, state := range []connectivity.State{oldState, newState} { + updateVal := 2*uint64(idx) - 1 // -1 for oldState and +1 for new. + switch state { + case connectivity.Ready: + cse.numReady += updateVal + case connectivity.Connecting: + cse.numConnecting += updateVal + case connectivity.TransientFailure: + cse.numTransientFailure += updateVal + } + } + + // Evaluate. + if cse.numReady > 0 { + return connectivity.Ready + } + if cse.numConnecting > 0 { + return connectivity.Connecting + } + return connectivity.TransientFailure +} diff --git a/clientv3/balancer/grpc1.7-health_test.go b/clientv3/balancer/grpc1.7-health_test.go index bf139a5b14b..1671f53386a 100644 --- a/clientv3/balancer/grpc1.7-health_test.go +++ b/clientv3/balancer/grpc1.7-health_test.go @@ -30,7 +30,7 @@ import ( var endpoints = []string{"localhost:2379", "localhost:22379", "localhost:32379"} -func TestBalancerGetUnblocking(t *testing.T) { +func TestOldHealthBalancerGetUnblocking(t *testing.T) { hb := NewGRPC17Health(endpoints, minHealthRetryDuration, func(ep string, dopts ...grpc.DialOption) (*grpc.ClientConn, error) { return nil, nil }) defer hb.Close() if addrs := <-hb.Notify(); len(addrs) != len(endpoints) { @@ -74,7 +74,7 @@ func TestBalancerGetUnblocking(t *testing.T) { } } -func TestBalancerGetBlocking(t *testing.T) { +func TestOldHealthBalancerGetBlocking(t *testing.T) { hb := NewGRPC17Health(endpoints, minHealthRetryDuration, func(ep string, dopts ...grpc.DialOption) (*grpc.ClientConn, error) { return nil, nil }) defer hb.Close() if addrs := <-hb.Notify(); len(addrs) != len(endpoints) { @@ -131,9 +131,9 @@ func TestBalancerGetBlocking(t *testing.T) { } } -// TestHealthBalancerGraylist checks one endpoint is tried after the other +// TestOldHealthBalancerGraylist checks one endpoint is tried after the other // due to gray listing. -func TestHealthBalancerGraylist(t *testing.T) { +func TestOldHealthBalancerGraylist(t *testing.T) { var wg sync.WaitGroup // Use 3 endpoints so gray list doesn't fallback to all connections // after failing on 2 endpoints. @@ -192,7 +192,7 @@ func TestHealthBalancerGraylist(t *testing.T) { // TestBalancerDoNotBlockOnClose ensures that balancer and grpc don't deadlock each other // due to rapid open/close conn. The deadlock causes balancer.Close() to block forever. // See issue: https://github.com/coreos/etcd/issues/7283 for more detail. -func TestBalancerDoNotBlockOnClose(t *testing.T) { +func TestOldHealthBalancerDoNotBlockOnClose(t *testing.T) { defer testutil.AfterTest(t) kcl := newKillConnListener(t, 3) diff --git a/clientv3/balancer/picker/doc.go b/clientv3/balancer/picker/doc.go new file mode 100644 index 00000000000..35dabf5532f --- /dev/null +++ b/clientv3/balancer/picker/doc.go @@ -0,0 +1,16 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package picker defines/implements client balancer picker policy. +package picker diff --git a/clientv3/balancer/picker/err.go b/clientv3/balancer/picker/err.go new file mode 100644 index 00000000000..c70ce158b68 --- /dev/null +++ b/clientv3/balancer/picker/err.go @@ -0,0 +1,34 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package picker + +import ( + "context" + + "google.golang.org/grpc/balancer" +) + +// NewErr returns a picker that always returns err on "Pick". +func NewErr(err error) Picker { + return &errPicker{err: err} +} + +type errPicker struct { + err error +} + +func (p *errPicker) Pick(context.Context, balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) { + return nil, nil, p.err +} diff --git a/clientv3/balancer/picker/picker.go b/clientv3/balancer/picker/picker.go new file mode 100644 index 00000000000..7ea761bdb57 --- /dev/null +++ b/clientv3/balancer/picker/picker.go @@ -0,0 +1,24 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package picker + +import ( + "google.golang.org/grpc/balancer" +) + +// Picker defines balancer Picker methods. +type Picker interface { + balancer.Picker +} diff --git a/clientv3/balancer/picker/picker_policy.go b/clientv3/balancer/picker/picker_policy.go new file mode 100644 index 00000000000..463ddc2a5c1 --- /dev/null +++ b/clientv3/balancer/picker/picker_policy.go @@ -0,0 +1,49 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package picker + +import "fmt" + +// Policy defines balancer picker policy. +type Policy uint8 + +const ( + // TODO: custom picker is not supported yet. + // custom defines custom balancer picker. + custom Policy = iota + + // RoundrobinBalanced balance loads over multiple endpoints + // and implements failover in roundrobin fashion. + RoundrobinBalanced Policy = iota + + // TODO: only send loads to pinned address "RoundrobinFailover" + // just like how 3.3 client works + // + // TODO: priotize leader + // TODO: health-check + // TODO: weighted roundrobin + // TODO: power of two random choice +) + +func (p Policy) String() string { + switch p { + case custom: + panic("'custom' picker policy is not supported yet") + case RoundrobinBalanced: + return "etcd-client-roundrobin-balanced" + default: + panic(fmt.Errorf("invalid balancer picker policy (%d)", p)) + } +} diff --git a/clientv3/balancer/picker/roundrobin_balanced.go b/clientv3/balancer/picker/roundrobin_balanced.go new file mode 100644 index 00000000000..b043d572dd7 --- /dev/null +++ b/clientv3/balancer/picker/roundrobin_balanced.go @@ -0,0 +1,92 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package picker + +import ( + "context" + "sync" + + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/resolver" +) + +// NewRoundrobinBalanced returns a new roundrobin balanced picker. +func NewRoundrobinBalanced( + lg *zap.Logger, + scs []balancer.SubConn, + addrToSc map[resolver.Address]balancer.SubConn, + scToAddr map[balancer.SubConn]resolver.Address, +) Picker { + return &rrBalanced{ + lg: lg, + scs: scs, + addrToSc: addrToSc, + scToAddr: scToAddr, + } +} + +type rrBalanced struct { + lg *zap.Logger + + mu sync.RWMutex + next int + scs []balancer.SubConn + + addrToSc map[resolver.Address]balancer.SubConn + scToAddr map[balancer.SubConn]resolver.Address +} + +// Pick is called for every client request. +func (rb *rrBalanced) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) { + rb.mu.RLock() + n := len(rb.scs) + rb.mu.RUnlock() + if n == 0 { + return nil, nil, balancer.ErrNoSubConnAvailable + } + + rb.mu.Lock() + cur := rb.next + sc := rb.scs[cur] + picked := rb.scToAddr[sc].Addr + rb.next = (rb.next + 1) % len(rb.scs) + rb.mu.Unlock() + + rb.lg.Debug( + "picked", + zap.String("address", picked), + zap.Int("subconn-index", cur), + zap.Int("subconn-size", n), + ) + + doneFunc := func(info balancer.DoneInfo) { + // TODO: error handling? + fss := []zapcore.Field{ + zap.Error(info.Err), + zap.String("address", picked), + zap.Bool("success", info.Err == nil), + zap.Bool("bytes-sent", info.BytesSent), + zap.Bool("bytes-received", info.BytesReceived), + } + if info.Err == nil { + rb.lg.Debug("balancer done", fss...) + } else { + rb.lg.Warn("balancer failed", fss...) + } + } + return sc, doneFunc, nil +} diff --git a/clientv3/balancer/resolver/endpoint/endpoint.go b/clientv3/balancer/resolver/endpoint/endpoint.go new file mode 100644 index 00000000000..104ec773c0b --- /dev/null +++ b/clientv3/balancer/resolver/endpoint/endpoint.go @@ -0,0 +1,229 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package endpoint resolves etcd entpoints using grpc targets of the form 'endpoint:///'. +package endpoint + +import ( + "fmt" + "net/url" + "strings" + "sync" + + "google.golang.org/grpc/resolver" +) + +const scheme = "endpoint" + +var ( + targetPrefix = fmt.Sprintf("%s://", scheme) + + bldr *builder +) + +func init() { + bldr = &builder{ + resolverGroups: make(map[string]*ResolverGroup), + } + resolver.Register(bldr) +} + +type builder struct { + mu sync.RWMutex + resolverGroups map[string]*ResolverGroup +} + +// NewResolverGroup creates a new ResolverGroup with the given id. +func NewResolverGroup(id string) (*ResolverGroup, error) { + return bldr.newResolverGroup(id) +} + +// ResolverGroup keeps all endpoints of resolvers using a common endpoint:/// target +// up-to-date. +type ResolverGroup struct { + mu sync.RWMutex + id string + endpoints []string + resolvers []*Resolver +} + +func (e *ResolverGroup) addResolver(r *Resolver) { + e.mu.Lock() + addrs := epsToAddrs(e.endpoints...) + e.resolvers = append(e.resolvers, r) + e.mu.Unlock() + r.cc.NewAddress(addrs) +} + +func (e *ResolverGroup) removeResolver(r *Resolver) { + e.mu.Lock() + for i, er := range e.resolvers { + if er == r { + e.resolvers = append(e.resolvers[:i], e.resolvers[i+1:]...) + break + } + } + e.mu.Unlock() +} + +// SetEndpoints updates the endpoints for ResolverGroup. All registered resolver are updated +// immediately with the new endpoints. +func (e *ResolverGroup) SetEndpoints(endpoints []string) { + addrs := epsToAddrs(endpoints...) + e.mu.Lock() + e.endpoints = endpoints + for _, r := range e.resolvers { + r.cc.NewAddress(addrs) + } + e.mu.Unlock() +} + +// Target constructs a endpoint target using the endpoint id of the ResolverGroup. +func (e *ResolverGroup) Target(endpoint string) string { + return Target(e.id, endpoint) +} + +// Target constructs a endpoint resolver target. +func Target(id, endpoint string) string { + return fmt.Sprintf("%s://%s/%s", scheme, id, endpoint) +} + +// IsTarget checks if a given target string in an endpoint resolver target. +func IsTarget(target string) bool { + return strings.HasPrefix(target, "endpoint://") +} + +func (e *ResolverGroup) Close() { + bldr.close(e.id) +} + +// Build creates or reuses an etcd resolver for the etcd cluster name identified by the authority part of the target. +func (b *builder) Build(target resolver.Target, cc resolver.ClientConn, opts resolver.BuildOption) (resolver.Resolver, error) { + if len(target.Authority) < 1 { + return nil, fmt.Errorf("'etcd' target scheme requires non-empty authority identifying etcd cluster being routed to") + } + id := target.Authority + es, err := b.getResolverGroup(id) + if err != nil { + return nil, fmt.Errorf("failed to build resolver: %v", err) + } + r := &Resolver{ + endpointID: id, + cc: cc, + } + es.addResolver(r) + return r, nil +} + +func (b *builder) newResolverGroup(id string) (*ResolverGroup, error) { + b.mu.RLock() + _, ok := b.resolverGroups[id] + b.mu.RUnlock() + if ok { + return nil, fmt.Errorf("Endpoint already exists for id: %s", id) + } + + es := &ResolverGroup{id: id} + b.mu.Lock() + b.resolverGroups[id] = es + b.mu.Unlock() + return es, nil +} + +func (b *builder) getResolverGroup(id string) (*ResolverGroup, error) { + b.mu.RLock() + es, ok := b.resolverGroups[id] + b.mu.RUnlock() + if !ok { + return nil, fmt.Errorf("ResolverGroup not found for id: %s", id) + } + return es, nil +} + +func (b *builder) close(id string) { + b.mu.Lock() + delete(b.resolverGroups, id) + b.mu.Unlock() +} + +func (r *builder) Scheme() string { + return scheme +} + +// Resolver provides a resolver for a single etcd cluster, identified by name. +type Resolver struct { + endpointID string + cc resolver.ClientConn + sync.RWMutex +} + +// TODO: use balancer.epsToAddrs +func epsToAddrs(eps ...string) (addrs []resolver.Address) { + addrs = make([]resolver.Address, 0, len(eps)) + for _, ep := range eps { + addrs = append(addrs, resolver.Address{Addr: ep}) + } + return addrs +} + +func (*Resolver) ResolveNow(o resolver.ResolveNowOption) {} + +func (r *Resolver) Close() { + es, err := bldr.getResolverGroup(r.endpointID) + if err != nil { + return + } + es.removeResolver(r) +} + +// ParseEndpoint endpoint parses an endpoint of the form +// (http|https)://*|(unix|unixs)://) +// and returns a protocol ('tcp' or 'unix'), +// host (or filepath if a unix socket), +// scheme (http, https, unix, unixs). +func ParseEndpoint(endpoint string) (proto string, host string, scheme string) { + proto = "tcp" + host = endpoint + url, uerr := url.Parse(endpoint) + if uerr != nil || !strings.Contains(endpoint, "://") { + return proto, host, scheme + } + scheme = url.Scheme + + // strip scheme:// prefix since grpc dials by host + host = url.Host + switch url.Scheme { + case "http", "https": + case "unix", "unixs": + proto = "unix" + host = url.Host + url.Path + default: + proto, host = "", "" + } + return proto, host, scheme +} + +// ParseTarget parses a endpoint:/// string and returns the parsed id and endpoint. +// If the target is malformed, an error is returned. +func ParseTarget(target string) (string, string, error) { + noPrefix := strings.TrimPrefix(target, targetPrefix) + if noPrefix == target { + return "", "", fmt.Errorf("malformed target, %s prefix is required: %s", targetPrefix, target) + } + parts := strings.SplitN(noPrefix, "/", 2) + if len(parts) != 2 { + return "", "", fmt.Errorf("malformed target, expected %s:///, but got %s", scheme, target) + } + return parts[0], parts[1], nil +} diff --git a/clientv3/balancer/utils.go b/clientv3/balancer/utils.go new file mode 100644 index 00000000000..a11faeb7e6c --- /dev/null +++ b/clientv3/balancer/utils.go @@ -0,0 +1,68 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package balancer + +import ( + "fmt" + "net/url" + "sort" + "sync/atomic" + "time" + + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/resolver" +) + +func scToString(sc balancer.SubConn) string { + return fmt.Sprintf("%p", sc) +} + +func scsToStrings(scs map[resolver.Address]balancer.SubConn) (ss []string) { + ss = make([]string, 0, len(scs)) + for a, sc := range scs { + ss = append(ss, fmt.Sprintf("%s (%s)", a.Addr, scToString(sc))) + } + sort.Strings(ss) + return ss +} + +func addrsToStrings(addrs []resolver.Address) (ss []string) { + ss = make([]string, len(addrs)) + for i := range addrs { + ss[i] = addrs[i].Addr + } + sort.Strings(ss) + return ss +} + +func epsToAddrs(eps ...string) (addrs []resolver.Address) { + addrs = make([]resolver.Address, 0, len(eps)) + for _, ep := range eps { + u, err := url.Parse(ep) + if err != nil { + addrs = append(addrs, resolver.Address{Addr: ep, Type: resolver.Backend}) + continue + } + addrs = append(addrs, resolver.Address{Addr: u.Host, Type: resolver.Backend}) + } + return addrs +} + +var genN = new(uint32) + +func genName() string { + now := time.Now().UnixNano() + return fmt.Sprintf("%X%X", now, atomic.AddUint32(genN, 1)) +} diff --git a/clientv3/balancer/utils_test.go b/clientv3/balancer/utils_test.go new file mode 100644 index 00000000000..e58cd349576 --- /dev/null +++ b/clientv3/balancer/utils_test.go @@ -0,0 +1,34 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package balancer + +import ( + "reflect" + "testing" + + "google.golang.org/grpc/resolver" +) + +func Test_epsToAddrs(t *testing.T) { + eps := []string{"https://example.com:2379", "127.0.0.1:2379"} + exp := []resolver.Address{ + {Addr: "example.com:2379", Type: resolver.Backend}, + {Addr: "127.0.0.1:2379", Type: resolver.Backend}, + } + rs := epsToAddrs(eps...) + if !reflect.DeepEqual(rs, exp) { + t.Fatalf("expected %v, got %v", exp, rs) + } +} diff --git a/clientv3/client.go b/clientv3/client.go index 00d621ffe8b..eae8491e4ea 100644 --- a/clientv3/client.go +++ b/clientv3/client.go @@ -21,13 +21,18 @@ import ( "fmt" "net" "net/url" + "os" "strconv" "strings" "sync" "time" "github.com/coreos/etcd/clientv3/balancer" + "github.com/coreos/etcd/clientv3/balancer/picker" + "github.com/coreos/etcd/clientv3/balancer/resolver/endpoint" "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" + "github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils" + "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/codes" @@ -40,8 +45,26 @@ import ( var ( ErrNoAvailableEndpoints = errors.New("etcdclient: no available endpoints") ErrOldCluster = errors.New("etcdclient: old cluster version") + + roundRobinBalancerName = fmt.Sprintf("etcd-%s", picker.RoundrobinBalanced.String()) ) +func init() { + lg := zap.NewNop() + if os.Getenv("ETCD_CLIENT_DEBUG") != "" { + var err error + lg, err = zap.NewProductionConfig().Build() // info level logging + if err != nil { + panic(err) + } + } + balancer.RegisterBuilder(balancer.Config{ + Policy: picker.RoundrobinBalanced, + Name: roundRobinBalancerName, + Logger: lg, + }) +} + // Client provides and manages an etcd v3 client session. type Client struct { Cluster @@ -51,13 +74,13 @@ type Client struct { Auth Maintenance - conn *grpc.ClientConn - dialerrc chan error + conn *grpc.ClientConn - cfg Config - creds *credentials.TransportCredentials - balancer *balancer.GRPC17Health - mu *sync.Mutex + cfg Config + creds *credentials.TransportCredentials + balancer balancer.Balancer + resolverGroup *endpoint.ResolverGroup + mu *sync.Mutex ctx context.Context cancel context.CancelFunc @@ -70,6 +93,8 @@ type Client struct { tokenCred *authTokenCredential callOpts []grpc.CallOption + + lg *zap.Logger } // New creates a new etcdv3 client from a given configuration. @@ -104,6 +129,9 @@ func (c *Client) Close() error { c.cancel() c.Watcher.Close() c.Lease.Close() + if c.resolverGroup != nil { + c.resolverGroup.Close() + } if c.conn != nil { return toErr(c.ctx, c.conn.Close()) } @@ -126,16 +154,9 @@ func (c *Client) Endpoints() (eps []string) { // SetEndpoints updates client's endpoints. func (c *Client) SetEndpoints(eps ...string) { c.mu.Lock() + defer c.mu.Unlock() c.cfg.Endpoints = eps - c.mu.Unlock() - c.balancer.UpdateAddrs(eps...) - - if c.balancer.NeedUpdate() { - select { - case c.balancer.UpdateAddrsC() <- balancer.NotifyNext: - case <-c.balancer.StopC(): - } - } + c.resolverGroup.SetEndpoints(eps) } // Sync synchronizes client's endpoints with the known endpoints from the etcd membership. @@ -189,28 +210,6 @@ func (cred authTokenCredential) GetRequestMetadata(ctx context.Context, s ...str }, nil } -func parseEndpoint(endpoint string) (proto string, host string, scheme string) { - proto = "tcp" - host = endpoint - url, uerr := url.Parse(endpoint) - if uerr != nil || !strings.Contains(endpoint, "://") { - return proto, host, scheme - } - scheme = url.Scheme - - // strip scheme:// prefix since grpc dials by host - host = url.Host - switch url.Scheme { - case "http", "https": - case "unix", "unixs": - proto = "unix" - host = url.Host + url.Path - default: - proto, host = "", "" - } - return proto, host, scheme -} - func (c *Client) processCreds(scheme string) (creds *credentials.TransportCredentials) { creds = c.creds switch scheme { @@ -231,10 +230,12 @@ func (c *Client) processCreds(scheme string) (creds *credentials.TransportCreden } // dialSetupOpts gives the dial opts prior to any authentication -func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) (opts []grpc.DialOption) { - if c.cfg.DialTimeout > 0 { - opts = []grpc.DialOption{grpc.WithTimeout(c.cfg.DialTimeout)} +func (c *Client) dialSetupOpts(target string, dopts ...grpc.DialOption) (opts []grpc.DialOption, err error) { + _, ep, err := endpoint.ParseTarget(target) + if err != nil { + return nil, fmt.Errorf("unable to parse target: %v", err) } + if c.cfg.DialKeepAliveTime > 0 { params := keepalive.ClientParameters{ Time: c.cfg.DialKeepAliveTime, @@ -244,12 +245,12 @@ func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) (opts } opts = append(opts, dopts...) - f := func(host string, t time.Duration) (net.Conn, error) { - proto, host, _ := parseEndpoint(c.balancer.Endpoint(host)) - if host == "" && endpoint != "" { + f := func(dialEp string, t time.Duration) (net.Conn, error) { + proto, host, _ := endpoint.ParseEndpoint(dialEp) + if host == "" && ep != "" { // dialing an endpoint not in the balancer; use // endpoint passed into dial - proto, host, _ = parseEndpoint(endpoint) + proto, host, _ = endpoint.ParseEndpoint(ep) } if proto == "" { return nil, fmt.Errorf("unknown scheme for %q", host) @@ -260,19 +261,12 @@ func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) (opts default: } dialer := &net.Dialer{Timeout: t} - conn, err := dialer.DialContext(c.ctx, proto, host) - if err != nil { - select { - case c.dialerrc <- err: - default: - } - } - return conn, err + return dialer.DialContext(c.ctx, proto, host) } opts = append(opts, grpc.WithDialer(f)) creds := c.creds - if _, _, scheme := parseEndpoint(endpoint); len(scheme) != 0 { + if _, _, scheme := endpoint.ParseEndpoint(ep); len(scheme) != 0 { creds = c.processCreds(scheme) } if creds != nil { @@ -281,7 +275,19 @@ func (c *Client) dialSetupOpts(endpoint string, dopts ...grpc.DialOption) (opts opts = append(opts, grpc.WithInsecure()) } - return opts + // Interceptor retry and backoff. + // TODO: Replace all of clientv3/retry.go with interceptor based retry, or with + // https://github.com/grpc/proposal/blob/master/A6-client-retries.md#retry-policy + // once it is available. + rrBackoff := withBackoff(c.roundRobinQuorumBackoff(defaultBackoffWaitBetween, defaultBackoffJitterFraction)) + opts = append(opts, + // Disable stream retry by default since go-grpc-middleware/retry does not support client streams. + // Streams that are safe to retry are enabled individually. + grpc.WithStreamInterceptor(c.streamClientInterceptor(c.lg, withMax(0), rrBackoff)), + grpc.WithUnaryInterceptor(c.unaryClientInterceptor(c.lg, withMax(defaultUnaryMaxRetries), rrBackoff)), + ) + + return opts, nil } // Dial connects to a single endpoint using the client's config. @@ -294,10 +300,18 @@ func (c *Client) getToken(ctx context.Context) error { var auth *authenticator for i := 0; i < len(c.cfg.Endpoints); i++ { - endpoint := c.cfg.Endpoints[i] - host := getHost(endpoint) + ep := c.cfg.Endpoints[i] // use dial options without dopts to avoid reusing the client balancer - auth, err = newAuthenticator(host, c.dialSetupOpts(endpoint), c) + var dOpts []grpc.DialOption + _, host, _ := endpoint.ParseEndpoint(ep) + target := c.resolverGroup.Target(host) + dOpts, err = c.dialSetupOpts(target, c.cfg.DialOptions...) + if err != nil { + err = fmt.Errorf("failed to configure auth dialer: %v", err) + continue + } + dOpts = append(dOpts, grpc.WithBalancerName(roundRobinBalancerName)) + auth, err = newAuthenticator(ctx, target, dOpts, c) if err != nil { continue } @@ -319,37 +333,52 @@ func (c *Client) getToken(ctx context.Context) error { return err } -func (c *Client) dial(endpoint string, dopts ...grpc.DialOption) (*grpc.ClientConn, error) { - opts := c.dialSetupOpts(endpoint, dopts...) - host := getHost(endpoint) +func (c *Client) dial(ep string, dopts ...grpc.DialOption) (*grpc.ClientConn, error) { + // We pass a target to DialContext of the form: endpoint:/// that + // does not include scheme (http/https/unix/unixs) or path parts. + _, host, _ := endpoint.ParseEndpoint(ep) + target := c.resolverGroup.Target(host) + + opts, err := c.dialSetupOpts(target, dopts...) + if err != nil { + return nil, fmt.Errorf("failed to configure dialer: %v", err) + } + if c.Username != "" && c.Password != "" { c.tokenCred = &authTokenCredential{ tokenMu: &sync.RWMutex{}, } - ctx := c.ctx + ctx, cancel := c.ctx, func() {} if c.cfg.DialTimeout > 0 { - cctx, cancel := context.WithTimeout(ctx, c.cfg.DialTimeout) - defer cancel() - ctx = cctx + ctx, cancel = context.WithTimeout(ctx, c.cfg.DialTimeout) } - err := c.getToken(ctx) + err = c.getToken(ctx) if err != nil { if toErr(ctx, err) != rpctypes.ErrAuthNotEnabled { if err == ctx.Err() && ctx.Err() != c.ctx.Err() { err = context.DeadlineExceeded } + cancel() return nil, err } } else { opts = append(opts, grpc.WithPerRPCCredentials(c.tokenCred)) } + cancel() } opts = append(opts, c.cfg.DialOptions...) - conn, err := grpc.DialContext(c.ctx, host, opts...) + dctx := c.ctx + if c.cfg.DialTimeout > 0 { + var cancel context.CancelFunc + dctx, cancel = context.WithTimeout(c.ctx, c.cfg.DialTimeout) + defer cancel() // TODO: Is this right for cases where grpc.WithBlock() is not set on the dial options? + } + + conn, err := grpc.DialContext(dctx, target, opts...) if err != nil { return nil, err } @@ -382,7 +411,6 @@ func newClient(cfg *Config) (*Client, error) { ctx, cancel := context.WithCancel(baseCtx) client := &Client{ conn: nil, - dialerrc: make(chan error, 1), cfg: *cfg, creds: creds, ctx: ctx, @@ -390,6 +418,17 @@ func newClient(cfg *Config) (*Client, error) { mu: new(sync.Mutex), callOpts: defaultCallOpts, } + + lcfg := DefaultLogConfig + if cfg.LogConfig != nil { + lcfg = *cfg.LogConfig + } + var err error + client.lg, err = lcfg.Build() + if err != nil { + return nil, err + } + if cfg.Username != "" && cfg.Password != "" { client.Username = cfg.Username client.Password = cfg.Password @@ -412,41 +451,32 @@ func newClient(cfg *Config) (*Client, error) { client.callOpts = callOpts } - client.balancer = balancer.NewGRPC17Health(cfg.Endpoints, cfg.DialTimeout, client.dial) + // Prepare a 'endpoint:///' resolver for the client and create a endpoint target to pass + // to dial so the client knows to use this resolver. + client.resolverGroup, err = endpoint.NewResolverGroup(fmt.Sprintf("client-%s", strconv.FormatInt(time.Now().UnixNano(), 36))) + if err != nil { + client.cancel() + return nil, err + } + client.resolverGroup.SetEndpoints(cfg.Endpoints) + + if len(cfg.Endpoints) < 1 { + return nil, fmt.Errorf("at least one Endpoint must is required in client config") + } + dialEndpoint := cfg.Endpoints[0] - // use Endpoints[0] so that for https:// without any tls config given, then + // Use an provided endpoint target so that for https:// without any tls config given, then // grpc will assume the certificate server name is the endpoint host. - conn, err := client.dial(cfg.Endpoints[0], grpc.WithBalancer(client.balancer)) + conn, err := client.dial(dialEndpoint, grpc.WithBalancerName(roundRobinBalancerName)) if err != nil { client.cancel() - client.balancer.Close() + client.resolverGroup.Close() return nil, err } + // TODO: With the old grpc balancer interface, we waited until the dial timeout + // for the balancer to be ready. Is there an equivalent wait we should do with the new grpc balancer interface? client.conn = conn - // wait for a connection - if cfg.DialTimeout > 0 { - hasConn := false - waitc := time.After(cfg.DialTimeout) - select { - case <-client.balancer.Ready(): - hasConn = true - case <-ctx.Done(): - case <-waitc: - } - if !hasConn { - err := context.DeadlineExceeded - select { - case err = <-client.dialerrc: - default: - } - client.cancel() - client.balancer.Close() - conn.Close() - return nil, err - } - } - client.Cluster = NewCluster(client) client.KV = NewKV(client) client.Lease = NewLease(client) @@ -465,6 +495,22 @@ func newClient(cfg *Config) (*Client, error) { return client, nil } +// roundRobinQuorumBackoff retries against quorum between each backoff. +// This is intended for use with a round robin load balancer. +func (c *Client) roundRobinQuorumBackoff(waitBetween time.Duration, jitterFraction float64) backoffFunc { + return func(attempt uint) time.Duration { + // after each round robin across quorum, backoff for our wait between duration + n := uint(len(c.Endpoints())) + quorum := (n/2 + 1) + if attempt%quorum == 0 { + c.lg.Info("backoff", zap.Uint("attempt", attempt), zap.Uint("quorum", quorum), zap.Duration("waitBetween", waitBetween), zap.Float64("jitterFraction", jitterFraction)) + return backoffutils.JitterUp(waitBetween, jitterFraction) + } + c.lg.Info("backoff skipped", zap.Uint("attempt", attempt), zap.Uint("quorum", quorum)) + return 0 + } +} + func (c *Client) checkVersion() (err error) { var wg sync.WaitGroup errc := make(chan error, len(c.cfg.Endpoints)) @@ -574,6 +620,26 @@ func canceledByCaller(stopCtx context.Context, err error) bool { return err == context.Canceled || err == context.DeadlineExceeded } +// IsConnCanceled returns true, if error is from a closed gRPC connection. +// ref. https://github.com/grpc/grpc-go/pull/1854 +func IsConnCanceled(err error) bool { + if err == nil { + return false + } + // >= gRPC v1.10.x + s, ok := status.FromError(err) + if ok { + // connection is canceled or server has already closed the connection + return s.Code() == codes.Canceled || s.Message() == "transport is closing" + } + // >= gRPC v1.10.x + if err == context.Canceled { + return true + } + // <= gRPC v1.7.x returns 'errors.New("grpc: the client connection is closing")' + return strings.Contains(err.Error(), "grpc: the client connection is closing") +} + func getHost(ep string) string { url, uerr := url.Parse(ep) if uerr != nil || !strings.Contains(ep, "://") { diff --git a/clientv3/client_test.go b/clientv3/client_test.go index d09e6433017..2f94fa62b7b 100644 --- a/clientv3/client_test.go +++ b/clientv3/client_test.go @@ -23,6 +23,8 @@ import ( "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" "github.com/coreos/etcd/pkg/testutil" + + "google.golang.org/grpc" ) func TestDialCancel(t *testing.T) { @@ -80,14 +82,17 @@ func TestDialCancel(t *testing.T) { func TestDialTimeout(t *testing.T) { defer testutil.AfterTest(t) + // grpc.WithBlock to block until connection up or timeout testCfgs := []Config{ { Endpoints: []string{"http://254.0.0.1:12345"}, DialTimeout: 2 * time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, }, { Endpoints: []string{"http://254.0.0.1:12345"}, DialTimeout: time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, Username: "abc", Password: "def", }, diff --git a/clientv3/config.go b/clientv3/config.go index 79d6e2a984f..4cea369076e 100644 --- a/clientv3/config.go +++ b/clientv3/config.go @@ -19,6 +19,7 @@ import ( "crypto/tls" "time" + "go.uber.org/zap" "google.golang.org/grpc" ) @@ -72,4 +73,27 @@ type Config struct { // Context is the default client context; it can be used to cancel grpc dial out and // other operations that do not have an explicit context. Context context.Context + + // LogConfig configures client-side logger. + // If nil, use the default logger. + // TODO: configure gRPC logger + LogConfig *zap.Config +} + +// DefaultLogConfig is the default client logging configuration. +// Default log level is "Warn". Use "zap.InfoLevel" for debugging. +// Use "/dev/null" for output paths, to discard all logs. +var DefaultLogConfig = zap.Config{ + Level: zap.NewAtomicLevelAt(zap.WarnLevel), + Development: false, + Sampling: &zap.SamplingConfig{ + Initial: 100, + Thereafter: 100, + }, + Encoding: "json", + EncoderConfig: zap.NewProductionEncoderConfig(), + + // Use "/dev/null" to discard all + OutputPaths: []string{"stderr"}, + ErrorOutputPaths: []string{"stderr"}, } diff --git a/clientv3/doc.go b/clientv3/doc.go index 717fbe435ea..cd6ef01d213 100644 --- a/clientv3/doc.go +++ b/clientv3/doc.go @@ -87,11 +87,20 @@ // go func() { cli.Close() }() // _, err := kvc.Get(ctx, "a") // if err != nil { +// // with etcd clientv3 <= v3.3 // if err == context.Canceled { // // grpc balancer calls 'Get' with an inflight client.Close // } else if err == grpc.ErrClientConnClosing { // // grpc balancer calls 'Get' after client.Close. // } +// // with etcd clientv3 >= v3.4 +// if clientv3.IsConnCanceled(err) { +// // gRPC client connection is closed +// } // } // +// The grpc load balancer is registered statically and is shared across etcd clients. +// To enable detailed load balancer logging, set the ETCD_CLIENT_DEBUG environment +// variable. E.g. "ETCD_CLIENT_DEBUG=1". +// package clientv3 diff --git a/clientv3/integration/black_hole_test.go b/clientv3/integration/black_hole_test.go index c59541e381d..151187e14b9 100644 --- a/clientv3/integration/black_hole_test.go +++ b/clientv3/integration/black_hole_test.go @@ -25,6 +25,7 @@ import ( "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" "github.com/coreos/etcd/integration" "github.com/coreos/etcd/pkg/testutil" + "google.golang.org/grpc" ) // TestBalancerUnderBlackholeKeepAliveWatch tests when watch discovers it cannot talk to @@ -44,6 +45,7 @@ func TestBalancerUnderBlackholeKeepAliveWatch(t *testing.T) { ccfg := clientv3.Config{ Endpoints: []string{eps[0]}, DialTimeout: 1 * time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, DialKeepAliveTime: 1 * time.Second, DialKeepAliveTimeout: 500 * time.Millisecond, } @@ -106,7 +108,7 @@ func TestBalancerUnderBlackholeKeepAliveWatch(t *testing.T) { func TestBalancerUnderBlackholeNoKeepAlivePut(t *testing.T) { testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error { _, err := cli.Put(ctx, "foo", "bar") - if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { + if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { return errExpected } return err @@ -116,7 +118,7 @@ func TestBalancerUnderBlackholeNoKeepAlivePut(t *testing.T) { func TestBalancerUnderBlackholeNoKeepAliveDelete(t *testing.T) { testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error { _, err := cli.Delete(ctx, "foo") - if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { + if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { return errExpected } return err @@ -129,7 +131,7 @@ func TestBalancerUnderBlackholeNoKeepAliveTxn(t *testing.T) { If(clientv3.Compare(clientv3.Version("foo"), "=", 0)). Then(clientv3.OpPut("foo", "bar")). Else(clientv3.OpPut("foo", "baz")).Commit() - if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { + if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { return errExpected } return err @@ -139,7 +141,7 @@ func TestBalancerUnderBlackholeNoKeepAliveTxn(t *testing.T) { func TestBalancerUnderBlackholeNoKeepAliveLinearizableGet(t *testing.T) { testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error { _, err := cli.Get(ctx, "a") - if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { + if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { return errExpected } return err @@ -149,7 +151,7 @@ func TestBalancerUnderBlackholeNoKeepAliveLinearizableGet(t *testing.T) { func TestBalancerUnderBlackholeNoKeepAliveSerializableGet(t *testing.T) { testBalancerUnderBlackholeNoKeepAlive(t, func(cli *clientv3.Client, ctx context.Context) error { _, err := cli.Get(ctx, "a", clientv3.WithSerializable()) - if err == context.DeadlineExceeded || isServerCtxTimeout(err) { + if isClientTimeout(err) || isServerCtxTimeout(err) { return errExpected } return err @@ -172,6 +174,7 @@ func testBalancerUnderBlackholeNoKeepAlive(t *testing.T, op func(*clientv3.Clien ccfg := clientv3.Config{ Endpoints: []string{eps[0]}, DialTimeout: 1 * time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, } cli, err := clientv3.New(ccfg) if err != nil { @@ -189,22 +192,23 @@ func testBalancerUnderBlackholeNoKeepAlive(t *testing.T, op func(*clientv3.Clien // blackhole eps[0] clus.Members[0].Blackhole() - // fail first due to blackhole, retry should succeed + // With round robin balancer, client will make a request to a healthy endpoint + // within a few requests. // TODO: first operation can succeed // when gRPC supports better retry on non-delivered request - for i := 0; i < 2; i++ { - ctx, cancel := context.WithTimeout(context.Background(), time.Second) + for i := 0; i < 5; i++ { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) err = op(cli, ctx) cancel() if err == nil { break - } - if i == 0 { - if err != errExpected { - t.Errorf("#%d: expected %v, got %v", i, errExpected, err) - } - } else if err != nil { + } else if err == errExpected { + t.Logf("#%d: current error %v", i, err) + } else { t.Errorf("#%d: failed with error %v", i, err) } } + if err != nil { + t.Fatal(err) + } } diff --git a/clientv3/integration/dial_test.go b/clientv3/integration/dial_test.go index 91eb597ea04..e996a132bbc 100644 --- a/clientv3/integration/dial_test.go +++ b/clientv3/integration/dial_test.go @@ -26,6 +26,7 @@ import ( "github.com/coreos/etcd/integration" "github.com/coreos/etcd/pkg/testutil" "github.com/coreos/etcd/pkg/transport" + "google.golang.org/grpc" ) var ( @@ -58,10 +59,11 @@ func TestDialTLSExpired(t *testing.T) { _, err = clientv3.New(clientv3.Config{ Endpoints: []string{clus.Members[0].GRPCAddr()}, DialTimeout: 3 * time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, TLS: tls, }) - if err != context.DeadlineExceeded { - t.Fatalf("expected %v, got %v", context.DeadlineExceeded, err) + if !isClientTimeout(err) { + t.Fatalf("expected dial timeout error, got %v", err) } } @@ -72,12 +74,24 @@ func TestDialTLSNoConfig(t *testing.T) { clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1, ClientTLS: &testTLSInfo, SkipCreatingClient: true}) defer clus.Terminate(t) // expect "signed by unknown authority" - _, err := clientv3.New(clientv3.Config{ + c, err := clientv3.New(clientv3.Config{ Endpoints: []string{clus.Members[0].GRPCAddr()}, DialTimeout: time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, }) - if err != context.DeadlineExceeded { - t.Fatalf("expected %v, got %v", context.DeadlineExceeded, err) + if err != nil { + t.Fatal(err) + } + defer c.Close() + + // TODO: this should not be required when we set grpc.WithBlock() + if c != nil { + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + _, err = c.KV.Get(ctx, "/") + cancel() + } + if !isClientTimeout(err) { + t.Fatalf("expected dial timeout error, got %v", err) } } @@ -104,7 +118,11 @@ func testDialSetEndpoints(t *testing.T, setBefore bool) { } toKill := rand.Intn(len(eps)) - cfg := clientv3.Config{Endpoints: []string{eps[toKill]}, DialTimeout: 1 * time.Second} + cfg := clientv3.Config{ + Endpoints: []string{eps[toKill]}, + DialTimeout: 1 * time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, + } cli, err := clientv3.New(cfg) if err != nil { t.Fatal(err) @@ -121,6 +139,7 @@ func testDialSetEndpoints(t *testing.T, setBefore bool) { if !setBefore { cli.SetEndpoints(eps[toKill%3], eps[(toKill+1)%3]) } + time.Sleep(time.Second * 2) ctx, cancel := context.WithTimeout(context.Background(), integration.RequestWaitTimeout) if _, err = cli.Get(ctx, "foo", clientv3.WithSerializable()); err != nil { t.Fatal(err) @@ -142,6 +161,7 @@ func TestSwitchSetEndpoints(t *testing.T) { clus.Members[0].InjectPartition(t, clus.Members[1:]...) cli.SetEndpoints(eps...) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() if _, err := cli.Get(ctx, "foo"); err != nil { @@ -158,6 +178,7 @@ func TestRejectOldCluster(t *testing.T) { cfg := clientv3.Config{ Endpoints: []string{clus.Members[0].GRPCAddr(), clus.Members[1].GRPCAddr()}, DialTimeout: 5 * time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, RejectOldCluster: true, } cli, err := clientv3.New(cfg) diff --git a/clientv3/integration/kv_test.go b/clientv3/integration/kv_test.go index 66de753cdad..1c7724acdfb 100644 --- a/clientv3/integration/kv_test.go +++ b/clientv3/integration/kv_test.go @@ -442,7 +442,7 @@ func TestKVGetErrConnClosed(t *testing.T) { go func() { defer close(donec) _, err := cli.Get(context.TODO(), "foo") - if err != nil && err != context.Canceled && err != grpc.ErrClientConnClosing { + if !clientv3.IsConnCanceled(err) { t.Fatalf("expected %v or %v, got %v", context.Canceled, grpc.ErrClientConnClosing, err) } }() @@ -474,7 +474,7 @@ func TestKVNewAfterClose(t *testing.T) { donec := make(chan struct{}) go func() { _, err := cli.Get(context.TODO(), "foo") - if err != context.Canceled && err != grpc.ErrClientConnClosing { + if !clientv3.IsConnCanceled(err) { t.Fatalf("expected %v or %v, got %v", context.Canceled, grpc.ErrClientConnClosing, err) } close(donec) @@ -708,9 +708,10 @@ func TestKVGetRetry(t *testing.T) { time.Sleep(100 * time.Millisecond) clus.Members[fIdx].Restart(t) + clus.Members[fIdx].WaitOK(t) select { - case <-time.After(5 * time.Second): + case <-time.After(20 * time.Second): t.Fatalf("timed out waiting for get") case <-donec: } @@ -750,7 +751,7 @@ func TestKVPutFailGetRetry(t *testing.T) { clus.Members[0].Restart(t) select { - case <-time.After(5 * time.Second): + case <-time.After(20 * time.Second): t.Fatalf("timed out waiting for get") case <-donec: } @@ -792,7 +793,7 @@ func TestKVGetStoppedServerAndClose(t *testing.T) { // this Get fails and triggers an asynchronous connection retry _, err := cli.Get(ctx, "abc") cancel() - if err != nil && err != context.DeadlineExceeded { + if err != nil && !(isCanceled(err) || isClientTimeout(err)) { t.Fatal(err) } } @@ -814,14 +815,15 @@ func TestKVPutStoppedServerAndClose(t *testing.T) { // grpc finds out the original connection is down due to the member shutdown. _, err := cli.Get(ctx, "abc") cancel() - if err != nil && err != context.DeadlineExceeded { + if err != nil && !(isCanceled(err) || isClientTimeout(err)) { t.Fatal(err) } + ctx, cancel = context.WithTimeout(context.TODO(), time.Second) // this Put fails and triggers an asynchronous connection retry _, err = cli.Put(ctx, "abc", "123") cancel() - if err != nil && err != context.DeadlineExceeded { + if err != nil && !(isCanceled(err) || isClientTimeout(err) || isUnavailable(err)) { t.Fatal(err) } } @@ -906,7 +908,7 @@ func TestKVLargeRequests(t *testing.T) { maxCallSendBytesClient: 10 * 1024 * 1024, maxCallRecvBytesClient: 0, valueSize: 10 * 1024 * 1024, - expectError: grpc.Errorf(codes.ResourceExhausted, "grpc: trying to send message larger than max "), + expectError: grpc.Errorf(codes.ResourceExhausted, "trying to send message larger than max "), }, { maxRequestBytesServer: 10 * 1024 * 1024, @@ -920,7 +922,7 @@ func TestKVLargeRequests(t *testing.T) { maxCallSendBytesClient: 10 * 1024 * 1024, maxCallRecvBytesClient: 0, valueSize: 10*1024*1024 + 5, - expectError: grpc.Errorf(codes.ResourceExhausted, "grpc: trying to send message larger than max "), + expectError: grpc.Errorf(codes.ResourceExhausted, "trying to send message larger than max "), }, } for i, test := range tests { @@ -940,7 +942,7 @@ func TestKVLargeRequests(t *testing.T) { t.Errorf("#%d: expected %v, got %v", i, test.expectError, err) } } else if err != nil && !strings.HasPrefix(err.Error(), test.expectError.Error()) { - t.Errorf("#%d: expected %v, got %v", i, test.expectError, err) + t.Errorf("#%d: expected error starting with '%s', got '%s'", i, test.expectError.Error(), err.Error()) } // put request went through, now expects large response back diff --git a/clientv3/integration/lease_test.go b/clientv3/integration/lease_test.go index ee7611b1e40..bd7ee897bbf 100644 --- a/clientv3/integration/lease_test.go +++ b/clientv3/integration/lease_test.go @@ -145,6 +145,10 @@ func TestLeaseKeepAlive(t *testing.T) { t.Errorf("chan is closed, want not closed") } + if kresp == nil { + t.Fatalf("unexpected null response") + } + if kresp.ID != resp.ID { t.Errorf("ID = %x, want %x", kresp.ID, resp.ID) } @@ -292,10 +296,10 @@ func TestLeaseGrantErrConnClosed(t *testing.T) { go func() { defer close(donec) _, err := cli.Grant(context.TODO(), 5) - if err != nil && err != grpc.ErrClientConnClosing && err != context.Canceled { + if !clientv3.IsConnCanceled(err) { // grpc.ErrClientConnClosing if grpc-go balancer calls 'Get' after client.Close. // context.Canceled if grpc-go balancer calls 'Get' with an inflight client.Close. - t.Fatalf("expected %v or %v, got %v", grpc.ErrClientConnClosing, context.Canceled, err) + t.Fatalf("expected %v, %v or server unavailable, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err) } }() @@ -324,8 +328,9 @@ func TestLeaseGrantNewAfterClose(t *testing.T) { donec := make(chan struct{}) go func() { - if _, err := cli.Grant(context.TODO(), 5); err != context.Canceled && err != grpc.ErrClientConnClosing { - t.Fatalf("expected %v or %v, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err) + _, err := cli.Grant(context.TODO(), 5) + if !clientv3.IsConnCanceled(err) { + t.Fatalf("expected %v, %v or server unavailable, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err) } close(donec) }() @@ -356,8 +361,9 @@ func TestLeaseRevokeNewAfterClose(t *testing.T) { donec := make(chan struct{}) go func() { - if _, err := cli.Revoke(context.TODO(), leaseID); err != context.Canceled && err != grpc.ErrClientConnClosing { - t.Fatalf("expected %v or %v, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err) + _, err := cli.Revoke(context.TODO(), leaseID) + if !clientv3.IsConnCanceled(err) { + t.Fatalf("expected %v, %v or server unavailable, got %v", err != context.Canceled, grpc.ErrClientConnClosing, err) } close(donec) }() diff --git a/clientv3/integration/leasing_test.go b/clientv3/integration/leasing_test.go index bfed1ea4221..7c3dcc3f179 100644 --- a/clientv3/integration/leasing_test.go +++ b/clientv3/integration/leasing_test.go @@ -1905,7 +1905,7 @@ func TestLeasingSessionExpire(t *testing.T) { } waitForExpireAck(t, lkv) clus.Members[0].Restart(t) - + integration.WaitClientV3(t, lkv2) if _, err = lkv2.Put(context.TODO(), "abc", "def"); err != nil { t.Fatal(err) } @@ -1920,10 +1920,6 @@ func TestLeasingSessionExpire(t *testing.T) { } func TestLeasingSessionExpireCancel(t *testing.T) { - defer testutil.AfterTest(t) - clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) - defer clus.Terminate(t) - tests := []func(context.Context, clientv3.KV) error{ func(ctx context.Context, kv clientv3.KV) error { _, err := kv.Get(ctx, "abc") @@ -1960,37 +1956,43 @@ func TestLeasingSessionExpireCancel(t *testing.T) { }, } for i := range tests { - lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/", concurrency.WithTTL(1)) - testutil.AssertNil(t, err) - defer closeLKV() + t.Run(fmt.Sprintf("test %d", i), func(t *testing.T) { + defer testutil.AfterTest(t) + clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3}) + defer clus.Terminate(t) - if _, err = lkv.Get(context.TODO(), "abc"); err != nil { - t.Fatal(err) - } + lkv, closeLKV, err := leasing.NewKV(clus.Client(0), "foo/", concurrency.WithTTL(1)) + testutil.AssertNil(t, err) + defer closeLKV() - // down endpoint lkv uses for keepalives - clus.Members[0].Stop(t) - if err := waitForLeasingExpire(clus.Client(1), "foo/abc"); err != nil { - t.Fatal(err) - } - waitForExpireAck(t, lkv) + if _, err = lkv.Get(context.TODO(), "abc"); err != nil { + t.Fatal(err) + } - ctx, cancel := context.WithCancel(context.TODO()) - errc := make(chan error, 1) - go func() { errc <- tests[i](ctx, lkv) }() - // some delay to get past for ctx.Err() != nil {} loops - time.Sleep(100 * time.Millisecond) - cancel() + // down endpoint lkv uses for keepalives + clus.Members[0].Stop(t) + if err := waitForLeasingExpire(clus.Client(1), "foo/abc"); err != nil { + t.Fatal(err) + } + waitForExpireAck(t, lkv) - select { - case err := <-errc: - if err != ctx.Err() { - t.Errorf("#%d: expected %v, got %v", i, ctx.Err(), err) + ctx, cancel := context.WithCancel(context.TODO()) + errc := make(chan error, 1) + go func() { errc <- tests[i](ctx, lkv) }() + // some delay to get past for ctx.Err() != nil {} loops + time.Sleep(100 * time.Millisecond) + cancel() + + select { + case err := <-errc: + if err != ctx.Err() { + t.Errorf("#%d: expected %v of server unavailable, got %v", i, ctx.Err(), err) + } + case <-time.After(5 * time.Second): + t.Errorf("#%d: timed out waiting for cancel", i) } - case <-time.After(5 * time.Second): - t.Errorf("#%d: timed out waiting for cancel", i) - } - clus.Members[0].Restart(t) + clus.Members[0].Restart(t) + }) } } @@ -2016,6 +2018,8 @@ func waitForExpireAck(t *testing.T, kv clientv3.KV) { cancel() if err == ctx.Err() { return + } else if err != nil { + t.Logf("current error: %v", err) } time.Sleep(time.Second) } diff --git a/clientv3/integration/maintenance_test.go b/clientv3/integration/maintenance_test.go index 1edc6fdff76..b4b73614a36 100644 --- a/clientv3/integration/maintenance_test.go +++ b/clientv3/integration/maintenance_test.go @@ -21,7 +21,6 @@ import ( "io" "io/ioutil" "path/filepath" - "strings" "testing" "time" @@ -131,8 +130,8 @@ func TestMaintenanceSnapshotError(t *testing.T) { time.Sleep(2 * time.Second) _, err = io.Copy(ioutil.Discard, rc2) - if err != nil && err != context.DeadlineExceeded { - t.Errorf("expected %v, got %v", context.DeadlineExceeded, err) + if err != nil && !isClientTimeout(err) { + t.Errorf("expected client timeout, got %v", err) } } @@ -157,9 +156,10 @@ func TestMaintenanceSnapshotErrorInflight(t *testing.T) { b.Close() clus.Members[0].Restart(t) + cli := clus.RandClient() // reading snapshot with canceled context should error out ctx, cancel := context.WithCancel(context.Background()) - rc1, err := clus.RandClient().Snapshot(ctx) + rc1, err := cli.Snapshot(ctx) if err != nil { t.Fatal(err) } @@ -189,7 +189,7 @@ func TestMaintenanceSnapshotErrorInflight(t *testing.T) { // 300ms left and expect timeout while snapshot reading is in progress time.Sleep(700 * time.Millisecond) _, err = io.Copy(ioutil.Discard, rc2) - if err != nil && !strings.Contains(err.Error(), context.DeadlineExceeded.Error()) { - t.Errorf("expected %v from gRPC, got %v", context.DeadlineExceeded, err) + if err != nil && !isClientTimeout(err) { + t.Errorf("expected client timeout, got %v", err) } } diff --git a/clientv3/integration/network_partition_test.go b/clientv3/integration/network_partition_test.go index e175aa4f266..f65807430ef 100644 --- a/clientv3/integration/network_partition_test.go +++ b/clientv3/integration/network_partition_test.go @@ -26,6 +26,7 @@ import ( "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" "github.com/coreos/etcd/integration" "github.com/coreos/etcd/pkg/testutil" + "google.golang.org/grpc" ) var errExpected = errors.New("expected error") @@ -36,7 +37,7 @@ var errExpected = errors.New("expected error") func TestBalancerUnderNetworkPartitionPut(t *testing.T) { testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error { _, err := cli.Put(ctx, "a", "b") - if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { + if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { return errExpected } return err @@ -46,7 +47,7 @@ func TestBalancerUnderNetworkPartitionPut(t *testing.T) { func TestBalancerUnderNetworkPartitionDelete(t *testing.T) { testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error { _, err := cli.Delete(ctx, "a") - if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { + if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { return errExpected } return err @@ -59,7 +60,7 @@ func TestBalancerUnderNetworkPartitionTxn(t *testing.T) { If(clientv3.Compare(clientv3.Version("foo"), "=", 0)). Then(clientv3.OpPut("foo", "bar")). Else(clientv3.OpPut("foo", "baz")).Commit() - if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { + if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout { return errExpected } return err @@ -72,6 +73,9 @@ func TestBalancerUnderNetworkPartitionTxn(t *testing.T) { func TestBalancerUnderNetworkPartitionLinearizableGetWithLongTimeout(t *testing.T) { testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error { _, err := cli.Get(ctx, "a") + if err == rpctypes.ErrTimeout { + return errExpected + } return err }, 7*time.Second) } @@ -82,7 +86,7 @@ func TestBalancerUnderNetworkPartitionLinearizableGetWithLongTimeout(t *testing. func TestBalancerUnderNetworkPartitionLinearizableGetWithShortTimeout(t *testing.T) { testBalancerUnderNetworkPartition(t, func(cli *clientv3.Client, ctx context.Context) error { _, err := cli.Get(ctx, "a") - if err == context.DeadlineExceeded || isServerCtxTimeout(err) { + if isClientTimeout(err) || isServerCtxTimeout(err) { return errExpected } return err @@ -111,6 +115,7 @@ func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, c ccfg := clientv3.Config{ Endpoints: []string{eps[0]}, DialTimeout: 3 * time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, } cli, err := clientv3.New(ccfg) if err != nil { @@ -123,9 +128,10 @@ func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, c // add other endpoints for later endpoint switch cli.SetEndpoints(eps...) + time.Sleep(time.Second * 2) clus.Members[0].InjectPartition(t, clus.Members[1:]...) - for i := 0; i < 2; i++ { + for i := 0; i < 5; i++ { ctx, cancel := context.WithTimeout(context.Background(), timeout) err = op(cli, ctx) cancel() @@ -133,7 +139,7 @@ func testBalancerUnderNetworkPartition(t *testing.T, op func(*clientv3.Client, c break } if err != errExpected { - t.Errorf("#%d: expected %v, got %v", i, errExpected, err) + t.Errorf("#%d: expected '%v', got '%v'", i, errExpected, err) } // give enough time for endpoint switch // TODO: remove random sleep by syncing directly with balancer @@ -165,16 +171,14 @@ func TestBalancerUnderNetworkPartitionLinearizableGetLeaderElection(t *testing.T cli, err := clientv3.New(clientv3.Config{ Endpoints: []string{eps[(lead+1)%2]}, - DialTimeout: 1 * time.Second, + DialTimeout: 2 * time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, }) if err != nil { t.Fatal(err) } defer cli.Close() - // wait for non-leader to be pinned - mustWaitPinReady(t, cli) - // add all eps to list, so that when the original pined one fails // the client can switch to other available eps cli.SetEndpoints(eps[lead], eps[(lead+1)%2]) @@ -182,10 +186,15 @@ func TestBalancerUnderNetworkPartitionLinearizableGetLeaderElection(t *testing.T // isolate leader clus.Members[lead].InjectPartition(t, clus.Members[(lead+1)%3], clus.Members[(lead+2)%3]) - // expects balancer endpoint switch while ongoing leader election - ctx, cancel := context.WithTimeout(context.TODO(), timeout) - _, err = cli.Get(ctx, "a") - cancel() + // expects balancer to round robin to leader within two attempts + for i := 0; i < 2; i++ { + ctx, cancel := context.WithTimeout(context.TODO(), timeout) + _, err = cli.Get(ctx, "a") + cancel() + if err == nil { + break + } + } if err != nil { t.Fatal(err) } diff --git a/clientv3/integration/server_shutdown_test.go b/clientv3/integration/server_shutdown_test.go index 0a9f7b492b8..1f889edac03 100644 --- a/clientv3/integration/server_shutdown_test.go +++ b/clientv3/integration/server_shutdown_test.go @@ -17,7 +17,6 @@ package integration import ( "bytes" "context" - "reflect" "strings" "testing" "time" @@ -29,6 +28,7 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/status" + "google.golang.org/grpc/transport" ) // TestBalancerUnderServerShutdownWatch expects that watch client @@ -105,7 +105,7 @@ func TestBalancerUnderServerShutdownWatch(t *testing.T) { if err == nil { break } - if err == context.DeadlineExceeded || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout || err == rpctypes.ErrTimeoutDueToLeaderFail { + if isClientTimeout(err) || isServerCtxTimeout(err) || err == rpctypes.ErrTimeout || err == rpctypes.ErrTimeoutDueToLeaderFail { continue } t.Fatal(err) @@ -338,14 +338,20 @@ func testBalancerUnderServerStopInflightRangeOnRestart(t *testing.T, linearizabl defer close(donec) ctx, cancel := context.WithTimeout(context.TODO(), clientTimeout) readyc <- struct{}{} - _, err := cli.Get(ctx, "abc", gops...) + + // TODO: The new grpc load balancer will not pin to an endpoint + // as intended by this test. But it will round robin member within + // two attempts. + // Remove retry loop once the new grpc load balancer provides retry. + for i := 0; i < 2; i++ { + _, err = cli.Get(ctx, "abc", gops...) + if err == nil { + break + } + } cancel() if err != nil { - if linearizable && strings.Contains(err.Error(), "context deadline exceeded") { - t.Logf("TODO: FIX THIS after balancer rewrite! %v %v", reflect.TypeOf(err), err) - } else { - t.Fatal(err) - } + t.Fatalf("unexpected error: %v", err) } }() @@ -373,3 +379,50 @@ func isServerCtxTimeout(err error) bool { code := ev.Code() return code == codes.DeadlineExceeded && strings.Contains(err.Error(), "context deadline exceeded") } + +// In grpc v1.11.3+ dial timeouts can error out with transport.ErrConnClosing. Previously dial timeouts +// would always error out with context.DeadlineExceeded. +func isClientTimeout(err error) bool { + if err == nil { + return false + } + if err == context.DeadlineExceeded { + return true + } + ev, ok := status.FromError(err) + if !ok { + return false + } + code := ev.Code() + return code == codes.DeadlineExceeded || ev.Message() == transport.ErrConnClosing.Desc +} + +func isCanceled(err error) bool { + if err == nil { + return false + } + if err == context.Canceled { + return true + } + ev, ok := status.FromError(err) + if !ok { + return false + } + code := ev.Code() + return code == codes.Canceled +} + +func isUnavailable(err error) bool { + if err == nil { + return false + } + if err == context.Canceled { + return true + } + ev, ok := status.FromError(err) + if !ok { + return false + } + code := ev.Code() + return code == codes.Unavailable +} diff --git a/clientv3/integration/txn_test.go b/clientv3/integration/txn_test.go index 14f8b81a7ba..ca49ec075d8 100644 --- a/clientv3/integration/txn_test.go +++ b/clientv3/integration/txn_test.go @@ -90,7 +90,7 @@ func TestTxnWriteFail(t *testing.T) { }() select { - case <-time.After(2 * clus.Members[1].ServerConfig.ReqTimeout()): + case <-time.After(5 * clus.Members[1].ServerConfig.ReqTimeout()): t.Fatalf("timed out waiting for get") case <-getc: } diff --git a/clientv3/integration/user_test.go b/clientv3/integration/user_test.go index 11548123ddb..da341d6604f 100644 --- a/clientv3/integration/user_test.go +++ b/clientv3/integration/user_test.go @@ -17,11 +17,13 @@ package integration import ( "context" "testing" + "time" "github.com/coreos/etcd/clientv3" "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" "github.com/coreos/etcd/integration" "github.com/coreos/etcd/pkg/testutil" + "google.golang.org/grpc" ) func TestUserError(t *testing.T) { @@ -68,7 +70,11 @@ func TestUserErrorAuth(t *testing.T) { } // wrong id or password - cfg := clientv3.Config{Endpoints: authapi.Endpoints()} + cfg := clientv3.Config{ + Endpoints: authapi.Endpoints(), + DialTimeout: 5 * time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, + } cfg.Username, cfg.Password = "wrong-id", "123" if _, err := clientv3.New(cfg); err != rpctypes.ErrAuthFailed { t.Fatalf("expected %v, got %v", rpctypes.ErrAuthFailed, err) diff --git a/clientv3/integration/watch_test.go b/clientv3/integration/watch_test.go index f9ac47b4ae4..efe4387abdd 100644 --- a/clientv3/integration/watch_test.go +++ b/clientv3/integration/watch_test.go @@ -30,7 +30,6 @@ import ( mvccpb "github.com/coreos/etcd/mvcc/mvccpb" "github.com/coreos/etcd/pkg/testutil" - "google.golang.org/grpc" "google.golang.org/grpc/metadata" ) @@ -667,8 +666,9 @@ func TestWatchErrConnClosed(t *testing.T) { go func() { defer close(donec) ch := cli.Watch(context.TODO(), "foo") - if wr := <-ch; grpc.ErrorDesc(wr.Err()) != grpc.ErrClientConnClosing.Error() { - t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, grpc.ErrorDesc(wr.Err())) + + if wr := <-ch; !isCanceled(wr.Err()) { + t.Fatalf("expected context canceled, got %v", wr.Err()) } }() @@ -699,8 +699,8 @@ func TestWatchAfterClose(t *testing.T) { donec := make(chan struct{}) go func() { cli.Watch(context.TODO(), "foo") - if err := cli.Close(); err != nil && err != grpc.ErrClientConnClosing { - t.Fatalf("expected %v, got %v", grpc.ErrClientConnClosing, err) + if err := cli.Close(); err != nil && err != context.Canceled { + t.Fatalf("expected %v, got %v", context.Canceled, err) } close(donec) }() diff --git a/clientv3/lease.go b/clientv3/lease.go index 4097b3afa2a..3d5ff4f7226 100644 --- a/clientv3/lease.go +++ b/clientv3/lease.go @@ -466,7 +466,7 @@ func (l *lessor) recvKeepAliveLoop() (gerr error) { // resetRecv opens a new lease stream and starts sending keep alive requests. func (l *lessor) resetRecv() (pb.Lease_LeaseKeepAliveClient, error) { sctx, cancel := context.WithCancel(l.stopCtx) - stream, err := l.remote.LeaseKeepAlive(sctx, l.callOpts...) + stream, err := l.remote.LeaseKeepAlive(sctx, append(l.callOpts, withMax(0))...) if err != nil { cancel() return nil, err diff --git a/clientv3/maintenance.go b/clientv3/maintenance.go index ce05b3b6594..f814874f2cb 100644 --- a/clientv3/maintenance.go +++ b/clientv3/maintenance.go @@ -16,6 +16,7 @@ package clientv3 import ( "context" + "fmt" "io" pb "github.com/coreos/etcd/etcdserver/etcdserverpb" @@ -77,7 +78,7 @@ func NewMaintenance(c *Client) Maintenance { dial: func(endpoint string) (pb.MaintenanceClient, func(), error) { conn, err := c.dial(endpoint) if err != nil { - return nil, nil, err + return nil, nil, fmt.Errorf("failed to dial endpoint %s with maintenance client: %v", endpoint, err) } cancel := func() { conn.Close() } return RetryMaintenanceClient(c, conn), cancel, nil @@ -175,6 +176,7 @@ func (m *maintenance) Status(ctx context.Context, endpoint string) (*StatusRespo func (m *maintenance) HashKV(ctx context.Context, endpoint string, rev int64) (*HashKVResponse, error) { remote, cancel, err := m.dial(endpoint) if err != nil { + return nil, toErr(ctx, err) } defer cancel() @@ -186,7 +188,7 @@ func (m *maintenance) HashKV(ctx context.Context, endpoint string, rev int64) (* } func (m *maintenance) Snapshot(ctx context.Context) (io.ReadCloser, error) { - ss, err := m.remote.Snapshot(ctx, &pb.SnapshotRequest{}, m.callOpts...) + ss, err := m.remote.Snapshot(ctx, &pb.SnapshotRequest{}, append(m.callOpts, withMax(defaultStreamMaxRetries))...) if err != nil { return nil, toErr(ctx, err) } diff --git a/clientv3/options.go b/clientv3/options.go index fa25811f3b0..b82b7554d7c 100644 --- a/clientv3/options.go +++ b/clientv3/options.go @@ -16,17 +16,17 @@ package clientv3 import ( "math" + "time" "google.golang.org/grpc" ) var ( - // Disable gRPC internal retrial logic - // TODO: enable when gRPC retry is stable (FailFast=false) - // Reference: - // - https://github.com/grpc/grpc-go/issues/1532 - // - https://github.com/grpc/proposal/blob/master/A6-client-retries.md - defaultFailFast = grpc.FailFast(true) + // client-side handling retrying of request failures where data was not written to the wire or + // where server indicates it did not process the data. gRPC default is default is "FailFast(true)" + // but for etcd we default to "FailFast(false)" to minimize client request error responses due to + // transient failures. + defaultFailFast = grpc.FailFast(false) // client-side request send limit, gRPC default is math.MaxInt32 // Make sure that "client-side send limit < server-side default send/recv limit" @@ -38,6 +38,22 @@ var ( // because range response can easily exceed request send limits // Default to math.MaxInt32; writes exceeding server-side send limit fails anyway defaultMaxCallRecvMsgSize = grpc.MaxCallRecvMsgSize(math.MaxInt32) + + // client-side non-streaming retry limit, only applied to requests where server responds with + // a error code clearly indicating it was unable to process the request such as codes.Unavailable. + // If set to 0, retry is disabled. + defaultUnaryMaxRetries uint = 100 + + // client-side streaming retry limit, only applied to requests where server responds with + // a error code clearly indicating it was unable to process the request such as codes.Unavailable. + // If set to 0, retry is disabled. + defaultStreamMaxRetries uint = ^uint(0) // max uint + + // client-side retry backoff wait between requests. + defaultBackoffWaitBetween = 25 * time.Millisecond + + // client-side retry backoff default jitter fraction. + defaultBackoffJitterFraction = 0.10 ) // defaultCallOpts defines a list of default "gRPC.CallOption". diff --git a/clientv3/ordering/kv_test.go b/clientv3/ordering/kv_test.go index 9e884f3b291..ebe802334e5 100644 --- a/clientv3/ordering/kv_test.go +++ b/clientv3/ordering/kv_test.go @@ -82,6 +82,7 @@ func TestDetectKvOrderViolation(t *testing.T) { clus.Members[2].Restart(t) // force OrderingKv to query the third member cli.SetEndpoints(clus.Members[2].GRPCAddr()) + time.Sleep(2 * time.Second) // FIXME: Figure out how pause SetEndpoints sufficiently that this is not needed _, err = orderingKv.Get(ctx, "foo", clientv3.WithSerializable()) if err != errOrderViolation { @@ -147,7 +148,7 @@ func TestDetectTxnOrderViolation(t *testing.T) { clus.Members[2].Restart(t) // force OrderingKv to query the third member cli.SetEndpoints(clus.Members[2].GRPCAddr()) - + time.Sleep(2 * time.Second) // FIXME: Figure out how pause SetEndpoints sufficiently that this is not needed _, err = orderingKv.Get(ctx, "foo", clientv3.WithSerializable()) if err != errOrderViolation { t.Fatalf("expected %v, got %v", errOrderViolation, err) diff --git a/clientv3/ordering/util.go b/clientv3/ordering/util.go index 190a5919a52..124aebf4174 100644 --- a/clientv3/ordering/util.go +++ b/clientv3/ordering/util.go @@ -43,6 +43,8 @@ func NewOrderViolationSwitchEndpointClosure(c clientv3.Client) OrderViolationFun // set available endpoints back to all endpoints in to ensure // the client has access to all the endpoints. c.SetEndpoints(eps...) + // give enough time for operation + time.Sleep(1 * time.Second) violationCount++ return nil } diff --git a/clientv3/ordering/util_test.go b/clientv3/ordering/util_test.go index 142c67072c2..a370e0d07cc 100644 --- a/clientv3/ordering/util_test.go +++ b/clientv3/ordering/util_test.go @@ -122,6 +122,7 @@ func TestUnresolvableOrderViolation(t *testing.T) { // NewOrderViolationSwitchEndpointClosure will be able to // access the full list of endpoints. cli.SetEndpoints(eps...) + time.Sleep(1 * time.Second) // give enough time for operation OrderingKv := NewKV(cli.KV, NewOrderViolationSwitchEndpointClosure(*cli)) // set prevRev to the first member's revision of "foo" such that // the revision is higher than the fourth and fifth members' revision of "foo" @@ -133,10 +134,17 @@ func TestUnresolvableOrderViolation(t *testing.T) { clus.Members[0].Stop(t) clus.Members[1].Stop(t) clus.Members[2].Stop(t) - clus.Members[3].Restart(t) - clus.Members[4].Restart(t) + err = clus.Members[3].Restart(t) + if err != nil { + t.Fatal(err) + } + err = clus.Members[4].Restart(t) + if err != nil { + t.Fatal(err) + } + clus.Members[3].WaitStarted(t) cli.SetEndpoints(clus.Members[3].GRPCAddr()) - time.Sleep(1 * time.Second) // give enough time for operation + time.Sleep(5 * time.Second) // give enough time for operation _, err = OrderingKv.Get(ctx, "foo", clientv3.WithSerializable()) if err != ErrNoGreaterRev { diff --git a/clientv3/retry.go b/clientv3/retry.go index 6226d787e95..6118aa55a05 100644 --- a/clientv3/retry.go +++ b/clientv3/retry.go @@ -32,467 +32,267 @@ const ( nonRepeatable ) +func (rp retryPolicy) String() string { + switch rp { + case repeatable: + return "repeatable" + case nonRepeatable: + return "nonRepeatable" + default: + return "UNKNOWN" + } +} + type rpcFunc func(ctx context.Context) error type retryRPCFunc func(context.Context, rpcFunc, retryPolicy) error type retryStopErrFunc func(error) bool +// isSafeRetryImmutableRPC returns "true" when an immutable request is safe for retry. +// // immutable requests (e.g. Get) should be retried unless it's // an obvious server-side error (e.g. rpctypes.ErrRequestTooLarge). // -// "isRepeatableStopError" returns "true" when an immutable request -// is interrupted by server-side or gRPC-side error and its status -// code is not transient (!= codes.Unavailable). -// -// Returning "true" means retry should stop, since client cannot +// Returning "false" means retry should stop, since client cannot // handle itself even with retries. -func isRepeatableStopError(err error) bool { +func isSafeRetryImmutableRPC(err error) bool { eErr := rpctypes.Error(err) - // always stop retry on etcd errors if serverErr, ok := eErr.(rpctypes.EtcdError); ok && serverErr.Code() != codes.Unavailable { - return true + // interrupted by non-transient server-side or gRPC-side error + // client cannot handle itself (e.g. rpctypes.ErrCompacted) + return false } // only retry if unavailable ev, ok := status.FromError(err) if !ok { + // all errors from RPC is typed "grpc/status.(*statusError)" + // (ref. https://github.com/grpc/grpc-go/pull/1782) + // + // if the error type is not "grpc/status.(*statusError)", + // it could be from "Dial" + // TODO: do not retry for now + // ref. https://github.com/grpc/grpc-go/issues/1581 return false } - return ev.Code() != codes.Unavailable + return ev.Code() == codes.Unavailable } +// isSafeRetryMutableRPC returns "true" when a mutable request is safe for retry. +// // mutable requests (e.g. Put, Delete, Txn) should only be retried // when the status code is codes.Unavailable when initial connection -// has not been established (no pinned endpoint). +// has not been established (no endpoint is up). // -// "isNonRepeatableStopError" returns "true" when a mutable request -// is interrupted by non-transient error that client cannot handle itself, -// or transient error while the connection has already been established -// (pinned endpoint exists). -// -// Returning "true" means retry should stop, otherwise it violates +// Returning "false" means retry should stop, otherwise it violates // write-at-most-once semantics. -func isNonRepeatableStopError(err error) bool { +func isSafeRetryMutableRPC(err error) bool { if ev, ok := status.FromError(err); ok && ev.Code() != codes.Unavailable { - return true + // not safe for mutable RPCs + // e.g. interrupted by non-transient error that client cannot handle itself, + // or transient error while the connection has already been established + return false } desc := rpctypes.ErrorDesc(err) - return desc != "there is no address available" && desc != "there is no connection available" -} - -func (c *Client) newRetryWrapper() retryRPCFunc { - return func(rpcCtx context.Context, f rpcFunc, rp retryPolicy) error { - var isStop retryStopErrFunc - switch rp { - case repeatable: - isStop = isRepeatableStopError - case nonRepeatable: - isStop = isNonRepeatableStopError - } - for { - if err := readyWait(rpcCtx, c.ctx, c.balancer.ConnectNotify()); err != nil { - return err - } - pinned := c.balancer.Pinned() - err := f(rpcCtx) - if err == nil { - return nil - } - lg.Lvl(4).Infof("clientv3/retry: error %q on pinned endpoint %q", err.Error(), pinned) - - if s, ok := status.FromError(err); ok && (s.Code() == codes.Unavailable || s.Code() == codes.DeadlineExceeded || s.Code() == codes.Internal) { - // mark this before endpoint switch is triggered - c.balancer.HostPortError(pinned, err) - c.balancer.Next() - lg.Lvl(4).Infof("clientv3/retry: switching from %q due to error %q", pinned, err.Error()) - } - - if isStop(err) { - return err - } - } - } -} - -func (c *Client) newAuthRetryWrapper(retryf retryRPCFunc) retryRPCFunc { - return func(rpcCtx context.Context, f rpcFunc, rp retryPolicy) error { - for { - pinned := c.balancer.Pinned() - err := retryf(rpcCtx, f, rp) - if err == nil { - return nil - } - lg.Lvl(4).Infof("clientv3/auth-retry: error %q on pinned endpoint %q", err.Error(), pinned) - // always stop retry on etcd errors other than invalid auth token - if rpctypes.Error(err) == rpctypes.ErrInvalidAuthToken { - gterr := c.getToken(rpcCtx) - if gterr != nil { - lg.Lvl(4).Infof("clientv3/auth-retry: cannot retry due to error %q(%q) on pinned endpoint %q", err.Error(), gterr.Error(), pinned) - return err // return the original error for simplicity - } - continue - } - return err - } - } + return desc == "there is no address available" || desc == "there is no connection available" } type retryKVClient struct { - kc pb.KVClient - retryf retryRPCFunc + kc pb.KVClient } // RetryKVClient implements a KVClient. func RetryKVClient(c *Client) pb.KVClient { return &retryKVClient{ - kc: pb.NewKVClient(c.conn), - retryf: c.newAuthRetryWrapper(c.newRetryWrapper()), + kc: pb.NewKVClient(c.conn), } } func (rkv *retryKVClient) Range(ctx context.Context, in *pb.RangeRequest, opts ...grpc.CallOption) (resp *pb.RangeResponse, err error) { - err = rkv.retryf(ctx, func(rctx context.Context) error { - resp, err = rkv.kc.Range(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rkv.kc.Range(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rkv *retryKVClient) Put(ctx context.Context, in *pb.PutRequest, opts ...grpc.CallOption) (resp *pb.PutResponse, err error) { - err = rkv.retryf(ctx, func(rctx context.Context) error { - resp, err = rkv.kc.Put(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rkv.kc.Put(ctx, in, opts...) } func (rkv *retryKVClient) DeleteRange(ctx context.Context, in *pb.DeleteRangeRequest, opts ...grpc.CallOption) (resp *pb.DeleteRangeResponse, err error) { - err = rkv.retryf(ctx, func(rctx context.Context) error { - resp, err = rkv.kc.DeleteRange(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rkv.kc.DeleteRange(ctx, in, opts...) } func (rkv *retryKVClient) Txn(ctx context.Context, in *pb.TxnRequest, opts ...grpc.CallOption) (resp *pb.TxnResponse, err error) { - // TODO: "repeatable" for read-only txn - err = rkv.retryf(ctx, func(rctx context.Context) error { - resp, err = rkv.kc.Txn(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rkv.kc.Txn(ctx, in, opts...) } func (rkv *retryKVClient) Compact(ctx context.Context, in *pb.CompactionRequest, opts ...grpc.CallOption) (resp *pb.CompactionResponse, err error) { - err = rkv.retryf(ctx, func(rctx context.Context) error { - resp, err = rkv.kc.Compact(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rkv.kc.Compact(ctx, in, opts...) } type retryLeaseClient struct { - lc pb.LeaseClient - retryf retryRPCFunc + lc pb.LeaseClient } // RetryLeaseClient implements a LeaseClient. func RetryLeaseClient(c *Client) pb.LeaseClient { return &retryLeaseClient{ - lc: pb.NewLeaseClient(c.conn), - retryf: c.newAuthRetryWrapper(c.newRetryWrapper()), + lc: pb.NewLeaseClient(c.conn), } } func (rlc *retryLeaseClient) LeaseTimeToLive(ctx context.Context, in *pb.LeaseTimeToLiveRequest, opts ...grpc.CallOption) (resp *pb.LeaseTimeToLiveResponse, err error) { - err = rlc.retryf(ctx, func(rctx context.Context) error { - resp, err = rlc.lc.LeaseTimeToLive(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rlc.lc.LeaseTimeToLive(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rlc *retryLeaseClient) LeaseLeases(ctx context.Context, in *pb.LeaseLeasesRequest, opts ...grpc.CallOption) (resp *pb.LeaseLeasesResponse, err error) { - err = rlc.retryf(ctx, func(rctx context.Context) error { - resp, err = rlc.lc.LeaseLeases(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rlc.lc.LeaseLeases(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rlc *retryLeaseClient) LeaseGrant(ctx context.Context, in *pb.LeaseGrantRequest, opts ...grpc.CallOption) (resp *pb.LeaseGrantResponse, err error) { - err = rlc.retryf(ctx, func(rctx context.Context) error { - resp, err = rlc.lc.LeaseGrant(rctx, in, opts...) - return err - }, repeatable) - return resp, err - + return rlc.lc.LeaseGrant(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rlc *retryLeaseClient) LeaseRevoke(ctx context.Context, in *pb.LeaseRevokeRequest, opts ...grpc.CallOption) (resp *pb.LeaseRevokeResponse, err error) { - err = rlc.retryf(ctx, func(rctx context.Context) error { - resp, err = rlc.lc.LeaseRevoke(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rlc.lc.LeaseRevoke(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rlc *retryLeaseClient) LeaseKeepAlive(ctx context.Context, opts ...grpc.CallOption) (stream pb.Lease_LeaseKeepAliveClient, err error) { - err = rlc.retryf(ctx, func(rctx context.Context) error { - stream, err = rlc.lc.LeaseKeepAlive(rctx, opts...) - return err - }, repeatable) - return stream, err + return rlc.lc.LeaseKeepAlive(ctx, append(opts, withRetryPolicy(repeatable))...) } type retryClusterClient struct { - cc pb.ClusterClient - retryf retryRPCFunc + cc pb.ClusterClient } // RetryClusterClient implements a ClusterClient. func RetryClusterClient(c *Client) pb.ClusterClient { return &retryClusterClient{ - cc: pb.NewClusterClient(c.conn), - retryf: c.newRetryWrapper(), + cc: pb.NewClusterClient(c.conn), } } func (rcc *retryClusterClient) MemberList(ctx context.Context, in *pb.MemberListRequest, opts ...grpc.CallOption) (resp *pb.MemberListResponse, err error) { - err = rcc.retryf(ctx, func(rctx context.Context) error { - resp, err = rcc.cc.MemberList(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rcc.cc.MemberList(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rcc *retryClusterClient) MemberAdd(ctx context.Context, in *pb.MemberAddRequest, opts ...grpc.CallOption) (resp *pb.MemberAddResponse, err error) { - err = rcc.retryf(ctx, func(rctx context.Context) error { - resp, err = rcc.cc.MemberAdd(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rcc.cc.MemberAdd(ctx, in, opts...) } func (rcc *retryClusterClient) MemberRemove(ctx context.Context, in *pb.MemberRemoveRequest, opts ...grpc.CallOption) (resp *pb.MemberRemoveResponse, err error) { - err = rcc.retryf(ctx, func(rctx context.Context) error { - resp, err = rcc.cc.MemberRemove(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rcc.cc.MemberRemove(ctx, in, opts...) } func (rcc *retryClusterClient) MemberUpdate(ctx context.Context, in *pb.MemberUpdateRequest, opts ...grpc.CallOption) (resp *pb.MemberUpdateResponse, err error) { - err = rcc.retryf(ctx, func(rctx context.Context) error { - resp, err = rcc.cc.MemberUpdate(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rcc.cc.MemberUpdate(ctx, in, opts...) } type retryMaintenanceClient struct { - mc pb.MaintenanceClient - retryf retryRPCFunc + mc pb.MaintenanceClient } // RetryMaintenanceClient implements a Maintenance. func RetryMaintenanceClient(c *Client, conn *grpc.ClientConn) pb.MaintenanceClient { return &retryMaintenanceClient{ - mc: pb.NewMaintenanceClient(conn), - retryf: c.newRetryWrapper(), + mc: pb.NewMaintenanceClient(conn), } } func (rmc *retryMaintenanceClient) Alarm(ctx context.Context, in *pb.AlarmRequest, opts ...grpc.CallOption) (resp *pb.AlarmResponse, err error) { - err = rmc.retryf(ctx, func(rctx context.Context) error { - resp, err = rmc.mc.Alarm(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rmc.mc.Alarm(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rmc *retryMaintenanceClient) Status(ctx context.Context, in *pb.StatusRequest, opts ...grpc.CallOption) (resp *pb.StatusResponse, err error) { - err = rmc.retryf(ctx, func(rctx context.Context) error { - resp, err = rmc.mc.Status(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rmc.mc.Status(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rmc *retryMaintenanceClient) Hash(ctx context.Context, in *pb.HashRequest, opts ...grpc.CallOption) (resp *pb.HashResponse, err error) { - err = rmc.retryf(ctx, func(rctx context.Context) error { - resp, err = rmc.mc.Hash(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rmc.mc.Hash(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rmc *retryMaintenanceClient) HashKV(ctx context.Context, in *pb.HashKVRequest, opts ...grpc.CallOption) (resp *pb.HashKVResponse, err error) { - err = rmc.retryf(ctx, func(rctx context.Context) error { - resp, err = rmc.mc.HashKV(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rmc.mc.HashKV(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rmc *retryMaintenanceClient) Snapshot(ctx context.Context, in *pb.SnapshotRequest, opts ...grpc.CallOption) (stream pb.Maintenance_SnapshotClient, err error) { - err = rmc.retryf(ctx, func(rctx context.Context) error { - stream, err = rmc.mc.Snapshot(rctx, in, opts...) - return err - }, repeatable) - return stream, err + return rmc.mc.Snapshot(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rmc *retryMaintenanceClient) MoveLeader(ctx context.Context, in *pb.MoveLeaderRequest, opts ...grpc.CallOption) (resp *pb.MoveLeaderResponse, err error) { - err = rmc.retryf(ctx, func(rctx context.Context) error { - resp, err = rmc.mc.MoveLeader(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rmc.mc.MoveLeader(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rmc *retryMaintenanceClient) Defragment(ctx context.Context, in *pb.DefragmentRequest, opts ...grpc.CallOption) (resp *pb.DefragmentResponse, err error) { - err = rmc.retryf(ctx, func(rctx context.Context) error { - resp, err = rmc.mc.Defragment(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rmc.mc.Defragment(ctx, in, opts...) } type retryAuthClient struct { - ac pb.AuthClient - retryf retryRPCFunc + ac pb.AuthClient } // RetryAuthClient implements a AuthClient. func RetryAuthClient(c *Client) pb.AuthClient { return &retryAuthClient{ - ac: pb.NewAuthClient(c.conn), - retryf: c.newRetryWrapper(), + ac: pb.NewAuthClient(c.conn), } } func (rac *retryAuthClient) UserList(ctx context.Context, in *pb.AuthUserListRequest, opts ...grpc.CallOption) (resp *pb.AuthUserListResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.UserList(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rac.ac.UserList(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rac *retryAuthClient) UserGet(ctx context.Context, in *pb.AuthUserGetRequest, opts ...grpc.CallOption) (resp *pb.AuthUserGetResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.UserGet(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rac.ac.UserGet(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rac *retryAuthClient) RoleGet(ctx context.Context, in *pb.AuthRoleGetRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleGetResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.RoleGet(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rac.ac.RoleGet(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rac *retryAuthClient) RoleList(ctx context.Context, in *pb.AuthRoleListRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleListResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.RoleList(rctx, in, opts...) - return err - }, repeatable) - return resp, err + return rac.ac.RoleList(ctx, in, append(opts, withRetryPolicy(repeatable))...) } func (rac *retryAuthClient) AuthEnable(ctx context.Context, in *pb.AuthEnableRequest, opts ...grpc.CallOption) (resp *pb.AuthEnableResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.AuthEnable(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.AuthEnable(ctx, in, opts...) } func (rac *retryAuthClient) AuthDisable(ctx context.Context, in *pb.AuthDisableRequest, opts ...grpc.CallOption) (resp *pb.AuthDisableResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.AuthDisable(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.AuthDisable(ctx, in, opts...) } func (rac *retryAuthClient) UserAdd(ctx context.Context, in *pb.AuthUserAddRequest, opts ...grpc.CallOption) (resp *pb.AuthUserAddResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.UserAdd(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.UserAdd(ctx, in, opts...) } func (rac *retryAuthClient) UserDelete(ctx context.Context, in *pb.AuthUserDeleteRequest, opts ...grpc.CallOption) (resp *pb.AuthUserDeleteResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.UserDelete(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.UserDelete(ctx, in, opts...) } func (rac *retryAuthClient) UserChangePassword(ctx context.Context, in *pb.AuthUserChangePasswordRequest, opts ...grpc.CallOption) (resp *pb.AuthUserChangePasswordResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.UserChangePassword(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.UserChangePassword(ctx, in, opts...) } func (rac *retryAuthClient) UserGrantRole(ctx context.Context, in *pb.AuthUserGrantRoleRequest, opts ...grpc.CallOption) (resp *pb.AuthUserGrantRoleResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.UserGrantRole(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.UserGrantRole(ctx, in, opts...) } func (rac *retryAuthClient) UserRevokeRole(ctx context.Context, in *pb.AuthUserRevokeRoleRequest, opts ...grpc.CallOption) (resp *pb.AuthUserRevokeRoleResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.UserRevokeRole(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.UserRevokeRole(ctx, in, opts...) } func (rac *retryAuthClient) RoleAdd(ctx context.Context, in *pb.AuthRoleAddRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleAddResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.RoleAdd(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.RoleAdd(ctx, in, opts...) } func (rac *retryAuthClient) RoleDelete(ctx context.Context, in *pb.AuthRoleDeleteRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleDeleteResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.RoleDelete(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.RoleDelete(ctx, in, opts...) } func (rac *retryAuthClient) RoleGrantPermission(ctx context.Context, in *pb.AuthRoleGrantPermissionRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleGrantPermissionResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.RoleGrantPermission(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.RoleGrantPermission(ctx, in, opts...) } func (rac *retryAuthClient) RoleRevokePermission(ctx context.Context, in *pb.AuthRoleRevokePermissionRequest, opts ...grpc.CallOption) (resp *pb.AuthRoleRevokePermissionResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.RoleRevokePermission(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.RoleRevokePermission(ctx, in, opts...) } func (rac *retryAuthClient) Authenticate(ctx context.Context, in *pb.AuthenticateRequest, opts ...grpc.CallOption) (resp *pb.AuthenticateResponse, err error) { - err = rac.retryf(ctx, func(rctx context.Context) error { - resp, err = rac.ac.Authenticate(rctx, in, opts...) - return err - }, nonRepeatable) - return resp, err + return rac.ac.Authenticate(ctx, in, opts...) } diff --git a/clientv3/retry_interceptor.go b/clientv3/retry_interceptor.go new file mode 100644 index 00000000000..9fcec4291c0 --- /dev/null +++ b/clientv3/retry_interceptor.go @@ -0,0 +1,382 @@ +// Copyright 2016 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Based on github.com/grpc-ecosystem/go-grpc-middleware/retry, but modified to support the more +// fine grained error checking required by write-at-most-once retry semantics of etcd. + +package clientv3 + +import ( + "context" + "io" + "sync" + "time" + + "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" + "github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/metadata" +) + +// unaryClientInterceptor returns a new retrying unary client interceptor. +// +// The default configuration of the interceptor is to not retry *at all*. This behaviour can be +// changed through options (e.g. WithMax) on creation of the interceptor or on call (through grpc.CallOptions). +func (c *Client) unaryClientInterceptor(logger *zap.Logger, optFuncs ...retryOption) grpc.UnaryClientInterceptor { + intOpts := reuseOrNewWithCallOptions(defaultOptions, optFuncs) + return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + grpcOpts, retryOpts := filterCallOptions(opts) + callOpts := reuseOrNewWithCallOptions(intOpts, retryOpts) + // short circuit for simplicity, and avoiding allocations. + if callOpts.max == 0 { + return invoker(ctx, method, req, reply, cc, grpcOpts...) + } + var lastErr error + for attempt := uint(0); attempt < callOpts.max; attempt++ { + if err := waitRetryBackoff(attempt, ctx, callOpts); err != nil { + return err + } + lastErr = invoker(ctx, method, req, reply, cc, grpcOpts...) + logger.Info("retry unary intercept", zap.Uint("attempt", attempt), zap.Error(lastErr)) + if lastErr == nil { + return nil + } + if isContextError(lastErr) { + if ctx.Err() != nil { + // its the context deadline or cancellation. + return lastErr + } + // its the callCtx deadline or cancellation, in which case try again. + continue + } + if callOpts.retryAuth && rpctypes.Error(lastErr) == rpctypes.ErrInvalidAuthToken { + gterr := c.getToken(ctx) + if gterr != nil { + logger.Info("retry failed to fetch new auth token", zap.Error(gterr)) + return lastErr // return the original error for simplicity + } + continue + } + if !isSafeRetry(c.lg, lastErr, callOpts) { + return lastErr + } + } + return lastErr + } +} + +// streamClientInterceptor returns a new retrying stream client interceptor for server side streaming calls. +// +// The default configuration of the interceptor is to not retry *at all*. This behaviour can be +// changed through options (e.g. WithMax) on creation of the interceptor or on call (through grpc.CallOptions). +// +// Retry logic is available *only for ServerStreams*, i.e. 1:n streams, as the internal logic needs +// to buffer the messages sent by the client. If retry is enabled on any other streams (ClientStreams, +// BidiStreams), the retry interceptor will fail the call. +func (c *Client) streamClientInterceptor(logger *zap.Logger, optFuncs ...retryOption) grpc.StreamClientInterceptor { + intOpts := reuseOrNewWithCallOptions(defaultOptions, optFuncs) + return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) { + grpcOpts, retryOpts := filterCallOptions(opts) + callOpts := reuseOrNewWithCallOptions(intOpts, retryOpts) + // short circuit for simplicity, and avoiding allocations. + if callOpts.max == 0 { + return streamer(ctx, desc, cc, method, grpcOpts...) + } + if desc.ClientStreams { + return nil, grpc.Errorf(codes.Unimplemented, "clientv3/retry_interceptor: cannot retry on ClientStreams, set Disable()") + } + newStreamer, err := streamer(ctx, desc, cc, method, grpcOpts...) + logger.Info("retry stream intercept", zap.Error(err)) + if err != nil { + // TODO(mwitkow): Maybe dial and transport errors should be retriable? + return nil, err + } + retryingStreamer := &serverStreamingRetryingStream{ + client: c, + ClientStream: newStreamer, + callOpts: callOpts, + ctx: ctx, + streamerCall: func(ctx context.Context) (grpc.ClientStream, error) { + return streamer(ctx, desc, cc, method, grpcOpts...) + }, + } + return retryingStreamer, nil + } +} + +// type serverStreamingRetryingStream is the implementation of grpc.ClientStream that acts as a +// proxy to the underlying call. If any of the RecvMsg() calls fail, it will try to reestablish +// a new ClientStream according to the retry policy. +type serverStreamingRetryingStream struct { + grpc.ClientStream + client *Client + bufferedSends []interface{} // single message that the client can sen + receivedGood bool // indicates whether any prior receives were successful + wasClosedSend bool // indicates that CloseSend was closed + ctx context.Context + callOpts *options + streamerCall func(ctx context.Context) (grpc.ClientStream, error) + mu sync.RWMutex +} + +func (s *serverStreamingRetryingStream) setStream(clientStream grpc.ClientStream) { + s.mu.Lock() + s.ClientStream = clientStream + s.mu.Unlock() +} + +func (s *serverStreamingRetryingStream) getStream() grpc.ClientStream { + s.mu.RLock() + defer s.mu.RUnlock() + return s.ClientStream +} + +func (s *serverStreamingRetryingStream) SendMsg(m interface{}) error { + s.mu.Lock() + s.bufferedSends = append(s.bufferedSends, m) + s.mu.Unlock() + return s.getStream().SendMsg(m) +} + +func (s *serverStreamingRetryingStream) CloseSend() error { + s.mu.Lock() + s.wasClosedSend = true + s.mu.Unlock() + return s.getStream().CloseSend() +} + +func (s *serverStreamingRetryingStream) Header() (metadata.MD, error) { + return s.getStream().Header() +} + +func (s *serverStreamingRetryingStream) Trailer() metadata.MD { + return s.getStream().Trailer() +} + +func (s *serverStreamingRetryingStream) RecvMsg(m interface{}) error { + attemptRetry, lastErr := s.receiveMsgAndIndicateRetry(m) + if !attemptRetry { + return lastErr // success or hard failure + } + // We start off from attempt 1, because zeroth was already made on normal SendMsg(). + for attempt := uint(1); attempt < s.callOpts.max; attempt++ { + if err := waitRetryBackoff(attempt, s.ctx, s.callOpts); err != nil { + return err + } + newStream, err := s.reestablishStreamAndResendBuffer(s.ctx) + if err != nil { + // TODO(mwitkow): Maybe dial and transport errors should be retriable? + return err + } + s.setStream(newStream) + attemptRetry, lastErr = s.receiveMsgAndIndicateRetry(m) + //fmt.Printf("Received message and indicate: %v %v\n", attemptRetry, lastErr) + if !attemptRetry { + return lastErr + } + } + return lastErr +} + +func (s *serverStreamingRetryingStream) receiveMsgAndIndicateRetry(m interface{}) (bool, error) { + s.mu.RLock() + wasGood := s.receivedGood + s.mu.RUnlock() + err := s.getStream().RecvMsg(m) + if err == nil || err == io.EOF { + s.mu.Lock() + s.receivedGood = true + s.mu.Unlock() + return false, err + } else if wasGood { + // previous RecvMsg in the stream succeeded, no retry logic should interfere + return false, err + } + if isContextError(err) { + if s.ctx.Err() != nil { + return false, err + } + // its the callCtx deadline or cancellation, in which case try again. + return true, err + } + if s.callOpts.retryAuth && rpctypes.Error(err) == rpctypes.ErrInvalidAuthToken { + gterr := s.client.getToken(s.ctx) + if gterr != nil { + s.client.lg.Info("retry failed to fetch new auth token", zap.Error(gterr)) + return false, err // return the original error for simplicity + } + return true, err + + } + return isSafeRetry(s.client.lg, err, s.callOpts), err +} + +func (s *serverStreamingRetryingStream) reestablishStreamAndResendBuffer(callCtx context.Context) (grpc.ClientStream, error) { + s.mu.RLock() + bufferedSends := s.bufferedSends + s.mu.RUnlock() + newStream, err := s.streamerCall(callCtx) + if err != nil { + return nil, err + } + for _, msg := range bufferedSends { + if err := newStream.SendMsg(msg); err != nil { + return nil, err + } + } + if err := newStream.CloseSend(); err != nil { + return nil, err + } + return newStream, nil +} + +func waitRetryBackoff(attempt uint, ctx context.Context, callOpts *options) error { + var waitTime time.Duration = 0 + if attempt > 0 { + waitTime = callOpts.backoffFunc(attempt) + } + if waitTime > 0 { + timer := time.NewTimer(waitTime) + select { + case <-ctx.Done(): + timer.Stop() + return contextErrToGrpcErr(ctx.Err()) + case <-timer.C: + } + } + return nil +} + +// isSafeRetry returns "true", if request is safe for retry with the given error. +func isSafeRetry(lg *zap.Logger, err error, callOpts *options) bool { + if isContextError(err) { + return false + } + switch callOpts.retryPolicy { + case repeatable: + return isSafeRetryImmutableRPC(err) + case nonRepeatable: + return isSafeRetryMutableRPC(err) + default: + lg.Warn("unrecognized retry policy", zap.String("retryPolicy", callOpts.retryPolicy.String())) + return false + } +} + +func isContextError(err error) bool { + return grpc.Code(err) == codes.DeadlineExceeded || grpc.Code(err) == codes.Canceled +} + +func contextErrToGrpcErr(err error) error { + switch err { + case context.DeadlineExceeded: + return grpc.Errorf(codes.DeadlineExceeded, err.Error()) + case context.Canceled: + return grpc.Errorf(codes.Canceled, err.Error()) + default: + return grpc.Errorf(codes.Unknown, err.Error()) + } +} + +var ( + defaultOptions = &options{ + retryPolicy: nonRepeatable, + max: 0, // disable + backoffFunc: backoffLinearWithJitter(50*time.Millisecond /*jitter*/, 0.10), + retryAuth: true, + } +) + +// backoffFunc denotes a family of functions that control the backoff duration between call retries. +// +// They are called with an identifier of the attempt, and should return a time the system client should +// hold off for. If the time returned is longer than the `context.Context.Deadline` of the request +// the deadline of the request takes precedence and the wait will be interrupted before proceeding +// with the next iteration. +type backoffFunc func(attempt uint) time.Duration + +// withRetryPolicy sets the retry policy of this call. +func withRetryPolicy(rp retryPolicy) retryOption { + return retryOption{applyFunc: func(o *options) { + o.retryPolicy = rp + }} +} + +// withAuthRetry sets enables authentication retries. +func withAuthRetry(retryAuth bool) retryOption { + return retryOption{applyFunc: func(o *options) { + o.retryAuth = retryAuth + }} +} + +// withMax sets the maximum number of retries on this call, or this interceptor. +func withMax(maxRetries uint) retryOption { + return retryOption{applyFunc: func(o *options) { + o.max = maxRetries + }} +} + +// WithBackoff sets the `BackoffFunc `used to control time between retries. +func withBackoff(bf backoffFunc) retryOption { + return retryOption{applyFunc: func(o *options) { + o.backoffFunc = bf + }} +} + +type options struct { + retryPolicy retryPolicy + max uint + backoffFunc backoffFunc + retryAuth bool +} + +// retryOption is a grpc.CallOption that is local to clientv3's retry interceptor. +type retryOption struct { + grpc.EmptyCallOption // make sure we implement private after() and before() fields so we don't panic. + applyFunc func(opt *options) +} + +func reuseOrNewWithCallOptions(opt *options, retryOptions []retryOption) *options { + if len(retryOptions) == 0 { + return opt + } + optCopy := &options{} + *optCopy = *opt + for _, f := range retryOptions { + f.applyFunc(optCopy) + } + return optCopy +} + +func filterCallOptions(callOptions []grpc.CallOption) (grpcOptions []grpc.CallOption, retryOptions []retryOption) { + for _, opt := range callOptions { + if co, ok := opt.(retryOption); ok { + retryOptions = append(retryOptions, co) + } else { + grpcOptions = append(grpcOptions, opt) + } + } + return grpcOptions, retryOptions +} + +// BackoffLinearWithJitter waits a set period of time, allowing for jitter (fractional adjustment). +// +// For example waitBetween=1s and jitter=0.10 can generate waits between 900ms and 1100ms. +func backoffLinearWithJitter(waitBetween time.Duration, jitterFraction float64) backoffFunc { + return func(attempt uint) time.Duration { + return backoffutils.JitterUp(waitBetween, jitterFraction) + } +} diff --git a/docs/client-architecture.rst b/docs/client-architecture.rst index b7d61cf7a13..192fb5b3817 100644 --- a/docs/client-architecture.rst +++ b/docs/client-architecture.rst @@ -1,24 +1,151 @@ .. _client-architecture: -######################## + etcd Client Architecture ######################## -.. image:: img/etcd.png - :width: 100px +Introduction +============ + +etcd server has proven its robustness with years of failure injection testing. Most complex application logic is already handled by etcd server and its data stores (e.g. cluster membership is transparent to clients, with Raft-layer forwarding proposals to leader). Although server components are correct, its composition with client requires a different set of intricate protocols to guarantee its correctness and high availability under faulty conditions. Ideally, etcd server provides one logical cluster view of many physical machines, and client implements automatic failover between replicas. This documents client architectural decisions and its implementation details. + + +Glossary +======== + +*clientv3*: etcd Official Go client for etcd v3 API. + +*clientv3-grpc1.0*: Official client implementation, with `grpc-go v1.0.x `_, which is used in latest etcd v3.1. + +*clientv3-grpc1.7*: Official client implementation, with `grpc-go v1.7.x `_, which is used in latest etcd v3.2 and v3.3. + +*clientv3-grpc1.12*: Official client implementation, with `grpc-go v1.12.x `_, which is used in latest etcd v3.4. + +*Balancer*: etcd client load balancer that implements retry and failover mechanism. etcd client should automatically balance loads between multiple endpoints. + +*Endpoints*: A list of etcd server endpoints that clients can connect to. Typically, 3 or 5 client URLs of an etcd cluster. + +*Pinned endpoint*: When configured with multiple endpoints, <= v3.3 client balancer chooses only one endpoint to establish a TCP connection, in order to conserve total open connections to etcd cluster. In v3.4, balancer round-robins pinned endpoints for every request, thus distributing loads more evenly. + +*Client Connection*: TCP connection that has been established to an etcd server, via gRPC Dial. + +*Sub Connection*: gRPC SubConn interface. Each sub-connection contains a list of addresses. Balancer creates a SubConn from a list of resolved addresses. gRPC ClientConn can map to multiple SubConn (e.g. example.com resolves to ``10.10.10.1`` and ``10.10.10.2`` of two sub-connections). etcd v3.4 balancer employs internal resolver to establish one sub-connection for each endpoint. + +*Transient disconnect*: When gRPC server returns a status error of `code Unavailable `_. + + +Client Requirements +=================== + +*Correctness*. Requests may fail in the presence of server faults. However, it never violates consistency guarantees: global ordering properties, never write corrupted data, at-most once semantics for mutable operations, watch never observes partial events, and so on. + +*Liveness*. Servers may fail or disconnect briefly. Clients should make progress in either way. Clients should `never deadlock `_ waiting for a server to come back from offline, unless configured to do so. Ideally, clients detect unavailable servers with HTTP/2 ping and failover to other nodes with clear error messages. + +*Effectiveness*. Clients should operate effectively with minimum resources: previous TCP connections should be `gracefully closed `_ after endpoint switch. Failover mechanism should effectively predict the next replica to connect, without wastefully retrying on failed nodes. + +*Portability*. Official client should be clearly documented and its implementation be applicable to other language bindings. Error handling between different language bindings should be consistent. Since etcd is fully committed to gRPC, implementation should be closely aligned with gRPC long-term design goals (e.g. pluggable retry policy should be compatible with `gRPC retry `_). Upgrades between two client versions should be non-disruptive. + + +Client Overview +=============== + +etcd client implements the following components: + +* balancer that establishes gRPC connections to an etcd cluster, +* API client that sends RPCs to an etcd server, and +* error handler that decides whether to retry a failed request or switch endpoints. + +Languages may differ in how to establish an initial connection (e.g. configure TLS), how to encode and send Protocol Buffer messages to server, how to handle stream RPCs, and so on. However, errors returned from etcd server will be the same. So should be error handling and retry policy. + +For example, etcd server may return ``"rpc error: code = Unavailable desc = etcdserver: request timed out"``, which is transient error that expects retries. Or return `rpc error: code = InvalidArgument desc = etcdserver: key is not provided`, which means request was invalid and should not be retried. Go client can parse errors with ``google.golang.org/grpc/status.FromError``, and Java client with ``io.grpc.Status.fromThrowable``. + + +clientv3-grpc1.0: Balancer Overview +----------------------------------- + +``clientv3-grpc1.0`` maintains multiple TCP connections when configured with multiple etcd endpoints. Then pick one address and use it to send all client requests. The pinned address is maintained until the client object is closed (see *Figure 1*). When the client receives an error, it randomly picks another and retries. + +.. image:: img/client-architecture-balancer-figure-01.png + :align: center + :alt: client-architecture-balancer-figure-01 + + +clientv3-grpc1.0: Balancer Limitation +------------------------------------- + +``clientv3-grpc1.0`` opening multiple TCP connections may provide faster balancer failover but requires more resources. The balancer does not understand node’s health status or cluster membership. So, it is possible that balancer gets stuck with one failed or partitioned node. + + +clientv3-grpc1.7: Balancer Overview +------------------------------------ + +``clientv3-grpc1.7`` maintains only one TCP connection to a chosen etcd server. When given multiple cluster endpoints, a client first tries to connect to them all. As soon as one connection is up, balancer pins the address, closing others (see *Figure 2*). The pinned address is to be maintained until the client object is closed. An error, from server or client network fault, is sent to client error handler (see *Figure 3*). + +.. image:: img/client-architecture-balancer-figure-02.png + :align: center + :alt: client-architecture-balancer-figure-02 + +.. image:: img/client-architecture-balancer-figure-03.png + :align: center + :alt: client-architecture-balancer-figure-03 + +The client error handler takes an error from gRPC server, and decides whether to retry on the same endpoint, or to switch to other addresses, based on the error code and message (see *Figure 4* and *Figure 5*). + +.. image:: img/client-architecture-balancer-figure-04.png + :align: center + :alt: client-architecture-balancer-figure-04 + +.. image:: img/client-architecture-balancer-figure-05.png + :align: center + :alt: client-architecture-balancer-figure-05 + +Stream RPCs, such as Watch and KeepAlive, are often requested with no timeouts. Instead, client can send periodic HTTP/2 pings to check the status of a pinned endpoint; if the server does not respond to the ping, balancer switches to other endpoints (see *Figure 6*). + +.. image:: img/client-architecture-balancer-figure-06.png + :align: center + :alt: client-architecture-balancer-figure-06 + + +clientv3-grpc1.7: Balancer Limitation +------------------------------------- + +``clientv3-grpc1.7`` balancer sends HTTP/2 keepalives to detect disconnects from streaming requests. It is a simple gRPC server ping mechanism and does not reason about cluster membership, thus unable to detect network partitions. Since partitioned gRPC server can still respond to client pings, balancer may get stuck with a partitioned node. Ideally, keepalive ping detects partition and triggers endpoint switch, before request time-out (see `issue#8673 `_ and *Figure 7*). + +.. image:: img/client-architecture-balancer-figure-07.png + :align: center + :alt: client-architecture-balancer-figure-07 + +``clientv3-grpc1.7`` balancer maintains a list of unhealthy endpoints. Disconnected addresses are added to “unhealthy” list, and considered unavailable until after wait duration, which is hard coded as dial timeout with default value 5-second. Balancer can have false positives on which endpoints are unhealthy. For instance, endpoint A may come back right after being blacklisted, but still unusable for next 5 seconds (see *Figure 8*). + +``clientv3-grpc1.0`` suffered the same problems above. + +.. image:: img/client-architecture-balancer-figure-08.png :align: center - :height: 100px + :alt: client-architecture-balancer-figure-08 + +Upstream gRPC Go had already migrated to new balancer interface. For example, ``clientv3-grpc1.7`` underlying balancer implementation uses new gRPC balancer and tries to be consistent with old balancer behaviors. While its compatibility has been maintained reasonably well, etcd client still `suffered from subtle breaking changes `_. Furthermore, gRPC maintainer recommends to `not rely on the old balancer interface `_. In general, to get better support from upstream, it is best to be in sync with latest gRPC releases. And new features, such as retry policy, may not be backported to gRPC 1.7 branch. Thus, both etcd server and client must migrate to latest gRPC versions. + +clientv3-grpc1.12: Balancer Overview +------------------------------------ -What is etcd Client? -==================== +``clientv3-grpc1.7`` is so tightly coupled with old gRPC interface, that every single gRPC dependency upgrade broke client behavior. Majority of development and debugging efforts were devoted to fixing those client behavior changes. As a result, its implementation has become overly complicated with bad assumptions on server connectivities. + +The primary goal of ``clientv3-grpc1.12`` is to simplify balancer failover logic; rather than maintaining a list of unhealthy endpoints, which may be stale, simply roundrobin to the next endpoint whenever client gets disconnected from the current endpoint. It does not assume endpoint status. Thus, no more complicated status tracking is needed (see *Figure 8* and above). Upgrading to ``clientv3-grpc1.12`` should be no issue; all changes were internal while keeping all the backward compatibilities. + +Internally, when given multiple endpoints, ``clientv3-grpc1.12`` creates multiple sub-connections (one sub-connection per each endpoint), while ``clientv3-grpc1.7`` creates only one connection to a pinned endpoint (see *Figure 9*). For instance, in 5-node cluster, ``clientv3-grpc1.12`` balancer would require 5 TCP connections, while ``clientv3-grpc1.7`` only requires one. By preserving the pool of TCP connections, ``clientv3-grpc1.12`` may consume more resources but provide more flexible load balancer with better failover performance. The default balancing policy is round robin but can be easily extended to support other types of balancers (e.g. power of two, pick leader, etc.). ``clientv3-grpc1.12`` uses gRPC resolver group and implements balancer picker policy, in order to delegate complex balancing work to upstream gRPC. On the other hand, ``clientv3-grpc1.7`` manually handles each gRPC connection and balancer failover, which complicates the implementation. ``clientv3-grpc1.12`` implements retry in the gRPC interceptor chain that automatically handles gRPC internal errors and enables more advanced retry policies like backoff, while ``clientv3-grpc1.7`` manually interprets gRPC errors for retries. + +.. image:: img/client-architecture-balancer-figure-09.png + :align: center + :alt: client-architecture-balancer-figure-09 -Hello World! +clientv3-grpc1.12: Balancer Limitation +-------------------------------------- -etcd Client Use Case? -===================== +Improvements can be made by caching the status of each endpoint. For instance, balancer can ping each server in advance to maintain a list of healthy candidates, and use this information when doing round-robin. Or when disconnected, balancer can prioritize healthy endpoints. This may complicate the balancer implementation, thus can be addressed in later versions. -Hello World! +Client-side keepalive ping still does not reason about network partitions. Streaming request may get stuck with a partitioned node. Advanced health checking service need to be implemented to understand the cluster membership (see `issue#8673 `_ for more detail). +Currently, retry logic is handled manually as an interceptor. This may be simplified via `official gRPC retries `_. diff --git a/docs/img/client-architecture-balancer-figure-01.png b/docs/img/client-architecture-balancer-figure-01.png new file mode 100644 index 00000000000..b11e325a430 Binary files /dev/null and b/docs/img/client-architecture-balancer-figure-01.png differ diff --git a/docs/img/client-architecture-balancer-figure-02.png b/docs/img/client-architecture-balancer-figure-02.png new file mode 100644 index 00000000000..814a224e504 Binary files /dev/null and b/docs/img/client-architecture-balancer-figure-02.png differ diff --git a/docs/img/client-architecture-balancer-figure-03.png b/docs/img/client-architecture-balancer-figure-03.png new file mode 100644 index 00000000000..4573a4db24e Binary files /dev/null and b/docs/img/client-architecture-balancer-figure-03.png differ diff --git a/docs/img/client-architecture-balancer-figure-04.png b/docs/img/client-architecture-balancer-figure-04.png new file mode 100644 index 00000000000..2a987b2db97 Binary files /dev/null and b/docs/img/client-architecture-balancer-figure-04.png differ diff --git a/docs/img/client-architecture-balancer-figure-05.png b/docs/img/client-architecture-balancer-figure-05.png new file mode 100644 index 00000000000..93484c7bd1b Binary files /dev/null and b/docs/img/client-architecture-balancer-figure-05.png differ diff --git a/docs/img/client-architecture-balancer-figure-06.png b/docs/img/client-architecture-balancer-figure-06.png new file mode 100644 index 00000000000..1e387223889 Binary files /dev/null and b/docs/img/client-architecture-balancer-figure-06.png differ diff --git a/docs/img/client-architecture-balancer-figure-07.png b/docs/img/client-architecture-balancer-figure-07.png new file mode 100644 index 00000000000..b699772dd34 Binary files /dev/null and b/docs/img/client-architecture-balancer-figure-07.png differ diff --git a/docs/img/client-architecture-balancer-figure-08.png b/docs/img/client-architecture-balancer-figure-08.png new file mode 100644 index 00000000000..f0fe778309f Binary files /dev/null and b/docs/img/client-architecture-balancer-figure-08.png differ diff --git a/docs/img/client-architecture-balancer-figure-09.png b/docs/img/client-architecture-balancer-figure-09.png new file mode 100644 index 00000000000..baa092e3bac Binary files /dev/null and b/docs/img/client-architecture-balancer-figure-09.png differ diff --git a/docs/index.rst b/docs/index.rst index 4cd4ff0b4b6..2ed39e15f28 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -1,8 +1,8 @@ -Welcome to etcd's documentation! -================================ +etcd Documentation +================== -* :ref:`client-architecture`: etcd Architecture. +* :ref:`client-architecture`: Describes etcd client components. .. toctree:: :maxdepth: 2 diff --git a/integration/cluster.go b/integration/cluster.go index a1840f73409..e3924e64d11 100644 --- a/integration/cluster.go +++ b/integration/cluster.go @@ -536,6 +536,7 @@ type member struct { PeerTLSInfo *transport.TLSInfo // ClientTLSInfo enables client TLS when set ClientTLSInfo *transport.TLSInfo + DialOptions []grpc.DialOption raftHandler *testutil.PauseableHandler s *etcdserver.EtcdServer @@ -733,6 +734,7 @@ func NewClientV3(m *member) (*clientv3.Client, error) { cfg := clientv3.Config{ Endpoints: []string{m.grpcAddr}, DialTimeout: 5 * time.Second, + DialOptions: []grpc.DialOption{grpc.WithBlock()}, MaxCallSendMsgSize: m.clientMaxCallSendMsgSize, MaxCallRecvMsgSize: m.clientMaxCallRecvMsgSize, } @@ -744,6 +746,9 @@ func NewClientV3(m *member) (*clientv3.Client, error) { } cfg.TLS = tls } + if m.DialOptions != nil { + cfg.DialOptions = append(cfg.DialOptions, m.DialOptions...) + } return newClientV3(cfg) } @@ -950,6 +955,13 @@ func (m *member) Launch() error { } func (m *member) WaitOK(t *testing.T) { + m.WaitStarted(t) + for m.s.Leader() == 0 { + time.Sleep(tickDuration) + } +} + +func (m *member) WaitStarted(t *testing.T) { cc := MustNewHTTPClient(t, []string{m.URL()}, m.ClientTLSInfo) kapi := client.NewKeysAPI(cc) for { @@ -962,9 +974,23 @@ func (m *member) WaitOK(t *testing.T) { cancel() break } - for m.s.Leader() == 0 { +} + +func WaitClientV3(t *testing.T, kv clientv3.KV) { + timeout := time.Now().Add(requestTimeout) + var err error + for time.Now().Before(timeout) { + ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) + _, err = kv.Get(ctx, "/") + cancel() + if err == nil { + return + } time.Sleep(tickDuration) } + if err != nil { + t.Fatalf("timed out waiting for client: %v", err) + } } func (m *member) URL() string { return m.ClientURLs[0].String() } diff --git a/integration/v3_alarm_test.go b/integration/v3_alarm_test.go index 01af6440689..e66cf0bab16 100644 --- a/integration/v3_alarm_test.go +++ b/integration/v3_alarm_test.go @@ -88,13 +88,16 @@ func TestV3StorageQuotaApply(t *testing.T) { } } + ctx, close := context.WithTimeout(context.TODO(), RequestWaitTimeout) + defer close() + // small quota machine should reject put - if _, err := kvc0.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err == nil { + if _, err := kvc0.Put(ctx, &pb.PutRequest{Key: key, Value: smallbuf}); err == nil { t.Fatalf("past-quota instance should reject put") } // large quota machine should reject put - if _, err := kvc1.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err == nil { + if _, err := kvc1.Put(ctx, &pb.PutRequest{Key: key, Value: smallbuf}); err == nil { t.Fatalf("past-quota instance should reject put") } @@ -175,12 +178,20 @@ func TestV3CorruptAlarm(t *testing.T) { s.Close() be.Close() + clus.Members[1].WaitOK(t) + clus.Members[2].WaitOK(t) + time.Sleep(time.Second * 2) + // Wait for cluster so Puts succeed in case member 0 was the leader. if _, err := clus.Client(1).Get(context.TODO(), "k"); err != nil { t.Fatal(err) } - clus.Client(1).Put(context.TODO(), "xyz", "321") - clus.Client(1).Put(context.TODO(), "abc", "fed") + if _, err := clus.Client(1).Put(context.TODO(), "xyz", "321"); err != nil { + t.Fatal(err) + } + if _, err := clus.Client(1).Put(context.TODO(), "abc", "fed"); err != nil { + t.Fatal(err) + } // Restart with corruption checking enabled. clus.Members[1].Stop(t) @@ -189,12 +200,15 @@ func TestV3CorruptAlarm(t *testing.T) { m.CorruptCheckTime = time.Second m.Restart(t) } - // Member 0 restarts into split brain. + clus.WaitLeader(t) + time.Sleep(time.Second * 2) + clus.Members[0].WaitStarted(t) resp0, err0 := clus.Client(0).Get(context.TODO(), "abc") if err0 != nil { t.Fatal(err0) } + clus.Members[1].WaitStarted(t) resp1, err1 := clus.Client(1).Get(context.TODO(), "abc") if err1 != nil { t.Fatal(err1) diff --git a/integration/v3_grpc_inflight_test.go b/integration/v3_grpc_inflight_test.go index dd0d180cc0d..08c1a1f2369 100644 --- a/integration/v3_grpc_inflight_test.go +++ b/integration/v3_grpc_inflight_test.go @@ -25,6 +25,7 @@ import ( "github.com/coreos/etcd/pkg/testutil" "google.golang.org/grpc" + "google.golang.org/grpc/transport" ) // TestV3MaintenanceDefragmentInflightRange ensures inflight range requests @@ -81,8 +82,9 @@ func TestV3KVInflightRangeRequests(t *testing.T) { defer wg.Done() _, err := kvc.Range(ctx, &pb.RangeRequest{Key: []byte("foo"), Serializable: true}, grpc.FailFast(false)) if err != nil { - if err != nil && rpctypes.ErrorDesc(err) != context.Canceled.Error() { - t.Fatalf("inflight request should be canceld with %v, got %v", context.Canceled, err) + errDesc := rpctypes.ErrorDesc(err) + if err != nil && !(errDesc == context.Canceled.Error() || errDesc == transport.ErrConnClosing.Desc) { + t.Fatalf("inflight request should be canceled with '%v' or '%v', got '%v'", context.Canceled.Error(), transport.ErrConnClosing.Desc, errDesc) } } }() diff --git a/integration/v3_grpc_test.go b/integration/v3_grpc_test.go index c936703d5e3..b193f661f3d 100644 --- a/integration/v3_grpc_test.go +++ b/integration/v3_grpc_test.go @@ -32,7 +32,9 @@ import ( "github.com/coreos/etcd/pkg/transport" "google.golang.org/grpc" + "google.golang.org/grpc/codes" "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" ) // TestV3PutOverwrite puts a key with the v3 api to a random cluster member, @@ -1570,6 +1572,7 @@ func TestTLSGRPCRejectSecureClient(t *testing.T) { defer clus.Terminate(t) clus.Members[0].ClientTLSInfo = &testTLSInfo + clus.Members[0].DialOptions = []grpc.DialOption{grpc.WithBlock()} client, err := NewClientV3(clus.Members[0]) if client != nil || err == nil { t.Fatalf("expected no client") @@ -1752,6 +1755,7 @@ func testTLSReload( continue } cli, cerr := clientv3.New(clientv3.Config{ + DialOptions: []grpc.DialOption{grpc.WithBlock()}, Endpoints: []string{clus.Members[0].GRPCAddr()}, DialTimeout: time.Second, TLS: cc, @@ -1932,7 +1936,18 @@ func eqErrGRPC(err1 error, err2 error) bool { // FailFast=false works with Put. func waitForRestart(t *testing.T, kvc pb.KVClient) { req := &pb.RangeRequest{Key: []byte("_"), Serializable: true} - if _, err := kvc.Range(context.TODO(), req, grpc.FailFast(false)); err != nil { - t.Fatal(err) + // TODO: Remove retry loop once the new grpc load balancer provides retry. + var err error + for i := 0; i < 10; i++ { + if _, err = kvc.Range(context.TODO(), req, grpc.FailFast(false)); err != nil { + if status, ok := status.FromError(err); ok && status.Code() == codes.Unavailable { + time.Sleep(time.Millisecond * 250) + } else { + t.Fatal(err) + } + } + } + if err != nil { + t.Fatalf("timed out waiting for restart: %v", err) } } diff --git a/pkg/mock/mockserver/doc.go b/pkg/mock/mockserver/doc.go new file mode 100644 index 00000000000..030b3b2ffb7 --- /dev/null +++ b/pkg/mock/mockserver/doc.go @@ -0,0 +1,16 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package mockserver provides mock implementations for etcdserver's server interface. +package mockserver diff --git a/pkg/mock/mockserver/mockserver.go b/pkg/mock/mockserver/mockserver.go new file mode 100644 index 00000000000..d72b40b45e7 --- /dev/null +++ b/pkg/mock/mockserver/mockserver.go @@ -0,0 +1,188 @@ +// Copyright 2018 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package mockserver + +import ( + "context" + "fmt" + "io/ioutil" + "net" + "os" + "sync" + + pb "github.com/coreos/etcd/etcdserver/etcdserverpb" + + "google.golang.org/grpc" + "google.golang.org/grpc/resolver" +) + +// MockServer provides a mocked out grpc server of the etcdserver interface. +type MockServer struct { + ln net.Listener + Network string + Address string + GrpcServer *grpc.Server +} + +func (ms *MockServer) ResolverAddress() resolver.Address { + switch ms.Network { + case "unix": + return resolver.Address{Addr: fmt.Sprintf("unix://%s", ms.Address)} + case "tcp": + return resolver.Address{Addr: ms.Address} + default: + panic("illegal network type: " + ms.Network) + } +} + +// MockServers provides a cluster of mocket out gprc servers of the etcdserver interface. +type MockServers struct { + mu sync.RWMutex + Servers []*MockServer + wg sync.WaitGroup +} + +// StartMockServers creates the desired count of mock servers +// and starts them. +func StartMockServers(count int) (ms *MockServers, err error) { + return StartMockServersOnNetwork(count, "tcp") +} + +// StartMockServersOnNetwork creates mock servers on either 'tcp' or 'unix' sockets. +func StartMockServersOnNetwork(count int, network string) (ms *MockServers, err error) { + switch network { + case "tcp": + return startMockServersTcp(count) + case "unix": + return startMockServersUnix(count) + default: + return nil, fmt.Errorf("unsupported network type: %s", network) + } +} + +func startMockServersTcp(count int) (ms *MockServers, err error) { + addrs := make([]string, 0, count) + for i := 0; i < count; i++ { + addrs = append(addrs, "localhost:0") + } + return startMockServers("tcp", addrs) +} + +func startMockServersUnix(count int) (ms *MockServers, err error) { + dir := os.TempDir() + addrs := make([]string, 0, count) + for i := 0; i < count; i++ { + f, err := ioutil.TempFile(dir, "etcd-unix-so-") + if err != nil { + return nil, fmt.Errorf("failed to allocate temp file for unix socket: %v", err) + } + fn := f.Name() + err = os.Remove(fn) + if err != nil { + return nil, fmt.Errorf("failed to remove temp file before creating unix socket: %v", err) + } + addrs = append(addrs, fn) + } + return startMockServers("unix", addrs) +} + +func startMockServers(network string, addrs []string) (ms *MockServers, err error) { + ms = &MockServers{ + Servers: make([]*MockServer, len(addrs)), + wg: sync.WaitGroup{}, + } + defer func() { + if err != nil { + ms.Stop() + } + }() + for idx, addr := range addrs { + ln, err := net.Listen(network, addr) + if err != nil { + return nil, fmt.Errorf("failed to listen %v", err) + } + ms.Servers[idx] = &MockServer{ln: ln, Network: network, Address: ln.Addr().String()} + ms.StartAt(idx) + } + return ms, nil +} + +// StartAt restarts mock server at given index. +func (ms *MockServers) StartAt(idx int) (err error) { + ms.mu.Lock() + defer ms.mu.Unlock() + + if ms.Servers[idx].ln == nil { + ms.Servers[idx].ln, err = net.Listen(ms.Servers[idx].Network, ms.Servers[idx].Address) + if err != nil { + return fmt.Errorf("failed to listen %v", err) + } + } + + svr := grpc.NewServer() + pb.RegisterKVServer(svr, &mockKVServer{}) + ms.Servers[idx].GrpcServer = svr + + ms.wg.Add(1) + go func(svr *grpc.Server, l net.Listener) { + svr.Serve(l) + }(ms.Servers[idx].GrpcServer, ms.Servers[idx].ln) + return nil +} + +// StopAt stops mock server at given index. +func (ms *MockServers) StopAt(idx int) { + ms.mu.Lock() + defer ms.mu.Unlock() + + if ms.Servers[idx].ln == nil { + return + } + + ms.Servers[idx].GrpcServer.Stop() + ms.Servers[idx].GrpcServer = nil + ms.Servers[idx].ln = nil + ms.wg.Done() +} + +// Stop stops the mock server, immediately closing all open connections and listeners. +func (ms *MockServers) Stop() { + for idx := range ms.Servers { + ms.StopAt(idx) + } + ms.wg.Wait() +} + +type mockKVServer struct{} + +func (m *mockKVServer) Range(context.Context, *pb.RangeRequest) (*pb.RangeResponse, error) { + return &pb.RangeResponse{}, nil +} + +func (m *mockKVServer) Put(context.Context, *pb.PutRequest) (*pb.PutResponse, error) { + return &pb.PutResponse{}, nil +} + +func (m *mockKVServer) DeleteRange(context.Context, *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) { + return &pb.DeleteRangeResponse{}, nil +} + +func (m *mockKVServer) Txn(context.Context, *pb.TxnRequest) (*pb.TxnResponse, error) { + return &pb.TxnResponse{}, nil +} + +func (m *mockKVServer) Compact(context.Context, *pb.CompactionRequest) (*pb.CompactionResponse, error) { + return &pb.CompactionResponse{}, nil +} diff --git a/proxy/grpcproxy/leader.go b/proxy/grpcproxy/leader.go index 042c949b708..2e01b466f20 100644 --- a/proxy/grpcproxy/leader.go +++ b/proxy/grpcproxy/leader.go @@ -20,10 +20,8 @@ import ( "sync" "github.com/coreos/etcd/clientv3" - "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" "golang.org/x/time/rate" - "google.golang.org/grpc" ) const ( @@ -69,7 +67,7 @@ func (l *leader) recvLoop() { } if cresp.Err() != nil { l.loseLeader() - if rpctypes.ErrorDesc(cresp.Err()) == grpc.ErrClientConnClosing.Error() { + if clientv3.IsConnCanceled(cresp.Err()) { close(l.disconnc) return } diff --git a/test b/test index f797c872dcc..61f8fda6553 100755 --- a/test +++ b/test @@ -380,7 +380,7 @@ function shellcheck_pass { function markdown_you_pass { # eschew you - yous=$(find . -name \*.md ! -path './vendor/*' ! -path './gopath.proto/*' -exec grep -E --color "[Yy]ou[r]?[ '.,;]" {} + | grep -v /v2/ || true) + yous=$(find . -name \*.md ! -path './vendor/*' ! -path './Documentation/v2/*' ! -path './gopath.proto/*' -exec grep -E --color "[Yy]ou[r]?[ '.,;]" {} + || true) if [ ! -z "$yous" ]; then echo -e "found 'you' in documentation:\\n${yous}" exit 255 diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/LICENSE b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/LICENSE new file mode 100644 index 00000000000..b2b065037fc --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. \ No newline at end of file diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/chain.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/chain.go new file mode 100644 index 00000000000..45a2f5f49a7 --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/chain.go @@ -0,0 +1,183 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +// gRPC Server Interceptor chaining middleware. + +package grpc_middleware + +import ( + "golang.org/x/net/context" + "google.golang.org/grpc" +) + +// ChainUnaryServer creates a single interceptor out of a chain of many interceptors. +// +// Execution is done in left-to-right order, including passing of context. +// For example ChainUnaryServer(one, two, three) will execute one before two before three, and three +// will see context changes of one and two. +func ChainUnaryServer(interceptors ...grpc.UnaryServerInterceptor) grpc.UnaryServerInterceptor { + n := len(interceptors) + + if n > 1 { + lastI := n - 1 + return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { + var ( + chainHandler grpc.UnaryHandler + curI int + ) + + chainHandler = func(currentCtx context.Context, currentReq interface{}) (interface{}, error) { + if curI == lastI { + return handler(currentCtx, currentReq) + } + curI++ + resp, err := interceptors[curI](currentCtx, currentReq, info, chainHandler) + curI-- + return resp, err + } + + return interceptors[0](ctx, req, info, chainHandler) + } + } + + if n == 1 { + return interceptors[0] + } + + // n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil. + return func(ctx context.Context, req interface{}, _ *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { + return handler(ctx, req) + } +} + +// ChainStreamServer creates a single interceptor out of a chain of many interceptors. +// +// Execution is done in left-to-right order, including passing of context. +// For example ChainUnaryServer(one, two, three) will execute one before two before three. +// If you want to pass context between interceptors, use WrapServerStream. +func ChainStreamServer(interceptors ...grpc.StreamServerInterceptor) grpc.StreamServerInterceptor { + n := len(interceptors) + + if n > 1 { + lastI := n - 1 + return func(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error { + var ( + chainHandler grpc.StreamHandler + curI int + ) + + chainHandler = func(currentSrv interface{}, currentStream grpc.ServerStream) error { + if curI == lastI { + return handler(currentSrv, currentStream) + } + curI++ + err := interceptors[curI](currentSrv, currentStream, info, chainHandler) + curI-- + return err + } + + return interceptors[0](srv, stream, info, chainHandler) + } + } + + if n == 1 { + return interceptors[0] + } + + // n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil. + return func(srv interface{}, stream grpc.ServerStream, _ *grpc.StreamServerInfo, handler grpc.StreamHandler) error { + return handler(srv, stream) + } +} + +// ChainUnaryClient creates a single interceptor out of a chain of many interceptors. +// +// Execution is done in left-to-right order, including passing of context. +// For example ChainUnaryClient(one, two, three) will execute one before two before three. +func ChainUnaryClient(interceptors ...grpc.UnaryClientInterceptor) grpc.UnaryClientInterceptor { + n := len(interceptors) + + if n > 1 { + lastI := n - 1 + return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + var ( + chainHandler grpc.UnaryInvoker + curI int + ) + + chainHandler = func(currentCtx context.Context, currentMethod string, currentReq, currentRepl interface{}, currentConn *grpc.ClientConn, currentOpts ...grpc.CallOption) error { + if curI == lastI { + return invoker(currentCtx, currentMethod, currentReq, currentRepl, currentConn, currentOpts...) + } + curI++ + err := interceptors[curI](currentCtx, currentMethod, currentReq, currentRepl, currentConn, chainHandler, currentOpts...) + curI-- + return err + } + + return interceptors[0](ctx, method, req, reply, cc, chainHandler, opts...) + } + } + + if n == 1 { + return interceptors[0] + } + + // n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil. + return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + return invoker(ctx, method, req, reply, cc, opts...) + } +} + +// ChainStreamClient creates a single interceptor out of a chain of many interceptors. +// +// Execution is done in left-to-right order, including passing of context. +// For example ChainStreamClient(one, two, three) will execute one before two before three. +func ChainStreamClient(interceptors ...grpc.StreamClientInterceptor) grpc.StreamClientInterceptor { + n := len(interceptors) + + if n > 1 { + lastI := n - 1 + return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) { + var ( + chainHandler grpc.Streamer + curI int + ) + + chainHandler = func(currentCtx context.Context, currentDesc *grpc.StreamDesc, currentConn *grpc.ClientConn, currentMethod string, currentOpts ...grpc.CallOption) (grpc.ClientStream, error) { + if curI == lastI { + return streamer(currentCtx, currentDesc, currentConn, currentMethod, currentOpts...) + } + curI++ + stream, err := interceptors[curI](currentCtx, currentDesc, currentConn, currentMethod, chainHandler, currentOpts...) + curI-- + return stream, err + } + + return interceptors[0](ctx, desc, cc, method, chainHandler, opts...) + } + } + + if n == 1 { + return interceptors[0] + } + + // n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil. + return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) { + return streamer(ctx, desc, cc, method, opts...) + } +} + +// Chain creates a single interceptor out of a chain of many interceptors. +// +// WithUnaryServerChain is a grpc.Server config option that accepts multiple unary interceptors. +// Basically syntactic sugar. +func WithUnaryServerChain(interceptors ...grpc.UnaryServerInterceptor) grpc.ServerOption { + return grpc.UnaryInterceptor(ChainUnaryServer(interceptors...)) +} + +// WithStreamServerChain is a grpc.Server config option that accepts multiple stream interceptors. +// Basically syntactic sugar. +func WithStreamServerChain(interceptors ...grpc.StreamServerInterceptor) grpc.ServerOption { + return grpc.StreamInterceptor(ChainStreamServer(interceptors...)) +} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/doc.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/doc.go new file mode 100644 index 00000000000..71689503642 --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/doc.go @@ -0,0 +1,69 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +/* +`grpc_middleware` is a collection of gRPC middleware packages: interceptors, helpers and tools. + +Middleware + +gRPC is a fantastic RPC middleware, which sees a lot of adoption in the Golang world. However, the +upstream gRPC codebase is relatively bare bones. + +This package, and most of its child packages provides commonly needed middleware for gRPC: +client-side interceptors for retires, server-side interceptors for input validation and auth, +functions for chaining said interceptors, metadata convenience methods and more. + +Chaining + +By default, gRPC doesn't allow one to have more than one interceptor either on the client nor on +the server side. `grpc_middleware` provides convenient chaining methods + +Simple way of turning a multiple interceptors into a single interceptor. Here's an example for +server chaining: + + myServer := grpc.NewServer( + grpc.StreamInterceptor(grpc_middleware.ChainStreamServer(loggingStream, monitoringStream, authStream)), + grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(loggingUnary, monitoringUnary, authUnary), + ) + +These interceptors will be executed from left to right: logging, monitoring and auth. + +Here's an example for client side chaining: + + clientConn, err = grpc.Dial( + address, + grpc.WithUnaryInterceptor(grpc_middleware.ChainUnaryClient(monitoringClientUnary, retryUnary)), + grpc.WithStreamInterceptor(grpc_middleware.ChainStreamClient(monitoringClientStream, retryStream)), + ) + client = pb_testproto.NewTestServiceClient(clientConn) + resp, err := client.PingEmpty(s.ctx, &myservice.Request{Msg: "hello"}) + +These interceptors will be executed from left to right: monitoring and then retry logic. + +The retry interceptor will call every interceptor that follows it whenever when a retry happens. + +Writing Your Own + +Implementing your own interceptor is pretty trivial: there are interfaces for that. But the interesting +bit exposing common data to handlers (and other middleware), similarly to HTTP Middleware design. +For example, you may want to pass the identity of the caller from the auth interceptor all the way +to the handling function. + +For example, a client side interceptor example for auth looks like: + + func FakeAuthUnaryInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { + newCtx := context.WithValue(ctx, "user_id", "john@example.com") + return handler(newCtx, req) + } + +Unfortunately, it's not as easy for streaming RPCs. These have the `context.Context` embedded within +the `grpc.ServerStream` object. To pass values through context, a wrapper (`WrappedServerStream`) is +needed. For example: + + func FakeAuthStreamingInterceptor(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error { + newStream := grpc_middleware.WrapServerStream(stream) + newStream.WrappedContext = context.WithValue(ctx, "user_id", "john@example.com") + return handler(srv, stream) + } +*/ +package grpc_middleware diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils/backoff.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils/backoff.go new file mode 100644 index 00000000000..10aa7f96c6c --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils/backoff.go @@ -0,0 +1,23 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +/* +Backoff Helper Utilities + +Implements common backoff features. +*/ +package backoffutils + +import ( + "math/rand" + "time" +) + +// JitterUp adds random jitter to the duration. +// +// This adds or substracts time from the duration within a given jitter fraction. +// For example for 10s and jitter 0.1, it will returna time within [9s, 11s]) +func JitterUp(duration time.Duration, jitter float64) time.Duration { + multiplier := jitter * (rand.Float64()*2 - 1) + return time.Duration(float64(duration) * (1 + multiplier)) +} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/wrappers.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/wrappers.go new file mode 100644 index 00000000000..597b862445f --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/wrappers.go @@ -0,0 +1,29 @@ +// Copyright 2016 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +package grpc_middleware + +import ( + "golang.org/x/net/context" + "google.golang.org/grpc" +) + +// WrappedServerStream is a thin wrapper around grpc.ServerStream that allows modifying context. +type WrappedServerStream struct { + grpc.ServerStream + // WrappedContext is the wrapper's own Context. You can assign it. + WrappedContext context.Context +} + +// Context returns the wrapper's WrappedContext, overwriting the nested grpc.ServerStream.Context() +func (w *WrappedServerStream) Context() context.Context { + return w.WrappedContext +} + +// WrapServerStream returns a ServerStream that has the ability to overwrite context. +func WrapServerStream(stream grpc.ServerStream) *WrappedServerStream { + if existing, ok := stream.(*WrappedServerStream); ok { + return existing + } + return &WrappedServerStream{ServerStream: stream, WrappedContext: stream.Context()} +} diff --git a/vendor/google.golang.org/grpc/backoff.go b/vendor/google.golang.org/grpc/backoff.go index 090fbe87c52..c40facce510 100644 --- a/vendor/google.golang.org/grpc/backoff.go +++ b/vendor/google.golang.org/grpc/backoff.go @@ -25,14 +25,12 @@ import ( // DefaultBackoffConfig uses values specified for backoff in // https://github.com/grpc/grpc/blob/master/doc/connection-backoff.md. -var ( - DefaultBackoffConfig = BackoffConfig{ - MaxDelay: 120 * time.Second, - baseDelay: 1.0 * time.Second, - factor: 1.6, - jitter: 0.2, - } -) +var DefaultBackoffConfig = BackoffConfig{ + MaxDelay: 120 * time.Second, + baseDelay: 1.0 * time.Second, + factor: 1.6, + jitter: 0.2, +} // backoffStrategy defines the methodology for backing off after a grpc // connection failure. diff --git a/vendor/google.golang.org/grpc/balancer.go b/vendor/google.golang.org/grpc/balancer.go index ab65049ddc1..e1730166cde 100644 --- a/vendor/google.golang.org/grpc/balancer.go +++ b/vendor/google.golang.org/grpc/balancer.go @@ -28,10 +28,12 @@ import ( "google.golang.org/grpc/credentials" "google.golang.org/grpc/grpclog" "google.golang.org/grpc/naming" + "google.golang.org/grpc/status" ) // Address represents a server the client connects to. -// This is the EXPERIMENTAL API and may be changed or extended in the future. +// +// Deprecated: please use package balancer. type Address struct { // Addr is the server address on which a connection will be established. Addr string @@ -41,6 +43,8 @@ type Address struct { } // BalancerConfig specifies the configurations for Balancer. +// +// Deprecated: please use package balancer. type BalancerConfig struct { // DialCreds is the transport credential the Balancer implementation can // use to dial to a remote load balancer server. The Balancer implementations @@ -53,7 +57,8 @@ type BalancerConfig struct { } // BalancerGetOptions configures a Get call. -// This is the EXPERIMENTAL API and may be changed or extended in the future. +// +// Deprecated: please use package balancer. type BalancerGetOptions struct { // BlockingWait specifies whether Get should block when there is no // connected address. @@ -61,7 +66,8 @@ type BalancerGetOptions struct { } // Balancer chooses network addresses for RPCs. -// This is the EXPERIMENTAL API and may be changed or extended in the future. +// +// Deprecated: please use package balancer. type Balancer interface { // Start does the initialization work to bootstrap a Balancer. For example, // this function may start the name resolution and watch the updates. It will @@ -134,6 +140,8 @@ func downErrorf(timeout, temporary bool, format string, a ...interface{}) downEr // RoundRobin returns a Balancer that selects addresses round-robin. It uses r to watch // the name resolution updates and updates the addresses available correspondingly. +// +// Deprecated: please use package balancer/roundrobin. func RoundRobin(r naming.Resolver) Balancer { return &roundRobin{r: r} } @@ -310,7 +318,7 @@ func (rr *roundRobin) Get(ctx context.Context, opts BalancerGetOptions) (addr Ad if !opts.BlockingWait { if len(rr.addrs) == 0 { rr.mu.Unlock() - err = Errorf(codes.Unavailable, "there is no address available") + err = status.Errorf(codes.Unavailable, "there is no address available") return } // Returns the next addr on rr.addrs for failfast RPCs. diff --git a/vendor/google.golang.org/grpc/balancer/balancer.go b/vendor/google.golang.org/grpc/balancer/balancer.go index 84e10b630e7..63b8d71371e 100644 --- a/vendor/google.golang.org/grpc/balancer/balancer.go +++ b/vendor/google.golang.org/grpc/balancer/balancer.go @@ -23,6 +23,7 @@ package balancer import ( "errors" "net" + "strings" "golang.org/x/net/context" "google.golang.org/grpc/connectivity" @@ -33,24 +34,26 @@ import ( var ( // m is a map from name to balancer builder. m = make(map[string]Builder) - // defaultBuilder is the default balancer to use. - defaultBuilder Builder // TODO(bar) install pickfirst as default. ) -// Register registers the balancer builder to the balancer map. -// b.Name will be used as the name registered with this builder. +// Register registers the balancer builder to the balancer map. b.Name +// (lowercased) will be used as the name registered with this builder. +// +// NOTE: this function must only be called during initialization time (i.e. in +// an init() function), and is not thread-safe. If multiple Balancers are +// registered with the same name, the one registered last will take effect. func Register(b Builder) { - m[b.Name()] = b + m[strings.ToLower(b.Name())] = b } // Get returns the resolver builder registered with the given name. -// If no builder is register with the name, the default pickfirst will -// be used. +// Note that the compare is done in a case-insenstive fashion. +// If no builder is register with the name, nil will be returned. func Get(name string) Builder { - if b, ok := m[name]; ok { + if b, ok := m[strings.ToLower(name)]; ok { return b } - return defaultBuilder + return nil } // SubConn represents a gRPC sub connection. @@ -66,6 +69,11 @@ func Get(name string) Builder { // When the connection encounters an error, it will reconnect immediately. // When the connection becomes IDLE, it will not reconnect unless Connect is // called. +// +// This interface is to be implemented by gRPC. Users should not need a +// brand new implementation of this interface. For the situations like +// testing, the new implementation should embed this interface. This allows +// gRPC to add new methods to this interface. type SubConn interface { // UpdateAddresses updates the addresses used in this SubConn. // gRPC checks if currently-connected address is still in the new list. @@ -83,6 +91,11 @@ type SubConn interface { type NewSubConnOptions struct{} // ClientConn represents a gRPC ClientConn. +// +// This interface is to be implemented by gRPC. Users should not need a +// brand new implementation of this interface. For the situations like +// testing, the new implementation should embed this interface. This allows +// gRPC to add new methods to this interface. type ClientConn interface { // NewSubConn is called by balancer to create a new SubConn. // It doesn't block and wait for the connections to be established. @@ -99,6 +112,9 @@ type ClientConn interface { // on the new picker to pick new SubConn. UpdateBalancerState(s connectivity.State, p Picker) + // ResolveNow is called by balancer to notify gRPC to do a name resolving. + ResolveNow(resolver.ResolveNowOption) + // Target returns the dial target for this ClientConn. Target() string } @@ -113,6 +129,8 @@ type BuildOptions struct { // to a remote load balancer server. The Balancer implementations // can ignore this if it doesn't need to talk to remote balancer. Dialer func(context.Context, string) (net.Conn, error) + // ChannelzParentID is the entity parent's channelz unique identification number. + ChannelzParentID int64 } // Builder creates a balancer. @@ -131,6 +149,10 @@ type PickOptions struct{} type DoneInfo struct { // Err is the rpc error the RPC finished with. It could be nil. Err error + // BytesSent indicates if any bytes have been sent to the server. + BytesSent bool + // BytesReceived indicates if any byte has been received from the server. + BytesReceived bool } var ( @@ -143,7 +165,7 @@ var ( ) // Picker is used by gRPC to pick a SubConn to send an RPC. -// Balancer is expected to generate a new picker from its snapshot everytime its +// Balancer is expected to generate a new picker from its snapshot every time its // internal state has changed. // // The pickers used by gRPC can be updated by ClientConn.UpdateBalancerState(). @@ -161,7 +183,7 @@ type Picker interface { // If a SubConn is returned: // - If it is READY, gRPC will send the RPC on it; // - If it is not ready, or becomes not ready after it's returned, gRPC will block - // this call until a new picker is updated and will call pick on the new picker. + // until UpdateBalancerState() is called and will call pick on the new picker. // // If the returned error is not nil: // - If the error is ErrNoSubConnAvailable, gRPC will block until UpdateBalancerState() diff --git a/vendor/google.golang.org/grpc/balancer/base/balancer.go b/vendor/google.golang.org/grpc/balancer/base/balancer.go new file mode 100644 index 00000000000..23d13511bb2 --- /dev/null +++ b/vendor/google.golang.org/grpc/balancer/base/balancer.go @@ -0,0 +1,208 @@ +/* + * + * Copyright 2017 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package base + +import ( + "golang.org/x/net/context" + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/resolver" +) + +type baseBuilder struct { + name string + pickerBuilder PickerBuilder +} + +func (bb *baseBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer { + return &baseBalancer{ + cc: cc, + pickerBuilder: bb.pickerBuilder, + + subConns: make(map[resolver.Address]balancer.SubConn), + scStates: make(map[balancer.SubConn]connectivity.State), + csEvltr: &connectivityStateEvaluator{}, + // Initialize picker to a picker that always return + // ErrNoSubConnAvailable, because when state of a SubConn changes, we + // may call UpdateBalancerState with this picker. + picker: NewErrPicker(balancer.ErrNoSubConnAvailable), + } +} + +func (bb *baseBuilder) Name() string { + return bb.name +} + +type baseBalancer struct { + cc balancer.ClientConn + pickerBuilder PickerBuilder + + csEvltr *connectivityStateEvaluator + state connectivity.State + + subConns map[resolver.Address]balancer.SubConn + scStates map[balancer.SubConn]connectivity.State + picker balancer.Picker +} + +func (b *baseBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) { + if err != nil { + grpclog.Infof("base.baseBalancer: HandleResolvedAddrs called with error %v", err) + return + } + grpclog.Infoln("base.baseBalancer: got new resolved addresses: ", addrs) + // addrsSet is the set converted from addrs, it's used for quick lookup of an address. + addrsSet := make(map[resolver.Address]struct{}) + for _, a := range addrs { + addrsSet[a] = struct{}{} + if _, ok := b.subConns[a]; !ok { + // a is a new address (not existing in b.subConns). + sc, err := b.cc.NewSubConn([]resolver.Address{a}, balancer.NewSubConnOptions{}) + if err != nil { + grpclog.Warningf("base.baseBalancer: failed to create new SubConn: %v", err) + continue + } + b.subConns[a] = sc + b.scStates[sc] = connectivity.Idle + sc.Connect() + } + } + for a, sc := range b.subConns { + // a was removed by resolver. + if _, ok := addrsSet[a]; !ok { + b.cc.RemoveSubConn(sc) + delete(b.subConns, a) + // Keep the state of this sc in b.scStates until sc's state becomes Shutdown. + // The entry will be deleted in HandleSubConnStateChange. + } + } +} + +// regeneratePicker takes a snapshot of the balancer, and generates a picker +// from it. The picker is +// - errPicker with ErrTransientFailure if the balancer is in TransientFailure, +// - built by the pickerBuilder with all READY SubConns otherwise. +func (b *baseBalancer) regeneratePicker() { + if b.state == connectivity.TransientFailure { + b.picker = NewErrPicker(balancer.ErrTransientFailure) + return + } + readySCs := make(map[resolver.Address]balancer.SubConn) + + // Filter out all ready SCs from full subConn map. + for addr, sc := range b.subConns { + if st, ok := b.scStates[sc]; ok && st == connectivity.Ready { + readySCs[addr] = sc + } + } + b.picker = b.pickerBuilder.Build(readySCs) +} + +func (b *baseBalancer) HandleSubConnStateChange(sc balancer.SubConn, s connectivity.State) { + grpclog.Infof("base.baseBalancer: handle SubConn state change: %p, %v", sc, s) + oldS, ok := b.scStates[sc] + if !ok { + grpclog.Infof("base.baseBalancer: got state changes for an unknown SubConn: %p, %v", sc, s) + return + } + b.scStates[sc] = s + switch s { + case connectivity.Idle: + sc.Connect() + case connectivity.Shutdown: + // When an address was removed by resolver, b called RemoveSubConn but + // kept the sc's state in scStates. Remove state for this sc here. + delete(b.scStates, sc) + } + + oldAggrState := b.state + b.state = b.csEvltr.recordTransition(oldS, s) + + // Regenerate picker when one of the following happens: + // - this sc became ready from not-ready + // - this sc became not-ready from ready + // - the aggregated state of balancer became TransientFailure from non-TransientFailure + // - the aggregated state of balancer became non-TransientFailure from TransientFailure + if (s == connectivity.Ready) != (oldS == connectivity.Ready) || + (b.state == connectivity.TransientFailure) != (oldAggrState == connectivity.TransientFailure) { + b.regeneratePicker() + } + + b.cc.UpdateBalancerState(b.state, b.picker) +} + +// Close is a nop because base balancer doesn't have internal state to clean up, +// and it doesn't need to call RemoveSubConn for the SubConns. +func (b *baseBalancer) Close() { +} + +// NewErrPicker returns a picker that always returns err on Pick(). +func NewErrPicker(err error) balancer.Picker { + return &errPicker{err: err} +} + +type errPicker struct { + err error // Pick() always returns this err. +} + +func (p *errPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) { + return nil, nil, p.err +} + +// connectivityStateEvaluator gets updated by addrConns when their +// states transition, based on which it evaluates the state of +// ClientConn. +type connectivityStateEvaluator struct { + numReady uint64 // Number of addrConns in ready state. + numConnecting uint64 // Number of addrConns in connecting state. + numTransientFailure uint64 // Number of addrConns in transientFailure. +} + +// recordTransition records state change happening in every subConn and based on +// that it evaluates what aggregated state should be. +// It can only transition between Ready, Connecting and TransientFailure. Other states, +// Idle and Shutdown are transitioned into by ClientConn; in the beginning of the connection +// before any subConn is created ClientConn is in idle state. In the end when ClientConn +// closes it is in Shutdown state. +// +// recordTransition should only be called synchronously from the same goroutine. +func (cse *connectivityStateEvaluator) recordTransition(oldState, newState connectivity.State) connectivity.State { + // Update counters. + for idx, state := range []connectivity.State{oldState, newState} { + updateVal := 2*uint64(idx) - 1 // -1 for oldState and +1 for new. + switch state { + case connectivity.Ready: + cse.numReady += updateVal + case connectivity.Connecting: + cse.numConnecting += updateVal + case connectivity.TransientFailure: + cse.numTransientFailure += updateVal + } + } + + // Evaluate. + if cse.numReady > 0 { + return connectivity.Ready + } + if cse.numConnecting > 0 { + return connectivity.Connecting + } + return connectivity.TransientFailure +} diff --git a/vendor/google.golang.org/grpc/balancer/base/base.go b/vendor/google.golang.org/grpc/balancer/base/base.go new file mode 100644 index 00000000000..012ace2f2f7 --- /dev/null +++ b/vendor/google.golang.org/grpc/balancer/base/base.go @@ -0,0 +1,52 @@ +/* + * + * Copyright 2017 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +// Package base defines a balancer base that can be used to build balancers with +// different picking algorithms. +// +// The base balancer creates a new SubConn for each resolved address. The +// provided picker will only be notified about READY SubConns. +// +// This package is the base of round_robin balancer, its purpose is to be used +// to build round_robin like balancers with complex picking algorithms. +// Balancers with more complicated logic should try to implement a balancer +// builder from scratch. +// +// All APIs in this package are experimental. +package base + +import ( + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/resolver" +) + +// PickerBuilder creates balancer.Picker. +type PickerBuilder interface { + // Build takes a slice of ready SubConns, and returns a picker that will be + // used by gRPC to pick a SubConn. + Build(readySCs map[resolver.Address]balancer.SubConn) balancer.Picker +} + +// NewBalancerBuilder returns a balancer builder. The balancers +// built by this builder will use the picker builder to build pickers. +func NewBalancerBuilder(name string, pb PickerBuilder) balancer.Builder { + return &baseBuilder{ + name: name, + pickerBuilder: pb, + } +} diff --git a/vendor/google.golang.org/grpc/balancer/roundrobin/roundrobin.go b/vendor/google.golang.org/grpc/balancer/roundrobin/roundrobin.go new file mode 100644 index 00000000000..2eda0a1c210 --- /dev/null +++ b/vendor/google.golang.org/grpc/balancer/roundrobin/roundrobin.go @@ -0,0 +1,79 @@ +/* + * + * Copyright 2017 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +// Package roundrobin defines a roundrobin balancer. Roundrobin balancer is +// installed as one of the default balancers in gRPC, users don't need to +// explicitly install this balancer. +package roundrobin + +import ( + "sync" + + "golang.org/x/net/context" + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/balancer/base" + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/resolver" +) + +// Name is the name of round_robin balancer. +const Name = "round_robin" + +// newBuilder creates a new roundrobin balancer builder. +func newBuilder() balancer.Builder { + return base.NewBalancerBuilder(Name, &rrPickerBuilder{}) +} + +func init() { + balancer.Register(newBuilder()) +} + +type rrPickerBuilder struct{} + +func (*rrPickerBuilder) Build(readySCs map[resolver.Address]balancer.SubConn) balancer.Picker { + grpclog.Infof("roundrobinPicker: newPicker called with readySCs: %v", readySCs) + var scs []balancer.SubConn + for _, sc := range readySCs { + scs = append(scs, sc) + } + return &rrPicker{ + subConns: scs, + } +} + +type rrPicker struct { + // subConns is the snapshot of the roundrobin balancer when this picker was + // created. The slice is immutable. Each Get() will do a round robin + // selection from it and return the selected SubConn. + subConns []balancer.SubConn + + mu sync.Mutex + next int +} + +func (p *rrPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) { + if len(p.subConns) <= 0 { + return nil, nil, balancer.ErrNoSubConnAvailable + } + + p.mu.Lock() + sc := p.subConns[p.next] + p.next = (p.next + 1) % len(p.subConns) + p.mu.Unlock() + return sc, nil, nil +} diff --git a/vendor/google.golang.org/grpc/balancer_conn_wrappers.go b/vendor/google.golang.org/grpc/balancer_conn_wrappers.go index f5dbc4ba201..c23f81706fb 100644 --- a/vendor/google.golang.org/grpc/balancer_conn_wrappers.go +++ b/vendor/google.golang.org/grpc/balancer_conn_wrappers.go @@ -19,6 +19,7 @@ package grpc import ( + "fmt" "sync" "google.golang.org/grpc/balancer" @@ -73,7 +74,7 @@ func (b *scStateUpdateBuffer) load() { } } -// get returns the channel that receives a recvMsg in the buffer. +// get returns the channel that the scStateUpdate will be sent to. // // Upon receiving, the caller should call load to send another // scStateChangeTuple onto the channel if there is any. @@ -96,6 +97,9 @@ type ccBalancerWrapper struct { stateChangeQueue *scStateUpdateBuffer resolverUpdateCh chan *resolverUpdate done chan struct{} + + mu sync.Mutex + subConns map[*acBalancerWrapper]struct{} } func newCCBalancerWrapper(cc *ClientConn, b balancer.Builder, bopts balancer.BuildOptions) *ccBalancerWrapper { @@ -104,21 +108,34 @@ func newCCBalancerWrapper(cc *ClientConn, b balancer.Builder, bopts balancer.Bui stateChangeQueue: newSCStateUpdateBuffer(), resolverUpdateCh: make(chan *resolverUpdate, 1), done: make(chan struct{}), + subConns: make(map[*acBalancerWrapper]struct{}), } go ccb.watcher() ccb.balancer = b.Build(ccb, bopts) return ccb } -// watcher balancer functions sequencially, so the balancer can be implemeneted +// watcher balancer functions sequentially, so the balancer can be implemented // lock-free. func (ccb *ccBalancerWrapper) watcher() { for { select { case t := <-ccb.stateChangeQueue.get(): ccb.stateChangeQueue.load() + select { + case <-ccb.done: + ccb.balancer.Close() + return + default: + } ccb.balancer.HandleSubConnStateChange(t.sc, t.state) case t := <-ccb.resolverUpdateCh: + select { + case <-ccb.done: + ccb.balancer.Close() + return + default: + } ccb.balancer.HandleResolvedAddrs(t.addrs, t.err) case <-ccb.done: } @@ -126,6 +143,13 @@ func (ccb *ccBalancerWrapper) watcher() { select { case <-ccb.done: ccb.balancer.Close() + ccb.mu.Lock() + scs := ccb.subConns + ccb.subConns = nil + ccb.mu.Unlock() + for acbw := range scs { + ccb.cc.removeAddrConn(acbw.getAddrConn(), errConnDrain) + } return default: } @@ -165,33 +189,54 @@ func (ccb *ccBalancerWrapper) handleResolvedAddrs(addrs []resolver.Address, err } func (ccb *ccBalancerWrapper) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) { - grpclog.Infof("ccBalancerWrapper: new subconn: %v", addrs) + if len(addrs) <= 0 { + return nil, fmt.Errorf("grpc: cannot create SubConn with empty address list") + } + ccb.mu.Lock() + defer ccb.mu.Unlock() + if ccb.subConns == nil { + return nil, fmt.Errorf("grpc: ClientConn balancer wrapper was closed") + } ac, err := ccb.cc.newAddrConn(addrs) if err != nil { return nil, err } acbw := &acBalancerWrapper{ac: ac} - ac.mu.Lock() + acbw.ac.mu.Lock() ac.acbw = acbw - ac.mu.Unlock() + acbw.ac.mu.Unlock() + ccb.subConns[acbw] = struct{}{} return acbw, nil } func (ccb *ccBalancerWrapper) RemoveSubConn(sc balancer.SubConn) { - grpclog.Infof("ccBalancerWrapper: removing subconn") acbw, ok := sc.(*acBalancerWrapper) if !ok { return } + ccb.mu.Lock() + defer ccb.mu.Unlock() + if ccb.subConns == nil { + return + } + delete(ccb.subConns, acbw) ccb.cc.removeAddrConn(acbw.getAddrConn(), errConnDrain) } func (ccb *ccBalancerWrapper) UpdateBalancerState(s connectivity.State, p balancer.Picker) { - grpclog.Infof("ccBalancerWrapper: updating state and picker called by balancer: %v, %p", s, p) + ccb.mu.Lock() + defer ccb.mu.Unlock() + if ccb.subConns == nil { + return + } ccb.cc.csMgr.updateState(s) ccb.cc.blockingpicker.updatePicker(p) } +func (ccb *ccBalancerWrapper) ResolveNow(o resolver.ResolveNowOption) { + ccb.cc.resolveNow(o) +} + func (ccb *ccBalancerWrapper) Target() string { return ccb.cc.target } @@ -204,9 +249,12 @@ type acBalancerWrapper struct { } func (acbw *acBalancerWrapper) UpdateAddresses(addrs []resolver.Address) { - grpclog.Infof("acBalancerWrapper: UpdateAddresses called with %v", addrs) acbw.mu.Lock() defer acbw.mu.Unlock() + if len(addrs) <= 0 { + acbw.ac.tearDown(errConnDrain) + return + } if !acbw.ac.tryUpdateAddrs(addrs) { cc := acbw.ac.cc acbw.ac.mu.Lock() @@ -234,7 +282,7 @@ func (acbw *acBalancerWrapper) UpdateAddresses(addrs []resolver.Address) { ac.acbw = acbw ac.mu.Unlock() if acState != connectivity.Idle { - ac.connect(false) + ac.connect() } } } @@ -242,7 +290,7 @@ func (acbw *acBalancerWrapper) UpdateAddresses(addrs []resolver.Address) { func (acbw *acBalancerWrapper) Connect() { acbw.mu.Lock() defer acbw.mu.Unlock() - acbw.ac.connect(false) + acbw.ac.connect() } func (acbw *acBalancerWrapper) getAddrConn() *addrConn { diff --git a/vendor/google.golang.org/grpc/balancer_v1_wrapper.go b/vendor/google.golang.org/grpc/balancer_v1_wrapper.go index 9d0616080a1..b7abc6b7457 100644 --- a/vendor/google.golang.org/grpc/balancer_v1_wrapper.go +++ b/vendor/google.golang.org/grpc/balancer_v1_wrapper.go @@ -19,6 +19,7 @@ package grpc import ( + "strings" "sync" "golang.org/x/net/context" @@ -27,6 +28,7 @@ import ( "google.golang.org/grpc/connectivity" "google.golang.org/grpc/grpclog" "google.golang.org/grpc/resolver" + "google.golang.org/grpc/status" ) type balancerWrapperBuilder struct { @@ -34,20 +36,27 @@ type balancerWrapperBuilder struct { } func (bwb *balancerWrapperBuilder) Build(cc balancer.ClientConn, opts balancer.BuildOptions) balancer.Balancer { - bwb.b.Start(cc.Target(), BalancerConfig{ + targetAddr := cc.Target() + targetSplitted := strings.Split(targetAddr, ":///") + if len(targetSplitted) >= 2 { + targetAddr = targetSplitted[1] + } + + bwb.b.Start(targetAddr, BalancerConfig{ DialCreds: opts.DialCreds, Dialer: opts.Dialer, }) _, pickfirst := bwb.b.(*pickFirst) bw := &balancerWrapper{ - balancer: bwb.b, - pickfirst: pickfirst, - cc: cc, - startCh: make(chan struct{}), - conns: make(map[resolver.Address]balancer.SubConn), - connSt: make(map[balancer.SubConn]*scState), - csEvltr: &connectivityStateEvaluator{}, - state: connectivity.Idle, + balancer: bwb.b, + pickfirst: pickfirst, + cc: cc, + targetAddr: targetAddr, + startCh: make(chan struct{}), + conns: make(map[resolver.Address]balancer.SubConn), + connSt: make(map[balancer.SubConn]*scState), + csEvltr: &connectivityStateEvaluator{}, + state: connectivity.Idle, } cc.UpdateBalancerState(connectivity.Idle, bw) go bw.lbWatcher() @@ -68,7 +77,8 @@ type balancerWrapper struct { balancer Balancer // The v1 balancer. pickfirst bool - cc balancer.ClientConn + cc balancer.ClientConn + targetAddr string // Target without the scheme. // To aggregate the connectivity state. csEvltr *connectivityStateEvaluator @@ -88,12 +98,11 @@ type balancerWrapper struct { // connections accordingly. func (bw *balancerWrapper) lbWatcher() { <-bw.startCh - grpclog.Infof("balancerWrapper: is pickfirst: %v\n", bw.pickfirst) notifyCh := bw.balancer.Notify() if notifyCh == nil { // There's no resolver in the balancer. Connect directly. a := resolver.Address{ - Addr: bw.cc.Target(), + Addr: bw.targetAddr, Type: resolver.Backend, } sc, err := bw.cc.NewSubConn([]resolver.Address{a}, balancer.NewSubConnOptions{}) @@ -103,7 +112,7 @@ func (bw *balancerWrapper) lbWatcher() { bw.mu.Lock() bw.conns[a] = sc bw.connSt[sc] = &scState{ - addr: Address{Addr: bw.cc.Target()}, + addr: Address{Addr: bw.targetAddr}, s: connectivity.Idle, } bw.mu.Unlock() @@ -165,10 +174,10 @@ func (bw *balancerWrapper) lbWatcher() { sc.Connect() } } else { - oldSC.UpdateAddresses(newAddrs) bw.mu.Lock() bw.connSt[oldSC].addr = addrs[0] bw.mu.Unlock() + oldSC.UpdateAddresses(newAddrs) } } else { var ( @@ -221,7 +230,6 @@ func (bw *balancerWrapper) lbWatcher() { } func (bw *balancerWrapper) HandleSubConnStateChange(sc balancer.SubConn, s connectivity.State) { - grpclog.Infof("balancerWrapper: handle subconn state change: %p, %v", sc, s) bw.mu.Lock() defer bw.mu.Unlock() scSt, ok := bw.connSt[sc] @@ -249,7 +257,6 @@ func (bw *balancerWrapper) HandleSubConnStateChange(sc balancer.SubConn, s conne // Remove state for this sc. delete(bw.connSt, sc) } - return } func (bw *balancerWrapper) HandleResolvedAddrs([]resolver.Address, error) { @@ -262,7 +269,6 @@ func (bw *balancerWrapper) HandleResolvedAddrs([]resolver.Address, error) { } // There should be a resolver inside the balancer. // All updates here, if any, are ignored. - return } func (bw *balancerWrapper) Close() { @@ -274,7 +280,6 @@ func (bw *balancerWrapper) Close() { close(bw.startCh) } bw.balancer.Close() - return } // The picker is the balancerWrapper itself. @@ -310,12 +315,12 @@ func (bw *balancerWrapper) Pick(ctx context.Context, opts balancer.PickOptions) Metadata: a.Metadata, }] if !ok && failfast { - return nil, nil, Errorf(codes.Unavailable, "there is no connection available") + return nil, nil, status.Errorf(codes.Unavailable, "there is no connection available") } if s, ok := bw.connSt[sc]; failfast && (!ok || s.s != connectivity.Ready) { // If the returned sc is not ready and RPC is failfast, // return error, and this RPC will fail. - return nil, nil, Errorf(codes.Unavailable, "there is no connection available") + return nil, nil, status.Errorf(codes.Unavailable, "there is no connection available") } } diff --git a/vendor/google.golang.org/grpc/call.go b/vendor/google.golang.org/grpc/call.go index 1ef2507c35f..f73b7d5528f 100644 --- a/vendor/google.golang.org/grpc/call.go +++ b/vendor/google.golang.org/grpc/call.go @@ -19,289 +19,75 @@ package grpc import ( - "bytes" - "io" - "time" - "golang.org/x/net/context" - "golang.org/x/net/trace" - "google.golang.org/grpc/balancer" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/peer" - "google.golang.org/grpc/stats" - "google.golang.org/grpc/status" - "google.golang.org/grpc/transport" ) -// recvResponse receives and parses an RPC response. -// On error, it returns the error and indicates whether the call should be retried. +// Invoke sends the RPC request on the wire and returns after response is +// received. This is typically called by generated code. // -// TODO(zhaoq): Check whether the received message sequence is valid. -// TODO ctx is used for stats collection and processing. It is the context passed from the application. -func recvResponse(ctx context.Context, dopts dialOptions, t transport.ClientTransport, c *callInfo, stream *transport.Stream, reply interface{}) (err error) { - // Try to acquire header metadata from the server if there is any. - defer func() { - if err != nil { - if _, ok := err.(transport.ConnectionError); !ok { - t.CloseStream(stream, err) - } - } - }() - c.headerMD, err = stream.Header() - if err != nil { - return - } - p := &parser{r: stream} - var inPayload *stats.InPayload - if dopts.copts.StatsHandler != nil { - inPayload = &stats.InPayload{ - Client: true, - } - } - for { - if c.maxReceiveMessageSize == nil { - return Errorf(codes.Internal, "callInfo maxReceiveMessageSize field uninitialized(nil)") - } - if err = recv(p, dopts.codec, stream, dopts.dc, reply, *c.maxReceiveMessageSize, inPayload); err != nil { - if err == io.EOF { - break - } - return - } - } - if inPayload != nil && err == io.EOF && stream.Status().Code() == codes.OK { - // TODO in the current implementation, inTrailer may be handled before inPayload in some cases. - // Fix the order if necessary. - dopts.copts.StatsHandler.HandleRPC(ctx, inPayload) - } - c.trailerMD = stream.Trailer() - return nil -} +// All errors returned by Invoke are compatible with the status package. +func (cc *ClientConn) Invoke(ctx context.Context, method string, args, reply interface{}, opts ...CallOption) error { + // allow interceptor to see all applicable call options, which means those + // configured as defaults from dial option as well as per-call options + opts = combine(cc.dopts.callOptions, opts) -// sendRequest writes out various information of an RPC such as Context and Message. -func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, c *callInfo, callHdr *transport.CallHdr, stream *transport.Stream, t transport.ClientTransport, args interface{}, opts *transport.Options) (err error) { - defer func() { - if err != nil { - // If err is connection error, t will be closed, no need to close stream here. - if _, ok := err.(transport.ConnectionError); !ok { - t.CloseStream(stream, err) - } - } - }() - var ( - cbuf *bytes.Buffer - outPayload *stats.OutPayload - ) - if compressor != nil { - cbuf = new(bytes.Buffer) - } - if dopts.copts.StatsHandler != nil { - outPayload = &stats.OutPayload{ - Client: true, - } - } - hdr, data, err := encode(dopts.codec, args, compressor, cbuf, outPayload) - if err != nil { - return err - } - if c.maxSendMessageSize == nil { - return Errorf(codes.Internal, "callInfo maxSendMessageSize field uninitialized(nil)") - } - if len(data) > *c.maxSendMessageSize { - return Errorf(codes.ResourceExhausted, "grpc: trying to send message larger than max (%d vs. %d)", len(data), *c.maxSendMessageSize) - } - err = t.Write(stream, hdr, data, opts) - if err == nil && outPayload != nil { - outPayload.SentTime = time.Now() - dopts.copts.StatsHandler.HandleRPC(ctx, outPayload) - } - // t.NewStream(...) could lead to an early rejection of the RPC (e.g., the service/method - // does not exist.) so that t.Write could get io.EOF from wait(...). Leave the following - // recvResponse to get the final status. - if err != nil && err != io.EOF { - return err - } - // Sent successfully. - return nil -} - -// Invoke sends the RPC request on the wire and returns after response is received. -// Invoke is called by generated code. Also users can call Invoke directly when it -// is really needed in their use cases. -func Invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) error { if cc.dopts.unaryInt != nil { return cc.dopts.unaryInt(ctx, method, args, reply, cc, invoke, opts...) } return invoke(ctx, method, args, reply, cc, opts...) } -func invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) (e error) { - c := defaultCallInfo() - mc := cc.GetMethodConfig(method) - if mc.WaitForReady != nil { - c.failFast = !*mc.WaitForReady - } - - if mc.Timeout != nil && *mc.Timeout >= 0 { - var cancel context.CancelFunc - ctx, cancel = context.WithTimeout(ctx, *mc.Timeout) - defer cancel() - } +func combine(o1 []CallOption, o2 []CallOption) []CallOption { + // we don't use append because o1 could have extra capacity whose + // elements would be overwritten, which could cause inadvertent + // sharing (and race connditions) between concurrent calls + if len(o1) == 0 { + return o2 + } else if len(o2) == 0 { + return o1 + } + ret := make([]CallOption, len(o1)+len(o2)) + copy(ret, o1) + copy(ret[len(o1):], o2) + return ret +} - opts = append(cc.dopts.callOptions, opts...) - for _, o := range opts { - if err := o.before(c); err != nil { - return toRPCErr(err) - } - } - defer func() { - for _, o := range opts { - o.after(c) - } - }() +// Invoke sends the RPC request on the wire and returns after response is +// received. This is typically called by generated code. +// +// DEPRECATED: Use ClientConn.Invoke instead. +func Invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) error { + return cc.Invoke(ctx, method, args, reply, opts...) +} - c.maxSendMessageSize = getMaxSize(mc.MaxReqSize, c.maxSendMessageSize, defaultClientMaxSendMessageSize) - c.maxReceiveMessageSize = getMaxSize(mc.MaxRespSize, c.maxReceiveMessageSize, defaultClientMaxReceiveMessageSize) +var unaryStreamDesc = &StreamDesc{ServerStreams: false, ClientStreams: false} - if EnableTracing { - c.traceInfo.tr = trace.New("grpc.Sent."+methodFamily(method), method) - defer c.traceInfo.tr.Finish() - c.traceInfo.firstLine.client = true - if deadline, ok := ctx.Deadline(); ok { - c.traceInfo.firstLine.deadline = deadline.Sub(time.Now()) - } - c.traceInfo.tr.LazyLog(&c.traceInfo.firstLine, false) - // TODO(dsymonds): Arrange for c.traceInfo.firstLine.remoteAddr to be set. - defer func() { - if e != nil { - c.traceInfo.tr.LazyLog(&fmtStringer{"%v", []interface{}{e}}, true) - c.traceInfo.tr.SetError() - } - }() - } - ctx = newContextWithRPCInfo(ctx, c.failFast) - sh := cc.dopts.copts.StatsHandler - if sh != nil { - ctx = sh.TagRPC(ctx, &stats.RPCTagInfo{FullMethodName: method, FailFast: c.failFast}) - begin := &stats.Begin{ - Client: true, - BeginTime: time.Now(), - FailFast: c.failFast, - } - sh.HandleRPC(ctx, begin) - defer func() { - end := &stats.End{ - Client: true, - EndTime: time.Now(), - Error: e, - } - sh.HandleRPC(ctx, end) - }() - } - topts := &transport.Options{ - Last: true, - Delay: false, - } +func invoke(ctx context.Context, method string, req, reply interface{}, cc *ClientConn, opts ...CallOption) error { + // TODO: implement retries in clientStream and make this simply + // newClientStream, SendMsg, RecvMsg. + firstAttempt := true for { - var ( - err error - t transport.ClientTransport - stream *transport.Stream - // Record the done handler from Balancer.Get(...). It is called once the - // RPC has completed or failed. - done func(balancer.DoneInfo) - ) - // TODO(zhaoq): Need a formal spec of fail-fast. - callHdr := &transport.CallHdr{ - Host: cc.authority, - Method: method, - } - if cc.dopts.cp != nil { - callHdr.SendCompress = cc.dopts.cp.Type() - } - if c.creds != nil { - callHdr.Creds = c.creds - } - - t, done, err = cc.getTransport(ctx, c.failFast) + csInt, err := newClientStream(ctx, unaryStreamDesc, cc, method, opts...) if err != nil { - // TODO(zhaoq): Probably revisit the error handling. - if _, ok := status.FromError(err); ok { - return err - } - if err == errConnClosing || err == errConnUnavailable { - if c.failFast { - return Errorf(codes.Unavailable, "%v", err) - } - continue - } - // All the other errors are treated as Internal errors. - return Errorf(codes.Internal, "%v", err) - } - if c.traceInfo.tr != nil { - c.traceInfo.tr.LazyLog(&payload{sent: true, msg: args}, true) - } - stream, err = t.NewStream(ctx, callHdr) - if err != nil { - if done != nil { - if _, ok := err.(transport.ConnectionError); ok { - // If error is connection error, transport was sending data on wire, - // and we are not sure if anything has been sent on wire. - // If error is not connection error, we are sure nothing has been sent. - updateRPCInfoInContext(ctx, rpcInfo{bytesSent: true, bytesReceived: false}) - } - done(balancer.DoneInfo{Err: err}) - } - if _, ok := err.(transport.ConnectionError); (ok || err == transport.ErrStreamDrain) && !c.failFast { - continue - } - return toRPCErr(err) - } - if peer, ok := peer.FromContext(stream.Context()); ok { - c.peer = peer + return err } - err = sendRequest(ctx, cc.dopts, cc.dopts.cp, c, callHdr, stream, t, args, topts) - if err != nil { - if done != nil { - updateRPCInfoInContext(ctx, rpcInfo{ - bytesSent: stream.BytesSent(), - bytesReceived: stream.BytesReceived(), - }) - done(balancer.DoneInfo{Err: err}) - } - // Retry a non-failfast RPC when - // i) there is a connection error; or - // ii) the server started to drain before this RPC was initiated. - if _, ok := err.(transport.ConnectionError); (ok || err == transport.ErrStreamDrain) && !c.failFast { + cs := csInt.(*clientStream) + if err := cs.SendMsg(req); err != nil { + if !cs.c.failFast && cs.attempt.s.Unprocessed() && firstAttempt { + // TODO: Add a field to header for grpc-transparent-retry-attempts + firstAttempt = false continue } - return toRPCErr(err) + return err } - err = recvResponse(ctx, cc.dopts, t, c, stream, reply) - if err != nil { - if done != nil { - updateRPCInfoInContext(ctx, rpcInfo{ - bytesSent: stream.BytesSent(), - bytesReceived: stream.BytesReceived(), - }) - done(balancer.DoneInfo{Err: err}) - } - if _, ok := err.(transport.ConnectionError); (ok || err == transport.ErrStreamDrain) && !c.failFast { + if err := cs.RecvMsg(reply); err != nil { + if !cs.c.failFast && cs.attempt.s.Unprocessed() && firstAttempt { + // TODO: Add a field to header for grpc-transparent-retry-attempts + firstAttempt = false continue } - return toRPCErr(err) - } - if c.traceInfo.tr != nil { - c.traceInfo.tr.LazyLog(&payload{sent: false, msg: reply}, true) - } - t.CloseStream(stream, nil) - if done != nil { - updateRPCInfoInContext(ctx, rpcInfo{ - bytesSent: stream.BytesSent(), - bytesReceived: stream.BytesReceived(), - }) - done(balancer.DoneInfo{Err: err}) + return err } - return stream.Status().Err() + return nil } } diff --git a/vendor/google.golang.org/grpc/channelz/funcs.go b/vendor/google.golang.org/grpc/channelz/funcs.go new file mode 100644 index 00000000000..586a0336b47 --- /dev/null +++ b/vendor/google.golang.org/grpc/channelz/funcs.go @@ -0,0 +1,573 @@ +/* + * + * Copyright 2018 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +// Package channelz defines APIs for enabling channelz service, entry +// registration/deletion, and accessing channelz data. It also defines channelz +// metric struct formats. +// +// All APIs in this package are experimental. +package channelz + +import ( + "sort" + "sync" + "sync/atomic" + + "google.golang.org/grpc/grpclog" +) + +var ( + db dbWrapper + idGen idGenerator + // EntryPerPage defines the number of channelz entries to be shown on a web page. + EntryPerPage = 50 + curState int32 +) + +// TurnOn turns on channelz data collection. +func TurnOn() { + if !IsOn() { + NewChannelzStorage() + atomic.StoreInt32(&curState, 1) + } +} + +// IsOn returns whether channelz data collection is on. +func IsOn() bool { + return atomic.CompareAndSwapInt32(&curState, 1, 1) +} + +// dbWarpper wraps around a reference to internal channelz data storage, and +// provide synchronized functionality to set and get the reference. +type dbWrapper struct { + mu sync.RWMutex + DB *channelMap +} + +func (d *dbWrapper) set(db *channelMap) { + d.mu.Lock() + d.DB = db + d.mu.Unlock() +} + +func (d *dbWrapper) get() *channelMap { + d.mu.RLock() + defer d.mu.RUnlock() + return d.DB +} + +// NewChannelzStorage initializes channelz data storage and id generator. +// +// Note: This function is exported for testing purpose only. User should not call +// it in most cases. +func NewChannelzStorage() { + db.set(&channelMap{ + topLevelChannels: make(map[int64]struct{}), + channels: make(map[int64]*channel), + listenSockets: make(map[int64]*listenSocket), + normalSockets: make(map[int64]*normalSocket), + servers: make(map[int64]*server), + subChannels: make(map[int64]*subChannel), + }) + idGen.reset() +} + +// GetTopChannels returns a slice of top channel's ChannelMetric, along with a +// boolean indicating whether there's more top channels to be queried for. +// +// The arg id specifies that only top channel with id at or above it will be included +// in the result. The returned slice is up to a length of EntryPerPage, and is +// sorted in ascending id order. +func GetTopChannels(id int64) ([]*ChannelMetric, bool) { + return db.get().GetTopChannels(id) +} + +// GetServers returns a slice of server's ServerMetric, along with a +// boolean indicating whether there's more servers to be queried for. +// +// The arg id specifies that only server with id at or above it will be included +// in the result. The returned slice is up to a length of EntryPerPage, and is +// sorted in ascending id order. +func GetServers(id int64) ([]*ServerMetric, bool) { + return db.get().GetServers(id) +} + +// GetServerSockets returns a slice of server's (identified by id) normal socket's +// SocketMetric, along with a boolean indicating whether there's more sockets to +// be queried for. +// +// The arg startID specifies that only sockets with id at or above it will be +// included in the result. The returned slice is up to a length of EntryPerPage, +// and is sorted in ascending id order. +func GetServerSockets(id int64, startID int64) ([]*SocketMetric, bool) { + return db.get().GetServerSockets(id, startID) +} + +// GetChannel returns the ChannelMetric for the channel (identified by id). +func GetChannel(id int64) *ChannelMetric { + return db.get().GetChannel(id) +} + +// GetSubChannel returns the SubChannelMetric for the subchannel (identified by id). +func GetSubChannel(id int64) *SubChannelMetric { + return db.get().GetSubChannel(id) +} + +// GetSocket returns the SocketInternalMetric for the socket (identified by id). +func GetSocket(id int64) *SocketMetric { + return db.get().GetSocket(id) +} + +// RegisterChannel registers the given channel c in channelz database with ref +// as its reference name, and add it to the child list of its parent (identified +// by pid). pid = 0 means no parent. It returns the unique channelz tracking id +// assigned to this channel. +func RegisterChannel(c Channel, pid int64, ref string) int64 { + id := idGen.genID() + cn := &channel{ + refName: ref, + c: c, + subChans: make(map[int64]string), + nestedChans: make(map[int64]string), + id: id, + pid: pid, + } + if pid == 0 { + db.get().addChannel(id, cn, true, pid, ref) + } else { + db.get().addChannel(id, cn, false, pid, ref) + } + return id +} + +// RegisterSubChannel registers the given channel c in channelz database with ref +// as its reference name, and add it to the child list of its parent (identified +// by pid). It returns the unique channelz tracking id assigned to this subchannel. +func RegisterSubChannel(c Channel, pid int64, ref string) int64 { + if pid == 0 { + grpclog.Error("a SubChannel's parent id cannot be 0") + return 0 + } + id := idGen.genID() + sc := &subChannel{ + refName: ref, + c: c, + sockets: make(map[int64]string), + id: id, + pid: pid, + } + db.get().addSubChannel(id, sc, pid, ref) + return id +} + +// RegisterServer registers the given server s in channelz database. It returns +// the unique channelz tracking id assigned to this server. +func RegisterServer(s Server, ref string) int64 { + id := idGen.genID() + svr := &server{ + refName: ref, + s: s, + sockets: make(map[int64]string), + listenSockets: make(map[int64]string), + id: id, + } + db.get().addServer(id, svr) + return id +} + +// RegisterListenSocket registers the given listen socket s in channelz database +// with ref as its reference name, and add it to the child list of its parent +// (identified by pid). It returns the unique channelz tracking id assigned to +// this listen socket. +func RegisterListenSocket(s Socket, pid int64, ref string) int64 { + if pid == 0 { + grpclog.Error("a ListenSocket's parent id cannot be 0") + return 0 + } + id := idGen.genID() + ls := &listenSocket{refName: ref, s: s, id: id, pid: pid} + db.get().addListenSocket(id, ls, pid, ref) + return id +} + +// RegisterNormalSocket registers the given normal socket s in channelz database +// with ref as its reference name, and add it to the child list of its parent +// (identified by pid). It returns the unique channelz tracking id assigned to +// this normal socket. +func RegisterNormalSocket(s Socket, pid int64, ref string) int64 { + if pid == 0 { + grpclog.Error("a NormalSocket's parent id cannot be 0") + return 0 + } + id := idGen.genID() + ns := &normalSocket{refName: ref, s: s, id: id, pid: pid} + db.get().addNormalSocket(id, ns, pid, ref) + return id +} + +// RemoveEntry removes an entry with unique channelz trakcing id to be id from +// channelz database. +func RemoveEntry(id int64) { + db.get().removeEntry(id) +} + +// channelMap is the storage data structure for channelz. +// Methods of channelMap can be divided in two two categories with respect to locking. +// 1. Methods acquire the global lock. +// 2. Methods that can only be called when global lock is held. +// A second type of method need always to be called inside a first type of method. +type channelMap struct { + mu sync.RWMutex + topLevelChannels map[int64]struct{} + servers map[int64]*server + channels map[int64]*channel + subChannels map[int64]*subChannel + listenSockets map[int64]*listenSocket + normalSockets map[int64]*normalSocket +} + +func (c *channelMap) addServer(id int64, s *server) { + c.mu.Lock() + s.cm = c + c.servers[id] = s + c.mu.Unlock() +} + +func (c *channelMap) addChannel(id int64, cn *channel, isTopChannel bool, pid int64, ref string) { + c.mu.Lock() + cn.cm = c + c.channels[id] = cn + if isTopChannel { + c.topLevelChannels[id] = struct{}{} + } else { + c.findEntry(pid).addChild(id, cn) + } + c.mu.Unlock() +} + +func (c *channelMap) addSubChannel(id int64, sc *subChannel, pid int64, ref string) { + c.mu.Lock() + sc.cm = c + c.subChannels[id] = sc + c.findEntry(pid).addChild(id, sc) + c.mu.Unlock() +} + +func (c *channelMap) addListenSocket(id int64, ls *listenSocket, pid int64, ref string) { + c.mu.Lock() + ls.cm = c + c.listenSockets[id] = ls + c.findEntry(pid).addChild(id, ls) + c.mu.Unlock() +} + +func (c *channelMap) addNormalSocket(id int64, ns *normalSocket, pid int64, ref string) { + c.mu.Lock() + ns.cm = c + c.normalSockets[id] = ns + c.findEntry(pid).addChild(id, ns) + c.mu.Unlock() +} + +// removeEntry triggers the removal of an entry, which may not indeed delete the +// entry, if it has to wait on the deletion of its children, or may lead to a chain +// of entry deletion. For example, deleting the last socket of a gracefully shutting +// down server will lead to the server being also deleted. +func (c *channelMap) removeEntry(id int64) { + c.mu.Lock() + c.findEntry(id).triggerDelete() + c.mu.Unlock() +} + +// c.mu must be held by the caller. +func (c *channelMap) findEntry(id int64) entry { + var v entry + var ok bool + if v, ok = c.channels[id]; ok { + return v + } + if v, ok = c.subChannels[id]; ok { + return v + } + if v, ok = c.servers[id]; ok { + return v + } + if v, ok = c.listenSockets[id]; ok { + return v + } + if v, ok = c.normalSockets[id]; ok { + return v + } + return &dummyEntry{idNotFound: id} +} + +// c.mu must be held by the caller +// deleteEntry simply deletes an entry from the channelMap. Before calling this +// method, caller must check this entry is ready to be deleted, i.e removeEntry() +// has been called on it, and no children still exist. +// Conditionals are ordered by the expected frequency of deletion of each entity +// type, in order to optimize performance. +func (c *channelMap) deleteEntry(id int64) { + var ok bool + if _, ok = c.normalSockets[id]; ok { + delete(c.normalSockets, id) + return + } + if _, ok = c.subChannels[id]; ok { + delete(c.subChannels, id) + return + } + if _, ok = c.channels[id]; ok { + delete(c.channels, id) + delete(c.topLevelChannels, id) + return + } + if _, ok = c.listenSockets[id]; ok { + delete(c.listenSockets, id) + return + } + if _, ok = c.servers[id]; ok { + delete(c.servers, id) + return + } +} + +type int64Slice []int64 + +func (s int64Slice) Len() int { return len(s) } +func (s int64Slice) Swap(i, j int) { s[i], s[j] = s[j], s[i] } +func (s int64Slice) Less(i, j int) bool { return s[i] < s[j] } + +func copyMap(m map[int64]string) map[int64]string { + n := make(map[int64]string) + for k, v := range m { + n[k] = v + } + return n +} + +func min(a, b int) int { + if a < b { + return a + } + return b +} + +func (c *channelMap) GetTopChannels(id int64) ([]*ChannelMetric, bool) { + c.mu.RLock() + l := len(c.topLevelChannels) + ids := make([]int64, 0, l) + cns := make([]*channel, 0, min(l, EntryPerPage)) + + for k := range c.topLevelChannels { + ids = append(ids, k) + } + sort.Sort(int64Slice(ids)) + idx := sort.Search(len(ids), func(i int) bool { return ids[i] >= id }) + count := 0 + var end bool + var t []*ChannelMetric + for i, v := range ids[idx:] { + if count == EntryPerPage { + break + } + if cn, ok := c.channels[v]; ok { + cns = append(cns, cn) + t = append(t, &ChannelMetric{ + NestedChans: copyMap(cn.nestedChans), + SubChans: copyMap(cn.subChans), + }) + count++ + } + if i == len(ids[idx:])-1 { + end = true + break + } + } + c.mu.RUnlock() + if count == 0 { + end = true + } + + for i, cn := range cns { + t[i].ChannelData = cn.c.ChannelzMetric() + t[i].ID = cn.id + t[i].RefName = cn.refName + } + return t, end +} + +func (c *channelMap) GetServers(id int64) ([]*ServerMetric, bool) { + c.mu.RLock() + l := len(c.servers) + ids := make([]int64, 0, l) + ss := make([]*server, 0, min(l, EntryPerPage)) + for k := range c.servers { + ids = append(ids, k) + } + sort.Sort(int64Slice(ids)) + idx := sort.Search(len(ids), func(i int) bool { return ids[i] >= id }) + count := 0 + var end bool + var s []*ServerMetric + for i, v := range ids[idx:] { + if count == EntryPerPage { + break + } + if svr, ok := c.servers[v]; ok { + ss = append(ss, svr) + s = append(s, &ServerMetric{ + ListenSockets: copyMap(svr.listenSockets), + }) + count++ + } + if i == len(ids[idx:])-1 { + end = true + break + } + } + c.mu.RUnlock() + if count == 0 { + end = true + } + + for i, svr := range ss { + s[i].ServerData = svr.s.ChannelzMetric() + s[i].ID = svr.id + s[i].RefName = svr.refName + } + return s, end +} + +func (c *channelMap) GetServerSockets(id int64, startID int64) ([]*SocketMetric, bool) { + var svr *server + var ok bool + c.mu.RLock() + if svr, ok = c.servers[id]; !ok { + // server with id doesn't exist. + c.mu.RUnlock() + return nil, true + } + svrskts := svr.sockets + l := len(svrskts) + ids := make([]int64, 0, l) + sks := make([]*normalSocket, 0, min(l, EntryPerPage)) + for k := range svrskts { + ids = append(ids, k) + } + sort.Sort((int64Slice(ids))) + idx := sort.Search(len(ids), func(i int) bool { return ids[i] >= id }) + count := 0 + var end bool + for i, v := range ids[idx:] { + if count == EntryPerPage { + break + } + if ns, ok := c.normalSockets[v]; ok { + sks = append(sks, ns) + count++ + } + if i == len(ids[idx:])-1 { + end = true + break + } + } + c.mu.RUnlock() + if count == 0 { + end = true + } + var s []*SocketMetric + for _, ns := range sks { + sm := &SocketMetric{} + sm.SocketData = ns.s.ChannelzMetric() + sm.ID = ns.id + sm.RefName = ns.refName + s = append(s, sm) + } + return s, end +} + +func (c *channelMap) GetChannel(id int64) *ChannelMetric { + cm := &ChannelMetric{} + var cn *channel + var ok bool + c.mu.RLock() + if cn, ok = c.channels[id]; !ok { + // channel with id doesn't exist. + c.mu.RUnlock() + return nil + } + cm.NestedChans = copyMap(cn.nestedChans) + cm.SubChans = copyMap(cn.subChans) + c.mu.RUnlock() + cm.ChannelData = cn.c.ChannelzMetric() + cm.ID = cn.id + cm.RefName = cn.refName + return cm +} + +func (c *channelMap) GetSubChannel(id int64) *SubChannelMetric { + cm := &SubChannelMetric{} + var sc *subChannel + var ok bool + c.mu.RLock() + if sc, ok = c.subChannels[id]; !ok { + // subchannel with id doesn't exist. + c.mu.RUnlock() + return nil + } + cm.Sockets = copyMap(sc.sockets) + c.mu.RUnlock() + cm.ChannelData = sc.c.ChannelzMetric() + cm.ID = sc.id + cm.RefName = sc.refName + return cm +} + +func (c *channelMap) GetSocket(id int64) *SocketMetric { + sm := &SocketMetric{} + c.mu.RLock() + if ls, ok := c.listenSockets[id]; ok { + c.mu.RUnlock() + sm.SocketData = ls.s.ChannelzMetric() + sm.ID = ls.id + sm.RefName = ls.refName + return sm + } + if ns, ok := c.normalSockets[id]; ok { + c.mu.RUnlock() + sm.SocketData = ns.s.ChannelzMetric() + sm.ID = ns.id + sm.RefName = ns.refName + return sm + } + c.mu.RUnlock() + return nil +} + +type idGenerator struct { + id int64 +} + +func (i *idGenerator) reset() { + atomic.StoreInt64(&i.id, 0) +} + +func (i *idGenerator) genID() int64 { + return atomic.AddInt64(&i.id, 1) +} diff --git a/vendor/google.golang.org/grpc/channelz/types.go b/vendor/google.golang.org/grpc/channelz/types.go new file mode 100644 index 00000000000..153d75340e4 --- /dev/null +++ b/vendor/google.golang.org/grpc/channelz/types.go @@ -0,0 +1,418 @@ +/* + * + * Copyright 2018 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package channelz + +import ( + "net" + "time" + + "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/grpclog" +) + +// entry represents a node in the channelz database. +type entry interface { + // addChild adds a child e, whose channelz id is id to child list + addChild(id int64, e entry) + // deleteChild deletes a child with channelz id to be id from child list + deleteChild(id int64) + // triggerDelete tries to delete self from channelz database. However, if child + // list is not empty, then deletion from the database is on hold until the last + // child is deleted from database. + triggerDelete() + // deleteSelfIfReady check whether triggerDelete() has been called before, and whether child + // list is now empty. If both conditions are met, then delete self from database. + deleteSelfIfReady() +} + +// dummyEntry is a fake entry to handle entry not found case. +type dummyEntry struct { + idNotFound int64 +} + +func (d *dummyEntry) addChild(id int64, e entry) { + // Note: It is possible for a normal program to reach here under race condition. + // For example, there could be a race between ClientConn.Close() info being propagated + // to addrConn and http2Client. ClientConn.Close() cancel the context and result + // in http2Client to error. The error info is then caught by transport monitor + // and before addrConn.tearDown() is called in side ClientConn.Close(). Therefore, + // the addrConn will create a new transport. And when registering the new transport in + // channelz, its parent addrConn could have already been torn down and deleted + // from channelz tracking, and thus reach the code here. + grpclog.Infof("attempt to add child of type %T with id %d to a parent (id=%d) that doesn't currently exist", e, id, d.idNotFound) +} + +func (d *dummyEntry) deleteChild(id int64) { + // It is possible for a normal program to reach here under race condition. + // Refer to the example described in addChild(). + grpclog.Infof("attempt to delete child with id %d from a parent (id=%d) that doesn't currently exist", id, d.idNotFound) +} + +func (d *dummyEntry) triggerDelete() { + grpclog.Warningf("attempt to delete an entry (id=%d) that doesn't currently exist", d.idNotFound) +} + +func (*dummyEntry) deleteSelfIfReady() { + // code should not reach here. deleteSelfIfReady is always called on an existing entry. +} + +// ChannelMetric defines the info channelz provides for a specific Channel, which +// includes ChannelInternalMetric and channelz-specific data, such as channelz id, +// child list, etc. +type ChannelMetric struct { + // ID is the channelz id of this channel. + ID int64 + // RefName is the human readable reference string of this channel. + RefName string + // ChannelData contains channel internal metric reported by the channel through + // ChannelzMetric(). + ChannelData *ChannelInternalMetric + // NestedChans tracks the nested channel type children of this channel in the format of + // a map from nested channel channelz id to corresponding reference string. + NestedChans map[int64]string + // SubChans tracks the subchannel type children of this channel in the format of a + // map from subchannel channelz id to corresponding reference string. + SubChans map[int64]string + // Sockets tracks the socket type children of this channel in the format of a map + // from socket channelz id to corresponding reference string. + // Note current grpc implementation doesn't allow channel having sockets directly, + // therefore, this is field is unused. + Sockets map[int64]string +} + +// SubChannelMetric defines the info channelz provides for a specific SubChannel, +// which includes ChannelInternalMetric and channelz-specific data, such as +// channelz id, child list, etc. +type SubChannelMetric struct { + // ID is the channelz id of this subchannel. + ID int64 + // RefName is the human readable reference string of this subchannel. + RefName string + // ChannelData contains subchannel internal metric reported by the subchannel + // through ChannelzMetric(). + ChannelData *ChannelInternalMetric + // NestedChans tracks the nested channel type children of this subchannel in the format of + // a map from nested channel channelz id to corresponding reference string. + // Note current grpc implementation doesn't allow subchannel to have nested channels + // as children, therefore, this field is unused. + NestedChans map[int64]string + // SubChans tracks the subchannel type children of this subchannel in the format of a + // map from subchannel channelz id to corresponding reference string. + // Note current grpc implementation doesn't allow subchannel to have subchannels + // as children, therefore, this field is unused. + SubChans map[int64]string + // Sockets tracks the socket type children of this subchannel in the format of a map + // from socket channelz id to corresponding reference string. + Sockets map[int64]string +} + +// ChannelInternalMetric defines the struct that the implementor of Channel interface +// should return from ChannelzMetric(). +type ChannelInternalMetric struct { + // current connectivity state of the channel. + State connectivity.State + // The target this channel originally tried to connect to. May be absent + Target string + // The number of calls started on the channel. + CallsStarted int64 + // The number of calls that have completed with an OK status. + CallsSucceeded int64 + // The number of calls that have a completed with a non-OK status. + CallsFailed int64 + // The last time a call was started on the channel. + LastCallStartedTimestamp time.Time + //TODO: trace +} + +// Channel is the interface that should be satisfied in order to be tracked by +// channelz as Channel or SubChannel. +type Channel interface { + ChannelzMetric() *ChannelInternalMetric +} + +type channel struct { + refName string + c Channel + closeCalled bool + nestedChans map[int64]string + subChans map[int64]string + id int64 + pid int64 + cm *channelMap +} + +func (c *channel) addChild(id int64, e entry) { + switch v := e.(type) { + case *subChannel: + c.subChans[id] = v.refName + case *channel: + c.nestedChans[id] = v.refName + default: + grpclog.Errorf("cannot add a child (id = %d) of type %T to a channel", id, e) + } +} + +func (c *channel) deleteChild(id int64) { + delete(c.subChans, id) + delete(c.nestedChans, id) + c.deleteSelfIfReady() +} + +func (c *channel) triggerDelete() { + c.closeCalled = true + c.deleteSelfIfReady() +} + +func (c *channel) deleteSelfIfReady() { + if !c.closeCalled || len(c.subChans)+len(c.nestedChans) != 0 { + return + } + c.cm.deleteEntry(c.id) + // not top channel + if c.pid != 0 { + c.cm.findEntry(c.pid).deleteChild(c.id) + } +} + +type subChannel struct { + refName string + c Channel + closeCalled bool + sockets map[int64]string + id int64 + pid int64 + cm *channelMap +} + +func (sc *subChannel) addChild(id int64, e entry) { + if v, ok := e.(*normalSocket); ok { + sc.sockets[id] = v.refName + } else { + grpclog.Errorf("cannot add a child (id = %d) of type %T to a subChannel", id, e) + } +} + +func (sc *subChannel) deleteChild(id int64) { + delete(sc.sockets, id) + sc.deleteSelfIfReady() +} + +func (sc *subChannel) triggerDelete() { + sc.closeCalled = true + sc.deleteSelfIfReady() +} + +func (sc *subChannel) deleteSelfIfReady() { + if !sc.closeCalled || len(sc.sockets) != 0 { + return + } + sc.cm.deleteEntry(sc.id) + sc.cm.findEntry(sc.pid).deleteChild(sc.id) +} + +// SocketMetric defines the info channelz provides for a specific Socket, which +// includes SocketInternalMetric and channelz-specific data, such as channelz id, etc. +type SocketMetric struct { + // ID is the channelz id of this socket. + ID int64 + // RefName is the human readable reference string of this socket. + RefName string + // SocketData contains socket internal metric reported by the socket through + // ChannelzMetric(). + SocketData *SocketInternalMetric +} + +// SocketInternalMetric defines the struct that the implementor of Socket interface +// should return from ChannelzMetric(). +type SocketInternalMetric struct { + // The number of streams that have been started. + StreamsStarted int64 + // The number of streams that have ended successfully: + // On client side, receiving frame with eos bit set. + // On server side, sending frame with eos bit set. + StreamsSucceeded int64 + // The number of streams that have ended unsuccessfully: + // On client side, termination without receiving frame with eos bit set. + // On server side, termination without sending frame with eos bit set. + StreamsFailed int64 + // The number of messages successfully sent on this socket. + MessagesSent int64 + MessagesReceived int64 + // The number of keep alives sent. This is typically implemented with HTTP/2 + // ping messages. + KeepAlivesSent int64 + // The last time a stream was created by this endpoint. Usually unset for + // servers. + LastLocalStreamCreatedTimestamp time.Time + // The last time a stream was created by the remote endpoint. Usually unset + // for clients. + LastRemoteStreamCreatedTimestamp time.Time + // The last time a message was sent by this endpoint. + LastMessageSentTimestamp time.Time + // The last time a message was received by this endpoint. + LastMessageReceivedTimestamp time.Time + // The amount of window, granted to the local endpoint by the remote endpoint. + // This may be slightly out of date due to network latency. This does NOT + // include stream level or TCP level flow control info. + LocalFlowControlWindow int64 + // The amount of window, granted to the remote endpoint by the local endpoint. + // This may be slightly out of date due to network latency. This does NOT + // include stream level or TCP level flow control info. + RemoteFlowControlWindow int64 + // The locally bound address. + LocalAddr net.Addr + // The remote bound address. May be absent. + RemoteAddr net.Addr + // Optional, represents the name of the remote endpoint, if different than + // the original target name. + RemoteName string + //TODO: socket options + //TODO: Security +} + +// Socket is the interface that should be satisfied in order to be tracked by +// channelz as Socket. +type Socket interface { + ChannelzMetric() *SocketInternalMetric +} + +type listenSocket struct { + refName string + s Socket + id int64 + pid int64 + cm *channelMap +} + +func (ls *listenSocket) addChild(id int64, e entry) { + grpclog.Errorf("cannot add a child (id = %d) of type %T to a listen socket", id, e) +} + +func (ls *listenSocket) deleteChild(id int64) { + grpclog.Errorf("cannot delete a child (id = %d) from a listen socket", id) +} + +func (ls *listenSocket) triggerDelete() { + ls.cm.deleteEntry(ls.id) + ls.cm.findEntry(ls.pid).deleteChild(ls.id) +} + +func (ls *listenSocket) deleteSelfIfReady() { + grpclog.Errorf("cannot call deleteSelfIfReady on a listen socket") +} + +type normalSocket struct { + refName string + s Socket + id int64 + pid int64 + cm *channelMap +} + +func (ns *normalSocket) addChild(id int64, e entry) { + grpclog.Errorf("cannot add a child (id = %d) of type %T to a normal socket", id, e) +} + +func (ns *normalSocket) deleteChild(id int64) { + grpclog.Errorf("cannot delete a child (id = %d) from a normal socket", id) +} + +func (ns *normalSocket) triggerDelete() { + ns.cm.deleteEntry(ns.id) + ns.cm.findEntry(ns.pid).deleteChild(ns.id) +} + +func (ns *normalSocket) deleteSelfIfReady() { + grpclog.Errorf("cannot call deleteSelfIfReady on a normal socket") +} + +// ServerMetric defines the info channelz provides for a specific Server, which +// includes ServerInternalMetric and channelz-specific data, such as channelz id, +// child list, etc. +type ServerMetric struct { + // ID is the channelz id of this server. + ID int64 + // RefName is the human readable reference string of this server. + RefName string + // ServerData contains server internal metric reported by the server through + // ChannelzMetric(). + ServerData *ServerInternalMetric + // ListenSockets tracks the listener socket type children of this server in the + // format of a map from socket channelz id to corresponding reference string. + ListenSockets map[int64]string +} + +// ServerInternalMetric defines the struct that the implementor of Server interface +// should return from ChannelzMetric(). +type ServerInternalMetric struct { + // The number of incoming calls started on the server. + CallsStarted int64 + // The number of incoming calls that have completed with an OK status. + CallsSucceeded int64 + // The number of incoming calls that have a completed with a non-OK status. + CallsFailed int64 + // The last time a call was started on the server. + LastCallStartedTimestamp time.Time + //TODO: trace +} + +// Server is the interface to be satisfied in order to be tracked by channelz as +// Server. +type Server interface { + ChannelzMetric() *ServerInternalMetric +} + +type server struct { + refName string + s Server + closeCalled bool + sockets map[int64]string + listenSockets map[int64]string + id int64 + cm *channelMap +} + +func (s *server) addChild(id int64, e entry) { + switch v := e.(type) { + case *normalSocket: + s.sockets[id] = v.refName + case *listenSocket: + s.listenSockets[id] = v.refName + default: + grpclog.Errorf("cannot add a child (id = %d) of type %T to a server", id, e) + } +} + +func (s *server) deleteChild(id int64) { + delete(s.sockets, id) + delete(s.listenSockets, id) + s.deleteSelfIfReady() +} + +func (s *server) triggerDelete() { + s.closeCalled = true + s.deleteSelfIfReady() +} + +func (s *server) deleteSelfIfReady() { + if !s.closeCalled || len(s.sockets)+len(s.listenSockets) != 0 { + return + } + s.cm.deleteEntry(s.id) +} diff --git a/vendor/google.golang.org/grpc/clientconn.go b/vendor/google.golang.org/grpc/clientconn.go index 71de2e50d2b..e8d95b43b74 100644 --- a/vendor/google.golang.org/grpc/clientconn.go +++ b/vendor/google.golang.org/grpc/clientconn.go @@ -31,24 +31,50 @@ import ( "golang.org/x/net/context" "golang.org/x/net/trace" "google.golang.org/grpc/balancer" + _ "google.golang.org/grpc/balancer/roundrobin" // To register roundrobin. + "google.golang.org/grpc/channelz" + "google.golang.org/grpc/codes" "google.golang.org/grpc/connectivity" "google.golang.org/grpc/credentials" "google.golang.org/grpc/grpclog" "google.golang.org/grpc/keepalive" "google.golang.org/grpc/resolver" + _ "google.golang.org/grpc/resolver/dns" // To register dns resolver. + _ "google.golang.org/grpc/resolver/passthrough" // To register passthrough resolver. "google.golang.org/grpc/stats" + "google.golang.org/grpc/status" "google.golang.org/grpc/transport" ) +const ( + // minimum time to give a connection to complete + minConnectTimeout = 20 * time.Second +) + var ( // ErrClientConnClosing indicates that the operation is illegal because // the ClientConn is closing. - ErrClientConnClosing = errors.New("grpc: the client connection is closing") - // ErrClientConnTimeout indicates that the ClientConn cannot establish the - // underlying connections within the specified timeout. - // DEPRECATED: Please use context.DeadlineExceeded instead. - ErrClientConnTimeout = errors.New("grpc: timed out when dialing") + // + // Deprecated: this error should not be relied upon by users; use the status + // code of Canceled instead. + ErrClientConnClosing = status.Error(codes.Canceled, "grpc: the client connection is closing") + // errConnDrain indicates that the connection starts to be drained and does not accept any new RPCs. + errConnDrain = errors.New("grpc: the connection is drained") + // errConnClosing indicates that the connection is closing. + errConnClosing = errors.New("grpc: the connection is closing") + // errConnUnavailable indicates that the connection is unavailable. + errConnUnavailable = errors.New("grpc: the connection is unavailable") + // errBalancerClosed indicates that the balancer is closed. + errBalancerClosed = errors.New("grpc: balancer is closed") + // We use an accessor so that minConnectTimeout can be + // atomically read and updated while testing. + getMinConnectTimeout = func() time.Duration { + return minConnectTimeout + } +) +// The following errors are returned from Dial and DialContext +var ( // errNoTransportSecurity indicates that there is no transport security // being set for ClientConn. Users should either set one or explicitly // call WithInsecure DialOption to disable security. @@ -62,16 +88,6 @@ var ( errCredentialsConflict = errors.New("grpc: transport credentials are set for an insecure connection (grpc.WithTransportCredentials() and grpc.WithInsecure() are both called)") // errNetworkIO indicates that the connection is down due to some network I/O error. errNetworkIO = errors.New("grpc: failed with network I/O error") - // errConnDrain indicates that the connection starts to be drained and does not accept any new RPCs. - errConnDrain = errors.New("grpc: the connection is drained") - // errConnClosing indicates that the connection is closing. - errConnClosing = errors.New("grpc: the connection is closing") - // errConnUnavailable indicates that the connection is unavailable. - errConnUnavailable = errors.New("grpc: the connection is unavailable") - // errBalancerClosed indicates that the balancer is closed. - errBalancerClosed = errors.New("grpc: balancer is closed") - // minimum time to give a connection to complete - minConnectTimeout = 20 * time.Second ) // dialOptions configure a Dial call. dialOptions are set by the DialOption @@ -79,7 +95,6 @@ var ( type dialOptions struct { unaryInt UnaryClientInterceptor streamInt StreamClientInterceptor - codec Codec cp Compressor dc Decompressor bs backoffStrategy @@ -89,8 +104,14 @@ type dialOptions struct { scChan <-chan ServiceConfig copts transport.ConnectOptions callOptions []CallOption - // This is to support v1 balancer. + // This is used by v1 balancer dial option WithBalancer to support v1 + // balancer, and also by WithBalancerName dial option. balancerBuilder balancer.Builder + // This is to support grpclb. + resolverBuilder resolver.Builder + waitForHandshake bool + channelzParentID int64 + disableServiceConfig bool } const ( @@ -98,9 +119,24 @@ const ( defaultClientMaxSendMessageSize = math.MaxInt32 ) +// RegisterChannelz turns on channelz service. +// This is an EXPERIMENTAL API. +func RegisterChannelz() { + channelz.TurnOn() +} + // DialOption configures how we set up the connection. type DialOption func(*dialOptions) +// WithWaitForHandshake blocks until the initial settings frame is received from the +// server before assigning RPCs to the connection. +// Experimental API. +func WithWaitForHandshake() DialOption { + return func(o *dialOptions) { + o.waitForHandshake = true + } +} + // WithWriteBufferSize lets you set the size of write buffer, this determines how much data can be batched // before doing a write on the wire. func WithWriteBufferSize(s int) DialOption { @@ -133,7 +169,9 @@ func WithInitialConnWindowSize(s int32) DialOption { } } -// WithMaxMsgSize returns a DialOption which sets the maximum message size the client can receive. Deprecated: use WithDefaultCallOptions(MaxCallRecvMsgSize(s)) instead. +// WithMaxMsgSize returns a DialOption which sets the maximum message size the client can receive. +// +// Deprecated: use WithDefaultCallOptions(MaxCallRecvMsgSize(s)) instead. func WithMaxMsgSize(s int) DialOption { return WithDefaultCallOptions(MaxCallRecvMsgSize(s)) } @@ -146,22 +184,32 @@ func WithDefaultCallOptions(cos ...CallOption) DialOption { } // WithCodec returns a DialOption which sets a codec for message marshaling and unmarshaling. +// +// Deprecated: use WithDefaultCallOptions(CallCustomCodec(c)) instead. func WithCodec(c Codec) DialOption { - return func(o *dialOptions) { - o.codec = c - } + return WithDefaultCallOptions(CallCustomCodec(c)) } -// WithCompressor returns a DialOption which sets a CompressorGenerator for generating message -// compressor. +// WithCompressor returns a DialOption which sets a Compressor to use for +// message compression. It has lower priority than the compressor set by +// the UseCompressor CallOption. +// +// Deprecated: use UseCompressor instead. func WithCompressor(cp Compressor) DialOption { return func(o *dialOptions) { o.cp = cp } } -// WithDecompressor returns a DialOption which sets a DecompressorGenerator for generating -// message decompressor. +// WithDecompressor returns a DialOption which sets a Decompressor to use for +// incoming message decompression. If incoming response messages are encoded +// using the decompressor's Type(), it will be used. Otherwise, the message +// encoding will be used to look up the compressor registered via +// encoding.RegisterCompressor, which will then be used to decompress the +// message. If no compressor is registered for the encoding, an Unimplemented +// status error will be returned. +// +// Deprecated: use encoding.RegisterCompressor instead. func WithDecompressor(dc Decompressor) DialOption { return func(o *dialOptions) { o.dc = dc @@ -170,7 +218,8 @@ func WithDecompressor(dc Decompressor) DialOption { // WithBalancer returns a DialOption which sets a load balancer with the v1 API. // Name resolver will be ignored if this DialOption is specified. -// Deprecated: use the new balancer APIs in balancer package instead. +// +// Deprecated: use the new balancer APIs in balancer package and WithBalancerName. func WithBalancer(b Balancer) DialOption { return func(o *dialOptions) { o.balancerBuilder = &balancerWrapperBuilder{ @@ -179,16 +228,35 @@ func WithBalancer(b Balancer) DialOption { } } -// WithBalancerBuilder is for testing only. Users using custom balancers should -// register their balancer and use service config to choose the balancer to use. -func WithBalancerBuilder(b balancer.Builder) DialOption { - // TODO(bar) remove this when switching balancer is done. +// WithBalancerName sets the balancer that the ClientConn will be initialized +// with. Balancer registered with balancerName will be used. This function +// panics if no balancer was registered by balancerName. +// +// The balancer cannot be overridden by balancer option specified by service +// config. +// +// This is an EXPERIMENTAL API. +func WithBalancerName(balancerName string) DialOption { + builder := balancer.Get(balancerName) + if builder == nil { + panic(fmt.Sprintf("grpc.WithBalancerName: no balancer is registered for name %v", balancerName)) + } + return func(o *dialOptions) { + o.balancerBuilder = builder + } +} + +// withResolverBuilder is only for grpclb. +func withResolverBuilder(b resolver.Builder) DialOption { return func(o *dialOptions) { - o.balancerBuilder = b + o.resolverBuilder = b } } // WithServiceConfig returns a DialOption which has a channel to read the service configuration. +// +// Deprecated: service config should be received through name resolver, as specified here. +// https://github.com/grpc/grpc/blob/master/doc/service_config.md func WithServiceConfig(c <-chan ServiceConfig) DialOption { return func(o *dialOptions) { o.scChan = c @@ -213,7 +281,7 @@ func WithBackoffConfig(b BackoffConfig) DialOption { return withBackoff(b) } -// withBackoff sets the backoff strategy used for retries after a +// withBackoff sets the backoff strategy used for connectRetryNum after a // failed connection attempt. // // This can be exported if arbitrary backoff strategies are allowed by gRPC. @@ -258,6 +326,7 @@ func WithPerRPCCredentials(creds credentials.PerRPCCredentials) DialOption { // WithTimeout returns a DialOption that configures a timeout for dialing a ClientConn // initially. This is valid if and only if WithBlock() is present. +// // Deprecated: use DialContext and context.WithTimeout instead. func WithTimeout(d time.Duration) DialOption { return func(o *dialOptions) { @@ -265,18 +334,23 @@ func WithTimeout(d time.Duration) DialOption { } } +func withContextDialer(f func(context.Context, string) (net.Conn, error)) DialOption { + return func(o *dialOptions) { + o.copts.Dialer = f + } +} + // WithDialer returns a DialOption that specifies a function to use for dialing network addresses. // If FailOnNonTempDialError() is set to true, and an error is returned by f, gRPC checks the error's // Temporary() method to decide if it should try to reconnect to the network address. func WithDialer(f func(string, time.Duration) (net.Conn, error)) DialOption { - return func(o *dialOptions) { - o.copts.Dialer = func(ctx context.Context, addr string) (net.Conn, error) { + return withContextDialer( + func(ctx context.Context, addr string) (net.Conn, error) { if deadline, ok := ctx.Deadline(); ok { return f(addr, deadline.Sub(time.Now())) } return f(addr, 0) - } - } + }) } // WithStatsHandler returns a DialOption that specifies the stats handler @@ -335,15 +409,44 @@ func WithAuthority(a string) DialOption { } } +// WithChannelzParentID returns a DialOption that specifies the channelz ID of current ClientConn's +// parent. This function is used in nested channel creation (e.g. grpclb dial). +func WithChannelzParentID(id int64) DialOption { + return func(o *dialOptions) { + o.channelzParentID = id + } +} + +// WithDisableServiceConfig returns a DialOption that causes grpc to ignore any +// service config provided by the resolver and provides a hint to the resolver +// to not fetch service configs. +func WithDisableServiceConfig() DialOption { + return func(o *dialOptions) { + o.disableServiceConfig = true + } +} + // Dial creates a client connection to the given target. func Dial(target string, opts ...DialOption) (*ClientConn, error) { return DialContext(context.Background(), target, opts...) } -// DialContext creates a client connection to the given target. ctx can be used to -// cancel or expire the pending connection. Once this function returns, the -// cancellation and expiration of ctx will be noop. Users should call ClientConn.Close -// to terminate all the pending operations after this function returns. +// DialContext creates a client connection to the given target. By default, it's +// a non-blocking dial (the function won't wait for connections to be +// established, and connecting happens in the background). To make it a blocking +// dial, use WithBlock() dial option. +// +// In the non-blocking case, the ctx does not act against the connection. It +// only controls the setup steps. +// +// In the blocking case, ctx can be used to cancel or expire the pending +// connection. Once this function returns, the cancellation and expiration of +// ctx will be noop. Users should call ClientConn.Close to terminate all the +// pending operations after this function returns. +// +// The target name syntax is defined in +// https://github.com/grpc/grpc/blob/master/doc/naming.md. +// e.g. to use dns resolver, a "dns:///" prefix should be applied to the target. func DialContext(ctx context.Context, target string, opts ...DialOption) (conn *ClientConn, err error) { cc := &ClientConn{ target: target, @@ -358,6 +461,14 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * opt(&cc.dopts) } + if channelz.IsOn() { + if cc.dopts.channelzParentID != 0 { + cc.channelzID = channelz.RegisterChannel(cc, cc.dopts.channelzParentID, target) + } else { + cc.channelzID = channelz.RegisterChannel(cc, 0, target) + } + } + if !cc.dopts.insecure { if cc.dopts.copts.TransportCredentials == nil { return nil, errNoTransportSecurity @@ -378,7 +489,8 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * if cc.dopts.copts.Dialer == nil { cc.dopts.copts.Dialer = newProxyDialer( func(ctx context.Context, addr string) (net.Conn, error) { - return (&net.Dialer{}).DialContext(ctx, "tcp", addr) + network, addr := parseDialTarget(addr) + return dialContext(ctx, network, addr) }, ) } @@ -419,58 +531,39 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * default: } } - // Set defaults. - if cc.dopts.codec == nil { - cc.dopts.codec = protoCodec{} - } if cc.dopts.bs == nil { cc.dopts.bs = DefaultBackoffConfig } + if cc.dopts.resolverBuilder == nil { + // Only try to parse target when resolver builder is not already set. + cc.parsedTarget = parseTarget(cc.target) + grpclog.Infof("parsed scheme: %q", cc.parsedTarget.Scheme) + cc.dopts.resolverBuilder = resolver.Get(cc.parsedTarget.Scheme) + if cc.dopts.resolverBuilder == nil { + // If resolver builder is still nil, the parse target's scheme is + // not registered. Fallback to default resolver and set Endpoint to + // the original unparsed target. + grpclog.Infof("scheme %q not registered, fallback to default scheme", cc.parsedTarget.Scheme) + cc.parsedTarget = resolver.Target{ + Scheme: resolver.GetDefaultScheme(), + Endpoint: target, + } + cc.dopts.resolverBuilder = resolver.Get(cc.parsedTarget.Scheme) + } + } else { + cc.parsedTarget = resolver.Target{Endpoint: target} + } creds := cc.dopts.copts.TransportCredentials if creds != nil && creds.Info().ServerName != "" { cc.authority = creds.Info().ServerName } else if cc.dopts.insecure && cc.dopts.copts.Authority != "" { cc.authority = cc.dopts.copts.Authority } else { - cc.authority = target + // Use endpoint from "scheme://authority/endpoint" as the default + // authority for ClientConn. + cc.authority = cc.parsedTarget.Endpoint } - if cc.dopts.balancerBuilder != nil { - var credsClone credentials.TransportCredentials - if creds != nil { - credsClone = creds.Clone() - } - buildOpts := balancer.BuildOptions{ - DialCreds: credsClone, - Dialer: cc.dopts.copts.Dialer, - } - // Build should not take long time. So it's ok to not have a goroutine for it. - // TODO(bar) init balancer after first resolver result to support service config balancer. - cc.balancerWrapper = newCCBalancerWrapper(cc, cc.dopts.balancerBuilder, buildOpts) - } else { - waitC := make(chan error, 1) - go func() { - defer close(waitC) - // No balancer, or no resolver within the balancer. Connect directly. - ac, err := cc.newAddrConn([]resolver.Address{{Addr: target}}) - if err != nil { - waitC <- err - return - } - if err := ac.connect(cc.dopts.block); err != nil { - waitC <- err - return - } - }() - select { - case <-ctx.Done(): - return nil, ctx.Err() - case err := <-waitC: - if err != nil { - return nil, err - } - } - } if cc.dopts.scChan != nil && !scSet { // Blocking wait for the initial service config. select { @@ -486,19 +579,29 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * go cc.scWatcher() } + var credsClone credentials.TransportCredentials + if creds := cc.dopts.copts.TransportCredentials; creds != nil { + credsClone = creds.Clone() + } + cc.balancerBuildOpts = balancer.BuildOptions{ + DialCreds: credsClone, + Dialer: cc.dopts.copts.Dialer, + ChannelzParentID: cc.channelzID, + } + // Build the resolver. cc.resolverWrapper, err = newCCResolverWrapper(cc) if err != nil { return nil, fmt.Errorf("failed to build resolver: %v", err) } - - if cc.balancerWrapper != nil && cc.resolverWrapper == nil { - // TODO(bar) there should always be a resolver (DNS as the default). - // Unblock balancer initialization with a fake resolver update if there's no resolver. - // The balancer wrapper will not read the addresses, so an empty list works. - // TODO(bar) remove this after the real resolver is started. - cc.balancerWrapper.handleResolvedAddrs([]resolver.Address{}, nil) - } + // Start the resolver wrapper goroutine after resolverWrapper is created. + // + // If the goroutine is started before resolverWrapper is ready, the + // following may happen: The goroutine sends updates to cc. cc forwards + // those to balancer. Balancer creates new addrConn. addrConn fails to + // connect, and calls resolveNow(). resolveNow() tries to use the non-ready + // resolverWrapper. + cc.resolverWrapper.start() // A blocking dial blocks until the clientConn is ready. if cc.dopts.block { @@ -565,21 +668,33 @@ type ClientConn struct { ctx context.Context cancel context.CancelFunc - target string - authority string - dopts dialOptions - csMgr *connectivityStateManager - - balancerWrapper *ccBalancerWrapper - resolverWrapper *ccResolverWrapper + target string + parsedTarget resolver.Target + authority string + dopts dialOptions + csMgr *connectivityStateManager - blockingpicker *pickerWrapper + balancerBuildOpts balancer.BuildOptions + resolverWrapper *ccResolverWrapper + blockingpicker *pickerWrapper mu sync.RWMutex sc ServiceConfig + scRaw string conns map[*addrConn]struct{} // Keepalive parameter can be updated if a GoAway is received. - mkp keepalive.ClientParameters + mkp keepalive.ClientParameters + curBalancerName string + preBalancerName string // previous balancer name. + curAddresses []resolver.Address + balancerWrapper *ccBalancerWrapper + + channelzID int64 // channelz unique identification number + czmu sync.RWMutex + callsStarted int64 + callsSucceeded int64 + callsFailed int64 + lastCallStartedTime time.Time } // WaitForStateChange waits until the connectivity.State of ClientConn changes from sourceState or @@ -615,6 +730,7 @@ func (cc *ClientConn) scWatcher() { // TODO: load balance policy runtime change is ignored. // We may revist this decision in the future. cc.sc = sc + cc.scRaw = "" cc.mu.Unlock() case <-cc.ctx.Done(): return @@ -622,7 +738,115 @@ func (cc *ClientConn) scWatcher() { } } +func (cc *ClientConn) handleResolvedAddrs(addrs []resolver.Address, err error) { + cc.mu.Lock() + defer cc.mu.Unlock() + if cc.conns == nil { + // cc was closed. + return + } + + if reflect.DeepEqual(cc.curAddresses, addrs) { + return + } + + cc.curAddresses = addrs + + if cc.dopts.balancerBuilder == nil { + // Only look at balancer types and switch balancer if balancer dial + // option is not set. + var isGRPCLB bool + for _, a := range addrs { + if a.Type == resolver.GRPCLB { + isGRPCLB = true + break + } + } + var newBalancerName string + if isGRPCLB { + newBalancerName = grpclbName + } else { + // Address list doesn't contain grpclb address. Try to pick a + // non-grpclb balancer. + newBalancerName = cc.curBalancerName + // If current balancer is grpclb, switch to the previous one. + if newBalancerName == grpclbName { + newBalancerName = cc.preBalancerName + } + // The following could be true in two cases: + // - the first time handling resolved addresses + // (curBalancerName="") + // - the first time handling non-grpclb addresses + // (curBalancerName="grpclb", preBalancerName="") + if newBalancerName == "" { + newBalancerName = PickFirstBalancerName + } + } + cc.switchBalancer(newBalancerName) + } else if cc.balancerWrapper == nil { + // Balancer dial option was set, and this is the first time handling + // resolved addresses. Build a balancer with dopts.balancerBuilder. + cc.balancerWrapper = newCCBalancerWrapper(cc, cc.dopts.balancerBuilder, cc.balancerBuildOpts) + } + + cc.balancerWrapper.handleResolvedAddrs(addrs, nil) +} + +// switchBalancer starts the switching from current balancer to the balancer +// with the given name. +// +// It will NOT send the current address list to the new balancer. If needed, +// caller of this function should send address list to the new balancer after +// this function returns. +// +// Caller must hold cc.mu. +func (cc *ClientConn) switchBalancer(name string) { + if cc.conns == nil { + return + } + + if strings.ToLower(cc.curBalancerName) == strings.ToLower(name) { + return + } + + grpclog.Infof("ClientConn switching balancer to %q", name) + if cc.dopts.balancerBuilder != nil { + grpclog.Infoln("ignoring balancer switching: Balancer DialOption used instead") + return + } + // TODO(bar switching) change this to two steps: drain and close. + // Keep track of sc in wrapper. + if cc.balancerWrapper != nil { + cc.balancerWrapper.close() + } + // Clear all stickiness state. + cc.blockingpicker.clearStickinessState() + + builder := balancer.Get(name) + if builder == nil { + grpclog.Infof("failed to get balancer builder for: %v, using pick_first instead", name) + builder = newPickfirstBuilder() + } + cc.preBalancerName = cc.curBalancerName + cc.curBalancerName = builder.Name() + cc.balancerWrapper = newCCBalancerWrapper(cc, builder, cc.balancerBuildOpts) +} + +func (cc *ClientConn) handleSubConnStateChange(sc balancer.SubConn, s connectivity.State) { + cc.mu.Lock() + if cc.conns == nil { + cc.mu.Unlock() + return + } + // TODO(bar switching) send updates to all balancer wrappers when balancer + // gracefully switching is supported. + cc.balancerWrapper.handleSubConnStateChange(sc, s) + cc.mu.Unlock() +} + // newAddrConn creates an addrConn for addrs and adds it to cc.conns. +// +// Caller needs to make sure len(addrs) > 0. func (cc *ClientConn) newAddrConn(addrs []resolver.Address) (*addrConn, error) { ac := &addrConn{ cc: cc, @@ -636,6 +860,9 @@ func (cc *ClientConn) newAddrConn(addrs []resolver.Address) (*addrConn, error) { cc.mu.Unlock() return nil, ErrClientConnClosing } + if channelz.IsOn() { + ac.channelzID = channelz.RegisterSubChannel(ac, cc.channelzID, "") + } cc.conns[ac] = struct{}{} cc.mu.Unlock() return ac, nil @@ -654,12 +881,48 @@ func (cc *ClientConn) removeAddrConn(ac *addrConn, err error) { ac.tearDown(err) } +// ChannelzMetric returns ChannelInternalMetric of current ClientConn. +// This is an EXPERIMENTAL API. +func (cc *ClientConn) ChannelzMetric() *channelz.ChannelInternalMetric { + state := cc.GetState() + cc.czmu.RLock() + defer cc.czmu.RUnlock() + return &channelz.ChannelInternalMetric{ + State: state, + Target: cc.target, + CallsStarted: cc.callsStarted, + CallsSucceeded: cc.callsSucceeded, + CallsFailed: cc.callsFailed, + LastCallStartedTimestamp: cc.lastCallStartedTime, + } +} + +func (cc *ClientConn) incrCallsStarted() { + cc.czmu.Lock() + cc.callsStarted++ + // TODO(yuxuanli): will make this a time.Time pointer improve performance? + cc.lastCallStartedTime = time.Now() + cc.czmu.Unlock() +} + +func (cc *ClientConn) incrCallsSucceeded() { + cc.czmu.Lock() + cc.callsSucceeded++ + cc.czmu.Unlock() +} + +func (cc *ClientConn) incrCallsFailed() { + cc.czmu.Lock() + cc.callsFailed++ + cc.czmu.Unlock() +} + // connect starts to creating transport and also starts the transport monitor // goroutine for this ac. // It does nothing if the ac is not IDLE. // TODO(bar) Move this to the addrConn section. // This was part of resetAddrConn, keep it here to make the diff look clean. -func (ac *addrConn) connect(block bool) error { +func (ac *addrConn) connect() error { ac.mu.Lock() if ac.state == connectivity.Shutdown { ac.mu.Unlock() @@ -670,39 +933,21 @@ func (ac *addrConn) connect(block bool) error { return nil } ac.state = connectivity.Connecting - if ac.cc.balancerWrapper != nil { - ac.cc.balancerWrapper.handleSubConnStateChange(ac.acbw, ac.state) - } else { - ac.cc.csMgr.updateState(ac.state) - } + ac.cc.handleSubConnStateChange(ac.acbw, ac.state) ac.mu.Unlock() - if block { + // Start a goroutine connecting to the server asynchronously. + go func() { if err := ac.resetTransport(); err != nil { + grpclog.Warningf("Failed to dial %s: %v; please retry.", ac.addrs[0].Addr, err) if err != errConnClosing { + // Keep this ac in cc.conns, to get the reason it's torn down. ac.tearDown(err) } - if e, ok := err.(transport.ConnectionError); ok && !e.Temporary() { - return e.Origin() - } - return err + return } - // Start to monitor the error status of transport. - go ac.transportMonitor() - } else { - // Start a goroutine connecting to the server asynchronously. - go func() { - if err := ac.resetTransport(); err != nil { - grpclog.Warningf("Failed to dial %s: %v; please retry.", ac.addrs[0].Addr, err) - if err != errConnClosing { - // Keep this ac in cc.conns, to get the reason it's torn down. - ac.tearDown(err) - } - return - } - ac.transportMonitor() - }() - } + ac.transportMonitor() + }() return nil } @@ -731,6 +976,7 @@ func (ac *addrConn) tryUpdateAddrs(addrs []resolver.Address) bool { grpclog.Infof("addrConn: tryUpdateAddrs curAddrFound: %v", curAddrFound) if curAddrFound { ac.addrs = addrs + ac.reconnectIdx = 0 // Start reconnecting from beginning in the new list. } return curAddrFound @@ -741,7 +987,7 @@ func (ac *addrConn) tryUpdateAddrs(addrs []resolver.Address) bool { // the corresponding MethodConfig. // If there isn't an exact match for the input method, we look for the default config // under the service (i.e /service/). If there is a default MethodConfig for -// the serivce, we return it. +// the service, we return it. // Otherwise, we return an empty MethodConfig. func (cc *ClientConn) GetMethodConfig(method string) MethodConfig { // TODO: Avoid the locking here. @@ -750,37 +996,12 @@ func (cc *ClientConn) GetMethodConfig(method string) MethodConfig { m, ok := cc.sc.Methods[method] if !ok { i := strings.LastIndex(method, "/") - m, _ = cc.sc.Methods[method[:i+1]] + m = cc.sc.Methods[method[:i+1]] } return m } func (cc *ClientConn) getTransport(ctx context.Context, failfast bool) (transport.ClientTransport, func(balancer.DoneInfo), error) { - if cc.balancerWrapper == nil { - // If balancer is nil, there should be only one addrConn available. - cc.mu.RLock() - if cc.conns == nil { - cc.mu.RUnlock() - // TODO this function returns toRPCErr and non-toRPCErr. Clean up - // the errors in ClientConn. - return nil, nil, toRPCErr(ErrClientConnClosing) - } - var ac *addrConn - for ac = range cc.conns { - // Break after the first iteration to get the first addrConn. - break - } - cc.mu.RUnlock() - if ac == nil { - return nil, nil, errConnClosing - } - t, err := ac.wait(ctx, false /*hasBalancer*/, failfast) - if err != nil { - return nil, nil, err - } - return t, nil, nil - } - t, done, err := cc.blockingpicker.pick(ctx, failfast, balancer.PickOptions{}) if err != nil { return nil, nil, toRPCErr(err) @@ -788,9 +1009,61 @@ func (cc *ClientConn) getTransport(ctx context.Context, failfast bool) (transpor return t, done, nil } +// handleServiceConfig parses the service config string in JSON format to Go native +// struct ServiceConfig, and store both the struct and the JSON string in ClientConn. +func (cc *ClientConn) handleServiceConfig(js string) error { + if cc.dopts.disableServiceConfig { + return nil + } + sc, err := parseServiceConfig(js) + if err != nil { + return err + } + cc.mu.Lock() + cc.scRaw = js + cc.sc = sc + if sc.LB != nil && *sc.LB != grpclbName { // "grpclb" is not a valid balancer option in service config. + if cc.curBalancerName == grpclbName { + // If current balancer is grpclb, there's at least one grpclb + // balancer address in the resolved list. Don't switch the balancer, + // but change the previous balancer name, so if a new resolved + // address list doesn't contain grpclb address, balancer will be + // switched to *sc.LB. + cc.preBalancerName = *sc.LB + } else { + cc.switchBalancer(*sc.LB) + cc.balancerWrapper.handleResolvedAddrs(cc.curAddresses, nil) + } + } + + if envConfigStickinessOn { + var newStickinessMDKey string + if sc.stickinessMetadataKey != nil && *sc.stickinessMetadataKey != "" { + newStickinessMDKey = *sc.stickinessMetadataKey + } + // newStickinessMDKey is "" if one of the following happens: + // - stickinessMetadataKey is set to "" + // - stickinessMetadataKey field doesn't exist in service config + cc.blockingpicker.updateStickinessMDKey(strings.ToLower(newStickinessMDKey)) + } + + cc.mu.Unlock() + return nil +} + +func (cc *ClientConn) resolveNow(o resolver.ResolveNowOption) { + cc.mu.Lock() + r := cc.resolverWrapper + cc.mu.Unlock() + if r == nil { + return + } + go r.resolveNow(o) +} + // Close tears down the ClientConn and all underlying connections. func (cc *ClientConn) Close() error { - cc.cancel() + defer cc.cancel() cc.mu.Lock() if cc.conns == nil { @@ -800,17 +1073,28 @@ func (cc *ClientConn) Close() error { conns := cc.conns cc.conns = nil cc.csMgr.updateState(connectivity.Shutdown) + + rWrapper := cc.resolverWrapper + cc.resolverWrapper = nil + bWrapper := cc.balancerWrapper + cc.balancerWrapper = nil cc.mu.Unlock() + cc.blockingpicker.close() - if cc.resolverWrapper != nil { - cc.resolverWrapper.close() + + if rWrapper != nil { + rWrapper.close() } - if cc.balancerWrapper != nil { - cc.balancerWrapper.close() + if bWrapper != nil { + bWrapper.close() } + for ac := range conns { ac.tearDown(ErrClientConnClosing) } + if channelz.IsOn() { + channelz.RemoveEntry(cc.channelzID) + } return nil } @@ -819,15 +1103,16 @@ type addrConn struct { ctx context.Context cancel context.CancelFunc - cc *ClientConn - curAddr resolver.Address - addrs []resolver.Address - dopts dialOptions - events trace.EventLog - acbw balancer.SubConn + cc *ClientConn + addrs []resolver.Address + dopts dialOptions + events trace.EventLog + acbw balancer.SubConn - mu sync.Mutex - state connectivity.State + mu sync.Mutex + curAddr resolver.Address + reconnectIdx int // The index in addrs list to start reconnecting from. + state connectivity.State // ready is closed and becomes nil when a new transport is up or failed // due to timeout. ready chan struct{} @@ -835,13 +1120,28 @@ type addrConn struct { // The reason this addrConn is torn down. tearDownErr error + + connectRetryNum int + // backoffDeadline is the time until which resetTransport needs to + // wait before increasing connectRetryNum count. + backoffDeadline time.Time + // connectDeadline is the time by which all connection + // negotiations must complete. + connectDeadline time.Time + + channelzID int64 // channelz unique identification number + czmu sync.RWMutex + callsStarted int64 + callsSucceeded int64 + callsFailed int64 + lastCallStartedTime time.Time } // adjustParams updates parameters used to create transports upon // receiving a GoAway. func (ac *addrConn) adjustParams(r transport.GoAwayReason) { switch r { - case transport.TooManyPings: + case transport.GoAwayTooManyPings: v := 2 * ac.dopts.copts.KeepaliveParams.Time ac.cc.mu.Lock() if v > ac.cc.mkp.Time { @@ -869,6 +1169,15 @@ func (ac *addrConn) errorf(format string, a ...interface{}) { // resetTransport recreates a transport to the address for ac. The old // transport will close itself on error or when the clientconn is closed. +// The created transport must receive initial settings frame from the server. +// In case that doesn't happen, transportMonitor will kill the newly created +// transport after connectDeadline has expired. +// In case there was an error on the transport before the settings frame was +// received, resetTransport resumes connecting to backends after the one that +// was previously connected to. In case end of the list is reached, resetTransport +// backs off until the original deadline. +// If the DialOption WithWaitForHandshake was set, resetTrasport returns +// successfully only after server settings are received. // // TODO(bar) make sure all state transitions are valid. func (ac *addrConn) resetTransport() error { @@ -882,19 +1191,38 @@ func (ac *addrConn) resetTransport() error { ac.ready = nil } ac.transport = nil - ac.curAddr = resolver.Address{} + ridx := ac.reconnectIdx ac.mu.Unlock() ac.cc.mu.RLock() ac.dopts.copts.KeepaliveParams = ac.cc.mkp ac.cc.mu.RUnlock() - for retries := 0; ; retries++ { - sleepTime := ac.dopts.bs.backoff(retries) - timeout := minConnectTimeout + var backoffDeadline, connectDeadline time.Time + for connectRetryNum := 0; ; connectRetryNum++ { ac.mu.Lock() - if timeout < time.Duration(int(sleepTime)/len(ac.addrs)) { - timeout = time.Duration(int(sleepTime) / len(ac.addrs)) + if ac.backoffDeadline.IsZero() { + // This means either a successful HTTP2 connection was established + // or this is the first time this addrConn is trying to establish a + // connection. + backoffFor := ac.dopts.bs.backoff(connectRetryNum) // time.Duration. + // This will be the duration that dial gets to finish. + dialDuration := getMinConnectTimeout() + if backoffFor > dialDuration { + // Give dial more time as we keep failing to connect. + dialDuration = backoffFor + } + start := time.Now() + backoffDeadline = start.Add(backoffFor) + connectDeadline = start.Add(dialDuration) + ridx = 0 // Start connecting from the beginning. + } else { + // Continue trying to connect with the same deadlines. + connectRetryNum = ac.connectRetryNum + backoffDeadline = ac.backoffDeadline + connectDeadline = ac.connectDeadline + ac.backoffDeadline = time.Time{} + ac.connectDeadline = time.Time{} + ac.connectRetryNum = 0 } - connectTime := time.Now() if ac.state == connectivity.Shutdown { ac.mu.Unlock() return errConnClosing @@ -902,116 +1230,178 @@ func (ac *addrConn) resetTransport() error { ac.printf("connecting") if ac.state != connectivity.Connecting { ac.state = connectivity.Connecting - // TODO(bar) remove condition once we always have a balancer. - if ac.cc.balancerWrapper != nil { - ac.cc.balancerWrapper.handleSubConnStateChange(ac.acbw, ac.state) - } else { - ac.cc.csMgr.updateState(ac.state) - } + ac.cc.handleSubConnStateChange(ac.acbw, ac.state) } // copy ac.addrs in case of race addrsIter := make([]resolver.Address, len(ac.addrs)) copy(addrsIter, ac.addrs) copts := ac.dopts.copts ac.mu.Unlock() - for _, addr := range addrsIter { + connected, err := ac.createTransport(connectRetryNum, ridx, backoffDeadline, connectDeadline, addrsIter, copts) + if err != nil { + return err + } + if connected { + return nil + } + } +} + +// createTransport creates a connection to one of the backends in addrs. +// It returns true if a connection was established. +func (ac *addrConn) createTransport(connectRetryNum, ridx int, backoffDeadline, connectDeadline time.Time, addrs []resolver.Address, copts transport.ConnectOptions) (bool, error) { + for i := ridx; i < len(addrs); i++ { + addr := addrs[i] + target := transport.TargetInfo{ + Addr: addr.Addr, + Metadata: addr.Metadata, + Authority: ac.cc.authority, + } + done := make(chan struct{}) + onPrefaceReceipt := func() { ac.mu.Lock() - if ac.state == connectivity.Shutdown { - // ac.tearDown(...) has been invoked. - ac.mu.Unlock() - return errConnClosing + close(done) + if !ac.backoffDeadline.IsZero() { + // If we haven't already started reconnecting to + // other backends. + // Note, this can happen when writer notices an error + // and triggers resetTransport while at the same time + // reader receives the preface and invokes this closure. + ac.backoffDeadline = time.Time{} + ac.connectDeadline = time.Time{} + ac.connectRetryNum = 0 } ac.mu.Unlock() - sinfo := transport.TargetInfo{ - Addr: addr.Addr, - Metadata: addr.Metadata, - } - newTransport, err := transport.NewClientTransport(ac.cc.ctx, sinfo, copts, timeout) - if err != nil { - if e, ok := err.(transport.ConnectionError); ok && !e.Temporary() { - ac.mu.Lock() - if ac.state != connectivity.Shutdown { - ac.state = connectivity.TransientFailure - if ac.cc.balancerWrapper != nil { - ac.cc.balancerWrapper.handleSubConnStateChange(ac.acbw, ac.state) - } else { - ac.cc.csMgr.updateState(ac.state) - } - } - ac.mu.Unlock() - return err - } - grpclog.Warningf("grpc: addrConn.resetTransport failed to create client transport: %v; Reconnecting to %v", err, addr) - ac.mu.Lock() - if ac.state == connectivity.Shutdown { - // ac.tearDown(...) has been invoked. - ac.mu.Unlock() - return errConnClosing - } - ac.mu.Unlock() - continue - } + } + // Do not cancel in the success path because of + // this issue in Go1.6: https://github.com/golang/go/issues/15078. + connectCtx, cancel := context.WithDeadline(ac.ctx, connectDeadline) + if channelz.IsOn() { + copts.ChannelzParentID = ac.channelzID + } + newTr, err := transport.NewClientTransport(connectCtx, ac.cc.ctx, target, copts, onPrefaceReceipt) + if err != nil { + cancel() + ac.cc.blockingpicker.updateConnectionError(err) ac.mu.Lock() - ac.printf("ready") if ac.state == connectivity.Shutdown { // ac.tearDown(...) has been invoked. ac.mu.Unlock() - newTransport.Close() - return errConnClosing - } - ac.state = connectivity.Ready - if ac.cc.balancerWrapper != nil { - ac.cc.balancerWrapper.handleSubConnStateChange(ac.acbw, ac.state) - } else { - ac.cc.csMgr.updateState(ac.state) - } - t := ac.transport - ac.transport = newTransport - if t != nil { - t.Close() - } - ac.curAddr = addr - if ac.ready != nil { - close(ac.ready) - ac.ready = nil + return false, errConnClosing } ac.mu.Unlock() - return nil + grpclog.Warningf("grpc: addrConn.createTransport failed to connect to %v. Err :%v. Reconnecting...", addr, err) + continue + } + if ac.dopts.waitForHandshake { + select { + case <-done: + case <-connectCtx.Done(): + // Didn't receive server preface, must kill this new transport now. + grpclog.Warningf("grpc: addrConn.createTransport failed to receive server preface before deadline.") + newTr.Close() + break + case <-ac.ctx.Done(): + } } ac.mu.Lock() - ac.state = connectivity.TransientFailure - if ac.cc.balancerWrapper != nil { - ac.cc.balancerWrapper.handleSubConnStateChange(ac.acbw, ac.state) - } else { - ac.cc.csMgr.updateState(ac.state) + if ac.state == connectivity.Shutdown { + ac.mu.Unlock() + // ac.tearDonn(...) has been invoked. + newTr.Close() + return false, errConnClosing } + ac.printf("ready") + ac.state = connectivity.Ready + ac.cc.handleSubConnStateChange(ac.acbw, ac.state) + ac.transport = newTr + ac.curAddr = addr if ac.ready != nil { close(ac.ready) ac.ready = nil } - ac.mu.Unlock() - timer := time.NewTimer(sleepTime - time.Since(connectTime)) select { - case <-timer.C: - case <-ac.ctx.Done(): - timer.Stop() - return ac.ctx.Err() + case <-done: + // If the server has responded back with preface already, + // don't set the reconnect parameters. + default: + ac.connectRetryNum = connectRetryNum + ac.backoffDeadline = backoffDeadline + ac.connectDeadline = connectDeadline + ac.reconnectIdx = i + 1 // Start reconnecting from the next backend in the list. } + ac.mu.Unlock() + return true, nil + } + ac.mu.Lock() + if ac.state == connectivity.Shutdown { + ac.mu.Unlock() + return false, errConnClosing + } + ac.state = connectivity.TransientFailure + ac.cc.handleSubConnStateChange(ac.acbw, ac.state) + ac.cc.resolveNow(resolver.ResolveNowOption{}) + if ac.ready != nil { + close(ac.ready) + ac.ready = nil + } + ac.mu.Unlock() + timer := time.NewTimer(backoffDeadline.Sub(time.Now())) + select { + case <-timer.C: + case <-ac.ctx.Done(): timer.Stop() + return false, ac.ctx.Err() } + return false, nil } // Run in a goroutine to track the error in transport and create the // new transport if an error happens. It returns when the channel is closing. func (ac *addrConn) transportMonitor() { for { + var timer *time.Timer + var cdeadline <-chan time.Time ac.mu.Lock() t := ac.transport + if !ac.connectDeadline.IsZero() { + timer = time.NewTimer(ac.connectDeadline.Sub(time.Now())) + cdeadline = timer.C + } ac.mu.Unlock() // Block until we receive a goaway or an error occurs. select { case <-t.GoAway(): + done := t.Error() + cleanup := t.Close + // Since this transport will be orphaned (won't have a transportMonitor) + // we need to launch a goroutine to keep track of clientConn.Close() + // happening since it might not be noticed by any other goroutine for a while. + go func() { + <-done + cleanup() + }() case <-t.Error(): + // In case this is triggered because clientConn.Close() + // was called, we want to immeditately close the transport + // since no other goroutine might notice it for a while. + t.Close() + case <-cdeadline: + ac.mu.Lock() + // This implies that client received server preface. + if ac.backoffDeadline.IsZero() { + ac.mu.Unlock() + continue + } + ac.mu.Unlock() + timer = nil + // No server preface received until deadline. + // Kill the connection. + grpclog.Warningf("grpc: addrConn.transportMonitor didn't get server preface after waiting. Closing the new transport now.") + t.Close() + } + if timer != nil { + timer.Stop() } // If a GoAway happened, regardless of error, adjust our keepalive // parameters as appropriate. @@ -1028,11 +1418,8 @@ func (ac *addrConn) transportMonitor() { // Set connectivity state to TransientFailure before calling // resetTransport. Transition READY->CONNECTING is not valid. ac.state = connectivity.TransientFailure - if ac.cc.balancerWrapper != nil { - ac.cc.balancerWrapper.handleSubConnStateChange(ac.acbw, ac.state) - } else { - ac.cc.csMgr.updateState(ac.state) - } + ac.cc.handleSubConnStateChange(ac.acbw, ac.state) + ac.cc.resolveNow(resolver.ResolveNowOption{}) ac.curAddr = resolver.Address{} ac.mu.Unlock() if err := ac.resetTransport(); err != nil { @@ -1106,7 +1493,7 @@ func (ac *addrConn) getReadyTransport() (transport.ClientTransport, bool) { ac.mu.Unlock() // Trigger idle ac to connect. if idle { - ac.connect(false) + ac.connect() } return nil, false } @@ -1119,8 +1506,11 @@ func (ac *addrConn) getReadyTransport() (transport.ClientTransport, bool) { func (ac *addrConn) tearDown(err error) { ac.cancel() ac.mu.Lock() - ac.curAddr = resolver.Address{} defer ac.mu.Unlock() + if ac.state == connectivity.Shutdown { + return + } + ac.curAddr = resolver.Address{} if err == errConnDrain && ac.transport != nil { // GracefulClose(...) may be executed multiple times when // i) receiving multiple GoAway frames from the server; or @@ -1128,16 +1518,9 @@ func (ac *addrConn) tearDown(err error) { // address removal and GoAway. ac.transport.GracefulClose() } - if ac.state == connectivity.Shutdown { - return - } ac.state = connectivity.Shutdown ac.tearDownErr = err - if ac.cc.balancerWrapper != nil { - ac.cc.balancerWrapper.handleSubConnStateChange(ac.acbw, ac.state) - } else { - ac.cc.csMgr.updateState(ac.state) - } + ac.cc.handleSubConnStateChange(ac.acbw, ac.state) if ac.events != nil { ac.events.Finish() ac.events = nil @@ -1146,7 +1529,9 @@ func (ac *addrConn) tearDown(err error) { close(ac.ready) ac.ready = nil } - return + if channelz.IsOn() { + channelz.RemoveEntry(ac.channelzID) + } } func (ac *addrConn) getState() connectivity.State { @@ -1154,3 +1539,53 @@ func (ac *addrConn) getState() connectivity.State { defer ac.mu.Unlock() return ac.state } + +func (ac *addrConn) getCurAddr() (ret resolver.Address) { + ac.mu.Lock() + ret = ac.curAddr + ac.mu.Unlock() + return +} + +func (ac *addrConn) ChannelzMetric() *channelz.ChannelInternalMetric { + ac.mu.Lock() + addr := ac.curAddr.Addr + ac.mu.Unlock() + state := ac.getState() + ac.czmu.RLock() + defer ac.czmu.RUnlock() + return &channelz.ChannelInternalMetric{ + State: state, + Target: addr, + CallsStarted: ac.callsStarted, + CallsSucceeded: ac.callsSucceeded, + CallsFailed: ac.callsFailed, + LastCallStartedTimestamp: ac.lastCallStartedTime, + } +} + +func (ac *addrConn) incrCallsStarted() { + ac.czmu.Lock() + ac.callsStarted++ + ac.lastCallStartedTime = time.Now() + ac.czmu.Unlock() +} + +func (ac *addrConn) incrCallsSucceeded() { + ac.czmu.Lock() + ac.callsSucceeded++ + ac.czmu.Unlock() +} + +func (ac *addrConn) incrCallsFailed() { + ac.czmu.Lock() + ac.callsFailed++ + ac.czmu.Unlock() +} + +// ErrClientConnTimeout indicates that the ClientConn cannot establish the +// underlying connections within the specified timeout. +// +// Deprecated: This error is never returned by grpc and should not be +// referenced by users. +var ErrClientConnTimeout = errors.New("grpc: timed out when dialing") diff --git a/vendor/google.golang.org/grpc/codec.go b/vendor/google.golang.org/grpc/codec.go index 905b048e2ac..12977654781 100644 --- a/vendor/google.golang.org/grpc/codec.go +++ b/vendor/google.golang.org/grpc/codec.go @@ -19,86 +19,32 @@ package grpc import ( - "math" - "sync" - - "github.com/golang/protobuf/proto" + "google.golang.org/grpc/encoding" + _ "google.golang.org/grpc/encoding/proto" // to register the Codec for "proto" ) +// baseCodec contains the functionality of both Codec and encoding.Codec, but +// omits the name/string, which vary between the two and are not needed for +// anything besides the registry in the encoding package. +type baseCodec interface { + Marshal(v interface{}) ([]byte, error) + Unmarshal(data []byte, v interface{}) error +} + +var _ baseCodec = Codec(nil) +var _ baseCodec = encoding.Codec(nil) + // Codec defines the interface gRPC uses to encode and decode messages. // Note that implementations of this interface must be thread safe; // a Codec's methods can be called from concurrent goroutines. +// +// Deprecated: use encoding.Codec instead. type Codec interface { // Marshal returns the wire format of v. Marshal(v interface{}) ([]byte, error) // Unmarshal parses the wire format into v. Unmarshal(data []byte, v interface{}) error - // String returns the name of the Codec implementation. The returned - // string will be used as part of content type in transmission. + // String returns the name of the Codec implementation. This is unused by + // gRPC. String() string } - -// protoCodec is a Codec implementation with protobuf. It is the default codec for gRPC. -type protoCodec struct { -} - -type cachedProtoBuffer struct { - lastMarshaledSize uint32 - proto.Buffer -} - -func capToMaxInt32(val int) uint32 { - if val > math.MaxInt32 { - return uint32(math.MaxInt32) - } - return uint32(val) -} - -func (p protoCodec) marshal(v interface{}, cb *cachedProtoBuffer) ([]byte, error) { - protoMsg := v.(proto.Message) - newSlice := make([]byte, 0, cb.lastMarshaledSize) - - cb.SetBuf(newSlice) - cb.Reset() - if err := cb.Marshal(protoMsg); err != nil { - return nil, err - } - out := cb.Bytes() - cb.lastMarshaledSize = capToMaxInt32(len(out)) - return out, nil -} - -func (p protoCodec) Marshal(v interface{}) ([]byte, error) { - cb := protoBufferPool.Get().(*cachedProtoBuffer) - out, err := p.marshal(v, cb) - - // put back buffer and lose the ref to the slice - cb.SetBuf(nil) - protoBufferPool.Put(cb) - return out, err -} - -func (p protoCodec) Unmarshal(data []byte, v interface{}) error { - cb := protoBufferPool.Get().(*cachedProtoBuffer) - cb.SetBuf(data) - v.(proto.Message).Reset() - err := cb.Unmarshal(v.(proto.Message)) - cb.SetBuf(nil) - protoBufferPool.Put(cb) - return err -} - -func (protoCodec) String() string { - return "proto" -} - -var ( - protoBufferPool = &sync.Pool{ - New: func() interface{} { - return &cachedProtoBuffer{ - Buffer: proto.Buffer{}, - lastMarshaledSize: 16, - } - }, - } -) diff --git a/vendor/google.golang.org/grpc/codes/code_string.go b/vendor/google.golang.org/grpc/codes/code_string.go index 259837060ab..0b206a57822 100644 --- a/vendor/google.golang.org/grpc/codes/code_string.go +++ b/vendor/google.golang.org/grpc/codes/code_string.go @@ -1,16 +1,62 @@ -// Code generated by "stringer -type=Code"; DO NOT EDIT. +/* + * + * Copyright 2017 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ package codes -import "fmt" +import "strconv" -const _Code_name = "OKCanceledUnknownInvalidArgumentDeadlineExceededNotFoundAlreadyExistsPermissionDeniedResourceExhaustedFailedPreconditionAbortedOutOfRangeUnimplementedInternalUnavailableDataLossUnauthenticated" - -var _Code_index = [...]uint8{0, 2, 10, 17, 32, 48, 56, 69, 85, 102, 120, 127, 137, 150, 158, 169, 177, 192} - -func (i Code) String() string { - if i >= Code(len(_Code_index)-1) { - return fmt.Sprintf("Code(%d)", i) +func (c Code) String() string { + switch c { + case OK: + return "OK" + case Canceled: + return "Canceled" + case Unknown: + return "Unknown" + case InvalidArgument: + return "InvalidArgument" + case DeadlineExceeded: + return "DeadlineExceeded" + case NotFound: + return "NotFound" + case AlreadyExists: + return "AlreadyExists" + case PermissionDenied: + return "PermissionDenied" + case ResourceExhausted: + return "ResourceExhausted" + case FailedPrecondition: + return "FailedPrecondition" + case Aborted: + return "Aborted" + case OutOfRange: + return "OutOfRange" + case Unimplemented: + return "Unimplemented" + case Internal: + return "Internal" + case Unavailable: + return "Unavailable" + case DataLoss: + return "DataLoss" + case Unauthenticated: + return "Unauthenticated" + default: + return "Code(" + strconv.FormatInt(int64(c), 10) + ")" } - return _Code_name[_Code_index[i]:_Code_index[i+1]] } diff --git a/vendor/google.golang.org/grpc/codes/codes.go b/vendor/google.golang.org/grpc/codes/codes.go index 21e7733a5fe..a8280ae660d 100644 --- a/vendor/google.golang.org/grpc/codes/codes.go +++ b/vendor/google.golang.org/grpc/codes/codes.go @@ -20,11 +20,13 @@ // consistent across various languages. package codes // import "google.golang.org/grpc/codes" +import ( + "fmt" +) + // A Code is an unsigned 32-bit error code as defined in the gRPC spec. type Code uint32 -//go:generate stringer -type=Code - const ( // OK is returned on success. OK Code = 0 @@ -32,9 +34,9 @@ const ( // Canceled indicates the operation was canceled (typically by the caller). Canceled Code = 1 - // Unknown error. An example of where this error may be returned is + // Unknown error. An example of where this error may be returned is // if a Status value received from another address space belongs to - // an error-space that is not known in this address space. Also + // an error-space that is not known in this address space. Also // errors raised by APIs that do not return enough error information // may be converted to this error. Unknown Code = 2 @@ -63,15 +65,11 @@ const ( // PermissionDenied indicates the caller does not have permission to // execute the specified operation. It must not be used for rejections // caused by exhausting some resource (use ResourceExhausted - // instead for those errors). It must not be + // instead for those errors). It must not be // used if the caller cannot be identified (use Unauthenticated // instead for those errors). PermissionDenied Code = 7 - // Unauthenticated indicates the request does not have valid - // authentication credentials for the operation. - Unauthenticated Code = 16 - // ResourceExhausted indicates some resource has been exhausted, perhaps // a per-user quota, or perhaps the entire file system is out of space. ResourceExhausted Code = 8 @@ -87,7 +85,7 @@ const ( // (b) Use Aborted if the client should retry at a higher-level // (e.g., restarting a read-modify-write sequence). // (c) Use FailedPrecondition if the client should not retry until - // the system state has been explicitly fixed. E.g., if an "rmdir" + // the system state has been explicitly fixed. E.g., if an "rmdir" // fails because the directory is non-empty, FailedPrecondition // should be returned since the client should not retry unless // they have first fixed up the directory by deleting files from it. @@ -116,7 +114,7 @@ const ( // file size. // // There is a fair bit of overlap between FailedPrecondition and - // OutOfRange. We recommend using OutOfRange (the more specific + // OutOfRange. We recommend using OutOfRange (the more specific // error) when it applies so that callers who are iterating through // a space can easily look for an OutOfRange error to detect when // they are done. @@ -126,8 +124,8 @@ const ( // supported/enabled in this service. Unimplemented Code = 12 - // Internal errors. Means some invariants expected by underlying - // system has been broken. If you see one of these errors, + // Internal errors. Means some invariants expected by underlying + // system has been broken. If you see one of these errors, // something is very broken. Internal Code = 13 @@ -141,4 +139,46 @@ const ( // DataLoss indicates unrecoverable data loss or corruption. DataLoss Code = 15 + + // Unauthenticated indicates the request does not have valid + // authentication credentials for the operation. + Unauthenticated Code = 16 ) + +var strToCode = map[string]Code{ + `"OK"`: OK, + `"CANCELLED"`:/* [sic] */ Canceled, + `"UNKNOWN"`: Unknown, + `"INVALID_ARGUMENT"`: InvalidArgument, + `"DEADLINE_EXCEEDED"`: DeadlineExceeded, + `"NOT_FOUND"`: NotFound, + `"ALREADY_EXISTS"`: AlreadyExists, + `"PERMISSION_DENIED"`: PermissionDenied, + `"RESOURCE_EXHAUSTED"`: ResourceExhausted, + `"FAILED_PRECONDITION"`: FailedPrecondition, + `"ABORTED"`: Aborted, + `"OUT_OF_RANGE"`: OutOfRange, + `"UNIMPLEMENTED"`: Unimplemented, + `"INTERNAL"`: Internal, + `"UNAVAILABLE"`: Unavailable, + `"DATA_LOSS"`: DataLoss, + `"UNAUTHENTICATED"`: Unauthenticated, +} + +// UnmarshalJSON unmarshals b into the Code. +func (c *Code) UnmarshalJSON(b []byte) error { + // From json.Unmarshaler: By convention, to approximate the behavior of + // Unmarshal itself, Unmarshalers implement UnmarshalJSON([]byte("null")) as + // a no-op. + if string(b) == "null" { + return nil + } + if c == nil { + return fmt.Errorf("nil receiver passed to UnmarshalJSON") + } + if jc, ok := strToCode[string(b)]; ok { + *c = jc + return nil + } + return fmt.Errorf("invalid code: %q", string(b)) +} diff --git a/vendor/google.golang.org/grpc/credentials/credentials.go b/vendor/google.golang.org/grpc/credentials/credentials.go index 946aa1f2b85..3351bf0ee5f 100644 --- a/vendor/google.golang.org/grpc/credentials/credentials.go +++ b/vendor/google.golang.org/grpc/credentials/credentials.go @@ -34,10 +34,8 @@ import ( "golang.org/x/net/context" ) -var ( - // alpnProtoStr are the specified application level protocols for gRPC. - alpnProtoStr = []string{"h2"} -) +// alpnProtoStr are the specified application level protocols for gRPC. +var alpnProtoStr = []string{"h2"} // PerRPCCredentials defines the common interface for the credentials which need to // attach security information to every RPC (e.g., oauth2). @@ -45,8 +43,9 @@ type PerRPCCredentials interface { // GetRequestMetadata gets the current request metadata, refreshing // tokens if required. This should be called by the transport layer on // each request, and the data should be populated in headers or other - // context. uri is the URI of the entry point for the request. When - // supported by the underlying implementation, ctx can be used for + // context. If a status code is returned, it will be used as the status + // for the RPC. uri is the URI of the entry point for the request. + // When supported by the underlying implementation, ctx can be used for // timeout and cancellation. // TODO(zhaoq): Define the set of the qualified keys instead of leaving // it as an arbitrary string. @@ -74,11 +73,9 @@ type AuthInfo interface { AuthType() string } -var ( - // ErrConnDispatched indicates that rawConn has been dispatched out of gRPC - // and the caller should not close rawConn. - ErrConnDispatched = errors.New("credentials: rawConn is dispatched out of gRPC") -) +// ErrConnDispatched indicates that rawConn has been dispatched out of gRPC +// and the caller should not close rawConn. +var ErrConnDispatched = errors.New("credentials: rawConn is dispatched out of gRPC") // TransportCredentials defines the common interface for all the live gRPC wire // protocols and supported transport security protocols (e.g., TLS, SSL). @@ -135,15 +132,15 @@ func (c tlsCreds) Info() ProtocolInfo { } } -func (c *tlsCreds) ClientHandshake(ctx context.Context, addr string, rawConn net.Conn) (_ net.Conn, _ AuthInfo, err error) { +func (c *tlsCreds) ClientHandshake(ctx context.Context, authority string, rawConn net.Conn) (_ net.Conn, _ AuthInfo, err error) { // use local cfg to avoid clobbering ServerName if using multiple endpoints cfg := cloneTLSConfig(c.config) if cfg.ServerName == "" { - colonPos := strings.LastIndex(addr, ":") + colonPos := strings.LastIndex(authority, ":") if colonPos == -1 { - colonPos = len(addr) + colonPos = len(authority) } - cfg.ServerName = addr[:colonPos] + cfg.ServerName = authority[:colonPos] } conn := tls.Client(rawConn, cfg) errChannel := make(chan error, 1) diff --git a/vendor/google.golang.org/grpc/encoding/encoding.go b/vendor/google.golang.org/grpc/encoding/encoding.go new file mode 100644 index 00000000000..ade8b7cec73 --- /dev/null +++ b/vendor/google.golang.org/grpc/encoding/encoding.go @@ -0,0 +1,118 @@ +/* + * + * Copyright 2017 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +// Package encoding defines the interface for the compressor and codec, and +// functions to register and retrieve compressors and codecs. +// +// This package is EXPERIMENTAL. +package encoding + +import ( + "io" + "strings" +) + +// Identity specifies the optional encoding for uncompressed streams. +// It is intended for grpc internal use only. +const Identity = "identity" + +// Compressor is used for compressing and decompressing when sending or +// receiving messages. +type Compressor interface { + // Compress writes the data written to wc to w after compressing it. If an + // error occurs while initializing the compressor, that error is returned + // instead. + Compress(w io.Writer) (io.WriteCloser, error) + // Decompress reads data from r, decompresses it, and provides the + // uncompressed data via the returned io.Reader. If an error occurs while + // initializing the decompressor, that error is returned instead. + Decompress(r io.Reader) (io.Reader, error) + // Name is the name of the compression codec and is used to set the content + // coding header. The result must be static; the result cannot change + // between calls. + Name() string +} + +var registeredCompressor = make(map[string]Compressor) + +// RegisterCompressor registers the compressor with gRPC by its name. It can +// be activated when sending an RPC via grpc.UseCompressor(). It will be +// automatically accessed when receiving a message based on the content coding +// header. Servers also use it to send a response with the same encoding as +// the request. +// +// NOTE: this function must only be called during initialization time (i.e. in +// an init() function), and is not thread-safe. If multiple Compressors are +// registered with the same name, the one registered last will take effect. +func RegisterCompressor(c Compressor) { + registeredCompressor[c.Name()] = c +} + +// GetCompressor returns Compressor for the given compressor name. +func GetCompressor(name string) Compressor { + return registeredCompressor[name] +} + +// Codec defines the interface gRPC uses to encode and decode messages. Note +// that implementations of this interface must be thread safe; a Codec's +// methods can be called from concurrent goroutines. +type Codec interface { + // Marshal returns the wire format of v. + Marshal(v interface{}) ([]byte, error) + // Unmarshal parses the wire format into v. + Unmarshal(data []byte, v interface{}) error + // Name returns the name of the Codec implementation. The returned string + // will be used as part of content type in transmission. The result must be + // static; the result cannot change between calls. + Name() string +} + +var registeredCodecs = make(map[string]Codec) + +// RegisterCodec registers the provided Codec for use with all gRPC clients and +// servers. +// +// The Codec will be stored and looked up by result of its Name() method, which +// should match the content-subtype of the encoding handled by the Codec. This +// is case-insensitive, and is stored and looked up as lowercase. If the +// result of calling Name() is an empty string, RegisterCodec will panic. See +// Content-Type on +// https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#requests for +// more details. +// +// NOTE: this function must only be called during initialization time (i.e. in +// an init() function), and is not thread-safe. If multiple Compressors are +// registered with the same name, the one registered last will take effect. +func RegisterCodec(codec Codec) { + if codec == nil { + panic("cannot register a nil Codec") + } + contentSubtype := strings.ToLower(codec.Name()) + if contentSubtype == "" { + panic("cannot register Codec with empty string result for String()") + } + registeredCodecs[contentSubtype] = codec +} + +// GetCodec gets a registered Codec by content-subtype, or nil if no Codec is +// registered for the content-subtype. +// +// The content-subtype is expected to be lowercase. +func GetCodec(contentSubtype string) Codec { + return registeredCodecs[contentSubtype] +} diff --git a/vendor/google.golang.org/grpc/encoding/proto/proto.go b/vendor/google.golang.org/grpc/encoding/proto/proto.go new file mode 100644 index 00000000000..66b97a6f692 --- /dev/null +++ b/vendor/google.golang.org/grpc/encoding/proto/proto.go @@ -0,0 +1,110 @@ +/* + * + * Copyright 2018 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +// Package proto defines the protobuf codec. Importing this package will +// register the codec. +package proto + +import ( + "math" + "sync" + + "github.com/golang/protobuf/proto" + "google.golang.org/grpc/encoding" +) + +// Name is the name registered for the proto compressor. +const Name = "proto" + +func init() { + encoding.RegisterCodec(codec{}) +} + +// codec is a Codec implementation with protobuf. It is the default codec for gRPC. +type codec struct{} + +type cachedProtoBuffer struct { + lastMarshaledSize uint32 + proto.Buffer +} + +func capToMaxInt32(val int) uint32 { + if val > math.MaxInt32 { + return uint32(math.MaxInt32) + } + return uint32(val) +} + +func marshal(v interface{}, cb *cachedProtoBuffer) ([]byte, error) { + protoMsg := v.(proto.Message) + newSlice := make([]byte, 0, cb.lastMarshaledSize) + + cb.SetBuf(newSlice) + cb.Reset() + if err := cb.Marshal(protoMsg); err != nil { + return nil, err + } + out := cb.Bytes() + cb.lastMarshaledSize = capToMaxInt32(len(out)) + return out, nil +} + +func (codec) Marshal(v interface{}) ([]byte, error) { + if pm, ok := v.(proto.Marshaler); ok { + // object can marshal itself, no need for buffer + return pm.Marshal() + } + + cb := protoBufferPool.Get().(*cachedProtoBuffer) + out, err := marshal(v, cb) + + // put back buffer and lose the ref to the slice + cb.SetBuf(nil) + protoBufferPool.Put(cb) + return out, err +} + +func (codec) Unmarshal(data []byte, v interface{}) error { + protoMsg := v.(proto.Message) + protoMsg.Reset() + + if pu, ok := protoMsg.(proto.Unmarshaler); ok { + // object can unmarshal itself, no need for buffer + return pu.Unmarshal(data) + } + + cb := protoBufferPool.Get().(*cachedProtoBuffer) + cb.SetBuf(data) + err := cb.Unmarshal(protoMsg) + cb.SetBuf(nil) + protoBufferPool.Put(cb) + return err +} + +func (codec) Name() string { + return Name +} + +var protoBufferPool = &sync.Pool{ + New: func() interface{} { + return &cachedProtoBuffer{ + Buffer: proto.Buffer{}, + lastMarshaledSize: 16, + } + }, +} diff --git a/vendor/google.golang.org/grpc/envconfig.go b/vendor/google.golang.org/grpc/envconfig.go new file mode 100644 index 00000000000..d50178e5171 --- /dev/null +++ b/vendor/google.golang.org/grpc/envconfig.go @@ -0,0 +1,37 @@ +/* + * + * Copyright 2018 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package grpc + +import ( + "os" + "strings" +) + +const ( + envConfigPrefix = "GRPC_GO_" + envConfigStickinessStr = envConfigPrefix + "STICKINESS" +) + +var ( + envConfigStickinessOn bool +) + +func init() { + envConfigStickinessOn = strings.EqualFold(os.Getenv(envConfigStickinessStr), "on") +} diff --git a/vendor/google.golang.org/grpc/go16.go b/vendor/google.golang.org/grpc/go16.go new file mode 100644 index 00000000000..535ee9356f3 --- /dev/null +++ b/vendor/google.golang.org/grpc/go16.go @@ -0,0 +1,70 @@ +// +build go1.6,!go1.7 + +/* + * + * Copyright 2016 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package grpc + +import ( + "fmt" + "io" + "net" + "net/http" + + "golang.org/x/net/context" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "google.golang.org/grpc/transport" +) + +// dialContext connects to the address on the named network. +func dialContext(ctx context.Context, network, address string) (net.Conn, error) { + return (&net.Dialer{Cancel: ctx.Done()}).Dial(network, address) +} + +func sendHTTPRequest(ctx context.Context, req *http.Request, conn net.Conn) error { + req.Cancel = ctx.Done() + if err := req.Write(conn); err != nil { + return fmt.Errorf("failed to write the HTTP request: %v", err) + } + return nil +} + +// toRPCErr converts an error into an error from the status package. +func toRPCErr(err error) error { + if err == nil || err == io.EOF { + return err + } + if _, ok := status.FromError(err); ok { + return err + } + switch e := err.(type) { + case transport.StreamError: + return status.Error(e.Code, e.Desc) + case transport.ConnectionError: + return status.Error(codes.Unavailable, e.Desc) + default: + switch err { + case context.DeadlineExceeded: + return status.Error(codes.DeadlineExceeded, err.Error()) + case context.Canceled: + return status.Error(codes.Canceled, err.Error()) + } + } + return status.Error(codes.Unknown, err.Error()) +} diff --git a/vendor/google.golang.org/grpc/go17.go b/vendor/google.golang.org/grpc/go17.go new file mode 100644 index 00000000000..ec676a93c39 --- /dev/null +++ b/vendor/google.golang.org/grpc/go17.go @@ -0,0 +1,71 @@ +// +build go1.7 + +/* + * + * Copyright 2016 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package grpc + +import ( + "context" + "fmt" + "io" + "net" + "net/http" + + netctx "golang.org/x/net/context" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "google.golang.org/grpc/transport" +) + +// dialContext connects to the address on the named network. +func dialContext(ctx context.Context, network, address string) (net.Conn, error) { + return (&net.Dialer{}).DialContext(ctx, network, address) +} + +func sendHTTPRequest(ctx context.Context, req *http.Request, conn net.Conn) error { + req = req.WithContext(ctx) + if err := req.Write(conn); err != nil { + return fmt.Errorf("failed to write the HTTP request: %v", err) + } + return nil +} + +// toRPCErr converts an error into an error from the status package. +func toRPCErr(err error) error { + if err == nil || err == io.EOF { + return err + } + if _, ok := status.FromError(err); ok { + return err + } + switch e := err.(type) { + case transport.StreamError: + return status.Error(e.Code, e.Desc) + case transport.ConnectionError: + return status.Error(codes.Unavailable, e.Desc) + default: + switch err { + case context.DeadlineExceeded, netctx.DeadlineExceeded: + return status.Error(codes.DeadlineExceeded, err.Error()) + case context.Canceled, netctx.Canceled: + return status.Error(codes.Canceled, err.Error()) + } + } + return status.Error(codes.Unknown, err.Error()) +} diff --git a/vendor/google.golang.org/grpc/grpclb.go b/vendor/google.golang.org/grpc/grpclb.go index db56ff36217..bc2b4452558 100644 --- a/vendor/google.golang.org/grpc/grpclb.go +++ b/vendor/google.golang.org/grpc/grpclb.go @@ -19,21 +19,32 @@ package grpc import ( - "errors" - "fmt" - "math/rand" - "net" + "strconv" + "strings" "sync" "time" "golang.org/x/net/context" - "google.golang.org/grpc/codes" - lbmpb "google.golang.org/grpc/grpclb/grpc_lb_v1/messages" + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/connectivity" + lbpb "google.golang.org/grpc/grpclb/grpc_lb_v1/messages" "google.golang.org/grpc/grpclog" - "google.golang.org/grpc/metadata" - "google.golang.org/grpc/naming" + "google.golang.org/grpc/resolver" ) +const ( + lbTokeyKey = "lb-token" + defaultFallbackTimeout = 10 * time.Second + grpclbName = "grpclb" +) + +func convertDuration(d *lbpb.Duration) time.Duration { + if d == nil { + return 0 + } + return time.Duration(d.Seconds)*time.Second + time.Duration(d.Nanos)*time.Nanosecond +} + // Client API for LoadBalancer service. // Mostly copied from generated pb.go file. // To avoid circular dependency. @@ -47,7 +58,7 @@ func (c *loadBalancerClient) BalanceLoad(ctx context.Context, opts ...CallOption ServerStreams: true, ClientStreams: true, } - stream, err := NewClientStream(ctx, desc, c.cc, "/grpc.lb.v1.LoadBalancer/BalanceLoad", opts...) + stream, err := c.cc.NewStream(ctx, desc, "/grpc.lb.v1.LoadBalancer/BalanceLoad", opts...) if err != nil { return nil, err } @@ -59,646 +70,272 @@ type balanceLoadClientStream struct { ClientStream } -func (x *balanceLoadClientStream) Send(m *lbmpb.LoadBalanceRequest) error { +func (x *balanceLoadClientStream) Send(m *lbpb.LoadBalanceRequest) error { return x.ClientStream.SendMsg(m) } -func (x *balanceLoadClientStream) Recv() (*lbmpb.LoadBalanceResponse, error) { - m := new(lbmpb.LoadBalanceResponse) +func (x *balanceLoadClientStream) Recv() (*lbpb.LoadBalanceResponse, error) { + m := new(lbpb.LoadBalanceResponse) if err := x.ClientStream.RecvMsg(m); err != nil { return nil, err } return m, nil } -// NewGRPCLBBalancer creates a grpclb load balancer. -func NewGRPCLBBalancer(r naming.Resolver) Balancer { - return &grpclbBalancer{ - r: r, - } +func init() { + balancer.Register(newLBBuilder()) } -type remoteBalancerInfo struct { - addr string - // the server name used for authentication with the remote LB server. - name string +// newLBBuilder creates a builder for grpclb. +func newLBBuilder() balancer.Builder { + return NewLBBuilderWithFallbackTimeout(defaultFallbackTimeout) } -// grpclbAddrInfo consists of the information of a backend server. -type grpclbAddrInfo struct { - addr Address - connected bool - // dropForRateLimiting indicates whether this particular request should be - // dropped by the client for rate limiting. - dropForRateLimiting bool - // dropForLoadBalancing indicates whether this particular request should be - // dropped by the client for load balancing. - dropForLoadBalancing bool +// NewLBBuilderWithFallbackTimeout creates a grpclb builder with the given +// fallbackTimeout. If no response is received from the remote balancer within +// fallbackTimeout, the backend addresses from the resolved address list will be +// used. +// +// Only call this function when a non-default fallback timeout is needed. +func NewLBBuilderWithFallbackTimeout(fallbackTimeout time.Duration) balancer.Builder { + return &lbBuilder{ + fallbackTimeout: fallbackTimeout, + } } -type grpclbBalancer struct { - r naming.Resolver - target string - mu sync.Mutex - seq int // a sequence number to make sure addrCh does not get stale addresses. - w naming.Watcher - addrCh chan []Address - rbs []remoteBalancerInfo - addrs []*grpclbAddrInfo - next int - waitCh chan struct{} - done bool - rand *rand.Rand - - clientStats lbmpb.ClientStats +type lbBuilder struct { + fallbackTimeout time.Duration } -func (b *grpclbBalancer) watchAddrUpdates(w naming.Watcher, ch chan []remoteBalancerInfo) error { - updates, err := w.Next() - if err != nil { - grpclog.Warningf("grpclb: failed to get next addr update from watcher: %v", err) - return err - } - b.mu.Lock() - defer b.mu.Unlock() - if b.done { - return ErrClientConnClosing - } - for _, update := range updates { - switch update.Op { - case naming.Add: - var exist bool - for _, v := range b.rbs { - // TODO: Is the same addr with different server name a different balancer? - if update.Addr == v.addr { - exist = true - break - } - } - if exist { - continue - } - md, ok := update.Metadata.(*naming.AddrMetadataGRPCLB) - if !ok { - // TODO: Revisit the handling here and may introduce some fallback mechanism. - grpclog.Errorf("The name resolution contains unexpected metadata %v", update.Metadata) - continue - } - switch md.AddrType { - case naming.Backend: - // TODO: Revisit the handling here and may introduce some fallback mechanism. - grpclog.Errorf("The name resolution does not give grpclb addresses") - continue - case naming.GRPCLB: - b.rbs = append(b.rbs, remoteBalancerInfo{ - addr: update.Addr, - name: md.ServerName, - }) - default: - grpclog.Errorf("Received unknow address type %d", md.AddrType) - continue - } - case naming.Delete: - for i, v := range b.rbs { - if update.Addr == v.addr { - copy(b.rbs[i:], b.rbs[i+1:]) - b.rbs = b.rbs[:len(b.rbs)-1] - break - } - } - default: - grpclog.Errorf("Unknown update.Op %v", update.Op) - } +func (b *lbBuilder) Name() string { + return grpclbName +} + +func (b *lbBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions) balancer.Balancer { + // This generates a manual resolver builder with a random scheme. This + // scheme will be used to dial to remote LB, so we can send filtered address + // updates to remote LB ClientConn using this manual resolver. + scheme := "grpclb_internal_" + strconv.FormatInt(time.Now().UnixNano(), 36) + r := &lbManualResolver{scheme: scheme, ccb: cc} + + var target string + targetSplitted := strings.Split(cc.Target(), ":///") + if len(targetSplitted) < 2 { + target = cc.Target() + } else { + target = targetSplitted[1] } - // TODO: Fall back to the basic round-robin load balancing if the resulting address is - // not a load balancer. - select { - case <-ch: - default: + + lb := &lbBalancer{ + cc: newLBCacheClientConn(cc), + target: target, + opt: opt, + fallbackTimeout: b.fallbackTimeout, + doneCh: make(chan struct{}), + + manualResolver: r, + csEvltr: &connectivityStateEvaluator{}, + subConns: make(map[resolver.Address]balancer.SubConn), + scStates: make(map[balancer.SubConn]connectivity.State), + picker: &errPicker{err: balancer.ErrNoSubConnAvailable}, + clientStats: &rpcStats{}, } - ch <- b.rbs - return nil + + return lb } -func convertDuration(d *lbmpb.Duration) time.Duration { - if d == nil { - return 0 - } - return time.Duration(d.Seconds)*time.Second + time.Duration(d.Nanos)*time.Nanosecond +type lbBalancer struct { + cc *lbCacheClientConn + target string + opt balancer.BuildOptions + fallbackTimeout time.Duration + doneCh chan struct{} + + // manualResolver is used in the remote LB ClientConn inside grpclb. When + // resolved address updates are received by grpclb, filtered updates will be + // send to remote LB ClientConn through this resolver. + manualResolver *lbManualResolver + // The ClientConn to talk to the remote balancer. + ccRemoteLB *ClientConn + + // Support client side load reporting. Each picker gets a reference to this, + // and will update its content. + clientStats *rpcStats + + mu sync.Mutex // guards everything following. + // The full server list including drops, used to check if the newly received + // serverList contains anything new. Each generate picker will also have + // reference to this list to do the first layer pick. + fullServerList []*lbpb.Server + // All backends addresses, with metadata set to nil. This list contains all + // backend addresses in the same order and with the same duplicates as in + // serverlist. When generating picker, a SubConn slice with the same order + // but with only READY SCs will be gerenated. + backendAddrs []resolver.Address + // Roundrobin functionalities. + csEvltr *connectivityStateEvaluator + state connectivity.State + subConns map[resolver.Address]balancer.SubConn // Used to new/remove SubConn. + scStates map[balancer.SubConn]connectivity.State // Used to filter READY SubConns. + picker balancer.Picker + // Support fallback to resolved backend addresses if there's no response + // from remote balancer within fallbackTimeout. + fallbackTimerExpired bool + serverListReceived bool + // resolvedBackendAddrs is resolvedAddrs minus remote balancers. It's set + // when resolved address updates are received, and read in the goroutine + // handling fallback. + resolvedBackendAddrs []resolver.Address } -func (b *grpclbBalancer) processServerList(l *lbmpb.ServerList, seq int) { - if l == nil { +// regeneratePicker takes a snapshot of the balancer, and generates a picker from +// it. The picker +// - always returns ErrTransientFailure if the balancer is in TransientFailure, +// - does two layer roundrobin pick otherwise. +// Caller must hold lb.mu. +func (lb *lbBalancer) regeneratePicker() { + if lb.state == connectivity.TransientFailure { + lb.picker = &errPicker{err: balancer.ErrTransientFailure} return } - servers := l.GetServers() - var ( - sl []*grpclbAddrInfo - addrs []Address - ) - for _, s := range servers { - md := metadata.Pairs("lb-token", s.LoadBalanceToken) - ip := net.IP(s.IpAddress) - ipStr := ip.String() - if ip.To4() == nil { - // Add square brackets to ipv6 addresses, otherwise net.Dial() and - // net.SplitHostPort() will return too many colons error. - ipStr = fmt.Sprintf("[%s]", ipStr) - } - addr := Address{ - Addr: fmt.Sprintf("%s:%d", ipStr, s.Port), - Metadata: &md, + var readySCs []balancer.SubConn + for _, a := range lb.backendAddrs { + if sc, ok := lb.subConns[a]; ok { + if st, ok := lb.scStates[sc]; ok && st == connectivity.Ready { + readySCs = append(readySCs, sc) + } } - sl = append(sl, &grpclbAddrInfo{ - addr: addr, - dropForRateLimiting: s.DropForRateLimiting, - dropForLoadBalancing: s.DropForLoadBalancing, - }) - addrs = append(addrs, addr) } - b.mu.Lock() - defer b.mu.Unlock() - if b.done || seq < b.seq { - return - } - if len(sl) > 0 { - // reset b.next to 0 when replacing the server list. - b.next = 0 - b.addrs = sl - b.addrCh <- addrs - } - return -} -func (b *grpclbBalancer) sendLoadReport(s *balanceLoadClientStream, interval time.Duration, done <-chan struct{}) { - ticker := time.NewTicker(interval) - defer ticker.Stop() - for { - select { - case <-ticker.C: - case <-done: - return - } - b.mu.Lock() - stats := b.clientStats - b.clientStats = lbmpb.ClientStats{} // Clear the stats. - b.mu.Unlock() - t := time.Now() - stats.Timestamp = &lbmpb.Timestamp{ - Seconds: t.Unix(), - Nanos: int32(t.Nanosecond()), - } - if err := s.Send(&lbmpb.LoadBalanceRequest{ - LoadBalanceRequestType: &lbmpb.LoadBalanceRequest_ClientStats{ - ClientStats: &stats, - }, - }); err != nil { - grpclog.Errorf("grpclb: failed to send load report: %v", err) + if len(lb.fullServerList) <= 0 { + if len(readySCs) <= 0 { + lb.picker = &errPicker{err: balancer.ErrNoSubConnAvailable} return } - } -} - -func (b *grpclbBalancer) callRemoteBalancer(lbc *loadBalancerClient, seq int) (retry bool) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - stream, err := lbc.BalanceLoad(ctx) - if err != nil { - grpclog.Errorf("grpclb: failed to perform RPC to the remote balancer %v", err) - return - } - b.mu.Lock() - if b.done { - b.mu.Unlock() - return - } - b.mu.Unlock() - initReq := &lbmpb.LoadBalanceRequest{ - LoadBalanceRequestType: &lbmpb.LoadBalanceRequest_InitialRequest{ - InitialRequest: &lbmpb.InitialLoadBalanceRequest{ - Name: b.target, - }, - }, - } - if err := stream.Send(initReq); err != nil { - grpclog.Errorf("grpclb: failed to send init request: %v", err) - // TODO: backoff on retry? - return true - } - reply, err := stream.Recv() - if err != nil { - grpclog.Errorf("grpclb: failed to recv init response: %v", err) - // TODO: backoff on retry? - return true - } - initResp := reply.GetInitialResponse() - if initResp == nil { - grpclog.Errorf("grpclb: reply from remote balancer did not include initial response.") - return - } - // TODO: Support delegation. - if initResp.LoadBalancerDelegate != "" { - // delegation - grpclog.Errorf("TODO: Delegation is not supported yet.") + lb.picker = &rrPicker{subConns: readySCs} return } - streamDone := make(chan struct{}) - defer close(streamDone) - b.mu.Lock() - b.clientStats = lbmpb.ClientStats{} // Clear client stats. - b.mu.Unlock() - if d := convertDuration(initResp.ClientStatsReportInterval); d > 0 { - go b.sendLoadReport(stream, d, streamDone) + lb.picker = &lbPicker{ + serverList: lb.fullServerList, + subConns: readySCs, + stats: lb.clientStats, } - // Retrieve the server list. - for { - reply, err := stream.Recv() - if err != nil { - grpclog.Errorf("grpclb: failed to recv server list: %v", err) - break - } - b.mu.Lock() - if b.done || seq < b.seq { - b.mu.Unlock() - return - } - b.seq++ // tick when receiving a new list of servers. - seq = b.seq - b.mu.Unlock() - if serverList := reply.GetServerList(); serverList != nil { - b.processServerList(serverList, seq) - } - } - return true } -func (b *grpclbBalancer) Start(target string, config BalancerConfig) error { - b.rand = rand.New(rand.NewSource(time.Now().Unix())) - // TODO: Fall back to the basic direct connection if there is no name resolver. - if b.r == nil { - return errors.New("there is no name resolver installed") +func (lb *lbBalancer) HandleSubConnStateChange(sc balancer.SubConn, s connectivity.State) { + grpclog.Infof("lbBalancer: handle SubConn state change: %p, %v", sc, s) + lb.mu.Lock() + defer lb.mu.Unlock() + + oldS, ok := lb.scStates[sc] + if !ok { + grpclog.Infof("lbBalancer: got state changes for an unknown SubConn: %p, %v", sc, s) + return } - b.target = target - b.mu.Lock() - if b.done { - b.mu.Unlock() - return ErrClientConnClosing + lb.scStates[sc] = s + switch s { + case connectivity.Idle: + sc.Connect() + case connectivity.Shutdown: + // When an address was removed by resolver, b called RemoveSubConn but + // kept the sc's state in scStates. Remove state for this sc here. + delete(lb.scStates, sc) } - b.addrCh = make(chan []Address) - w, err := b.r.Resolve(target) - if err != nil { - b.mu.Unlock() - grpclog.Errorf("grpclb: failed to resolve address: %v, err: %v", target, err) - return err - } - b.w = w - b.mu.Unlock() - balancerAddrsCh := make(chan []remoteBalancerInfo, 1) - // Spawn a goroutine to monitor the name resolution of remote load balancer. - go func() { - for { - if err := b.watchAddrUpdates(w, balancerAddrsCh); err != nil { - grpclog.Warningf("grpclb: the naming watcher stops working due to %v.\n", err) - close(balancerAddrsCh) - return - } - } - }() - // Spawn a goroutine to talk to the remote load balancer. - go func() { - var ( - cc *ClientConn - // ccError is closed when there is an error in the current cc. - // A new rb should be picked from rbs and connected. - ccError chan struct{} - rb *remoteBalancerInfo - rbs []remoteBalancerInfo - rbIdx int - ) - - defer func() { - if ccError != nil { - select { - case <-ccError: - default: - close(ccError) - } - } - if cc != nil { - cc.Close() - } - }() - - for { - var ok bool - select { - case rbs, ok = <-balancerAddrsCh: - if !ok { - return - } - foundIdx := -1 - if rb != nil { - for i, trb := range rbs { - if trb == *rb { - foundIdx = i - break - } - } - } - if foundIdx >= 0 { - if foundIdx >= 1 { - // Move the address in use to the beginning of the list. - b.rbs[0], b.rbs[foundIdx] = b.rbs[foundIdx], b.rbs[0] - rbIdx = 0 - } - continue // If found, don't dial new cc. - } else if len(rbs) > 0 { - // Pick a random one from the list, instead of always using the first one. - if l := len(rbs); l > 1 && rb != nil { - tmpIdx := b.rand.Intn(l - 1) - b.rbs[0], b.rbs[tmpIdx] = b.rbs[tmpIdx], b.rbs[0] - } - rbIdx = 0 - rb = &rbs[0] - } else { - // foundIdx < 0 && len(rbs) <= 0. - rb = nil - } - case <-ccError: - ccError = nil - if rbIdx < len(rbs)-1 { - rbIdx++ - rb = &rbs[rbIdx] - } else { - rb = nil - } - } - if rb == nil { - continue - } + oldAggrState := lb.state + lb.state = lb.csEvltr.recordTransition(oldS, s) - if cc != nil { - cc.Close() - } - // Talk to the remote load balancer to get the server list. - var ( - err error - dopts []DialOption - ) - if creds := config.DialCreds; creds != nil { - if rb.name != "" { - if err := creds.OverrideServerName(rb.name); err != nil { - grpclog.Warningf("grpclb: failed to override the server name in the credentials: %v", err) - continue - } - } - dopts = append(dopts, WithTransportCredentials(creds)) - } else { - dopts = append(dopts, WithInsecure()) - } - if dialer := config.Dialer; dialer != nil { - // WithDialer takes a different type of function, so we instead use a special DialOption here. - dopts = append(dopts, func(o *dialOptions) { o.copts.Dialer = dialer }) - } - dopts = append(dopts, WithBlock()) - ccError = make(chan struct{}) - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - cc, err = DialContext(ctx, rb.addr, dopts...) - cancel() - if err != nil { - grpclog.Warningf("grpclb: failed to setup a connection to the remote balancer %v: %v", rb.addr, err) - close(ccError) - continue - } - b.mu.Lock() - b.seq++ // tick when getting a new balancer address - seq := b.seq - b.next = 0 - b.mu.Unlock() - go func(cc *ClientConn, ccError chan struct{}) { - lbc := &loadBalancerClient{cc} - b.callRemoteBalancer(lbc, seq) - cc.Close() - select { - case <-ccError: - default: - close(ccError) - } - }(cc, ccError) - } - }() - return nil -} - -func (b *grpclbBalancer) down(addr Address, err error) { - b.mu.Lock() - defer b.mu.Unlock() - for _, a := range b.addrs { - if addr == a.addr { - a.connected = false - break - } + // Regenerate picker when one of the following happens: + // - this sc became ready from not-ready + // - this sc became not-ready from ready + // - the aggregated state of balancer became TransientFailure from non-TransientFailure + // - the aggregated state of balancer became non-TransientFailure from TransientFailure + if (oldS == connectivity.Ready) != (s == connectivity.Ready) || + (lb.state == connectivity.TransientFailure) != (oldAggrState == connectivity.TransientFailure) { + lb.regeneratePicker() } + + lb.cc.UpdateBalancerState(lb.state, lb.picker) } -func (b *grpclbBalancer) Up(addr Address) func(error) { - b.mu.Lock() - defer b.mu.Unlock() - if b.done { - return nil - } - var cnt int - for _, a := range b.addrs { - if a.addr == addr { - if a.connected { - return nil - } - a.connected = true - } - if a.connected && !a.dropForRateLimiting && !a.dropForLoadBalancing { - cnt++ - } - } - // addr is the only one which is connected. Notify the Get() callers who are blocking. - if cnt == 1 && b.waitCh != nil { - close(b.waitCh) - b.waitCh = nil +// fallbackToBackendsAfter blocks for fallbackTimeout and falls back to use +// resolved backends (backends received from resolver, not from remote balancer) +// if no connection to remote balancers was successful. +func (lb *lbBalancer) fallbackToBackendsAfter(fallbackTimeout time.Duration) { + timer := time.NewTimer(fallbackTimeout) + defer timer.Stop() + select { + case <-timer.C: + case <-lb.doneCh: + return } - return func(err error) { - b.down(addr, err) + lb.mu.Lock() + if lb.serverListReceived { + lb.mu.Unlock() + return } + lb.fallbackTimerExpired = true + lb.refreshSubConns(lb.resolvedBackendAddrs) + lb.mu.Unlock() } -func (b *grpclbBalancer) Get(ctx context.Context, opts BalancerGetOptions) (addr Address, put func(), err error) { - var ch chan struct{} - b.mu.Lock() - if b.done { - b.mu.Unlock() - err = ErrClientConnClosing +// HandleResolvedAddrs sends the updated remoteLB addresses to remoteLB +// clientConn. The remoteLB clientConn will handle creating/removing remoteLB +// connections. +func (lb *lbBalancer) HandleResolvedAddrs(addrs []resolver.Address, err error) { + grpclog.Infof("lbBalancer: handleResolvedResult: %+v", addrs) + if len(addrs) <= 0 { return } - seq := b.seq - defer func() { - if err != nil { - return - } - put = func() { - s, ok := rpcInfoFromContext(ctx) - if !ok { - return - } - b.mu.Lock() - defer b.mu.Unlock() - if b.done || seq < b.seq { - return - } - b.clientStats.NumCallsFinished++ - if !s.bytesSent { - b.clientStats.NumCallsFinishedWithClientFailedToSend++ - } else if s.bytesReceived { - b.clientStats.NumCallsFinishedKnownReceived++ - } + var remoteBalancerAddrs, backendAddrs []resolver.Address + for _, a := range addrs { + if a.Type == resolver.GRPCLB { + remoteBalancerAddrs = append(remoteBalancerAddrs, a) + } else { + backendAddrs = append(backendAddrs, a) } - }() - - b.clientStats.NumCallsStarted++ - if len(b.addrs) > 0 { - if b.next >= len(b.addrs) { - b.next = 0 - } - next := b.next - for { - a := b.addrs[next] - next = (next + 1) % len(b.addrs) - if a.connected { - if !a.dropForRateLimiting && !a.dropForLoadBalancing { - addr = a.addr - b.next = next - b.mu.Unlock() - return - } - if !opts.BlockingWait { - b.next = next - if a.dropForLoadBalancing { - b.clientStats.NumCallsFinished++ - b.clientStats.NumCallsFinishedWithDropForLoadBalancing++ - } else if a.dropForRateLimiting { - b.clientStats.NumCallsFinished++ - b.clientStats.NumCallsFinishedWithDropForRateLimiting++ - } - b.mu.Unlock() - err = Errorf(codes.Unavailable, "%s drops requests", a.addr.Addr) - return - } - } - if next == b.next { - // Has iterated all the possible address but none is connected. - break - } - } - } - if !opts.BlockingWait { - b.clientStats.NumCallsFinished++ - b.clientStats.NumCallsFinishedWithClientFailedToSend++ - b.mu.Unlock() - err = Errorf(codes.Unavailable, "there is no address available") - return } - // Wait on b.waitCh for non-failfast RPCs. - if b.waitCh == nil { - ch = make(chan struct{}) - b.waitCh = ch - } else { - ch = b.waitCh - } - b.mu.Unlock() - for { - select { - case <-ctx.Done(): - b.mu.Lock() - b.clientStats.NumCallsFinished++ - b.clientStats.NumCallsFinishedWithClientFailedToSend++ - b.mu.Unlock() - err = ctx.Err() - return - case <-ch: - b.mu.Lock() - if b.done { - b.clientStats.NumCallsFinished++ - b.clientStats.NumCallsFinishedWithClientFailedToSend++ - b.mu.Unlock() - err = ErrClientConnClosing - return - } - if len(b.addrs) > 0 { - if b.next >= len(b.addrs) { - b.next = 0 - } - next := b.next - for { - a := b.addrs[next] - next = (next + 1) % len(b.addrs) - if a.connected { - if !a.dropForRateLimiting && !a.dropForLoadBalancing { - addr = a.addr - b.next = next - b.mu.Unlock() - return - } - if !opts.BlockingWait { - b.next = next - if a.dropForLoadBalancing { - b.clientStats.NumCallsFinished++ - b.clientStats.NumCallsFinishedWithDropForLoadBalancing++ - } else if a.dropForRateLimiting { - b.clientStats.NumCallsFinished++ - b.clientStats.NumCallsFinishedWithDropForRateLimiting++ - } - b.mu.Unlock() - err = Errorf(codes.Unavailable, "drop requests for the addreess %s", a.addr.Addr) - return - } - } - if next == b.next { - // Has iterated all the possible address but none is connected. - break - } - } - } - // The newly added addr got removed by Down() again. - if b.waitCh == nil { - ch = make(chan struct{}) - b.waitCh = ch - } else { - ch = b.waitCh - } - b.mu.Unlock() + if lb.ccRemoteLB == nil { + if len(remoteBalancerAddrs) <= 0 { + grpclog.Errorf("grpclb: no remote balancer address is available, should never happen") + return } + // First time receiving resolved addresses, create a cc to remote + // balancers. + lb.dialRemoteLB(remoteBalancerAddrs[0].ServerName) + // Start the fallback goroutine. + go lb.fallbackToBackendsAfter(lb.fallbackTimeout) } -} -func (b *grpclbBalancer) Notify() <-chan []Address { - return b.addrCh + // cc to remote balancers uses lb.manualResolver. Send the updated remote + // balancer addresses to it through manualResolver. + lb.manualResolver.NewAddress(remoteBalancerAddrs) + + lb.mu.Lock() + lb.resolvedBackendAddrs = backendAddrs + // If serverListReceived is true, connection to remote balancer was + // successful and there's no need to do fallback anymore. + // If fallbackTimerExpired is false, fallback hasn't happened yet. + if !lb.serverListReceived && lb.fallbackTimerExpired { + // This means we received a new list of resolved backends, and we are + // still in fallback mode. Need to update the list of backends we are + // using to the new list of backends. + lb.refreshSubConns(lb.resolvedBackendAddrs) + } + lb.mu.Unlock() } -func (b *grpclbBalancer) Close() error { - b.mu.Lock() - defer b.mu.Unlock() - if b.done { - return errBalancerClosed - } - b.done = true - if b.waitCh != nil { - close(b.waitCh) - } - if b.addrCh != nil { - close(b.addrCh) +func (lb *lbBalancer) Close() { + select { + case <-lb.doneCh: + return + default: } - if b.w != nil { - b.w.Close() + close(lb.doneCh) + if lb.ccRemoteLB != nil { + lb.ccRemoteLB.Close() } - return nil + lb.cc.close() } diff --git a/vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/messages/messages.pb.go b/vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/messages/messages.pb.go index f4a27125a4f..b3b32b48e86 100644 --- a/vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/messages/messages.pb.go +++ b/vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/messages/messages.pb.go @@ -1,24 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // source: grpc_lb_v1/messages/messages.proto -/* -Package messages is a generated protocol buffer package. - -It is generated from these files: - grpc_lb_v1/messages/messages.proto - -It has these top-level messages: - Duration - Timestamp - LoadBalanceRequest - InitialLoadBalanceRequest - ClientStats - LoadBalanceResponse - InitialLoadBalanceResponse - ServerList - Server -*/ -package messages +package messages // import "google.golang.org/grpc/grpclb/grpc_lb_v1/messages" import proto "github.com/golang/protobuf/proto" import fmt "fmt" @@ -45,13 +28,35 @@ type Duration struct { // of one second or more, a non-zero value for the `nanos` field must be // of the same sign as the `seconds` field. Must be from -999,999,999 // to +999,999,999 inclusive. - Nanos int32 `protobuf:"varint,2,opt,name=nanos" json:"nanos,omitempty"` + Nanos int32 `protobuf:"varint,2,opt,name=nanos" json:"nanos,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Duration) Reset() { *m = Duration{} } -func (m *Duration) String() string { return proto.CompactTextString(m) } -func (*Duration) ProtoMessage() {} -func (*Duration) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } +func (m *Duration) Reset() { *m = Duration{} } +func (m *Duration) String() string { return proto.CompactTextString(m) } +func (*Duration) ProtoMessage() {} +func (*Duration) Descriptor() ([]byte, []int) { + return fileDescriptor_messages_b81c731f0e83edbd, []int{0} +} +func (m *Duration) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_Duration.Unmarshal(m, b) +} +func (m *Duration) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_Duration.Marshal(b, m, deterministic) +} +func (dst *Duration) XXX_Merge(src proto.Message) { + xxx_messageInfo_Duration.Merge(dst, src) +} +func (m *Duration) XXX_Size() int { + return xxx_messageInfo_Duration.Size(m) +} +func (m *Duration) XXX_DiscardUnknown() { + xxx_messageInfo_Duration.DiscardUnknown(m) +} + +var xxx_messageInfo_Duration proto.InternalMessageInfo func (m *Duration) GetSeconds() int64 { if m != nil { @@ -76,13 +81,35 @@ type Timestamp struct { // second values with fractions must still have non-negative nanos values // that count forward in time. Must be from 0 to 999,999,999 // inclusive. - Nanos int32 `protobuf:"varint,2,opt,name=nanos" json:"nanos,omitempty"` + Nanos int32 `protobuf:"varint,2,opt,name=nanos" json:"nanos,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Timestamp) Reset() { *m = Timestamp{} } +func (m *Timestamp) String() string { return proto.CompactTextString(m) } +func (*Timestamp) ProtoMessage() {} +func (*Timestamp) Descriptor() ([]byte, []int) { + return fileDescriptor_messages_b81c731f0e83edbd, []int{1} +} +func (m *Timestamp) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_Timestamp.Unmarshal(m, b) +} +func (m *Timestamp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_Timestamp.Marshal(b, m, deterministic) +} +func (dst *Timestamp) XXX_Merge(src proto.Message) { + xxx_messageInfo_Timestamp.Merge(dst, src) +} +func (m *Timestamp) XXX_Size() int { + return xxx_messageInfo_Timestamp.Size(m) +} +func (m *Timestamp) XXX_DiscardUnknown() { + xxx_messageInfo_Timestamp.DiscardUnknown(m) } -func (m *Timestamp) Reset() { *m = Timestamp{} } -func (m *Timestamp) String() string { return proto.CompactTextString(m) } -func (*Timestamp) ProtoMessage() {} -func (*Timestamp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } +var xxx_messageInfo_Timestamp proto.InternalMessageInfo func (m *Timestamp) GetSeconds() int64 { if m != nil { @@ -103,12 +130,34 @@ type LoadBalanceRequest struct { // *LoadBalanceRequest_InitialRequest // *LoadBalanceRequest_ClientStats LoadBalanceRequestType isLoadBalanceRequest_LoadBalanceRequestType `protobuf_oneof:"load_balance_request_type"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LoadBalanceRequest) Reset() { *m = LoadBalanceRequest{} } +func (m *LoadBalanceRequest) String() string { return proto.CompactTextString(m) } +func (*LoadBalanceRequest) ProtoMessage() {} +func (*LoadBalanceRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_messages_b81c731f0e83edbd, []int{2} +} +func (m *LoadBalanceRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_LoadBalanceRequest.Unmarshal(m, b) +} +func (m *LoadBalanceRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_LoadBalanceRequest.Marshal(b, m, deterministic) +} +func (dst *LoadBalanceRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_LoadBalanceRequest.Merge(dst, src) +} +func (m *LoadBalanceRequest) XXX_Size() int { + return xxx_messageInfo_LoadBalanceRequest.Size(m) +} +func (m *LoadBalanceRequest) XXX_DiscardUnknown() { + xxx_messageInfo_LoadBalanceRequest.DiscardUnknown(m) } -func (m *LoadBalanceRequest) Reset() { *m = LoadBalanceRequest{} } -func (m *LoadBalanceRequest) String() string { return proto.CompactTextString(m) } -func (*LoadBalanceRequest) ProtoMessage() {} -func (*LoadBalanceRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } +var xxx_messageInfo_LoadBalanceRequest proto.InternalMessageInfo type isLoadBalanceRequest_LoadBalanceRequestType interface { isLoadBalanceRequest_LoadBalanceRequestType() @@ -204,12 +253,12 @@ func _LoadBalanceRequest_OneofSizer(msg proto.Message) (n int) { switch x := m.LoadBalanceRequestType.(type) { case *LoadBalanceRequest_InitialRequest: s := proto.Size(x.InitialRequest) - n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *LoadBalanceRequest_ClientStats: s := proto.Size(x.ClientStats) - n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -222,13 +271,35 @@ func _LoadBalanceRequest_OneofSizer(msg proto.Message) (n int) { type InitialLoadBalanceRequest struct { // Name of load balanced service (IE, balancer.service.com) // length should be less than 256 bytes. - Name string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"` + Name string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *InitialLoadBalanceRequest) Reset() { *m = InitialLoadBalanceRequest{} } +func (m *InitialLoadBalanceRequest) String() string { return proto.CompactTextString(m) } +func (*InitialLoadBalanceRequest) ProtoMessage() {} +func (*InitialLoadBalanceRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_messages_b81c731f0e83edbd, []int{3} +} +func (m *InitialLoadBalanceRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_InitialLoadBalanceRequest.Unmarshal(m, b) +} +func (m *InitialLoadBalanceRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_InitialLoadBalanceRequest.Marshal(b, m, deterministic) +} +func (dst *InitialLoadBalanceRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_InitialLoadBalanceRequest.Merge(dst, src) +} +func (m *InitialLoadBalanceRequest) XXX_Size() int { + return xxx_messageInfo_InitialLoadBalanceRequest.Size(m) +} +func (m *InitialLoadBalanceRequest) XXX_DiscardUnknown() { + xxx_messageInfo_InitialLoadBalanceRequest.DiscardUnknown(m) } -func (m *InitialLoadBalanceRequest) Reset() { *m = InitialLoadBalanceRequest{} } -func (m *InitialLoadBalanceRequest) String() string { return proto.CompactTextString(m) } -func (*InitialLoadBalanceRequest) ProtoMessage() {} -func (*InitialLoadBalanceRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } +var xxx_messageInfo_InitialLoadBalanceRequest proto.InternalMessageInfo func (m *InitialLoadBalanceRequest) GetName() string { if m != nil { @@ -256,13 +327,35 @@ type ClientStats struct { NumCallsFinishedWithClientFailedToSend int64 `protobuf:"varint,6,opt,name=num_calls_finished_with_client_failed_to_send,json=numCallsFinishedWithClientFailedToSend" json:"num_calls_finished_with_client_failed_to_send,omitempty"` // The total number of RPCs that finished and are known to have been received // by a server. - NumCallsFinishedKnownReceived int64 `protobuf:"varint,7,opt,name=num_calls_finished_known_received,json=numCallsFinishedKnownReceived" json:"num_calls_finished_known_received,omitempty"` + NumCallsFinishedKnownReceived int64 `protobuf:"varint,7,opt,name=num_calls_finished_known_received,json=numCallsFinishedKnownReceived" json:"num_calls_finished_known_received,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ClientStats) Reset() { *m = ClientStats{} } +func (m *ClientStats) String() string { return proto.CompactTextString(m) } +func (*ClientStats) ProtoMessage() {} +func (*ClientStats) Descriptor() ([]byte, []int) { + return fileDescriptor_messages_b81c731f0e83edbd, []int{4} +} +func (m *ClientStats) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ClientStats.Unmarshal(m, b) +} +func (m *ClientStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ClientStats.Marshal(b, m, deterministic) +} +func (dst *ClientStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientStats.Merge(dst, src) +} +func (m *ClientStats) XXX_Size() int { + return xxx_messageInfo_ClientStats.Size(m) +} +func (m *ClientStats) XXX_DiscardUnknown() { + xxx_messageInfo_ClientStats.DiscardUnknown(m) } -func (m *ClientStats) Reset() { *m = ClientStats{} } -func (m *ClientStats) String() string { return proto.CompactTextString(m) } -func (*ClientStats) ProtoMessage() {} -func (*ClientStats) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } +var xxx_messageInfo_ClientStats proto.InternalMessageInfo func (m *ClientStats) GetTimestamp() *Timestamp { if m != nil { @@ -318,12 +411,34 @@ type LoadBalanceResponse struct { // *LoadBalanceResponse_InitialResponse // *LoadBalanceResponse_ServerList LoadBalanceResponseType isLoadBalanceResponse_LoadBalanceResponseType `protobuf_oneof:"load_balance_response_type"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LoadBalanceResponse) Reset() { *m = LoadBalanceResponse{} } +func (m *LoadBalanceResponse) String() string { return proto.CompactTextString(m) } +func (*LoadBalanceResponse) ProtoMessage() {} +func (*LoadBalanceResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_messages_b81c731f0e83edbd, []int{5} +} +func (m *LoadBalanceResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_LoadBalanceResponse.Unmarshal(m, b) +} +func (m *LoadBalanceResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_LoadBalanceResponse.Marshal(b, m, deterministic) +} +func (dst *LoadBalanceResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_LoadBalanceResponse.Merge(dst, src) +} +func (m *LoadBalanceResponse) XXX_Size() int { + return xxx_messageInfo_LoadBalanceResponse.Size(m) +} +func (m *LoadBalanceResponse) XXX_DiscardUnknown() { + xxx_messageInfo_LoadBalanceResponse.DiscardUnknown(m) } -func (m *LoadBalanceResponse) Reset() { *m = LoadBalanceResponse{} } -func (m *LoadBalanceResponse) String() string { return proto.CompactTextString(m) } -func (*LoadBalanceResponse) ProtoMessage() {} -func (*LoadBalanceResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } +var xxx_messageInfo_LoadBalanceResponse proto.InternalMessageInfo type isLoadBalanceResponse_LoadBalanceResponseType interface { isLoadBalanceResponse_LoadBalanceResponseType() @@ -419,12 +534,12 @@ func _LoadBalanceResponse_OneofSizer(msg proto.Message) (n int) { switch x := m.LoadBalanceResponseType.(type) { case *LoadBalanceResponse_InitialResponse: s := proto.Size(x.InitialResponse) - n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *LoadBalanceResponse_ServerList: s := proto.Size(x.ServerList) - n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -445,12 +560,34 @@ type InitialLoadBalanceResponse struct { // to the load balancer. Stats should only be reported when the duration is // positive. ClientStatsReportInterval *Duration `protobuf:"bytes,2,opt,name=client_stats_report_interval,json=clientStatsReportInterval" json:"client_stats_report_interval,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *InitialLoadBalanceResponse) Reset() { *m = InitialLoadBalanceResponse{} } +func (m *InitialLoadBalanceResponse) String() string { return proto.CompactTextString(m) } +func (*InitialLoadBalanceResponse) ProtoMessage() {} +func (*InitialLoadBalanceResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_messages_b81c731f0e83edbd, []int{6} +} +func (m *InitialLoadBalanceResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_InitialLoadBalanceResponse.Unmarshal(m, b) +} +func (m *InitialLoadBalanceResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_InitialLoadBalanceResponse.Marshal(b, m, deterministic) +} +func (dst *InitialLoadBalanceResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_InitialLoadBalanceResponse.Merge(dst, src) +} +func (m *InitialLoadBalanceResponse) XXX_Size() int { + return xxx_messageInfo_InitialLoadBalanceResponse.Size(m) +} +func (m *InitialLoadBalanceResponse) XXX_DiscardUnknown() { + xxx_messageInfo_InitialLoadBalanceResponse.DiscardUnknown(m) } -func (m *InitialLoadBalanceResponse) Reset() { *m = InitialLoadBalanceResponse{} } -func (m *InitialLoadBalanceResponse) String() string { return proto.CompactTextString(m) } -func (*InitialLoadBalanceResponse) ProtoMessage() {} -func (*InitialLoadBalanceResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } +var xxx_messageInfo_InitialLoadBalanceResponse proto.InternalMessageInfo func (m *InitialLoadBalanceResponse) GetLoadBalancerDelegate() string { if m != nil { @@ -471,13 +608,35 @@ type ServerList struct { // be updated when server resolutions change or as needed to balance load // across more servers. The client should consume the server list in order // unless instructed otherwise via the client_config. - Servers []*Server `protobuf:"bytes,1,rep,name=servers" json:"servers,omitempty"` + Servers []*Server `protobuf:"bytes,1,rep,name=servers" json:"servers,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ServerList) Reset() { *m = ServerList{} } +func (m *ServerList) String() string { return proto.CompactTextString(m) } +func (*ServerList) ProtoMessage() {} +func (*ServerList) Descriptor() ([]byte, []int) { + return fileDescriptor_messages_b81c731f0e83edbd, []int{7} +} +func (m *ServerList) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ServerList.Unmarshal(m, b) +} +func (m *ServerList) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ServerList.Marshal(b, m, deterministic) +} +func (dst *ServerList) XXX_Merge(src proto.Message) { + xxx_messageInfo_ServerList.Merge(dst, src) +} +func (m *ServerList) XXX_Size() int { + return xxx_messageInfo_ServerList.Size(m) +} +func (m *ServerList) XXX_DiscardUnknown() { + xxx_messageInfo_ServerList.DiscardUnknown(m) } -func (m *ServerList) Reset() { *m = ServerList{} } -func (m *ServerList) String() string { return proto.CompactTextString(m) } -func (*ServerList) ProtoMessage() {} -func (*ServerList) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } +var xxx_messageInfo_ServerList proto.InternalMessageInfo func (m *ServerList) GetServers() []*Server { if m != nil { @@ -508,13 +667,35 @@ type Server struct { DropForRateLimiting bool `protobuf:"varint,4,opt,name=drop_for_rate_limiting,json=dropForRateLimiting" json:"drop_for_rate_limiting,omitempty"` // Indicates whether this particular request should be dropped by the client // for load balancing. - DropForLoadBalancing bool `protobuf:"varint,5,opt,name=drop_for_load_balancing,json=dropForLoadBalancing" json:"drop_for_load_balancing,omitempty"` + DropForLoadBalancing bool `protobuf:"varint,5,opt,name=drop_for_load_balancing,json=dropForLoadBalancing" json:"drop_for_load_balancing,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Server) Reset() { *m = Server{} } -func (m *Server) String() string { return proto.CompactTextString(m) } -func (*Server) ProtoMessage() {} -func (*Server) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } +func (m *Server) Reset() { *m = Server{} } +func (m *Server) String() string { return proto.CompactTextString(m) } +func (*Server) ProtoMessage() {} +func (*Server) Descriptor() ([]byte, []int) { + return fileDescriptor_messages_b81c731f0e83edbd, []int{8} +} +func (m *Server) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_Server.Unmarshal(m, b) +} +func (m *Server) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_Server.Marshal(b, m, deterministic) +} +func (dst *Server) XXX_Merge(src proto.Message) { + xxx_messageInfo_Server.Merge(dst, src) +} +func (m *Server) XXX_Size() int { + return xxx_messageInfo_Server.Size(m) +} +func (m *Server) XXX_DiscardUnknown() { + xxx_messageInfo_Server.DiscardUnknown(m) +} + +var xxx_messageInfo_Server proto.InternalMessageInfo func (m *Server) GetIpAddress() []byte { if m != nil { @@ -563,53 +744,56 @@ func init() { proto.RegisterType((*Server)(nil), "grpc.lb.v1.Server") } -func init() { proto.RegisterFile("grpc_lb_v1/messages/messages.proto", fileDescriptor0) } - -var fileDescriptor0 = []byte{ - // 709 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xdd, 0x4e, 0x1b, 0x3b, - 0x10, 0x26, 0x27, 0x01, 0x92, 0x09, 0x3a, 0xe4, 0x98, 0x1c, 0x08, 0x14, 0x24, 0xba, 0x52, 0x69, - 0x54, 0xd1, 0x20, 0xa0, 0xbd, 0xe8, 0xcf, 0x45, 0x1b, 0x10, 0x0a, 0x2d, 0x17, 0x95, 0x43, 0x55, - 0xa9, 0x52, 0x65, 0x39, 0xd9, 0x21, 0x58, 0x6c, 0xec, 0xad, 0xed, 0x04, 0xf5, 0x11, 0xfa, 0x28, - 0x7d, 0x8c, 0xaa, 0xcf, 0xd0, 0xf7, 0xa9, 0xd6, 0xbb, 0x9b, 0x5d, 0x20, 0x80, 0x7a, 0x67, 0x8f, - 0xbf, 0xf9, 0xbe, 0xf1, 0xac, 0xbf, 0x59, 0xf0, 0x06, 0x3a, 0xec, 0xb3, 0xa0, 0xc7, 0xc6, 0xbb, - 0x3b, 0x43, 0x34, 0x86, 0x0f, 0xd0, 0x4c, 0x16, 0xad, 0x50, 0x2b, 0xab, 0x08, 0x44, 0x98, 0x56, - 0xd0, 0x6b, 0x8d, 0x77, 0xbd, 0x97, 0x50, 0x3e, 0x1c, 0x69, 0x6e, 0x85, 0x92, 0xa4, 0x01, 0xf3, - 0x06, 0xfb, 0x4a, 0xfa, 0xa6, 0x51, 0xd8, 0x2c, 0x34, 0x8b, 0x34, 0xdd, 0x92, 0x3a, 0xcc, 0x4a, - 0x2e, 0x95, 0x69, 0xfc, 0xb3, 0x59, 0x68, 0xce, 0xd2, 0x78, 0xe3, 0xbd, 0x82, 0xca, 0xa9, 0x18, - 0xa2, 0xb1, 0x7c, 0x18, 0xfe, 0x75, 0xf2, 0xcf, 0x02, 0x90, 0x13, 0xc5, 0xfd, 0x36, 0x0f, 0xb8, - 0xec, 0x23, 0xc5, 0xaf, 0x23, 0x34, 0x96, 0x7c, 0x80, 0x45, 0x21, 0x85, 0x15, 0x3c, 0x60, 0x3a, - 0x0e, 0x39, 0xba, 0xea, 0xde, 0xa3, 0x56, 0x56, 0x75, 0xeb, 0x38, 0x86, 0xdc, 0xcc, 0xef, 0xcc, - 0xd0, 0x7f, 0x93, 0xfc, 0x94, 0xf1, 0x35, 0x2c, 0xf4, 0x03, 0x81, 0xd2, 0x32, 0x63, 0xb9, 0x8d, - 0xab, 0xa8, 0xee, 0xad, 0xe4, 0xe9, 0x0e, 0xdc, 0x79, 0x37, 0x3a, 0xee, 0xcc, 0xd0, 0x6a, 0x3f, - 0xdb, 0xb6, 0x1f, 0xc0, 0x6a, 0xa0, 0xb8, 0xcf, 0x7a, 0xb1, 0x4c, 0x5a, 0x14, 0xb3, 0xdf, 0x42, - 0xf4, 0x76, 0x60, 0xf5, 0xd6, 0x4a, 0x08, 0x81, 0x92, 0xe4, 0x43, 0x74, 0xe5, 0x57, 0xa8, 0x5b, - 0x7b, 0xdf, 0x4b, 0x50, 0xcd, 0x89, 0x91, 0x7d, 0xa8, 0xd8, 0xb4, 0x83, 0xc9, 0x3d, 0xff, 0xcf, - 0x17, 0x36, 0x69, 0x2f, 0xcd, 0x70, 0xe4, 0x09, 0xfc, 0x27, 0x47, 0x43, 0xd6, 0xe7, 0x41, 0x60, - 0xa2, 0x3b, 0x69, 0x8b, 0xbe, 0xbb, 0x55, 0x91, 0x2e, 0xca, 0xd1, 0xf0, 0x20, 0x8a, 0x77, 0xe3, - 0x30, 0xd9, 0x06, 0x92, 0x61, 0xcf, 0x84, 0x14, 0xe6, 0x1c, 0xfd, 0x46, 0xd1, 0x81, 0x6b, 0x29, - 0xf8, 0x28, 0x89, 0x13, 0x06, 0xad, 0x9b, 0x68, 0x76, 0x29, 0xec, 0x39, 0xf3, 0xb5, 0x0a, 0xd9, - 0x99, 0xd2, 0x4c, 0x73, 0x8b, 0x2c, 0x10, 0x43, 0x61, 0x85, 0x1c, 0x34, 0x4a, 0x8e, 0xe9, 0xf1, - 0x75, 0xa6, 0x4f, 0xc2, 0x9e, 0x1f, 0x6a, 0x15, 0x1e, 0x29, 0x4d, 0xb9, 0xc5, 0x93, 0x04, 0x4e, - 0x38, 0xec, 0xdc, 0x2b, 0x90, 0x6b, 0x77, 0xa4, 0x30, 0xeb, 0x14, 0x9a, 0x77, 0x28, 0x64, 0xbd, - 0x8f, 0x24, 0xbe, 0xc0, 0xd3, 0xdb, 0x24, 0x92, 0x67, 0x70, 0xc6, 0x45, 0x80, 0x3e, 0xb3, 0x8a, - 0x19, 0x94, 0x7e, 0x63, 0xce, 0x09, 0x6c, 0x4d, 0x13, 0x88, 0x3f, 0xd5, 0x91, 0xc3, 0x9f, 0xaa, - 0x2e, 0x4a, 0x9f, 0x74, 0xe0, 0xe1, 0x14, 0xfa, 0x0b, 0xa9, 0x2e, 0x25, 0xd3, 0xd8, 0x47, 0x31, - 0x46, 0xbf, 0x31, 0xef, 0x28, 0x37, 0xae, 0x53, 0xbe, 0x8f, 0x50, 0x34, 0x01, 0x79, 0xbf, 0x0a, - 0xb0, 0x74, 0xe5, 0xd9, 0x98, 0x50, 0x49, 0x83, 0xa4, 0x0b, 0xb5, 0xcc, 0x01, 0x71, 0x2c, 0x79, - 0x1a, 0x5b, 0xf7, 0x59, 0x20, 0x46, 0x77, 0x66, 0xe8, 0xe2, 0xc4, 0x03, 0x09, 0xe9, 0x0b, 0xa8, - 0x1a, 0xd4, 0x63, 0xd4, 0x2c, 0x10, 0xc6, 0x26, 0x1e, 0x58, 0xce, 0xf3, 0x75, 0xdd, 0xf1, 0x89, - 0x70, 0x1e, 0x02, 0x33, 0xd9, 0xb5, 0xd7, 0x61, 0xed, 0x9a, 0x03, 0x62, 0xce, 0xd8, 0x02, 0x3f, - 0x0a, 0xb0, 0x76, 0x7b, 0x29, 0xe4, 0x19, 0x2c, 0xe7, 0x93, 0x35, 0xf3, 0x31, 0xc0, 0x01, 0xb7, - 0xa9, 0x2d, 0xea, 0x41, 0x96, 0xa4, 0x0f, 0x93, 0x33, 0xf2, 0x11, 0xd6, 0xf3, 0x96, 0x65, 0x1a, - 0x43, 0xa5, 0x2d, 0x13, 0xd2, 0xa2, 0x1e, 0xf3, 0x20, 0x29, 0xbf, 0x9e, 0x2f, 0x3f, 0x1d, 0x62, - 0x74, 0x35, 0xe7, 0x5e, 0xea, 0xf2, 0x8e, 0x93, 0x34, 0xef, 0x0d, 0x40, 0x76, 0x4b, 0xb2, 0x1d, - 0x0d, 0xac, 0x68, 0x17, 0x0d, 0xac, 0x62, 0xb3, 0xba, 0x47, 0x6e, 0xb6, 0x83, 0xa6, 0x90, 0x77, - 0xa5, 0x72, 0xb1, 0x56, 0xf2, 0x7e, 0x17, 0x60, 0x2e, 0x3e, 0x21, 0x1b, 0x00, 0x22, 0x64, 0xdc, - 0xf7, 0x35, 0x9a, 0x78, 0xe4, 0x2d, 0xd0, 0x8a, 0x08, 0xdf, 0xc6, 0x81, 0xc8, 0xfd, 0x91, 0x76, - 0x32, 0xf3, 0xdc, 0x3a, 0x32, 0xe3, 0x95, 0x4e, 0x5a, 0x75, 0x81, 0xd2, 0x99, 0xb1, 0x42, 0x6b, - 0xb9, 0x46, 0x9c, 0x46, 0x71, 0xb2, 0x0f, 0xcb, 0x77, 0x98, 0xae, 0x4c, 0x97, 0xfc, 0x29, 0x06, - 0x7b, 0x0e, 0x2b, 0x77, 0x19, 0xa9, 0x4c, 0xeb, 0xfe, 0x14, 0xd3, 0xb4, 0xe1, 0x73, 0x39, 0xfd, - 0x47, 0xf4, 0xe6, 0xdc, 0x4f, 0x62, 0xff, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xa3, 0x36, 0x86, - 0xa6, 0x4a, 0x06, 0x00, 0x00, +func init() { + proto.RegisterFile("grpc_lb_v1/messages/messages.proto", fileDescriptor_messages_b81c731f0e83edbd) +} + +var fileDescriptor_messages_b81c731f0e83edbd = []byte{ + // 731 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xdd, 0x4e, 0x1b, 0x39, + 0x14, 0x26, 0x9b, 0x00, 0xc9, 0x09, 0x5a, 0xb2, 0x26, 0x0b, 0x81, 0x05, 0x89, 0x1d, 0x69, 0xd9, + 0x68, 0xc5, 0x4e, 0x04, 0xd9, 0xbd, 0xe8, 0xcf, 0x45, 0x1b, 0x10, 0x0a, 0x2d, 0x17, 0x95, 0x43, + 0x55, 0xa9, 0x52, 0x65, 0x39, 0x19, 0x33, 0x58, 0x38, 0xf6, 0xd4, 0x76, 0x82, 0xfa, 0x08, 0x7d, + 0x94, 0x3e, 0x46, 0xd5, 0x67, 0xe8, 0xfb, 0x54, 0xe3, 0x99, 0xc9, 0x0c, 0x10, 0x40, 0xbd, 0x89, + 0xec, 0xe3, 0xef, 0x7c, 0xdf, 0xf1, 0x89, 0xbf, 0x33, 0xe0, 0x85, 0x3a, 0x1a, 0x11, 0x31, 0x24, + 0xd3, 0x83, 0xce, 0x98, 0x19, 0x43, 0x43, 0x66, 0x66, 0x0b, 0x3f, 0xd2, 0xca, 0x2a, 0x04, 0x31, + 0xc6, 0x17, 0x43, 0x7f, 0x7a, 0xe0, 0x3d, 0x85, 0xea, 0xf1, 0x44, 0x53, 0xcb, 0x95, 0x44, 0x2d, + 0x58, 0x36, 0x6c, 0xa4, 0x64, 0x60, 0x5a, 0xa5, 0xdd, 0x52, 0xbb, 0x8c, 0xb3, 0x2d, 0x6a, 0xc2, + 0xa2, 0xa4, 0x52, 0x99, 0xd6, 0x2f, 0xbb, 0xa5, 0xf6, 0x22, 0x4e, 0x36, 0xde, 0x33, 0xa8, 0x9d, + 0xf3, 0x31, 0x33, 0x96, 0x8e, 0xa3, 0x9f, 0x4e, 0xfe, 0x5a, 0x02, 0x74, 0xa6, 0x68, 0xd0, 0xa3, + 0x82, 0xca, 0x11, 0xc3, 0xec, 0xe3, 0x84, 0x19, 0x8b, 0xde, 0xc0, 0x2a, 0x97, 0xdc, 0x72, 0x2a, + 0x88, 0x4e, 0x42, 0x8e, 0xae, 0x7e, 0xf8, 0x97, 0x9f, 0x57, 0xed, 0x9f, 0x26, 0x90, 0xbb, 0xf9, + 0xfd, 0x05, 0xfc, 0x6b, 0x9a, 0x9f, 0x31, 0x3e, 0x87, 0x95, 0x91, 0xe0, 0x4c, 0x5a, 0x62, 0x2c, + 0xb5, 0x49, 0x15, 0xf5, 0xc3, 0x8d, 0x22, 0xdd, 0x91, 0x3b, 0x1f, 0xc4, 0xc7, 0xfd, 0x05, 0x5c, + 0x1f, 0xe5, 0xdb, 0xde, 0x1f, 0xb0, 0x29, 0x14, 0x0d, 0xc8, 0x30, 0x91, 0xc9, 0x8a, 0x22, 0xf6, + 0x53, 0xc4, 0xbc, 0x0e, 0x6c, 0xde, 0x5b, 0x09, 0x42, 0x50, 0x91, 0x74, 0xcc, 0x5c, 0xf9, 0x35, + 0xec, 0xd6, 0xde, 0xe7, 0x0a, 0xd4, 0x0b, 0x62, 0xa8, 0x0b, 0x35, 0x9b, 0x75, 0x30, 0xbd, 0xe7, + 0xef, 0xc5, 0xc2, 0x66, 0xed, 0xc5, 0x39, 0x0e, 0xfd, 0x03, 0xbf, 0xc9, 0xc9, 0x98, 0x8c, 0xa8, + 0x10, 0x26, 0xbe, 0x93, 0xb6, 0x2c, 0x70, 0xb7, 0x2a, 0xe3, 0x55, 0x39, 0x19, 0x1f, 0xc5, 0xf1, + 0x41, 0x12, 0x46, 0xfb, 0x80, 0x72, 0xec, 0x05, 0x97, 0xdc, 0x5c, 0xb2, 0xa0, 0x55, 0x76, 0xe0, + 0x46, 0x06, 0x3e, 0x49, 0xe3, 0x88, 0x80, 0x7f, 0x17, 0x4d, 0xae, 0xb9, 0xbd, 0x24, 0x81, 0x56, + 0x11, 0xb9, 0x50, 0x9a, 0x68, 0x6a, 0x19, 0x11, 0x7c, 0xcc, 0x2d, 0x97, 0x61, 0xab, 0xe2, 0x98, + 0xfe, 0xbe, 0xcd, 0xf4, 0x8e, 0xdb, 0xcb, 0x63, 0xad, 0xa2, 0x13, 0xa5, 0x31, 0xb5, 0xec, 0x2c, + 0x85, 0x23, 0x0a, 0x9d, 0x47, 0x05, 0x0a, 0xed, 0x8e, 0x15, 0x16, 0x9d, 0x42, 0xfb, 0x01, 0x85, + 0xbc, 0xf7, 0xb1, 0xc4, 0x07, 0xf8, 0xf7, 0x3e, 0x89, 0xf4, 0x19, 0x5c, 0x50, 0x2e, 0x58, 0x40, + 0xac, 0x22, 0x86, 0xc9, 0xa0, 0xb5, 0xe4, 0x04, 0xf6, 0xe6, 0x09, 0x24, 0x7f, 0xd5, 0x89, 0xc3, + 0x9f, 0xab, 0x01, 0x93, 0x01, 0xea, 0xc3, 0x9f, 0x73, 0xe8, 0xaf, 0xa4, 0xba, 0x96, 0x44, 0xb3, + 0x11, 0xe3, 0x53, 0x16, 0xb4, 0x96, 0x1d, 0xe5, 0xce, 0x6d, 0xca, 0xd7, 0x31, 0x0a, 0xa7, 0x20, + 0xef, 0x5b, 0x09, 0xd6, 0x6e, 0x3c, 0x1b, 0x13, 0x29, 0x69, 0x18, 0x1a, 0x40, 0x23, 0x77, 0x40, + 0x12, 0x4b, 0x9f, 0xc6, 0xde, 0x63, 0x16, 0x48, 0xd0, 0xfd, 0x05, 0xbc, 0x3a, 0xf3, 0x40, 0x4a, + 0xfa, 0x04, 0xea, 0x86, 0xe9, 0x29, 0xd3, 0x44, 0x70, 0x63, 0x53, 0x0f, 0xac, 0x17, 0xf9, 0x06, + 0xee, 0xf8, 0x8c, 0x3b, 0x0f, 0x81, 0x99, 0xed, 0x7a, 0xdb, 0xb0, 0x75, 0xcb, 0x01, 0x09, 0x67, + 0x62, 0x81, 0x2f, 0x25, 0xd8, 0xba, 0xbf, 0x14, 0xf4, 0x1f, 0xac, 0x17, 0x93, 0x35, 0x09, 0x98, + 0x60, 0x21, 0xb5, 0x99, 0x2d, 0x9a, 0x22, 0x4f, 0xd2, 0xc7, 0xe9, 0x19, 0x7a, 0x0b, 0xdb, 0x45, + 0xcb, 0x12, 0xcd, 0x22, 0xa5, 0x2d, 0xe1, 0xd2, 0x32, 0x3d, 0xa5, 0x22, 0x2d, 0xbf, 0x59, 0x2c, + 0x3f, 0x1b, 0x62, 0x78, 0xb3, 0xe0, 0x5e, 0xec, 0xf2, 0x4e, 0xd3, 0x34, 0xef, 0x05, 0x40, 0x7e, + 0x4b, 0xb4, 0x1f, 0x0f, 0xac, 0x78, 0x17, 0x0f, 0xac, 0x72, 0xbb, 0x7e, 0x88, 0xee, 0xb6, 0x03, + 0x67, 0x90, 0x57, 0x95, 0x6a, 0xb9, 0x51, 0xf1, 0xbe, 0x97, 0x60, 0x29, 0x39, 0x41, 0x3b, 0x00, + 0x3c, 0x22, 0x34, 0x08, 0x34, 0x33, 0xc9, 0xc8, 0x5b, 0xc1, 0x35, 0x1e, 0xbd, 0x4c, 0x02, 0xb1, + 0xfb, 0x63, 0xed, 0x74, 0xe6, 0xb9, 0x75, 0x6c, 0xc6, 0x1b, 0x9d, 0xb4, 0xea, 0x8a, 0x49, 0x67, + 0xc6, 0x1a, 0x6e, 0x14, 0x1a, 0x71, 0x1e, 0xc7, 0x51, 0x17, 0xd6, 0x1f, 0x30, 0x5d, 0x15, 0xaf, + 0x05, 0x73, 0x0c, 0xf6, 0x3f, 0x6c, 0x3c, 0x64, 0xa4, 0x2a, 0x6e, 0x06, 0x73, 0x4c, 0xd3, 0xeb, + 0xbe, 0x3f, 0x08, 0x95, 0x0a, 0x05, 0xf3, 0x43, 0x25, 0xa8, 0x0c, 0x7d, 0xa5, 0xc3, 0x4e, 0xdc, + 0x0d, 0xf7, 0x23, 0x86, 0x9d, 0x39, 0x5f, 0x95, 0xe1, 0x92, 0xfb, 0x9a, 0x74, 0x7f, 0x04, 0x00, + 0x00, 0xff, 0xff, 0x8e, 0xd0, 0x70, 0xb7, 0x73, 0x06, 0x00, 0x00, } diff --git a/vendor/google.golang.org/grpc/grpclb_picker.go b/vendor/google.golang.org/grpc/grpclb_picker.go new file mode 100644 index 00000000000..872c7ccea0e --- /dev/null +++ b/vendor/google.golang.org/grpc/grpclb_picker.go @@ -0,0 +1,159 @@ +/* + * + * Copyright 2017 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package grpc + +import ( + "sync" + "sync/atomic" + + "golang.org/x/net/context" + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/codes" + lbpb "google.golang.org/grpc/grpclb/grpc_lb_v1/messages" + "google.golang.org/grpc/status" +) + +type rpcStats struct { + NumCallsStarted int64 + NumCallsFinished int64 + NumCallsFinishedWithDropForRateLimiting int64 + NumCallsFinishedWithDropForLoadBalancing int64 + NumCallsFinishedWithClientFailedToSend int64 + NumCallsFinishedKnownReceived int64 +} + +// toClientStats converts rpcStats to lbpb.ClientStats, and clears rpcStats. +func (s *rpcStats) toClientStats() *lbpb.ClientStats { + stats := &lbpb.ClientStats{ + NumCallsStarted: atomic.SwapInt64(&s.NumCallsStarted, 0), + NumCallsFinished: atomic.SwapInt64(&s.NumCallsFinished, 0), + NumCallsFinishedWithDropForRateLimiting: atomic.SwapInt64(&s.NumCallsFinishedWithDropForRateLimiting, 0), + NumCallsFinishedWithDropForLoadBalancing: atomic.SwapInt64(&s.NumCallsFinishedWithDropForLoadBalancing, 0), + NumCallsFinishedWithClientFailedToSend: atomic.SwapInt64(&s.NumCallsFinishedWithClientFailedToSend, 0), + NumCallsFinishedKnownReceived: atomic.SwapInt64(&s.NumCallsFinishedKnownReceived, 0), + } + return stats +} + +func (s *rpcStats) dropForRateLimiting() { + atomic.AddInt64(&s.NumCallsStarted, 1) + atomic.AddInt64(&s.NumCallsFinishedWithDropForRateLimiting, 1) + atomic.AddInt64(&s.NumCallsFinished, 1) +} + +func (s *rpcStats) dropForLoadBalancing() { + atomic.AddInt64(&s.NumCallsStarted, 1) + atomic.AddInt64(&s.NumCallsFinishedWithDropForLoadBalancing, 1) + atomic.AddInt64(&s.NumCallsFinished, 1) +} + +func (s *rpcStats) failedToSend() { + atomic.AddInt64(&s.NumCallsStarted, 1) + atomic.AddInt64(&s.NumCallsFinishedWithClientFailedToSend, 1) + atomic.AddInt64(&s.NumCallsFinished, 1) +} + +func (s *rpcStats) knownReceived() { + atomic.AddInt64(&s.NumCallsStarted, 1) + atomic.AddInt64(&s.NumCallsFinishedKnownReceived, 1) + atomic.AddInt64(&s.NumCallsFinished, 1) +} + +type errPicker struct { + // Pick always returns this err. + err error +} + +func (p *errPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) { + return nil, nil, p.err +} + +// rrPicker does roundrobin on subConns. It's typically used when there's no +// response from remote balancer, and grpclb falls back to the resolved +// backends. +// +// It guaranteed that len(subConns) > 0. +type rrPicker struct { + mu sync.Mutex + subConns []balancer.SubConn // The subConns that were READY when taking the snapshot. + subConnsNext int +} + +func (p *rrPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) { + p.mu.Lock() + defer p.mu.Unlock() + sc := p.subConns[p.subConnsNext] + p.subConnsNext = (p.subConnsNext + 1) % len(p.subConns) + return sc, nil, nil +} + +// lbPicker does two layers of picks: +// +// First layer: roundrobin on all servers in serverList, including drops and backends. +// - If it picks a drop, the RPC will fail as being dropped. +// - If it picks a backend, do a second layer pick to pick the real backend. +// +// Second layer: roundrobin on all READY backends. +// +// It's guaranteed that len(serverList) > 0. +type lbPicker struct { + mu sync.Mutex + serverList []*lbpb.Server + serverListNext int + subConns []balancer.SubConn // The subConns that were READY when taking the snapshot. + subConnsNext int + + stats *rpcStats +} + +func (p *lbPicker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer.SubConn, func(balancer.DoneInfo), error) { + p.mu.Lock() + defer p.mu.Unlock() + + // Layer one roundrobin on serverList. + s := p.serverList[p.serverListNext] + p.serverListNext = (p.serverListNext + 1) % len(p.serverList) + + // If it's a drop, return an error and fail the RPC. + if s.DropForRateLimiting { + p.stats.dropForRateLimiting() + return nil, nil, status.Errorf(codes.Unavailable, "request dropped by grpclb") + } + if s.DropForLoadBalancing { + p.stats.dropForLoadBalancing() + return nil, nil, status.Errorf(codes.Unavailable, "request dropped by grpclb") + } + + // If not a drop but there's no ready subConns. + if len(p.subConns) <= 0 { + return nil, nil, balancer.ErrNoSubConnAvailable + } + + // Return the next ready subConn in the list, also collect rpc stats. + sc := p.subConns[p.subConnsNext] + p.subConnsNext = (p.subConnsNext + 1) % len(p.subConns) + done := func(info balancer.DoneInfo) { + if !info.BytesSent { + p.stats.failedToSend() + } else if info.BytesReceived { + p.stats.knownReceived() + } + } + return sc, done, nil +} diff --git a/vendor/google.golang.org/grpc/grpclb_remote_balancer.go b/vendor/google.golang.org/grpc/grpclb_remote_balancer.go new file mode 100644 index 00000000000..b8dd4f18ce5 --- /dev/null +++ b/vendor/google.golang.org/grpc/grpclb_remote_balancer.go @@ -0,0 +1,266 @@ +/* + * + * Copyright 2017 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package grpc + +import ( + "fmt" + "net" + "reflect" + "time" + + "golang.org/x/net/context" + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/channelz" + + "google.golang.org/grpc/connectivity" + lbpb "google.golang.org/grpc/grpclb/grpc_lb_v1/messages" + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/resolver" +) + +// processServerList updates balaner's internal state, create/remove SubConns +// and regenerates picker using the received serverList. +func (lb *lbBalancer) processServerList(l *lbpb.ServerList) { + grpclog.Infof("lbBalancer: processing server list: %+v", l) + lb.mu.Lock() + defer lb.mu.Unlock() + + // Set serverListReceived to true so fallback will not take effect if it has + // not hit timeout. + lb.serverListReceived = true + + // If the new server list == old server list, do nothing. + if reflect.DeepEqual(lb.fullServerList, l.Servers) { + grpclog.Infof("lbBalancer: new serverlist same as the previous one, ignoring") + return + } + lb.fullServerList = l.Servers + + var backendAddrs []resolver.Address + for _, s := range l.Servers { + if s.DropForLoadBalancing || s.DropForRateLimiting { + continue + } + + md := metadata.Pairs(lbTokeyKey, s.LoadBalanceToken) + ip := net.IP(s.IpAddress) + ipStr := ip.String() + if ip.To4() == nil { + // Add square brackets to ipv6 addresses, otherwise net.Dial() and + // net.SplitHostPort() will return too many colons error. + ipStr = fmt.Sprintf("[%s]", ipStr) + } + addr := resolver.Address{ + Addr: fmt.Sprintf("%s:%d", ipStr, s.Port), + Metadata: &md, + } + + backendAddrs = append(backendAddrs, addr) + } + + // Call refreshSubConns to create/remove SubConns. + lb.refreshSubConns(backendAddrs) + // Regenerate and update picker no matter if there's update on backends (if + // any SubConn will be newed/removed). Because since the full serverList was + // different, there might be updates in drops or pick weights(different + // number of duplicates). We need to update picker with the fulllist. + // + // Now with cache, even if SubConn was newed/removed, there might be no + // state changes. + lb.regeneratePicker() + lb.cc.UpdateBalancerState(lb.state, lb.picker) +} + +// refreshSubConns creates/removes SubConns with backendAddrs. It returns a bool +// indicating whether the backendAddrs are different from the cached +// backendAddrs (whether any SubConn was newed/removed). +// Caller must hold lb.mu. +func (lb *lbBalancer) refreshSubConns(backendAddrs []resolver.Address) bool { + lb.backendAddrs = nil + var backendsUpdated bool + // addrsSet is the set converted from backendAddrs, it's used to quick + // lookup for an address. + addrsSet := make(map[resolver.Address]struct{}) + // Create new SubConns. + for _, addr := range backendAddrs { + addrWithoutMD := addr + addrWithoutMD.Metadata = nil + addrsSet[addrWithoutMD] = struct{}{} + lb.backendAddrs = append(lb.backendAddrs, addrWithoutMD) + + if _, ok := lb.subConns[addrWithoutMD]; !ok { + backendsUpdated = true + + // Use addrWithMD to create the SubConn. + sc, err := lb.cc.NewSubConn([]resolver.Address{addr}, balancer.NewSubConnOptions{}) + if err != nil { + grpclog.Warningf("roundrobinBalancer: failed to create new SubConn: %v", err) + continue + } + lb.subConns[addrWithoutMD] = sc // Use the addr without MD as key for the map. + if _, ok := lb.scStates[sc]; !ok { + // Only set state of new sc to IDLE. The state could already be + // READY for cached SubConns. + lb.scStates[sc] = connectivity.Idle + } + sc.Connect() + } + } + + for a, sc := range lb.subConns { + // a was removed by resolver. + if _, ok := addrsSet[a]; !ok { + backendsUpdated = true + + lb.cc.RemoveSubConn(sc) + delete(lb.subConns, a) + // Keep the state of this sc in b.scStates until sc's state becomes Shutdown. + // The entry will be deleted in HandleSubConnStateChange. + } + } + + return backendsUpdated +} + +func (lb *lbBalancer) readServerList(s *balanceLoadClientStream) error { + for { + reply, err := s.Recv() + if err != nil { + return fmt.Errorf("grpclb: failed to recv server list: %v", err) + } + if serverList := reply.GetServerList(); serverList != nil { + lb.processServerList(serverList) + } + } +} + +func (lb *lbBalancer) sendLoadReport(s *balanceLoadClientStream, interval time.Duration) { + ticker := time.NewTicker(interval) + defer ticker.Stop() + for { + select { + case <-ticker.C: + case <-s.Context().Done(): + return + } + stats := lb.clientStats.toClientStats() + t := time.Now() + stats.Timestamp = &lbpb.Timestamp{ + Seconds: t.Unix(), + Nanos: int32(t.Nanosecond()), + } + if err := s.Send(&lbpb.LoadBalanceRequest{ + LoadBalanceRequestType: &lbpb.LoadBalanceRequest_ClientStats{ + ClientStats: stats, + }, + }); err != nil { + return + } + } +} + +func (lb *lbBalancer) callRemoteBalancer() error { + lbClient := &loadBalancerClient{cc: lb.ccRemoteLB} + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + stream, err := lbClient.BalanceLoad(ctx, FailFast(false)) + if err != nil { + return fmt.Errorf("grpclb: failed to perform RPC to the remote balancer %v", err) + } + + // grpclb handshake on the stream. + initReq := &lbpb.LoadBalanceRequest{ + LoadBalanceRequestType: &lbpb.LoadBalanceRequest_InitialRequest{ + InitialRequest: &lbpb.InitialLoadBalanceRequest{ + Name: lb.target, + }, + }, + } + if err := stream.Send(initReq); err != nil { + return fmt.Errorf("grpclb: failed to send init request: %v", err) + } + reply, err := stream.Recv() + if err != nil { + return fmt.Errorf("grpclb: failed to recv init response: %v", err) + } + initResp := reply.GetInitialResponse() + if initResp == nil { + return fmt.Errorf("grpclb: reply from remote balancer did not include initial response") + } + if initResp.LoadBalancerDelegate != "" { + return fmt.Errorf("grpclb: Delegation is not supported") + } + + go func() { + if d := convertDuration(initResp.ClientStatsReportInterval); d > 0 { + lb.sendLoadReport(stream, d) + } + }() + return lb.readServerList(stream) +} + +func (lb *lbBalancer) watchRemoteBalancer() { + for { + err := lb.callRemoteBalancer() + select { + case <-lb.doneCh: + return + default: + if err != nil { + grpclog.Error(err) + } + } + + } +} + +func (lb *lbBalancer) dialRemoteLB(remoteLBName string) { + var dopts []DialOption + if creds := lb.opt.DialCreds; creds != nil { + if err := creds.OverrideServerName(remoteLBName); err == nil { + dopts = append(dopts, WithTransportCredentials(creds)) + } else { + grpclog.Warningf("grpclb: failed to override the server name in the credentials: %v, using Insecure", err) + dopts = append(dopts, WithInsecure()) + } + } else { + dopts = append(dopts, WithInsecure()) + } + if lb.opt.Dialer != nil { + // WithDialer takes a different type of function, so we instead use a + // special DialOption here. + dopts = append(dopts, withContextDialer(lb.opt.Dialer)) + } + // Explicitly set pickfirst as the balancer. + dopts = append(dopts, WithBalancerName(PickFirstBalancerName)) + dopts = append(dopts, withResolverBuilder(lb.manualResolver)) + if channelz.IsOn() { + dopts = append(dopts, WithChannelzParentID(lb.opt.ChannelzParentID)) + } + + // DialContext using manualResolver.Scheme, which is a random scheme generated + // when init grpclb. The target name is not important. + cc, err := DialContext(context.Background(), "grpclb:///grpclb.server", dopts...) + if err != nil { + grpclog.Fatalf("failed to dial: %v", err) + } + lb.ccRemoteLB = cc + go lb.watchRemoteBalancer() +} diff --git a/vendor/google.golang.org/grpc/grpclb_util.go b/vendor/google.golang.org/grpc/grpclb_util.go new file mode 100644 index 00000000000..063ba9d8590 --- /dev/null +++ b/vendor/google.golang.org/grpc/grpclb_util.go @@ -0,0 +1,214 @@ +/* + * + * Copyright 2016 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package grpc + +import ( + "fmt" + "sync" + "time" + + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/resolver" +) + +// The parent ClientConn should re-resolve when grpclb loses connection to the +// remote balancer. When the ClientConn inside grpclb gets a TransientFailure, +// it calls lbManualResolver.ResolveNow(), which calls parent ClientConn's +// ResolveNow, and eventually results in re-resolve happening in parent +// ClientConn's resolver (DNS for example). +// +// parent +// ClientConn +// +-----------------------------------------------------------------+ +// | parent +---------------------------------+ | +// | DNS ClientConn | grpclb | | +// | resolver balancerWrapper | | | +// | + + | grpclb grpclb | | +// | | | | ManualResolver ClientConn | | +// | | | | + + | | +// | | | | | | Transient | | +// | | | | | | Failure | | +// | | | | | <--------- | | | +// | | | <--------------- | ResolveNow | | | +// | | <--------- | ResolveNow | | | | | +// | | ResolveNow | | | | | | +// | | | | | | | | +// | + + | + + | | +// | +---------------------------------+ | +// +-----------------------------------------------------------------+ + +// lbManualResolver is used by the ClientConn inside grpclb. It's a manual +// resolver with a special ResolveNow() function. +// +// When ResolveNow() is called, it calls ResolveNow() on the parent ClientConn, +// so when grpclb client lose contact with remote balancers, the parent +// ClientConn's resolver will re-resolve. +type lbManualResolver struct { + scheme string + ccr resolver.ClientConn + + ccb balancer.ClientConn +} + +func (r *lbManualResolver) Build(_ resolver.Target, cc resolver.ClientConn, _ resolver.BuildOption) (resolver.Resolver, error) { + r.ccr = cc + return r, nil +} + +func (r *lbManualResolver) Scheme() string { + return r.scheme +} + +// ResolveNow calls resolveNow on the parent ClientConn. +func (r *lbManualResolver) ResolveNow(o resolver.ResolveNowOption) { + r.ccb.ResolveNow(o) +} + +// Close is a noop for Resolver. +func (*lbManualResolver) Close() {} + +// NewAddress calls cc.NewAddress. +func (r *lbManualResolver) NewAddress(addrs []resolver.Address) { + r.ccr.NewAddress(addrs) +} + +// NewServiceConfig calls cc.NewServiceConfig. +func (r *lbManualResolver) NewServiceConfig(sc string) { + r.ccr.NewServiceConfig(sc) +} + +const subConnCacheTime = time.Second * 10 + +// lbCacheClientConn is a wrapper balancer.ClientConn with a SubConn cache. +// SubConns will be kept in cache for subConnCacheTime before being removed. +// +// Its new and remove methods are updated to do cache first. +type lbCacheClientConn struct { + cc balancer.ClientConn + timeout time.Duration + + mu sync.Mutex + // subConnCache only keeps subConns that are being deleted. + subConnCache map[resolver.Address]*subConnCacheEntry + subConnToAddr map[balancer.SubConn]resolver.Address +} + +type subConnCacheEntry struct { + sc balancer.SubConn + + cancel func() + abortDeleting bool +} + +func newLBCacheClientConn(cc balancer.ClientConn) *lbCacheClientConn { + return &lbCacheClientConn{ + cc: cc, + timeout: subConnCacheTime, + subConnCache: make(map[resolver.Address]*subConnCacheEntry), + subConnToAddr: make(map[balancer.SubConn]resolver.Address), + } +} + +func (ccc *lbCacheClientConn) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) { + if len(addrs) != 1 { + return nil, fmt.Errorf("grpclb calling NewSubConn with addrs of length %v", len(addrs)) + } + addrWithoutMD := addrs[0] + addrWithoutMD.Metadata = nil + + ccc.mu.Lock() + defer ccc.mu.Unlock() + if entry, ok := ccc.subConnCache[addrWithoutMD]; ok { + // If entry is in subConnCache, the SubConn was being deleted. + // cancel function will never be nil. + entry.cancel() + delete(ccc.subConnCache, addrWithoutMD) + return entry.sc, nil + } + + scNew, err := ccc.cc.NewSubConn(addrs, opts) + if err != nil { + return nil, err + } + + ccc.subConnToAddr[scNew] = addrWithoutMD + return scNew, nil +} + +func (ccc *lbCacheClientConn) RemoveSubConn(sc balancer.SubConn) { + ccc.mu.Lock() + defer ccc.mu.Unlock() + addr, ok := ccc.subConnToAddr[sc] + if !ok { + return + } + + if entry, ok := ccc.subConnCache[addr]; ok { + if entry.sc != sc { + // This could happen if NewSubConn was called multiple times for the + // same address, and those SubConns are all removed. We remove sc + // immediately here. + delete(ccc.subConnToAddr, sc) + ccc.cc.RemoveSubConn(sc) + } + return + } + + entry := &subConnCacheEntry{ + sc: sc, + } + ccc.subConnCache[addr] = entry + + timer := time.AfterFunc(ccc.timeout, func() { + ccc.mu.Lock() + if entry.abortDeleting { + return + } + ccc.cc.RemoveSubConn(sc) + delete(ccc.subConnToAddr, sc) + delete(ccc.subConnCache, addr) + ccc.mu.Unlock() + }) + entry.cancel = func() { + if !timer.Stop() { + // If stop was not successful, the timer has fired (this can only + // happen in a race). But the deleting function is blocked on ccc.mu + // because the mutex was held by the caller of this function. + // + // Set abortDeleting to true to abort the deleting function. When + // the lock is released, the deleting function will acquire the + // lock, check the value of abortDeleting and return. + entry.abortDeleting = true + } + } +} + +func (ccc *lbCacheClientConn) UpdateBalancerState(s connectivity.State, p balancer.Picker) { + ccc.cc.UpdateBalancerState(s, p) +} + +func (ccc *lbCacheClientConn) close() { + ccc.mu.Lock() + // Only cancel all existing timers. There's no need to remove SubConns. + for _, entry := range ccc.subConnCache { + entry.cancel() + } + ccc.mu.Unlock() +} diff --git a/vendor/google.golang.org/grpc/grpclog/grpclog.go b/vendor/google.golang.org/grpc/grpclog/grpclog.go index 16a7d88867e..1fabb11e1ba 100644 --- a/vendor/google.golang.org/grpc/grpclog/grpclog.go +++ b/vendor/google.golang.org/grpc/grpclog/grpclog.go @@ -105,18 +105,21 @@ func Fatalln(args ...interface{}) { } // Print prints to the logger. Arguments are handled in the manner of fmt.Print. +// // Deprecated: use Info. func Print(args ...interface{}) { logger.Info(args...) } // Printf prints to the logger. Arguments are handled in the manner of fmt.Printf. +// // Deprecated: use Infof. func Printf(format string, args ...interface{}) { logger.Infof(format, args...) } // Println prints to the logger. Arguments are handled in the manner of fmt.Println. +// // Deprecated: use Infoln. func Println(args ...interface{}) { logger.Infoln(args...) diff --git a/vendor/google.golang.org/grpc/grpclog/logger.go b/vendor/google.golang.org/grpc/grpclog/logger.go index d03b2397bfa..097494f710f 100644 --- a/vendor/google.golang.org/grpc/grpclog/logger.go +++ b/vendor/google.golang.org/grpc/grpclog/logger.go @@ -19,6 +19,7 @@ package grpclog // Logger mimics golang's standard Logger as an interface. +// // Deprecated: use LoggerV2. type Logger interface { Fatal(args ...interface{}) @@ -31,6 +32,7 @@ type Logger interface { // SetLogger sets the logger that is used in grpc. Call only from // init() functions. +// // Deprecated: use SetLoggerV2. func SetLogger(l Logger) { logger = &loggerWrapper{Logger: l} diff --git a/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go b/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go index fdcbb9e0b7d..e5906de7d43 100644 --- a/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go +++ b/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go @@ -1,17 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // source: grpc_health_v1/health.proto -/* -Package grpc_health_v1 is a generated protocol buffer package. - -It is generated from these files: - grpc_health_v1/health.proto - -It has these top-level messages: - HealthCheckRequest - HealthCheckResponse -*/ -package grpc_health_v1 +package grpc_health_v1 // import "google.golang.org/grpc/health/grpc_health_v1" import proto "github.com/golang/protobuf/proto" import fmt "fmt" @@ -56,17 +46,39 @@ func (x HealthCheckResponse_ServingStatus) String() string { return proto.EnumName(HealthCheckResponse_ServingStatus_name, int32(x)) } func (HealthCheckResponse_ServingStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor0, []int{1, 0} + return fileDescriptor_health_8e5b8a3074428511, []int{1, 0} } type HealthCheckRequest struct { - Service string `protobuf:"bytes,1,opt,name=service" json:"service,omitempty"` + Service string `protobuf:"bytes,1,opt,name=service" json:"service,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *HealthCheckRequest) Reset() { *m = HealthCheckRequest{} } +func (m *HealthCheckRequest) String() string { return proto.CompactTextString(m) } +func (*HealthCheckRequest) ProtoMessage() {} +func (*HealthCheckRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_health_8e5b8a3074428511, []int{0} +} +func (m *HealthCheckRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_HealthCheckRequest.Unmarshal(m, b) +} +func (m *HealthCheckRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_HealthCheckRequest.Marshal(b, m, deterministic) +} +func (dst *HealthCheckRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_HealthCheckRequest.Merge(dst, src) +} +func (m *HealthCheckRequest) XXX_Size() int { + return xxx_messageInfo_HealthCheckRequest.Size(m) +} +func (m *HealthCheckRequest) XXX_DiscardUnknown() { + xxx_messageInfo_HealthCheckRequest.DiscardUnknown(m) } -func (m *HealthCheckRequest) Reset() { *m = HealthCheckRequest{} } -func (m *HealthCheckRequest) String() string { return proto.CompactTextString(m) } -func (*HealthCheckRequest) ProtoMessage() {} -func (*HealthCheckRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } +var xxx_messageInfo_HealthCheckRequest proto.InternalMessageInfo func (m *HealthCheckRequest) GetService() string { if m != nil { @@ -76,13 +88,35 @@ func (m *HealthCheckRequest) GetService() string { } type HealthCheckResponse struct { - Status HealthCheckResponse_ServingStatus `protobuf:"varint,1,opt,name=status,enum=grpc.health.v1.HealthCheckResponse_ServingStatus" json:"status,omitempty"` + Status HealthCheckResponse_ServingStatus `protobuf:"varint,1,opt,name=status,enum=grpc.health.v1.HealthCheckResponse_ServingStatus" json:"status,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *HealthCheckResponse) Reset() { *m = HealthCheckResponse{} } +func (m *HealthCheckResponse) String() string { return proto.CompactTextString(m) } +func (*HealthCheckResponse) ProtoMessage() {} +func (*HealthCheckResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_health_8e5b8a3074428511, []int{1} +} +func (m *HealthCheckResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_HealthCheckResponse.Unmarshal(m, b) +} +func (m *HealthCheckResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_HealthCheckResponse.Marshal(b, m, deterministic) +} +func (dst *HealthCheckResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_HealthCheckResponse.Merge(dst, src) +} +func (m *HealthCheckResponse) XXX_Size() int { + return xxx_messageInfo_HealthCheckResponse.Size(m) +} +func (m *HealthCheckResponse) XXX_DiscardUnknown() { + xxx_messageInfo_HealthCheckResponse.DiscardUnknown(m) } -func (m *HealthCheckResponse) Reset() { *m = HealthCheckResponse{} } -func (m *HealthCheckResponse) String() string { return proto.CompactTextString(m) } -func (*HealthCheckResponse) ProtoMessage() {} -func (*HealthCheckResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } +var xxx_messageInfo_HealthCheckResponse proto.InternalMessageInfo func (m *HealthCheckResponse) GetStatus() HealthCheckResponse_ServingStatus { if m != nil { @@ -169,10 +203,10 @@ var _Health_serviceDesc = grpc.ServiceDesc{ Metadata: "grpc_health_v1/health.proto", } -func init() { proto.RegisterFile("grpc_health_v1/health.proto", fileDescriptor0) } +func init() { proto.RegisterFile("grpc_health_v1/health.proto", fileDescriptor_health_8e5b8a3074428511) } -var fileDescriptor0 = []byte{ - // 213 bytes of a gzipped FileDescriptorProto +var fileDescriptor_health_8e5b8a3074428511 = []byte{ + // 269 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4e, 0x2f, 0x2a, 0x48, 0x8e, 0xcf, 0x48, 0x4d, 0xcc, 0x29, 0xc9, 0x88, 0x2f, 0x33, 0xd4, 0x87, 0xb0, 0xf4, 0x0a, 0x8a, 0xf2, 0x4b, 0xf2, 0x85, 0xf8, 0x40, 0x92, 0x7a, 0x50, 0xa1, 0x32, 0x43, 0x25, 0x3d, 0x2e, 0x21, @@ -185,6 +219,9 @@ var fileDescriptor0 = []byte{ 0x0f, 0xf7, 0x13, 0x60, 0x00, 0x71, 0x82, 0x5d, 0x83, 0xc2, 0x3c, 0xfd, 0xdc, 0x05, 0x18, 0x85, 0xf8, 0xb9, 0xb8, 0xfd, 0xfc, 0x43, 0xe2, 0x61, 0x02, 0x4c, 0x46, 0x51, 0x5c, 0x6c, 0x10, 0x8b, 0x84, 0x02, 0xb8, 0x58, 0xc1, 0x96, 0x09, 0x29, 0xe1, 0x75, 0x09, 0xd8, 0xbf, 0x52, 0xca, 0x44, - 0xb8, 0x36, 0x89, 0x0d, 0x1c, 0x82, 0xc6, 0x80, 0x00, 0x00, 0x00, 0xff, 0xff, 0x53, 0x2b, 0x65, - 0x20, 0x60, 0x01, 0x00, 0x00, + 0xb8, 0xd6, 0x29, 0x91, 0x4b, 0x30, 0x33, 0x1f, 0x4d, 0xa1, 0x13, 0x37, 0x44, 0x65, 0x00, 0x28, + 0x70, 0x03, 0x18, 0xa3, 0x74, 0xd2, 0xf3, 0xf3, 0xd3, 0x73, 0x52, 0xf5, 0xd2, 0xf3, 0x73, 0x12, + 0xf3, 0xd2, 0xf5, 0xf2, 0x8b, 0xd2, 0xf5, 0x41, 0x1a, 0xa0, 0x71, 0xa0, 0x8f, 0x1a, 0x33, 0xab, + 0x98, 0xf8, 0xdc, 0x41, 0xa6, 0x41, 0x8c, 0xd0, 0x0b, 0x33, 0x4c, 0x62, 0x03, 0x47, 0x92, 0x31, + 0x20, 0x00, 0x00, 0xff, 0xff, 0xb7, 0x70, 0xc4, 0xa7, 0xc3, 0x01, 0x00, 0x00, } diff --git a/vendor/google.golang.org/grpc/health/health.go b/vendor/google.golang.org/grpc/health/health.go index c6212f406f7..de7f9ba79e2 100644 --- a/vendor/google.golang.org/grpc/health/health.go +++ b/vendor/google.golang.org/grpc/health/health.go @@ -16,7 +16,7 @@ * */ -//go:generate protoc --go_out=plugins=grpc:. grpc_health_v1/health.proto +//go:generate protoc --go_out=plugins=grpc,paths=source_relative:. grpc_health_v1/health.proto // Package health provides some utility functions to health-check a server. The implementation // is based on protobuf. Users need to write their own implementations if other IDLs are used. @@ -26,9 +26,9 @@ import ( "sync" "golang.org/x/net/context" - "google.golang.org/grpc" "google.golang.org/grpc/codes" healthpb "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/status" ) // Server implements `service Health`. @@ -60,7 +60,7 @@ func (s *Server) Check(ctx context.Context, in *healthpb.HealthCheckRequest) (*h Status: status, }, nil } - return nil, grpc.Errorf(codes.NotFound, "unknown service") + return nil, status.Error(codes.NotFound, "unknown service") } // SetServingStatus is called when need to reset the serving status of a service diff --git a/vendor/google.golang.org/grpc/interceptor.go b/vendor/google.golang.org/grpc/interceptor.go index 06dc825b9fb..1f6ef678035 100644 --- a/vendor/google.golang.org/grpc/interceptor.go +++ b/vendor/google.golang.org/grpc/interceptor.go @@ -48,7 +48,9 @@ type UnaryServerInfo struct { } // UnaryHandler defines the handler invoked by UnaryServerInterceptor to complete the normal -// execution of a unary RPC. +// execution of a unary RPC. If a UnaryHandler returns an error, it should be produced by the +// status package, or else gRPC will use codes.Unknown as the status code and err.Error() as +// the status message of the RPC. type UnaryHandler func(ctx context.Context, req interface{}) (interface{}, error) // UnaryServerInterceptor provides a hook to intercept the execution of a unary RPC on the server. info diff --git a/vendor/google.golang.org/grpc/internal/internal.go b/vendor/google.golang.org/grpc/internal/internal.go index 07083832c3c..53f1775201c 100644 --- a/vendor/google.golang.org/grpc/internal/internal.go +++ b/vendor/google.golang.org/grpc/internal/internal.go @@ -19,13 +19,6 @@ // the godoc of the top-level grpc package. package internal -// TestingCloseConns closes all existing transports but keeps -// grpcServer.lis accepting new connections. -// -// The provided grpcServer must be of type *grpc.Server. It is untyped -// for circular dependency reasons. -var TestingCloseConns func(grpcServer interface{}) - // TestingUseHandlerImpl enables the http.Handler-based server implementation. // It must be called before Serve and requires TLS credentials. // diff --git a/vendor/google.golang.org/grpc/metadata/metadata.go b/vendor/google.golang.org/grpc/metadata/metadata.go index ccfea5d4530..bd2eaf40837 100644 --- a/vendor/google.golang.org/grpc/metadata/metadata.go +++ b/vendor/google.golang.org/grpc/metadata/metadata.go @@ -17,7 +17,8 @@ */ // Package metadata define the structure of the metadata supported by gRPC library. -// Please refer to https://grpc.io/docs/guides/wire.html for more information about custom-metadata. +// Please refer to https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md +// for more information about custom-metadata. package metadata // import "google.golang.org/grpc/metadata" import ( @@ -27,7 +28,9 @@ import ( "golang.org/x/net/context" ) -// DecodeKeyValue returns k, v, nil. It is deprecated and should not be used. +// DecodeKeyValue returns k, v, nil. +// +// Deprecated: use k and v directly instead. func DecodeKeyValue(k, v string) (string, string, error) { return k, v, nil } @@ -94,6 +97,30 @@ func (md MD) Copy() MD { return Join(md) } +// Get obtains the values for a given key. +func (md MD) Get(k string) []string { + k = strings.ToLower(k) + return md[k] +} + +// Set sets the value of a given key with a slice of values. +func (md MD) Set(k string, vals ...string) { + if len(vals) == 0 { + return + } + k = strings.ToLower(k) + md[k] = vals +} + +// Append adds the values to key k, not overwriting what was already stored at that key. +func (md MD) Append(k string, vals ...string) { + if len(vals) == 0 { + return + } + k = strings.ToLower(k) + md[k] = append(md[k], vals...) +} + // Join joins any number of mds into a single MD. // The order of values for each key is determined by the order in which // the mds containing those values are presented to Join. @@ -115,9 +142,26 @@ func NewIncomingContext(ctx context.Context, md MD) context.Context { return context.WithValue(ctx, mdIncomingKey{}, md) } -// NewOutgoingContext creates a new context with outgoing md attached. +// NewOutgoingContext creates a new context with outgoing md attached. If used +// in conjunction with AppendToOutgoingContext, NewOutgoingContext will +// overwrite any previously-appended metadata. func NewOutgoingContext(ctx context.Context, md MD) context.Context { - return context.WithValue(ctx, mdOutgoingKey{}, md) + return context.WithValue(ctx, mdOutgoingKey{}, rawMD{md: md}) +} + +// AppendToOutgoingContext returns a new context with the provided kv merged +// with any existing metadata in the context. Please refer to the +// documentation of Pairs for a description of kv. +func AppendToOutgoingContext(ctx context.Context, kv ...string) context.Context { + if len(kv)%2 == 1 { + panic(fmt.Sprintf("metadata: AppendToOutgoingContext got an odd number of input pairs for metadata: %d", len(kv))) + } + md, _ := ctx.Value(mdOutgoingKey{}).(rawMD) + added := make([][]string, len(md.added)+1) + copy(added, md.added) + added[len(added)-1] = make([]string, len(kv)) + copy(added[len(added)-1], kv) + return context.WithValue(ctx, mdOutgoingKey{}, rawMD{md: md.md, added: added}) } // FromIncomingContext returns the incoming metadata in ctx if it exists. The @@ -128,10 +172,39 @@ func FromIncomingContext(ctx context.Context) (md MD, ok bool) { return } +// FromOutgoingContextRaw returns the un-merged, intermediary contents +// of rawMD. Remember to perform strings.ToLower on the keys. The returned +// MD should not be modified. Writing to it may cause races. Modification +// should be made to copies of the returned MD. +// +// This is intended for gRPC-internal use ONLY. +func FromOutgoingContextRaw(ctx context.Context) (MD, [][]string, bool) { + raw, ok := ctx.Value(mdOutgoingKey{}).(rawMD) + if !ok { + return nil, nil, false + } + + return raw.md, raw.added, true +} + // FromOutgoingContext returns the outgoing metadata in ctx if it exists. The // returned MD should not be modified. Writing to it may cause races. -// Modification should be made to the copies of the returned MD. -func FromOutgoingContext(ctx context.Context) (md MD, ok bool) { - md, ok = ctx.Value(mdOutgoingKey{}).(MD) - return +// Modification should be made to copies of the returned MD. +func FromOutgoingContext(ctx context.Context) (MD, bool) { + raw, ok := ctx.Value(mdOutgoingKey{}).(rawMD) + if !ok { + return nil, false + } + + mds := make([]MD, 0, len(raw.added)+1) + mds = append(mds, raw.md) + for _, vv := range raw.added { + mds = append(mds, Pairs(vv...)) + } + return Join(mds...), ok +} + +type rawMD struct { + md MD + added [][]string } diff --git a/vendor/google.golang.org/grpc/naming/dns_resolver.go b/vendor/google.golang.org/grpc/naming/dns_resolver.go index 7e69a2ca0a6..0f8a908ea9c 100644 --- a/vendor/google.golang.org/grpc/naming/dns_resolver.go +++ b/vendor/google.golang.org/grpc/naming/dns_resolver.go @@ -153,10 +153,10 @@ type ipWatcher struct { updateChan chan *Update } -// Next returns the adrress resolution Update for the target. For IP address, -// the resolution is itself, thus polling name server is unncessary. Therefore, +// Next returns the address resolution Update for the target. For IP address, +// the resolution is itself, thus polling name server is unnecessary. Therefore, // Next() will return an Update the first time it is called, and will be blocked -// for all following calls as no Update exisits until watcher is closed. +// for all following calls as no Update exists until watcher is closed. func (i *ipWatcher) Next() ([]*Update, error) { u, ok := <-i.updateChan if !ok { diff --git a/vendor/google.golang.org/grpc/naming/go17.go b/vendor/google.golang.org/grpc/naming/go17.go index 8bdf21e7998..57b65d7b889 100644 --- a/vendor/google.golang.org/grpc/naming/go17.go +++ b/vendor/google.golang.org/grpc/naming/go17.go @@ -1,4 +1,4 @@ -// +build go1.7, !go1.8 +// +build go1.6,!go1.8 /* * diff --git a/vendor/google.golang.org/grpc/naming/naming.go b/vendor/google.golang.org/grpc/naming/naming.go index 1af7e32f86d..8cc39e93758 100644 --- a/vendor/google.golang.org/grpc/naming/naming.go +++ b/vendor/google.golang.org/grpc/naming/naming.go @@ -18,20 +18,26 @@ // Package naming defines the naming API and related data structures for gRPC. // The interface is EXPERIMENTAL and may be suject to change. +// +// Deprecated: please use package resolver. package naming // Operation defines the corresponding operations for a name resolution change. +// +// Deprecated: please use package resolver. type Operation uint8 const ( // Add indicates a new address is added. Add Operation = iota - // Delete indicates an exisiting address is deleted. + // Delete indicates an existing address is deleted. Delete ) // Update defines a name resolution update. Notice that it is not valid having both // empty string Addr and nil Metadata in an Update. +// +// Deprecated: please use package resolver. type Update struct { // Op indicates the operation of the update. Op Operation @@ -43,12 +49,16 @@ type Update struct { } // Resolver creates a Watcher for a target to track its resolution changes. +// +// Deprecated: please use package resolver. type Resolver interface { // Resolve creates a Watcher for target. Resolve(target string) (Watcher, error) } // Watcher watches for the updates on the specified target. +// +// Deprecated: please use package resolver. type Watcher interface { // Next blocks until an update or error happens. It may return one or more // updates. The first call should get the full set of the results. It should diff --git a/vendor/google.golang.org/grpc/picker_wrapper.go b/vendor/google.golang.org/grpc/picker_wrapper.go index 9085dbc9c98..0a984e6c8af 100644 --- a/vendor/google.golang.org/grpc/picker_wrapper.go +++ b/vendor/google.golang.org/grpc/picker_wrapper.go @@ -19,12 +19,17 @@ package grpc import ( + "io" "sync" + "sync/atomic" "golang.org/x/net/context" "google.golang.org/grpc/balancer" + "google.golang.org/grpc/channelz" "google.golang.org/grpc/codes" "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/resolver" "google.golang.org/grpc/status" "google.golang.org/grpc/transport" ) @@ -36,13 +41,57 @@ type pickerWrapper struct { done bool blockingCh chan struct{} picker balancer.Picker + + // The latest connection happened. + connErrMu sync.Mutex + connErr error + + stickinessMDKey atomic.Value + stickiness *stickyStore } func newPickerWrapper() *pickerWrapper { - bp := &pickerWrapper{blockingCh: make(chan struct{})} + bp := &pickerWrapper{ + blockingCh: make(chan struct{}), + stickiness: newStickyStore(), + } return bp } +func (bp *pickerWrapper) updateConnectionError(err error) { + bp.connErrMu.Lock() + bp.connErr = err + bp.connErrMu.Unlock() +} + +func (bp *pickerWrapper) connectionError() error { + bp.connErrMu.Lock() + err := bp.connErr + bp.connErrMu.Unlock() + return err +} + +func (bp *pickerWrapper) updateStickinessMDKey(newKey string) { + // No need to check ok because mdKey == "" if ok == false. + if oldKey, _ := bp.stickinessMDKey.Load().(string); oldKey != newKey { + bp.stickinessMDKey.Store(newKey) + bp.stickiness.reset(newKey) + } +} + +func (bp *pickerWrapper) getStickinessMDKey() string { + // No need to check ok because mdKey == "" if ok == false. + mdKey, _ := bp.stickinessMDKey.Load().(string) + return mdKey +} + +func (bp *pickerWrapper) clearStickinessState() { + if oldKey := bp.getStickinessMDKey(); oldKey != "" { + // There's no need to reset store if mdKey was "". + bp.stickiness.reset(oldKey) + } +} + // updatePicker is called by UpdateBalancerState. It unblocks all blocked pick. func (bp *pickerWrapper) updatePicker(p balancer.Picker) { bp.mu.Lock() @@ -57,6 +106,23 @@ func (bp *pickerWrapper) updatePicker(p balancer.Picker) { bp.mu.Unlock() } +func doneChannelzWrapper(acw *acBalancerWrapper, done func(balancer.DoneInfo)) func(balancer.DoneInfo) { + acw.mu.Lock() + ac := acw.ac + acw.mu.Unlock() + ac.incrCallsStarted() + return func(b balancer.DoneInfo) { + if b.Err != nil && b.Err != io.EOF { + ac.incrCallsFailed() + } else { + ac.incrCallsSucceeded() + } + if done != nil { + done(b) + } + } +} + // pick returns the transport that will be used for the RPC. // It may block in the following cases: // - there's no picker @@ -65,6 +131,27 @@ func (bp *pickerWrapper) updatePicker(p balancer.Picker) { // - 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) { + + mdKey := bp.getStickinessMDKey() + stickyKey, isSticky := stickyKeyFromContext(ctx, mdKey) + + // Potential race here: if stickinessMDKey is updated after the above two + // lines, and this pick is a sticky pick, the following put could add an + // entry to sticky store with an outdated sticky key. + // + // The solution: keep the current md key in sticky store, and at the + // beginning of each get/put, check the mdkey against store.curMDKey. + // - Cons: one more string comparing for each get/put. + // - Pros: the string matching happens inside get/put, so the overhead for + // non-sticky RPCs will be minimal. + + if isSticky { + if t, ok := bp.stickiness.get(mdKey, stickyKey); ok { + // Done function returned is always nil. + return t, nil, nil + } + } + var ( p balancer.Picker ch chan struct{} @@ -97,7 +184,7 @@ func (bp *pickerWrapper) pick(ctx context.Context, failfast bool, opts balancer. p = bp.picker bp.mu.Unlock() - subConn, put, err := p.Pick(ctx, opts) + subConn, done, err := p.Pick(ctx, opts) if err != nil { switch err { @@ -107,7 +194,7 @@ func (bp *pickerWrapper) pick(ctx context.Context, failfast bool, opts balancer. if !failfast { continue } - return nil, nil, status.Errorf(codes.Unavailable, "%v", err) + return nil, nil, status.Errorf(codes.Unavailable, "%v, latest connection error: %v", err, bp.connectionError()) default: // err is some other error. return nil, nil, toRPCErr(err) @@ -120,7 +207,13 @@ func (bp *pickerWrapper) pick(ctx context.Context, failfast bool, opts balancer. continue } if t, ok := acw.getAddrConn().getReadyTransport(); ok { - return t, put, nil + if isSticky { + bp.stickiness.put(mdKey, stickyKey, acw) + } + if channelz.IsOn() { + return t, doneChannelzWrapper(acw, done), nil + } + return t, done, nil } grpclog.Infof("blockingPicker: the picked transport is not ready, loop back to repick") // If ok == false, ac.state is not READY. @@ -139,3 +232,100 @@ func (bp *pickerWrapper) close() { bp.done = true close(bp.blockingCh) } + +type stickyStoreEntry struct { + acw *acBalancerWrapper + addr resolver.Address +} + +type stickyStore struct { + mu sync.Mutex + // curMDKey is check before every get/put to avoid races. The operation will + // abort immediately when the given mdKey is different from the curMDKey. + curMDKey string + store map[string]*stickyStoreEntry +} + +func newStickyStore() *stickyStore { + return &stickyStore{ + store: make(map[string]*stickyStoreEntry), + } +} + +// reset clears the map in stickyStore, and set the currentMDKey to newMDKey. +func (ss *stickyStore) reset(newMDKey string) { + ss.mu.Lock() + ss.curMDKey = newMDKey + ss.store = make(map[string]*stickyStoreEntry) + ss.mu.Unlock() +} + +// stickyKey is the key to look up in store. mdKey will be checked against +// curMDKey to avoid races. +func (ss *stickyStore) put(mdKey, stickyKey string, acw *acBalancerWrapper) { + ss.mu.Lock() + defer ss.mu.Unlock() + if mdKey != ss.curMDKey { + return + } + // TODO(stickiness): limit the total number of entries. + ss.store[stickyKey] = &stickyStoreEntry{ + acw: acw, + addr: acw.getAddrConn().getCurAddr(), + } +} + +// stickyKey is the key to look up in store. mdKey will be checked against +// curMDKey to avoid races. +func (ss *stickyStore) get(mdKey, stickyKey string) (transport.ClientTransport, bool) { + ss.mu.Lock() + defer ss.mu.Unlock() + if mdKey != ss.curMDKey { + return nil, false + } + entry, ok := ss.store[stickyKey] + if !ok { + return nil, false + } + ac := entry.acw.getAddrConn() + if ac.getCurAddr() != entry.addr { + delete(ss.store, stickyKey) + return nil, false + } + t, ok := ac.getReadyTransport() + if !ok { + delete(ss.store, stickyKey) + return nil, false + } + return t, true +} + +// Get one value from metadata in ctx with key stickinessMDKey. +// +// It returns "", false if stickinessMDKey is an empty string. +func stickyKeyFromContext(ctx context.Context, stickinessMDKey string) (string, bool) { + if stickinessMDKey == "" { + return "", false + } + + md, added, ok := metadata.FromOutgoingContextRaw(ctx) + if !ok { + return "", false + } + + if vv, ok := md[stickinessMDKey]; ok { + if len(vv) > 0 { + return vv[0], true + } + } + + for _, ss := range added { + for i := 0; i < len(ss)-1; i += 2 { + if ss[i] == stickinessMDKey { + return ss[i+1], true + } + } + } + + return "", false +} diff --git a/vendor/google.golang.org/grpc/pickfirst.go b/vendor/google.golang.org/grpc/pickfirst.go index 7f993ef5a38..bf659d49d2f 100644 --- a/vendor/google.golang.org/grpc/pickfirst.go +++ b/vendor/google.golang.org/grpc/pickfirst.go @@ -26,6 +26,9 @@ import ( "google.golang.org/grpc/resolver" ) +// PickFirstBalancerName is the name of the pick_first balancer. +const PickFirstBalancerName = "pick_first" + func newPickfirstBuilder() balancer.Builder { return &pickfirstBuilder{} } @@ -37,7 +40,7 @@ func (*pickfirstBuilder) Build(cc balancer.ClientConn, opt balancer.BuildOptions } func (*pickfirstBuilder) Name() string { - return "pickfirst" + return PickFirstBalancerName } type pickfirstBalancer struct { @@ -57,14 +60,20 @@ func (b *pickfirstBalancer) HandleResolvedAddrs(addrs []resolver.Address, err er return } b.cc.UpdateBalancerState(connectivity.Idle, &picker{sc: b.sc}) + b.sc.Connect() } else { b.sc.UpdateAddresses(addrs) + b.sc.Connect() } } 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 { + if b.sc != sc { + grpclog.Infof("pickfirstBalancer: ignored state change because sc is not recognized") + return + } + if s == connectivity.Shutdown { b.sc = nil return } @@ -93,3 +102,7 @@ func (p *picker) Pick(ctx context.Context, opts balancer.PickOptions) (balancer. } return p.sc, nil, nil } + +func init() { + balancer.Register(newPickfirstBuilder()) +} diff --git a/vendor/google.golang.org/grpc/proxy.go b/vendor/google.golang.org/grpc/proxy.go index 3e17efec61b..2d40236e218 100644 --- a/vendor/google.golang.org/grpc/proxy.go +++ b/vendor/google.golang.org/grpc/proxy.go @@ -82,8 +82,7 @@ func doHTTPConnectHandshake(ctx context.Context, conn net.Conn, addr string) (_ Header: map[string][]string{"User-Agent": {grpcUA}}, }) - req = req.WithContext(ctx) - if err := req.Write(conn); err != nil { + if err := sendHTTPRequest(ctx, req, conn); err != nil { return nil, fmt.Errorf("failed to write the HTTP request: %v", err) } diff --git a/vendor/google.golang.org/grpc/resolver/dns/dns_resolver.go b/vendor/google.golang.org/grpc/resolver/dns/dns_resolver.go new file mode 100644 index 00000000000..c1cabfc995f --- /dev/null +++ b/vendor/google.golang.org/grpc/resolver/dns/dns_resolver.go @@ -0,0 +1,379 @@ +/* + * + * 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 dns implements a dns resolver to be installed as the default resolver +// in grpc. +package dns + +import ( + "encoding/json" + "errors" + "fmt" + "math/rand" + "net" + "os" + "strconv" + "strings" + "sync" + "time" + + "golang.org/x/net/context" + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/resolver" +) + +func init() { + resolver.Register(NewBuilder()) +} + +const ( + defaultPort = "443" + defaultFreq = time.Minute * 30 + golang = "GO" + // In DNS, service config is encoded in a TXT record via the mechanism + // described in RFC-1464 using the attribute name grpc_config. + txtAttribute = "grpc_config=" +) + +var ( + errMissingAddr = errors.New("missing address") + randomGen = rand.New(rand.NewSource(time.Now().UnixNano())) +) + +// NewBuilder creates a dnsBuilder which is used to factory DNS resolvers. +func NewBuilder() resolver.Builder { + return &dnsBuilder{freq: defaultFreq} +} + +type dnsBuilder struct { + // frequency of polling the DNS server. + freq time.Duration +} + +// Build creates and starts a DNS resolver that watches the name resolution of the target. +func (b *dnsBuilder) Build(target resolver.Target, cc resolver.ClientConn, opts resolver.BuildOption) (resolver.Resolver, error) { + host, port, err := parseTarget(target.Endpoint) + if err != nil { + return nil, err + } + + // IP address. + if net.ParseIP(host) != nil { + host, _ = formatIP(host) + addr := []resolver.Address{{Addr: host + ":" + port}} + i := &ipResolver{ + cc: cc, + ip: addr, + rn: make(chan struct{}, 1), + q: make(chan struct{}), + } + cc.NewAddress(addr) + go i.watcher() + return i, nil + } + + // DNS address (non-IP). + ctx, cancel := context.WithCancel(context.Background()) + d := &dnsResolver{ + freq: b.freq, + host: host, + port: port, + ctx: ctx, + cancel: cancel, + cc: cc, + t: time.NewTimer(0), + rn: make(chan struct{}, 1), + disableServiceConfig: opts.DisableServiceConfig, + } + + d.wg.Add(1) + go d.watcher() + return d, nil +} + +// Scheme returns the naming scheme of this resolver builder, which is "dns". +func (b *dnsBuilder) Scheme() string { + return "dns" +} + +// ipResolver watches for the name resolution update for an IP address. +type ipResolver struct { + cc resolver.ClientConn + ip []resolver.Address + // rn channel is used by ResolveNow() to force an immediate resolution of the target. + rn chan struct{} + q chan struct{} +} + +// ResolveNow resend the address it stores, no resolution is needed. +func (i *ipResolver) ResolveNow(opt resolver.ResolveNowOption) { + select { + case i.rn <- struct{}{}: + default: + } +} + +// Close closes the ipResolver. +func (i *ipResolver) Close() { + close(i.q) +} + +func (i *ipResolver) watcher() { + for { + select { + case <-i.rn: + i.cc.NewAddress(i.ip) + case <-i.q: + return + } + } +} + +// dnsResolver watches for the name resolution update for a non-IP target. +type dnsResolver struct { + freq time.Duration + host string + port string + ctx context.Context + cancel context.CancelFunc + cc resolver.ClientConn + // rn channel is used by ResolveNow() to force an immediate resolution of the target. + rn chan struct{} + t *time.Timer + // wg is used to enforce Close() to return after the watcher() goroutine has finished. + // Otherwise, data race will be possible. [Race Example] in dns_resolver_test we + // replace the real lookup functions with mocked ones to facilitate testing. + // If Close() doesn't wait for watcher() goroutine finishes, race detector sometimes + // will warns lookup (READ the lookup function pointers) inside watcher() goroutine + // has data race with replaceNetFunc (WRITE the lookup function pointers). + wg sync.WaitGroup + disableServiceConfig bool +} + +// ResolveNow invoke an immediate resolution of the target that this dnsResolver watches. +func (d *dnsResolver) ResolveNow(opt resolver.ResolveNowOption) { + select { + case d.rn <- struct{}{}: + default: + } +} + +// Close closes the dnsResolver. +func (d *dnsResolver) Close() { + d.cancel() + d.wg.Wait() + d.t.Stop() +} + +func (d *dnsResolver) watcher() { + defer d.wg.Done() + for { + select { + case <-d.ctx.Done(): + return + case <-d.t.C: + case <-d.rn: + } + result, sc := d.lookup() + // Next lookup should happen after an interval defined by d.freq. + d.t.Reset(d.freq) + d.cc.NewServiceConfig(sc) + d.cc.NewAddress(result) + } +} + +func (d *dnsResolver) lookupSRV() []resolver.Address { + var newAddrs []resolver.Address + _, srvs, err := lookupSRV(d.ctx, "grpclb", "tcp", d.host) + if err != nil { + grpclog.Infof("grpc: failed dns SRV record lookup due to %v.\n", err) + return nil + } + for _, s := range srvs { + lbAddrs, err := lookupHost(d.ctx, s.Target) + if err != nil { + grpclog.Infof("grpc: failed load balancer address dns lookup due to %v.\n", err) + continue + } + for _, a := range lbAddrs { + a, ok := formatIP(a) + if !ok { + grpclog.Errorf("grpc: failed IP parsing due to %v.\n", err) + continue + } + addr := a + ":" + strconv.Itoa(int(s.Port)) + newAddrs = append(newAddrs, resolver.Address{Addr: addr, Type: resolver.GRPCLB, ServerName: s.Target}) + } + } + return newAddrs +} + +func (d *dnsResolver) lookupTXT() string { + ss, err := lookupTXT(d.ctx, d.host) + if err != nil { + grpclog.Infof("grpc: failed dns TXT record lookup due to %v.\n", err) + return "" + } + var res string + for _, s := range ss { + res += s + } + + // TXT record must have "grpc_config=" attribute in order to be used as service config. + if !strings.HasPrefix(res, txtAttribute) { + grpclog.Warningf("grpc: TXT record %v missing %v attribute", res, txtAttribute) + return "" + } + return strings.TrimPrefix(res, txtAttribute) +} + +func (d *dnsResolver) lookupHost() []resolver.Address { + var newAddrs []resolver.Address + addrs, err := lookupHost(d.ctx, d.host) + if err != nil { + grpclog.Warningf("grpc: failed dns A record lookup due to %v.\n", err) + return nil + } + for _, a := range addrs { + a, ok := formatIP(a) + if !ok { + grpclog.Errorf("grpc: failed IP parsing due to %v.\n", err) + continue + } + addr := a + ":" + d.port + newAddrs = append(newAddrs, resolver.Address{Addr: addr}) + } + return newAddrs +} + +func (d *dnsResolver) lookup() ([]resolver.Address, string) { + newAddrs := d.lookupSRV() + // Support fallback to non-balancer address. + newAddrs = append(newAddrs, d.lookupHost()...) + if d.disableServiceConfig { + return newAddrs, "" + } + sc := d.lookupTXT() + return newAddrs, canaryingSC(sc) +} + +// formatIP returns ok = false if addr is not a valid textual representation of an IP address. +// If addr is an IPv4 address, return the addr and ok = true. +// If addr is an IPv6 address, return the addr enclosed in square brackets and ok = true. +func formatIP(addr string) (addrIP string, ok bool) { + ip := net.ParseIP(addr) + if ip == nil { + return "", false + } + if ip.To4() != nil { + return addr, true + } + return "[" + addr + "]", true +} + +// parseTarget takes the user input target string, returns formatted host and port info. +// If target doesn't specify a port, set the port to be the defaultPort. +// If target is in IPv6 format and host-name is enclosed in sqarue brackets, brackets +// are strippd when setting the host. +// examples: +// target: "www.google.com" returns host: "www.google.com", port: "443" +// target: "ipv4-host:80" returns host: "ipv4-host", port: "80" +// target: "[ipv6-host]" returns host: "ipv6-host", port: "443" +// target: ":80" returns host: "localhost", port: "80" +// target: ":" returns host: "localhost", port: "443" +func parseTarget(target string) (host, port string, err error) { + if target == "" { + return "", "", errMissingAddr + } + if ip := net.ParseIP(target); ip != nil { + // target is an IPv4 or IPv6(without brackets) address + return target, defaultPort, nil + } + if host, port, err = net.SplitHostPort(target); err == nil { + // target has port, i.e ipv4-host:port, [ipv6-host]:port, host-name:port + if host == "" { + // Keep consistent with net.Dial(): If the host is empty, as in ":80", the local system is assumed. + host = "localhost" + } + if port == "" { + // If the port field is empty(target ends with colon), e.g. "[::1]:", defaultPort is used. + port = defaultPort + } + return host, port, nil + } + if host, port, err = net.SplitHostPort(target + ":" + defaultPort); err == nil { + // target doesn't have port + return host, port, nil + } + return "", "", fmt.Errorf("invalid target address %v, error info: %v", target, err) +} + +type rawChoice struct { + ClientLanguage *[]string `json:"clientLanguage,omitempty"` + Percentage *int `json:"percentage,omitempty"` + ClientHostName *[]string `json:"clientHostName,omitempty"` + ServiceConfig *json.RawMessage `json:"serviceConfig,omitempty"` +} + +func containsString(a *[]string, b string) bool { + if a == nil { + return true + } + for _, c := range *a { + if c == b { + return true + } + } + return false +} + +func chosenByPercentage(a *int) bool { + if a == nil { + return true + } + return randomGen.Intn(100)+1 <= *a +} + +func canaryingSC(js string) string { + if js == "" { + return "" + } + var rcs []rawChoice + err := json.Unmarshal([]byte(js), &rcs) + if err != nil { + grpclog.Warningf("grpc: failed to parse service config json string due to %v.\n", err) + return "" + } + cliHostname, err := os.Hostname() + if err != nil { + grpclog.Warningf("grpc: failed to get client hostname due to %v.\n", err) + return "" + } + var sc string + for _, c := range rcs { + if !containsString(c.ClientLanguage, golang) || + !chosenByPercentage(c.Percentage) || + !containsString(c.ClientHostName, cliHostname) || + c.ServiceConfig == nil { + continue + } + sc = string(*c.ServiceConfig) + break + } + return sc +} diff --git a/vendor/google.golang.org/grpc/resolver/dns/go17.go b/vendor/google.golang.org/grpc/resolver/dns/go17.go new file mode 100644 index 00000000000..b466bc8f6d4 --- /dev/null +++ b/vendor/google.golang.org/grpc/resolver/dns/go17.go @@ -0,0 +1,35 @@ +// +build go1.6, !go1.8 + +/* + * + * 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 dns + +import ( + "net" + + "golang.org/x/net/context" +) + +var ( + lookupHost = func(ctx context.Context, host string) ([]string, error) { return net.LookupHost(host) } + lookupSRV = func(ctx context.Context, service, proto, name string) (string, []*net.SRV, error) { + return net.LookupSRV(service, proto, name) + } + lookupTXT = func(ctx context.Context, name string) ([]string, error) { return net.LookupTXT(name) } +) diff --git a/vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/doc.go b/vendor/google.golang.org/grpc/resolver/dns/go18.go similarity index 76% rename from vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/doc.go rename to vendor/google.golang.org/grpc/resolver/dns/go18.go index aba962840c8..fa34f14cad4 100644 --- a/vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/doc.go +++ b/vendor/google.golang.org/grpc/resolver/dns/go18.go @@ -1,3 +1,5 @@ +// +build go1.8 + /* * * Copyright 2017 gRPC authors. @@ -16,6 +18,12 @@ * */ -// Package grpc_lb_v1 is the parent package of all gRPC loadbalancer -// message and service protobuf definitions. -package grpc_lb_v1 +package dns + +import "net" + +var ( + lookupHost = net.DefaultResolver.LookupHost + lookupSRV = net.DefaultResolver.LookupSRV + lookupTXT = net.DefaultResolver.LookupTXT +) diff --git a/vendor/google.golang.org/grpc/resolver/passthrough/passthrough.go b/vendor/google.golang.org/grpc/resolver/passthrough/passthrough.go new file mode 100644 index 00000000000..b76010d74d1 --- /dev/null +++ b/vendor/google.golang.org/grpc/resolver/passthrough/passthrough.go @@ -0,0 +1,57 @@ +/* + * + * 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 passthrough implements a pass-through resolver. It sends the target +// name without scheme back to gRPC as resolved address. +package passthrough + +import "google.golang.org/grpc/resolver" + +const scheme = "passthrough" + +type passthroughBuilder struct{} + +func (*passthroughBuilder) Build(target resolver.Target, cc resolver.ClientConn, opts resolver.BuildOption) (resolver.Resolver, error) { + r := &passthroughResolver{ + target: target, + cc: cc, + } + r.start() + return r, nil +} + +func (*passthroughBuilder) Scheme() string { + return scheme +} + +type passthroughResolver struct { + target resolver.Target + cc resolver.ClientConn +} + +func (r *passthroughResolver) start() { + r.cc.NewAddress([]resolver.Address{{Addr: r.target.Endpoint}}) +} + +func (*passthroughResolver) ResolveNow(o resolver.ResolveNowOption) {} + +func (*passthroughResolver) Close() {} + +func init() { + resolver.Register(&passthroughBuilder{}) +} diff --git a/vendor/google.golang.org/grpc/resolver/resolver.go b/vendor/google.golang.org/grpc/resolver/resolver.go index 49307e8fe9e..506afac88ae 100644 --- a/vendor/google.golang.org/grpc/resolver/resolver.go +++ b/vendor/google.golang.org/grpc/resolver/resolver.go @@ -24,42 +24,42 @@ var ( // m is a map from scheme to resolver builder. m = make(map[string]Builder) // defaultScheme is the default scheme to use. - defaultScheme string + defaultScheme = "passthrough" ) // 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. +// Register registers the resolver builder to the resolver map. b.Scheme will be +// used as the scheme registered with this builder. +// +// NOTE: this function must only be called during initialization time (i.e. in +// an init() function), and is not thread-safe. If multiple Resolvers are +// registered with the same name, the one registered last will take effect. 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. +// +// If no builder is register 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". +// The default default scheme is "passthrough". func SetDefaultScheme(scheme string) { defaultScheme = scheme } +// GetDefaultScheme gets the default scheme that will be used. +func GetDefaultScheme() string { + return defaultScheme +} + // AddressType indicates the address type returned by name resolution. type AddressType uint8 @@ -78,7 +78,9 @@ type Address struct { // 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. + // + // e.g. if Type is GRPCLB, ServerName should be the name of the remote load + // balancer, not the name of the backend. ServerName string // Metadata is the information associated with Addr, which may be used // to make load balancing decision. @@ -88,10 +90,17 @@ type Address struct { // BuildOption includes additional information for the builder to create // the resolver. type BuildOption struct { + // DisableServiceConfig indicates whether resolver should fetch service config data. + DisableServiceConfig bool } // ClientConn contains the callbacks for resolver to notify any updates // to the gRPC ClientConn. +// +// This interface is to be implemented by gRPC. Users should not need a +// brand new implementation of this interface. For the situations like +// testing, the new implementation should embed this interface. This allows +// gRPC to add new methods to this interface. type ClientConn interface { // NewAddress is called by resolver to notify ClientConn a new list // of resolved addresses. @@ -128,8 +137,10 @@ 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 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. + // + // It could be called multiple times concurrently. ResolveNow(ResolveNowOption) // Close closes the resolver. Close() diff --git a/vendor/google.golang.org/grpc/resolver_conn_wrapper.go b/vendor/google.golang.org/grpc/resolver_conn_wrapper.go index 7d53964d094..1b493db2e6c 100644 --- a/vendor/google.golang.org/grpc/resolver_conn_wrapper.go +++ b/vendor/google.golang.org/grpc/resolver_conn_wrapper.go @@ -19,6 +19,7 @@ package grpc import ( + "fmt" "strings" "google.golang.org/grpc/grpclog" @@ -36,20 +37,30 @@ type ccResolverWrapper 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) { +// If sep is not found, it returns ("", s, false) instead. +func split2(s, sep string) (string, string, bool) { spl := strings.SplitN(s, sep, 2) if len(spl) < 2 { - return "", s + return "", "", false } - return spl[0], spl[1] + return spl[0], spl[1], true } // parseTarget splits target into a struct containing scheme, authority and // endpoint. +// +// If target is not a valid scheme://authority/endpoint, it returns {Endpoint: +// target}. func parseTarget(target string) (ret resolver.Target) { - ret.Scheme, ret.Endpoint = split2(target, "://") - ret.Authority, ret.Endpoint = split2(ret.Endpoint, "/") + var ok bool + ret.Scheme, ret.Endpoint, ok = split2(target, "://") + if !ok { + return resolver.Target{Endpoint: target} + } + ret.Authority, ret.Endpoint, ok = split2(ret.Endpoint, "/") + if !ok { + return resolver.Target{Endpoint: target} + } return ret } @@ -57,18 +68,12 @@ func parseTarget(target string) (ret resolver.Target) { // 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. +// If withResolverBuilder dial option is set, the specified resolver will be +// used instead. 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) + rb := cc.dopts.resolverBuilder 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 + return nil, fmt.Errorf("could not get resolver for scheme: %q", cc.parsedTarget.Scheme) } ccr := &ccResolverWrapper{ @@ -79,15 +84,18 @@ func newCCResolverWrapper(cc *ClientConn) (*ccResolverWrapper, error) { } var err error - ccr.resolver, err = rb.Build(target, ccr, resolver.BuildOption{}) + ccr.resolver, err = rb.Build(cc.parsedTarget, ccr, resolver.BuildOption{DisableServiceConfig: cc.dopts.disableServiceConfig}) if err != nil { return nil, err } - go ccr.watcher() return ccr, nil } -// watcher processes address updates and service config updates sequencially. +func (ccr *ccResolverWrapper) start() { + go ccr.watcher() +} + +// watcher processes address updates and service config updates sequentially. // Otherwise, we need to resolve possible races between address and service // config (e.g. they specify different balancer types). func (ccr *ccResolverWrapper) watcher() { @@ -100,20 +108,31 @@ func (ccr *ccResolverWrapper) watcher() { 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) + select { + case <-ccr.done: + return + default: } + grpclog.Infof("ccResolverWrapper: sending new addresses to cc: %v", addrs) + ccr.cc.handleResolvedAddrs(addrs, nil) case sc := <-ccr.scCh: + select { + case <-ccr.done: + return + default: + } grpclog.Infof("ccResolverWrapper: got new service config: %v", sc) + ccr.cc.handleServiceConfig(sc) case <-ccr.done: return } } } +func (ccr *ccResolverWrapper) resolveNow(o resolver.ResolveNowOption) { + ccr.resolver.ResolveNow(o) +} + func (ccr *ccResolverWrapper) close() { ccr.resolver.Close() close(ccr.done) diff --git a/vendor/google.golang.org/grpc/rpc_util.go b/vendor/google.golang.org/grpc/rpc_util.go index 188a75fff94..5de1b031ec2 100644 --- a/vendor/google.golang.org/grpc/rpc_util.go +++ b/vendor/google.golang.org/grpc/rpc_util.go @@ -21,18 +21,21 @@ package grpc import ( "bytes" "compress/gzip" - stdctx "context" "encoding/binary" + "fmt" "io" "io/ioutil" "math" - "os" + "net/url" + "strings" "sync" "time" "golang.org/x/net/context" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" + "google.golang.org/grpc/encoding" + "google.golang.org/grpc/encoding/proto" "google.golang.org/grpc/metadata" "google.golang.org/grpc/peer" "google.golang.org/grpc/stats" @@ -41,6 +44,8 @@ import ( ) // Compressor defines the interface gRPC uses to compress a message. +// +// Deprecated: use package encoding. type Compressor interface { // Do compresses p into w. Do(w io.Writer, p []byte) error @@ -53,14 +58,34 @@ type gzipCompressor struct { } // NewGZIPCompressor creates a Compressor based on GZIP. +// +// Deprecated: use package encoding/gzip. func NewGZIPCompressor() Compressor { + c, _ := NewGZIPCompressorWithLevel(gzip.DefaultCompression) + return c +} + +// NewGZIPCompressorWithLevel is like NewGZIPCompressor but specifies the gzip compression level instead +// of assuming DefaultCompression. +// +// The error returned will be nil if the level is valid. +// +// Deprecated: use package encoding/gzip. +func NewGZIPCompressorWithLevel(level int) (Compressor, error) { + if level < gzip.DefaultCompression || level > gzip.BestCompression { + return nil, fmt.Errorf("grpc: invalid compression level: %d", level) + } return &gzipCompressor{ pool: sync.Pool{ New: func() interface{} { - return gzip.NewWriter(ioutil.Discard) + w, err := gzip.NewWriterLevel(ioutil.Discard, level) + if err != nil { + panic(err) + } + return w }, }, - } + }, nil } func (c *gzipCompressor) Do(w io.Writer, p []byte) error { @@ -78,6 +103,8 @@ func (c *gzipCompressor) Type() string { } // Decompressor defines the interface gRPC uses to decompress a message. +// +// Deprecated: use package encoding. type Decompressor interface { // Do reads the data from r and uncompress them. Do(r io.Reader) ([]byte, error) @@ -90,6 +117,8 @@ type gzipDecompressor struct { } // NewGZIPDecompressor creates a Decompressor based on GZIP. +// +// Deprecated: use package encoding/gzip. func NewGZIPDecompressor() Decompressor { return &gzipDecompressor{} } @@ -124,14 +153,15 @@ func (d *gzipDecompressor) Type() string { // callInfo contains all related configuration and information about an RPC. type callInfo struct { + compressorType string failFast bool - headerMD metadata.MD - trailerMD metadata.MD - peer *peer.Peer + stream *clientStream traceInfo traceInfo // in trace.go maxReceiveMessageSize *int maxSendMessageSize *int creds credentials.PerRPCCredentials + contentSubtype string + codec baseCodec } func defaultCallInfo() *callInfo { @@ -158,81 +188,233 @@ type EmptyCallOption struct{} func (EmptyCallOption) before(*callInfo) error { return nil } func (EmptyCallOption) after(*callInfo) {} -type beforeCall func(c *callInfo) error - -func (o beforeCall) before(c *callInfo) error { return o(c) } -func (o beforeCall) after(c *callInfo) {} - -type afterCall func(c *callInfo) - -func (o afterCall) before(c *callInfo) error { return nil } -func (o afterCall) after(c *callInfo) { o(c) } - // Header returns a CallOptions that retrieves the header metadata // for a unary RPC. func Header(md *metadata.MD) CallOption { - return afterCall(func(c *callInfo) { - *md = c.headerMD - }) + return HeaderCallOption{HeaderAddr: md} +} + +// HeaderCallOption is a CallOption for collecting response header metadata. +// The metadata field will be populated *after* the RPC completes. +// This is an EXPERIMENTAL API. +type HeaderCallOption struct { + HeaderAddr *metadata.MD +} + +func (o HeaderCallOption) before(c *callInfo) error { return nil } +func (o HeaderCallOption) after(c *callInfo) { + if c.stream != nil { + *o.HeaderAddr, _ = c.stream.Header() + } } // Trailer returns a CallOptions that retrieves the trailer metadata // for a unary RPC. func Trailer(md *metadata.MD) CallOption { - return afterCall(func(c *callInfo) { - *md = c.trailerMD - }) + return TrailerCallOption{TrailerAddr: md} +} + +// TrailerCallOption is a CallOption for collecting response trailer metadata. +// The metadata field will be populated *after* the RPC completes. +// This is an EXPERIMENTAL API. +type TrailerCallOption struct { + TrailerAddr *metadata.MD +} + +func (o TrailerCallOption) before(c *callInfo) error { return nil } +func (o TrailerCallOption) after(c *callInfo) { + if c.stream != nil { + *o.TrailerAddr = c.stream.Trailer() + } +} + +// Peer returns a CallOption that retrieves peer information for a unary RPC. +// The peer field will be populated *after* the RPC completes. +func Peer(p *peer.Peer) CallOption { + return PeerCallOption{PeerAddr: p} } -// Peer returns a CallOption that retrieves peer information for a -// unary RPC. -func Peer(peer *peer.Peer) CallOption { - return afterCall(func(c *callInfo) { - if c.peer != nil { - *peer = *c.peer +// PeerCallOption is a CallOption for collecting the identity of the remote +// peer. The peer field will be populated *after* the RPC completes. +// This is an EXPERIMENTAL API. +type PeerCallOption struct { + PeerAddr *peer.Peer +} + +func (o PeerCallOption) before(c *callInfo) error { return nil } +func (o PeerCallOption) after(c *callInfo) { + if c.stream != nil { + if x, ok := peer.FromContext(c.stream.Context()); ok { + *o.PeerAddr = *x } - }) + } } // FailFast configures the action to take when an RPC is attempted on broken -// connections or unreachable servers. If failfast is true, the RPC will fail +// connections or unreachable servers. If failFast is true, the RPC will fail // immediately. Otherwise, the RPC client will block the call until a -// connection is available (or the call is canceled or times out) and will retry -// the call if it fails due to a transient error. Please refer to +// connection is available (or the call is canceled or times out) and will +// retry the call if it fails due to a transient error. gRPC will not retry if +// data was written to the wire unless the server indicates it did not process +// the data. Please refer to // https://github.com/grpc/grpc/blob/master/doc/wait-for-ready.md. -// Note: failFast is default to true. +// +// By default, RPCs are "Fail Fast". func FailFast(failFast bool) CallOption { - return beforeCall(func(c *callInfo) error { - c.failFast = failFast - return nil - }) + return FailFastCallOption{FailFast: failFast} +} + +// FailFastCallOption is a CallOption for indicating whether an RPC should fail +// fast or not. +// This is an EXPERIMENTAL API. +type FailFastCallOption struct { + FailFast bool } +func (o FailFastCallOption) before(c *callInfo) error { + c.failFast = o.FailFast + return nil +} +func (o FailFastCallOption) after(c *callInfo) {} + // MaxCallRecvMsgSize returns a CallOption which sets the maximum message size the client can receive. func MaxCallRecvMsgSize(s int) CallOption { - return beforeCall(func(o *callInfo) error { - o.maxReceiveMessageSize = &s - return nil - }) + return MaxRecvMsgSizeCallOption{MaxRecvMsgSize: s} +} + +// MaxRecvMsgSizeCallOption is a CallOption that indicates the maximum message +// size the client can receive. +// This is an EXPERIMENTAL API. +type MaxRecvMsgSizeCallOption struct { + MaxRecvMsgSize int } +func (o MaxRecvMsgSizeCallOption) before(c *callInfo) error { + c.maxReceiveMessageSize = &o.MaxRecvMsgSize + return nil +} +func (o MaxRecvMsgSizeCallOption) after(c *callInfo) {} + // MaxCallSendMsgSize returns a CallOption which sets the maximum message size the client can send. func MaxCallSendMsgSize(s int) CallOption { - return beforeCall(func(o *callInfo) error { - o.maxSendMessageSize = &s - return nil - }) + return MaxSendMsgSizeCallOption{MaxSendMsgSize: s} } +// MaxSendMsgSizeCallOption is a CallOption that indicates the maximum message +// size the client can send. +// This is an EXPERIMENTAL API. +type MaxSendMsgSizeCallOption struct { + MaxSendMsgSize int +} + +func (o MaxSendMsgSizeCallOption) before(c *callInfo) error { + c.maxSendMessageSize = &o.MaxSendMsgSize + return nil +} +func (o MaxSendMsgSizeCallOption) after(c *callInfo) {} + // PerRPCCredentials returns a CallOption that sets credentials.PerRPCCredentials // for a call. func PerRPCCredentials(creds credentials.PerRPCCredentials) CallOption { - return beforeCall(func(c *callInfo) error { - c.creds = creds - return nil - }) + return PerRPCCredsCallOption{Creds: creds} +} + +// PerRPCCredsCallOption is a CallOption that indicates the per-RPC +// credentials to use for the call. +// This is an EXPERIMENTAL API. +type PerRPCCredsCallOption struct { + Creds credentials.PerRPCCredentials +} + +func (o PerRPCCredsCallOption) before(c *callInfo) error { + c.creds = o.Creds + return nil +} +func (o PerRPCCredsCallOption) after(c *callInfo) {} + +// UseCompressor returns a CallOption which sets the compressor used when +// sending the request. If WithCompressor is also set, UseCompressor has +// higher priority. +// +// This API is EXPERIMENTAL. +func UseCompressor(name string) CallOption { + return CompressorCallOption{CompressorType: name} +} + +// CompressorCallOption is a CallOption that indicates the compressor to use. +// This is an EXPERIMENTAL API. +type CompressorCallOption struct { + CompressorType string +} + +func (o CompressorCallOption) before(c *callInfo) error { + c.compressorType = o.CompressorType + return nil +} +func (o CompressorCallOption) after(c *callInfo) {} + +// CallContentSubtype returns a CallOption that will set the content-subtype +// for a call. For example, if content-subtype is "json", the Content-Type over +// the wire will be "application/grpc+json". The content-subtype is converted +// to lowercase before being included in Content-Type. See Content-Type on +// https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#requests for +// more details. +// +// If CallCustomCodec is not also used, the content-subtype will be used to +// look up the Codec to use in the registry controlled by RegisterCodec. See +// the documentation on RegisterCodec for details on registration. The lookup +// of content-subtype is case-insensitive. If no such Codec is found, the call +// will result in an error with code codes.Internal. +// +// If CallCustomCodec is also used, that Codec will be used for all request and +// response messages, with the content-subtype set to the given contentSubtype +// here for requests. +func CallContentSubtype(contentSubtype string) CallOption { + return ContentSubtypeCallOption{ContentSubtype: strings.ToLower(contentSubtype)} +} + +// ContentSubtypeCallOption is a CallOption that indicates the content-subtype +// used for marshaling messages. +// This is an EXPERIMENTAL API. +type ContentSubtypeCallOption struct { + ContentSubtype string } +func (o ContentSubtypeCallOption) before(c *callInfo) error { + c.contentSubtype = o.ContentSubtype + return nil +} +func (o ContentSubtypeCallOption) after(c *callInfo) {} + +// CallCustomCodec returns a CallOption that will set the given Codec to be +// used for all request and response messages for a call. The result of calling +// String() will be used as the content-subtype in a case-insensitive manner. +// +// See Content-Type on +// https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#requests for +// more details. Also see the documentation on RegisterCodec and +// CallContentSubtype for more details on the interaction between Codec and +// content-subtype. +// +// This function is provided for advanced users; prefer to use only +// CallContentSubtype to select a registered codec instead. +func CallCustomCodec(codec Codec) CallOption { + return CustomCodecCallOption{Codec: codec} +} + +// CustomCodecCallOption is a CallOption that indicates the codec used for +// marshaling messages. +// This is an EXPERIMENTAL API. +type CustomCodecCallOption struct { + Codec Codec +} + +func (o CustomCodecCallOption) before(c *callInfo) error { + c.codec = o.Codec + return nil +} +func (o CustomCodecCallOption) after(c *callInfo) {} + // The format of the payload: compressed or not? type payloadFormat uint8 @@ -248,8 +430,8 @@ type parser struct { // error types. r io.Reader - // The header of a gRPC message. Find more detail - // at https://grpc.io/docs/guides/wire.html. + // The header of a gRPC message. Find more detail at + // https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md header [5]byte } @@ -277,8 +459,11 @@ func (p *parser) recvMsg(maxReceiveMessageSize int) (pf payloadFormat, msg []byt if length == 0 { return pf, nil, nil } - if length > uint32(maxReceiveMessageSize) { - return 0, nil, Errorf(codes.ResourceExhausted, "grpc: received message larger than max (%d vs. %d)", length, maxReceiveMessageSize) + if int64(length) > int64(maxInt) { + return 0, nil, status.Errorf(codes.ResourceExhausted, "grpc: received message larger than max length allowed on current machine (%d vs. %d)", length, maxInt) + } + if int(length) > maxReceiveMessageSize { + return 0, nil, status.Errorf(codes.ResourceExhausted, "grpc: received message larger than max (%d vs. %d)", length, maxReceiveMessageSize) } // TODO(bradfitz,zhaoq): garbage. reuse buffer after proto decoding instead // of making it for each message: @@ -294,18 +479,21 @@ func (p *parser) recvMsg(maxReceiveMessageSize int) (pf payloadFormat, msg []byt // encode serializes msg and returns a buffer of message header and a buffer of msg. // If msg is nil, it generates the message header and an empty msg buffer. -func encode(c Codec, msg interface{}, cp Compressor, cbuf *bytes.Buffer, outPayload *stats.OutPayload) ([]byte, []byte, error) { - var b []byte +// TODO(ddyihai): eliminate extra Compressor parameter. +func encode(c baseCodec, msg interface{}, cp Compressor, outPayload *stats.OutPayload, compressor encoding.Compressor) ([]byte, []byte, error) { + var ( + b []byte + cbuf *bytes.Buffer + ) const ( payloadLen = 1 sizeLen = 4 ) - if msg != nil { var err error b, err = c.Marshal(msg) if err != nil { - return nil, nil, Errorf(codes.Internal, "grpc: error while marshaling: %v", err.Error()) + return nil, nil, status.Errorf(codes.Internal, "grpc: error while marshaling: %v", err.Error()) } if outPayload != nil { outPayload.Payload = msg @@ -313,24 +501,35 @@ func encode(c Codec, msg interface{}, cp Compressor, cbuf *bytes.Buffer, outPayl outPayload.Data = b outPayload.Length = len(b) } - if cp != nil { - if err := cp.Do(cbuf, b); err != nil { - return nil, nil, Errorf(codes.Internal, "grpc: error while compressing: %v", err.Error()) + if compressor != nil || cp != nil { + cbuf = new(bytes.Buffer) + // Has compressor, check Compressor is set by UseCompressor first. + if compressor != nil { + z, _ := compressor.Compress(cbuf) + if _, err := z.Write(b); err != nil { + return nil, nil, status.Errorf(codes.Internal, "grpc: error while compressing: %v", err.Error()) + } + z.Close() + } else { + // If Compressor is not set by UseCompressor, use default Compressor + if err := cp.Do(cbuf, b); err != nil { + return nil, nil, status.Errorf(codes.Internal, "grpc: error while compressing: %v", err.Error()) + } } b = cbuf.Bytes() } } - if uint(len(b)) > math.MaxUint32 { - return nil, nil, Errorf(codes.ResourceExhausted, "grpc: message too large (%d bytes)", len(b)) + return nil, nil, status.Errorf(codes.ResourceExhausted, "grpc: message too large (%d bytes)", len(b)) } bufHeader := make([]byte, payloadLen+sizeLen) - if cp == nil { - bufHeader[0] = byte(compressionNone) - } else { + if compressor != nil || cp != nil { bufHeader[0] = byte(compressionMade) + } else { + bufHeader[0] = byte(compressionNone) } + // Write length of b into buf binary.BigEndian.PutUint32(bufHeader[payloadLen:], uint32(len(b))) if outPayload != nil { @@ -339,20 +538,26 @@ func encode(c Codec, msg interface{}, cp Compressor, cbuf *bytes.Buffer, outPayl return bufHeader, b, nil } -func checkRecvPayload(pf payloadFormat, recvCompress string, dc Decompressor) error { +func checkRecvPayload(pf payloadFormat, recvCompress string, haveCompressor bool) *status.Status { switch pf { case compressionNone: case compressionMade: - if dc == nil || recvCompress != dc.Type() { - return Errorf(codes.Unimplemented, "grpc: Decompressor is not installed for grpc-encoding %q", recvCompress) + if recvCompress == "" || recvCompress == encoding.Identity { + return status.New(codes.Internal, "grpc: compressed flag set with identity or empty encoding") + } + if !haveCompressor { + return status.Newf(codes.Unimplemented, "grpc: Decompressor is not installed for grpc-encoding %q", recvCompress) } default: - return Errorf(codes.Internal, "grpc: received unexpected payload format %d", pf) + return status.Newf(codes.Internal, "grpc: received unexpected payload format %d", pf) } return nil } -func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{}, maxReceiveMessageSize int, inPayload *stats.InPayload) error { +// For the two compressor parameters, both should not be set, but if they are, +// dc takes precedence over compressor. +// TODO(dfawley): wrap the old compressor/decompressor using the new API? +func recv(p *parser, c baseCodec, s *transport.Stream, dc Decompressor, m interface{}, maxReceiveMessageSize int, inPayload *stats.InPayload, compressor encoding.Compressor) error { pf, d, err := p.recvMsg(maxReceiveMessageSize) if err != nil { return err @@ -360,22 +565,37 @@ func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{ if inPayload != nil { inPayload.WireLength = len(d) } - if err := checkRecvPayload(pf, s.RecvCompress(), dc); err != nil { - return err + + if st := checkRecvPayload(pf, s.RecvCompress(), compressor != nil || dc != nil); st != nil { + return st.Err() } + if pf == compressionMade { - d, err = dc.Do(bytes.NewReader(d)) - if err != nil { - return Errorf(codes.Internal, "grpc: failed to decompress the received message %v", err) + // To match legacy behavior, if the decompressor is set by WithDecompressor or RPCDecompressor, + // use this decompressor as the default. + if dc != nil { + d, err = dc.Do(bytes.NewReader(d)) + if err != nil { + return status.Errorf(codes.Internal, "grpc: failed to decompress the received message %v", err) + } + } else { + dcReader, err := compressor.Decompress(bytes.NewReader(d)) + if err != nil { + return status.Errorf(codes.Internal, "grpc: failed to decompress the received message %v", err) + } + d, err = ioutil.ReadAll(dcReader) + if err != nil { + return status.Errorf(codes.Internal, "grpc: failed to decompress the received message %v", err) + } } } if len(d) > maxReceiveMessageSize { // TODO: Revisit the error code. Currently keep it consistent with java // implementation. - return Errorf(codes.ResourceExhausted, "grpc: received message larger than max (%d vs. %d)", len(d), maxReceiveMessageSize) + return status.Errorf(codes.ResourceExhausted, "grpc: received message larger than max (%d vs. %d)", len(d), maxReceiveMessageSize) } if err := c.Unmarshal(d, m); err != nil { - return Errorf(codes.Internal, "grpc: failed to unmarshal the received message %v", err) + return status.Errorf(codes.Internal, "grpc: failed to unmarshal the received message %v", err) } if inPayload != nil { inPayload.RecvTime = time.Now() @@ -388,9 +608,7 @@ func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{ } type rpcInfo struct { - failfast bool - bytesSent bool - bytesReceived bool + failfast bool } type rpcInfoContextKey struct{} @@ -404,69 +622,10 @@ func rpcInfoFromContext(ctx context.Context) (s *rpcInfo, ok bool) { return } -func updateRPCInfoInContext(ctx context.Context, s rpcInfo) { - if ss, ok := rpcInfoFromContext(ctx); ok { - 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. // -// Deprecated; use status.FromError and Code method instead. +// Deprecated: use status.FromError and Code method instead. func Code(err error) codes.Code { if s, ok := status.FromError(err); ok { return s.Code() @@ -477,7 +636,7 @@ func Code(err error) codes.Code { // ErrorDesc returns the error description of err if it was produced by the rpc system. // Otherwise, it returns err.Error() or empty string when err is nil. // -// Deprecated; use status.FromError and Message method instead. +// Deprecated: use status.FromError and Message method instead. func ErrorDesc(err error) string { if s, ok := status.FromError(err); ok { return s.Message() @@ -488,85 +647,81 @@ func ErrorDesc(err error) string { // Errorf returns an error containing an error code and a description; // Errorf returns nil if c is OK. // -// Deprecated; use status.Errorf instead. +// Deprecated: use status.Errorf instead. func Errorf(c codes.Code, format string, a ...interface{}) error { return status.Errorf(c, format, a...) } -// MethodConfig defines the configuration recommended by the service providers for a -// particular method. -// This is EXPERIMENTAL and subject to change. -type MethodConfig struct { - // WaitForReady indicates whether RPCs sent to this method should wait until - // the connection is ready by default (!failfast). The value specified via the - // gRPC client API will override the value set here. - WaitForReady *bool - // Timeout is the default timeout for RPCs sent to this method. The actual - // deadline used will be 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 RPC has no deadline. - Timeout *time.Duration - // 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 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 - // MaxRespSize is the maximum allowed payload size for an individual response in a - // stream (server->client) in bytes. - MaxRespSize *int -} - -// ServiceConfig is provided by the service provider and contains parameters for how -// clients that connect to the service should behave. -// This is EXPERIMENTAL and subject to change. -type ServiceConfig struct { - // LB is the load balancer the service providers recommends. The balancer specified - // via grpc.WithBalancer will override this. - LB Balancer - // Methods contains a map for the methods in this service. - // If there is an exact match for a method (i.e. /service/method) in the map, use the corresponding MethodConfig. - // If there's no exact match, look for the default config for the service (/service/) and use the corresponding MethodConfig if it exists. - // Otherwise, the method has no MethodConfig to use. - Methods map[string]MethodConfig -} - -func min(a, b *int) *int { - if *a < *b { - return a +// setCallInfoCodec should only be called after CallOptions have been applied. +func setCallInfoCodec(c *callInfo) error { + if c.codec != nil { + // codec was already set by a CallOption; use it. + return nil } - return b -} -func getMaxSize(mcMax, doptMax *int, defaultVal int) *int { - if mcMax == nil && doptMax == nil { - return &defaultVal - } - if mcMax != nil && doptMax != nil { - return min(mcMax, doptMax) + if c.contentSubtype == "" { + // No codec specified in CallOptions; use proto by default. + c.codec = encoding.GetCodec(proto.Name) + return nil } - if mcMax != nil { - return mcMax + + // c.contentSubtype is already lowercased in CallContentSubtype + c.codec = encoding.GetCodec(c.contentSubtype) + if c.codec == nil { + return status.Errorf(codes.Internal, "no codec registered for content-subtype %s", c.contentSubtype) } - return doptMax + return nil } -// SupportPackageIsVersion3 is referenced from generated protocol buffer files. -// The latest support package version is 4. -// SupportPackageIsVersion3 is kept for compatibility. It will be removed in the -// next support package version update. -const SupportPackageIsVersion3 = true +// parseDialTarget returns the network and address to pass to dialer +func parseDialTarget(target string) (net string, addr string) { + net = "tcp" + + m1 := strings.Index(target, ":") + m2 := strings.Index(target, ":/") + + // handle unix:addr which will fail with url.Parse + if m1 >= 0 && m2 < 0 { + if n := target[0:m1]; n == "unix" { + net = n + addr = target[m1+1:] + return net, addr + } + } + if m2 >= 0 { + t, err := url.Parse(target) + if err != nil { + return net, target + } + scheme := t.Scheme + addr = t.Path + if scheme == "unix" { + net = scheme + if addr == "" { + addr = t.Host + } + return net, addr + } + } + + return net, target +} -// SupportPackageIsVersion4 is referenced from generated protocol buffer files -// to assert that that code is compatible with this version of the grpc package. +// The SupportPackageIsVersion variables are referenced from generated protocol +// buffer files to ensure compatibility with the gRPC version used. The latest +// support package version is 5. +// +// Older versions are kept for compatibility. They may be removed if +// compatibility cannot be maintained. // -// This constant may be renamed in the future if a change in the generated code -// requires a synchronised update of grpc-go and protoc-gen-go. This constant -// should not be referenced from any other code. -const SupportPackageIsVersion4 = true +// These constants should not be referenced from any other code. +const ( + SupportPackageIsVersion3 = true + SupportPackageIsVersion4 = true + SupportPackageIsVersion5 = true +) // Version is the current grpc version. -const Version = "1.7.5" +const Version = "1.12.2" const grpcUA = "grpc-go/" + Version diff --git a/vendor/google.golang.org/grpc/server.go b/vendor/google.golang.org/grpc/server.go index 787665dfeb3..4969331cb3d 100644 --- a/vendor/google.golang.org/grpc/server.go +++ b/vendor/google.golang.org/grpc/server.go @@ -32,11 +32,17 @@ import ( "sync" "time" + "io/ioutil" + "golang.org/x/net/context" "golang.org/x/net/http2" "golang.org/x/net/trace" + + "google.golang.org/grpc/channelz" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" + "google.golang.org/grpc/encoding" + "google.golang.org/grpc/encoding/proto" "google.golang.org/grpc/grpclog" "google.golang.org/grpc/internal" "google.golang.org/grpc/keepalive" @@ -89,18 +95,28 @@ type Server struct { conns map[io.Closer]bool serve bool drain bool - ctx context.Context - cancel context.CancelFunc - // A CondVar to let GracefulStop() blocks until all the pending RPCs are finished - // and all the transport goes away. - cv *sync.Cond + cv *sync.Cond // signaled when connections close for GracefulStop m map[string]*service // service name -> service info events trace.EventLog + + quit chan struct{} + done chan struct{} + quitOnce sync.Once + doneOnce sync.Once + channelzRemoveOnce sync.Once + serveWG sync.WaitGroup // counts active Serve goroutines for GracefulStop + + channelzID int64 // channelz unique identification number + czmu sync.RWMutex + callsStarted int64 + callsFailed int64 + callsSucceeded int64 + lastCallStartedTime time.Time } type options struct { creds credentials.TransportCredentials - codec Codec + codec baseCodec cp Compressor dc Decompressor unaryInt UnaryServerInterceptor @@ -177,20 +193,32 @@ func KeepaliveEnforcementPolicy(kep keepalive.EnforcementPolicy) ServerOption { } // CustomCodec returns a ServerOption that sets a codec for message marshaling and unmarshaling. +// +// This will override any lookups by content-subtype for Codecs registered with RegisterCodec. func CustomCodec(codec Codec) ServerOption { return func(o *options) { o.codec = codec } } -// RPCCompressor returns a ServerOption that sets a compressor for outbound messages. +// RPCCompressor returns a ServerOption that sets a compressor for outbound +// messages. For backward compatibility, all outbound messages will be sent +// using this compressor, regardless of incoming message compression. By +// default, server messages will be sent using the same compressor with which +// request messages were sent. +// +// Deprecated: use encoding.RegisterCompressor instead. func RPCCompressor(cp Compressor) ServerOption { return func(o *options) { o.cp = cp } } -// RPCDecompressor returns a ServerOption that sets a decompressor for inbound messages. +// RPCDecompressor returns a ServerOption that sets a decompressor for inbound +// messages. It has higher priority than decompressors registered via +// encoding.RegisterCompressor. +// +// Deprecated: use encoding.RegisterCompressor instead. func RPCDecompressor(dc Decompressor) ServerOption { return func(o *options) { o.dc = dc @@ -198,7 +226,9 @@ func RPCDecompressor(dc Decompressor) ServerOption { } // MaxMsgSize returns a ServerOption to set the max message size in bytes the server can receive. -// If this is not set, gRPC uses the default limit. Deprecated: use MaxRecvMsgSize instead. +// If this is not set, gRPC uses the default limit. +// +// Deprecated: use MaxRecvMsgSize instead. func MaxMsgSize(m int) ServerOption { return MaxRecvMsgSize(m) } @@ -297,6 +327,8 @@ func UnknownServiceHandler(streamHandler StreamHandler) ServerOption { // connection establishment (up to and including HTTP/2 handshaking) for all // new connections. If this is not set, the default is 120 seconds. A zero or // negative value will result in an immediate timeout. +// +// This API is EXPERIMENTAL. func ConnectionTimeout(d time.Duration) ServerOption { return func(o *options) { o.connectionTimeout = d @@ -310,22 +342,23 @@ func NewServer(opt ...ServerOption) *Server { for _, o := range opt { o(&opts) } - if opts.codec == nil { - // Set the default codec. - opts.codec = protoCodec{} - } s := &Server{ lis: make(map[net.Listener]bool), opts: opts, conns: make(map[io.Closer]bool), m: make(map[string]*service), + quit: make(chan struct{}), + done: make(chan struct{}), } s.cv = sync.NewCond(&s.mu) - s.ctx, s.cancel = context.WithCancel(context.Background()) if EnableTracing { _, file, line, _ := runtime.Caller(1) s.events = trace.NewEventLog("grpc.Server", fmt.Sprintf("%s:%d", file, line)) } + + if channelz.IsOn() { + s.channelzID = channelz.RegisterServer(s, "") + } return s } @@ -430,11 +463,9 @@ func (s *Server) GetServiceInfo() map[string]ServiceInfo { return ret } -var ( - // ErrServerStopped indicates that the operation is now illegal because of - // the server being stopped. - ErrServerStopped = errors.New("grpc: the server has been stopped") -) +// ErrServerStopped indicates that the operation is now illegal because of +// the server being stopped. +var ErrServerStopped = errors.New("grpc: the server has been stopped") func (s *Server) useTransportAuthenticator(rawConn net.Conn) (net.Conn, credentials.AuthInfo, error) { if s.opts.creds == nil { @@ -443,28 +474,66 @@ func (s *Server) useTransportAuthenticator(rawConn net.Conn) (net.Conn, credenti return s.opts.creds.ServerHandshake(rawConn) } +type listenSocket struct { + net.Listener + channelzID int64 +} + +func (l *listenSocket) ChannelzMetric() *channelz.SocketInternalMetric { + return &channelz.SocketInternalMetric{ + LocalAddr: l.Listener.Addr(), + } +} + +func (l *listenSocket) Close() error { + err := l.Listener.Close() + if channelz.IsOn() { + channelz.RemoveEntry(l.channelzID) + } + return err +} + // Serve accepts incoming connections on the listener lis, creating a new // ServerTransport and service goroutine for each. The service goroutines // read gRPC requests and then call the registered handlers to reply to them. // Serve returns when lis.Accept fails with fatal errors. lis will be closed when // this method returns. -// Serve always returns non-nil error. +// Serve will return a non-nil error unless Stop or GracefulStop is called. func (s *Server) Serve(lis net.Listener) error { s.mu.Lock() s.printf("serving") s.serve = true if s.lis == nil { + // Serve called after Stop or GracefulStop. s.mu.Unlock() lis.Close() return ErrServerStopped } - s.lis[lis] = true + + s.serveWG.Add(1) + defer func() { + s.serveWG.Done() + select { + // Stop or GracefulStop called; block until done and return nil. + case <-s.quit: + <-s.done + default: + } + }() + + ls := &listenSocket{Listener: lis} + s.lis[ls] = true + + if channelz.IsOn() { + ls.channelzID = channelz.RegisterListenSocket(ls, s.channelzID, "") + } s.mu.Unlock() + defer func() { s.mu.Lock() - if s.lis != nil && s.lis[lis] { - lis.Close() - delete(s.lis, lis) + if s.lis != nil && s.lis[ls] { + ls.Close() + delete(s.lis, ls) } s.mu.Unlock() }() @@ -491,25 +560,39 @@ func (s *Server) Serve(lis net.Listener) error { timer := time.NewTimer(tempDelay) select { case <-timer.C: - case <-s.ctx.Done(): + case <-s.quit: + timer.Stop() + return nil } - timer.Stop() continue } s.mu.Lock() s.printf("done serving; Accept = %v", err) s.mu.Unlock() + + select { + case <-s.quit: + return nil + default: + } return err } tempDelay = 0 - // Start a new goroutine to deal with rawConn - // so we don't stall this Accept loop goroutine. - go s.handleRawConn(rawConn) + // Start a new goroutine to deal with rawConn so we don't stall this Accept + // loop goroutine. + // + // Make sure we account for the goroutine so GracefulStop doesn't nil out + // s.conns before this conn can be added. + s.serveWG.Add(1) + go func() { + s.handleRawConn(rawConn) + s.serveWG.Done() + }() } } -// handleRawConn is run in its own goroutine and handles a just-accepted -// connection that has not had any I/O performed on it yet. +// handleRawConn forks a goroutine to handle a just-accepted connection that +// has not had any I/O performed on it yet. func (s *Server) handleRawConn(rawConn net.Conn) { rawConn.SetDeadline(time.Now().Add(s.opts.connectionTimeout)) conn, authInfo, err := s.useTransportAuthenticator(rawConn) @@ -534,17 +617,28 @@ func (s *Server) handleRawConn(rawConn net.Conn) { } s.mu.Unlock() + var serve func() + c := conn.(io.Closer) if s.opts.useHandlerImpl { - rawConn.SetDeadline(time.Time{}) - s.serveUsingHandler(conn) + serve = func() { s.serveUsingHandler(conn) } } else { + // Finish handshaking (HTTP2) st := s.newHTTP2Transport(conn, authInfo) if st == nil { return } - rawConn.SetDeadline(time.Time{}) - s.serveStreams(st) + c = st + serve = func() { s.serveStreams(st) } + } + + rawConn.SetDeadline(time.Time{}) + if !s.addConn(c) { + return } + go func() { + serve() + s.removeConn(c) + }() } // newHTTP2Transport sets up a http/2 transport (using the @@ -561,6 +655,7 @@ func (s *Server) newHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo) tr InitialConnWindowSize: s.opts.initialConnWindowSize, WriteBufferSize: s.opts.writeBufferSize, ReadBufferSize: s.opts.readBufferSize, + ChannelzParentID: s.channelzID, } st, err := transport.NewServerTransport("http2", c, config) if err != nil { @@ -571,15 +666,11 @@ func (s *Server) newHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo) tr grpclog.Warningln("grpc: Server.Serve failed to create ServerTransport: ", err) return nil } - if !s.addConn(st) { - st.Close() - return nil - } + return st } func (s *Server) serveStreams(st transport.ServerTransport) { - defer s.removeConn(st) defer st.Close() var wg sync.WaitGroup st.HandleStreams(func(stream *transport.Stream) { @@ -613,11 +704,6 @@ var _ http.Handler = (*Server)(nil) // // conn is the *tls.Conn that's already been authenticated. func (s *Server) serveUsingHandler(conn net.Conn) { - if !s.addConn(conn) { - conn.Close() - return - } - defer s.removeConn(conn) h2s := &http2.Server{ MaxConcurrentStreams: s.opts.maxConcurrentStreams, } @@ -651,13 +737,12 @@ func (s *Server) serveUsingHandler(conn net.Conn) { // available through grpc-go's HTTP/2 server, and it is currently EXPERIMENTAL // and subject to change. func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { - st, err := transport.NewServerHandlerTransport(w, r) + st, err := transport.NewServerHandlerTransport(w, r, s.opts.statsHandler) if err != nil { http.Error(w, err.Error(), http.StatusInternalServerError) return } if !s.addConn(st) { - st.Close() return } defer s.removeConn(st) @@ -687,9 +772,15 @@ func (s *Server) traceInfo(st transport.ServerTransport, stream *transport.Strea func (s *Server) addConn(c io.Closer) bool { s.mu.Lock() defer s.mu.Unlock() - if s.conns == nil || s.drain { + if s.conns == nil { + c.Close() return false } + if s.drain { + // Transport added after we drained our existing conns: drain it + // immediately. + c.(transport.ServerTransport).Drain() + } s.conns[c] = true return true } @@ -703,18 +794,46 @@ func (s *Server) removeConn(c io.Closer) { } } -func (s *Server) sendResponse(t transport.ServerTransport, stream *transport.Stream, msg interface{}, cp Compressor, opts *transport.Options) error { +// ChannelzMetric returns ServerInternalMetric of current server. +// This is an EXPERIMENTAL API. +func (s *Server) ChannelzMetric() *channelz.ServerInternalMetric { + s.czmu.RLock() + defer s.czmu.RUnlock() + return &channelz.ServerInternalMetric{ + CallsStarted: s.callsStarted, + CallsSucceeded: s.callsSucceeded, + CallsFailed: s.callsFailed, + LastCallStartedTimestamp: s.lastCallStartedTime, + } +} + +func (s *Server) incrCallsStarted() { + s.czmu.Lock() + s.callsStarted++ + s.lastCallStartedTime = time.Now() + s.czmu.Unlock() +} + +func (s *Server) incrCallsSucceeded() { + s.czmu.Lock() + s.callsSucceeded++ + s.czmu.Unlock() +} + +func (s *Server) incrCallsFailed() { + s.czmu.Lock() + s.callsFailed++ + s.czmu.Unlock() +} + +func (s *Server) sendResponse(t transport.ServerTransport, stream *transport.Stream, msg interface{}, cp Compressor, opts *transport.Options, comp encoding.Compressor) error { var ( - cbuf *bytes.Buffer outPayload *stats.OutPayload ) - if cp != nil { - cbuf = new(bytes.Buffer) - } if s.opts.statsHandler != nil { outPayload = &stats.OutPayload{} } - hdr, data, err := encode(s.opts.codec, msg, cp, cbuf, outPayload) + hdr, data, err := encode(s.getCodec(stream.ContentSubtype()), msg, cp, outPayload, comp) if err != nil { grpclog.Errorln("grpc: server failed to encode response: ", err) return err @@ -731,15 +850,27 @@ func (s *Server) sendResponse(t transport.ServerTransport, stream *transport.Str } func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport.Stream, srv *service, md *MethodDesc, trInfo *traceInfo) (err error) { + if channelz.IsOn() { + s.incrCallsStarted() + defer func() { + if err != nil && err != io.EOF { + s.incrCallsFailed() + } else { + s.incrCallsSucceeded() + } + }() + } sh := s.opts.statsHandler if sh != nil { + beginTime := time.Now() begin := &stats.Begin{ - BeginTime: time.Now(), + BeginTime: beginTime, } sh.HandleRPC(stream.Context(), begin) defer func() { end := &stats.End{ - EndTime: time.Now(), + BeginTime: beginTime, + EndTime: time.Now(), } if err != nil && err != io.EOF { end.Error = toRPCErr(err) @@ -758,10 +889,43 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. } }() } + + // comp and cp are used for compression. decomp and dc are used for + // decompression. If comp and decomp are both set, they are the same; + // however they are kept separate to ensure that at most one of the + // compressor/decompressor variable pairs are set for use later. + var comp, decomp encoding.Compressor + var cp Compressor + var dc Decompressor + + // If dc is set and matches the stream's compression, use it. Otherwise, try + // to find a matching registered compressor for decomp. + if rc := stream.RecvCompress(); s.opts.dc != nil && s.opts.dc.Type() == rc { + dc = s.opts.dc + } else if rc != "" && rc != encoding.Identity { + decomp = encoding.GetCompressor(rc) + if decomp == nil { + st := status.Newf(codes.Unimplemented, "grpc: Decompressor is not installed for grpc-encoding %q", rc) + t.WriteStatus(stream, st) + return st.Err() + } + } + + // If cp is set, use it. Otherwise, attempt to compress the response using + // the incoming message compression method. + // + // NOTE: this needs to be ahead of all handling, https://github.com/grpc/grpc-go/issues/686. if s.opts.cp != nil { - // NOTE: this needs to be ahead of all handling, https://github.com/grpc/grpc-go/issues/686. - stream.SetSendCompress(s.opts.cp.Type()) + cp = s.opts.cp + stream.SetSendCompress(cp.Type()) + } else if rc := stream.RecvCompress(); rc != "" && rc != encoding.Identity { + // Legacy compressor not specified; attempt to respond with same encoding. + comp = encoding.GetCompressor(rc) + if comp != nil { + stream.SetSendCompress(rc) + } } + p := &parser{r: stream} pf, req, err := p.recvMsg(s.opts.maxReceiveMessageSize) if err == io.EOF { @@ -769,7 +933,7 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. return err } if err == io.ErrUnexpectedEOF { - err = Errorf(codes.Internal, io.ErrUnexpectedEOF.Error()) + err = status.Errorf(codes.Internal, io.ErrUnexpectedEOF.Error()) } if err != nil { if st, ok := status.FromError(err); ok { @@ -790,19 +954,14 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. } return err } - - if err := checkRecvPayload(pf, stream.RecvCompress(), s.opts.dc); err != nil { - if st, ok := status.FromError(err); ok { - if e := t.WriteStatus(stream, st); e != nil { - grpclog.Warningf("grpc: Server.processUnaryRPC failed to write status %v", e) - } - return err - } - if e := t.WriteStatus(stream, status.New(codes.Internal, err.Error())); e != nil { + if channelz.IsOn() { + t.IncrMsgRecv() + } + if st := checkRecvPayload(pf, stream.RecvCompress(), dc != nil || decomp != nil); st != nil { + if e := t.WriteStatus(stream, st); e != nil { grpclog.Warningf("grpc: Server.processUnaryRPC failed to write status %v", e) } - - // TODO checkRecvPayload always return RPC error. Add a return here if necessary. + return st.Err() } var inPayload *stats.InPayload if sh != nil { @@ -816,9 +975,17 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. } if pf == compressionMade { var err error - req, err = s.opts.dc.Do(bytes.NewReader(req)) - if err != nil { - return Errorf(codes.Internal, err.Error()) + if dc != nil { + req, err = dc.Do(bytes.NewReader(req)) + if err != nil { + return status.Errorf(codes.Internal, err.Error()) + } + } else { + tmp, _ := decomp.Decompress(bytes.NewReader(req)) + req, err = ioutil.ReadAll(tmp) + if err != nil { + return status.Errorf(codes.Internal, "grpc: failed to decompress the received message %v", err) + } } } if len(req) > s.opts.maxReceiveMessageSize { @@ -826,7 +993,7 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. // java implementation. return status.Errorf(codes.ResourceExhausted, "grpc: received message larger than max (%d vs. %d)", len(req), s.opts.maxReceiveMessageSize) } - if err := s.opts.codec.Unmarshal(req, v); err != nil { + if err := s.getCodec(stream.ContentSubtype()).Unmarshal(req, v); err != nil { return status.Errorf(codes.Internal, "grpc: error unmarshalling request: %v", err) } if inPayload != nil { @@ -840,12 +1007,13 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. } return nil } - reply, appErr := md.Handler(srv.server, stream.Context(), df, s.opts.unaryInt) + ctx := NewContextWithServerTransportStream(stream.Context(), stream) + reply, appErr := md.Handler(srv.server, ctx, df, s.opts.unaryInt) if appErr != nil { appStatus, ok := status.FromError(appErr) if !ok { // Convert appErr if it is not a grpc status error. - appErr = status.Error(convertCode(appErr), appErr.Error()) + appErr = status.Error(codes.Unknown, appErr.Error()) appStatus, _ = status.FromError(appErr) } if trInfo != nil { @@ -864,7 +1032,8 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. Last: true, Delay: false, } - if err := s.sendResponse(t, stream, reply, s.opts.cp, opts); err != nil { + + if err := s.sendResponse(t, stream, reply, cp, opts, comp); err != nil { if err == io.EOF { // The entire stream is done (for unary RPC only). return err @@ -887,6 +1056,9 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. } return err } + if channelz.IsOn() { + t.IncrMsgSent() + } if trInfo != nil { trInfo.tr.LazyLog(&payload{sent: true, msg: reply}, true) } @@ -897,15 +1069,27 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. } func (s *Server) processStreamingRPC(t transport.ServerTransport, stream *transport.Stream, srv *service, sd *StreamDesc, trInfo *traceInfo) (err error) { + if channelz.IsOn() { + s.incrCallsStarted() + defer func() { + if err != nil && err != io.EOF { + s.incrCallsFailed() + } else { + s.incrCallsSucceeded() + } + }() + } sh := s.opts.statsHandler if sh != nil { + beginTime := time.Now() begin := &stats.Begin{ - BeginTime: time.Now(), + BeginTime: beginTime, } sh.HandleRPC(stream.Context(), begin) defer func() { end := &stats.End{ - EndTime: time.Now(), + BeginTime: beginTime, + EndTime: time.Now(), } if err != nil && err != io.EOF { end.Error = toRPCErr(err) @@ -913,21 +1097,47 @@ func (s *Server) processStreamingRPC(t transport.ServerTransport, stream *transp sh.HandleRPC(stream.Context(), end) }() } - if s.opts.cp != nil { - stream.SetSendCompress(s.opts.cp.Type()) - } + ctx := NewContextWithServerTransportStream(stream.Context(), stream) ss := &serverStream{ + ctx: ctx, t: t, s: stream, p: &parser{r: stream}, - codec: s.opts.codec, - cp: s.opts.cp, - dc: s.opts.dc, + codec: s.getCodec(stream.ContentSubtype()), maxReceiveMessageSize: s.opts.maxReceiveMessageSize, maxSendMessageSize: s.opts.maxSendMessageSize, trInfo: trInfo, statsHandler: sh, } + + // If dc is set and matches the stream's compression, use it. Otherwise, try + // to find a matching registered compressor for decomp. + if rc := stream.RecvCompress(); s.opts.dc != nil && s.opts.dc.Type() == rc { + ss.dc = s.opts.dc + } else if rc != "" && rc != encoding.Identity { + ss.decomp = encoding.GetCompressor(rc) + if ss.decomp == nil { + st := status.Newf(codes.Unimplemented, "grpc: Decompressor is not installed for grpc-encoding %q", rc) + t.WriteStatus(ss.s, st) + return st.Err() + } + } + + // If cp is set, use it. Otherwise, attempt to compress the response using + // the incoming message compression method. + // + // NOTE: this needs to be ahead of all handling, https://github.com/grpc/grpc-go/issues/686. + if s.opts.cp != nil { + ss.cp = s.opts.cp + stream.SetSendCompress(s.opts.cp.Type()) + } else if rc := stream.RecvCompress(); rc != "" && rc != encoding.Identity { + // Legacy compressor not specified; attempt to respond with same encoding. + ss.comp = encoding.GetCompressor(rc) + if ss.comp != nil { + stream.SetSendCompress(rc) + } + } + if trInfo != nil { trInfo.tr.LazyLog(&trInfo.firstLine, false) defer func() { @@ -963,7 +1173,7 @@ func (s *Server) processStreamingRPC(t transport.ServerTransport, stream *transp case transport.StreamError: appStatus = status.New(err.Code, err.Desc) default: - appStatus = status.New(convertCode(appErr), appErr.Error()) + appStatus = status.New(codes.Unknown, appErr.Error()) } appErr = appStatus.Err() } @@ -983,7 +1193,6 @@ func (s *Server) processStreamingRPC(t transport.ServerTransport, stream *transp ss.mu.Unlock() } return t.WriteStatus(ss.s, status.New(codes.OK, "")) - } func (s *Server) handleStream(t transport.ServerTransport, stream *transport.Stream, trInfo *traceInfo) { @@ -1065,12 +1274,65 @@ func (s *Server) handleStream(t transport.ServerTransport, stream *transport.Str } } +// The key to save ServerTransportStream in the context. +type streamKey struct{} + +// NewContextWithServerTransportStream creates a new context from ctx and +// attaches stream to it. +// +// This API is EXPERIMENTAL. +func NewContextWithServerTransportStream(ctx context.Context, stream ServerTransportStream) context.Context { + return context.WithValue(ctx, streamKey{}, stream) +} + +// ServerTransportStream is a minimal interface that a transport stream must +// implement. This can be used to mock an actual transport stream for tests of +// handler code that use, for example, grpc.SetHeader (which requires some +// stream to be in context). +// +// See also NewContextWithServerTransportStream. +// +// This API is EXPERIMENTAL. +type ServerTransportStream interface { + Method() string + SetHeader(md metadata.MD) error + SendHeader(md metadata.MD) error + SetTrailer(md metadata.MD) error +} + +// ServerTransportStreamFromContext returns the ServerTransportStream saved in +// ctx. Returns nil if the given context has no stream associated with it +// (which implies it is not an RPC invocation context). +// +// This API is EXPERIMENTAL. +func ServerTransportStreamFromContext(ctx context.Context) ServerTransportStream { + s, _ := ctx.Value(streamKey{}).(ServerTransportStream) + return s +} + // Stop stops the gRPC server. It immediately closes all open // connections and listeners. // It cancels all active RPCs on the server side and the corresponding // pending RPCs on the client side will get notified by connection // errors. func (s *Server) Stop() { + s.quitOnce.Do(func() { + close(s.quit) + }) + + defer func() { + s.serveWG.Wait() + s.doneOnce.Do(func() { + close(s.done) + }) + }() + + s.channelzRemoveOnce.Do(func() { + if channelz.IsOn() { + channelz.RemoveEntry(s.channelzID) + } + }) + s.mu.Lock() listeners := s.lis s.lis = nil @@ -1088,7 +1350,6 @@ func (s *Server) Stop() { } s.mu.Lock() - s.cancel() if s.events != nil { s.events.Finish() s.events = nil @@ -1100,22 +1361,44 @@ func (s *Server) Stop() { // accepting new connections and RPCs and blocks until all the pending RPCs are // finished. func (s *Server) GracefulStop() { + s.quitOnce.Do(func() { + close(s.quit) + }) + + defer func() { + s.doneOnce.Do(func() { + close(s.done) + }) + }() + + s.channelzRemoveOnce.Do(func() { + if channelz.IsOn() { + channelz.RemoveEntry(s.channelzID) + } + }) s.mu.Lock() - defer s.mu.Unlock() if s.conns == nil { + s.mu.Unlock() return } + for lis := range s.lis { lis.Close() } s.lis = nil - s.cancel() if !s.drain { for c := range s.conns { c.(transport.ServerTransport).Drain() } s.drain = true } + + // Wait for serving threads to be ready to exit. Only then can we be sure no + // new conns will be created. + s.mu.Unlock() + s.serveWG.Wait() + s.mu.Lock() + for len(s.conns) != 0 { s.cv.Wait() } @@ -1124,26 +1407,29 @@ func (s *Server) GracefulStop() { s.events.Finish() s.events = nil } + s.mu.Unlock() } func init() { - internal.TestingCloseConns = func(arg interface{}) { - arg.(*Server).testingCloseConns() - } internal.TestingUseHandlerImpl = func(arg interface{}) { arg.(*Server).opts.useHandlerImpl = true } } -// testingCloseConns closes all existing transports but keeps s.lis -// accepting new connections. -func (s *Server) testingCloseConns() { - s.mu.Lock() - for c := range s.conns { - c.Close() - delete(s.conns, c) +// contentSubtype must be lowercase +// cannot return nil +func (s *Server) getCodec(contentSubtype string) baseCodec { + if s.opts.codec != nil { + return s.opts.codec } - s.mu.Unlock() + if contentSubtype == "" { + return encoding.GetCodec(proto.Name) + } + codec := encoding.GetCodec(contentSubtype) + if codec == nil { + return encoding.GetCodec(proto.Name) + } + return codec } // SetHeader sets the header metadata. @@ -1156,9 +1442,9 @@ func SetHeader(ctx context.Context, md metadata.MD) error { if md.Len() == 0 { return nil } - stream, ok := transport.StreamFromContext(ctx) - if !ok { - return Errorf(codes.Internal, "grpc: failed to fetch the stream from the context %v", ctx) + stream := ServerTransportStreamFromContext(ctx) + if stream == nil { + return status.Errorf(codes.Internal, "grpc: failed to fetch the stream from the context %v", ctx) } return stream.SetHeader(md) } @@ -1166,15 +1452,11 @@ func SetHeader(ctx context.Context, md metadata.MD) error { // SendHeader sends header metadata. It may be called at most once. // The provided md and headers set by SetHeader() will be sent. func SendHeader(ctx context.Context, md metadata.MD) error { - stream, ok := transport.StreamFromContext(ctx) - if !ok { - return Errorf(codes.Internal, "grpc: failed to fetch the stream from the context %v", ctx) - } - t := stream.ServerTransport() - if t == nil { - grpclog.Fatalf("grpc: SendHeader: %v has no ServerTransport to send header metadata.", stream) + stream := ServerTransportStreamFromContext(ctx) + if stream == nil { + return status.Errorf(codes.Internal, "grpc: failed to fetch the stream from the context %v", ctx) } - if err := t.WriteHeader(stream, md); err != nil { + if err := stream.SendHeader(md); err != nil { return toRPCErr(err) } return nil @@ -1186,9 +1468,19 @@ func SetTrailer(ctx context.Context, md metadata.MD) error { if md.Len() == 0 { return nil } - stream, ok := transport.StreamFromContext(ctx) - if !ok { - return Errorf(codes.Internal, "grpc: failed to fetch the stream from the context %v", ctx) + stream := ServerTransportStreamFromContext(ctx) + if stream == nil { + return status.Errorf(codes.Internal, "grpc: failed to fetch the stream from the context %v", ctx) } return stream.SetTrailer(md) } + +// Method returns the method string for the server context. The returned +// string is in the format of "/service/method". +func Method(ctx context.Context) (string, bool) { + s := ServerTransportStreamFromContext(ctx) + if s == nil { + return "", false + } + return s.Method(), true +} diff --git a/vendor/google.golang.org/grpc/service_config.go b/vendor/google.golang.org/grpc/service_config.go new file mode 100644 index 00000000000..015631d8d38 --- /dev/null +++ b/vendor/google.golang.org/grpc/service_config.go @@ -0,0 +1,233 @@ +/* + * + * 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 ( + "encoding/json" + "fmt" + "strconv" + "strings" + "time" + + "google.golang.org/grpc/grpclog" +) + +const maxInt = int(^uint(0) >> 1) + +// MethodConfig defines the configuration recommended by the service providers for a +// particular method. +// +// Deprecated: Users should not use this struct. Service config should be received +// through name resolver, as specified here +// https://github.com/grpc/grpc/blob/master/doc/service_config.md +type MethodConfig struct { + // WaitForReady indicates whether RPCs sent to this method should wait until + // the connection is ready by default (!failfast). The value specified via the + // gRPC client API will override the value set here. + WaitForReady *bool + // Timeout is the default timeout for RPCs sent to this method. The actual + // deadline used will be 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 RPC has no deadline. + Timeout *time.Duration + // 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 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 + // MaxRespSize is the maximum allowed payload size for an individual response in a + // stream (server->client) in bytes. + MaxRespSize *int +} + +// ServiceConfig is provided by the service provider and contains parameters for how +// clients that connect to the service should behave. +// +// Deprecated: Users should not use this struct. Service config should be received +// through name resolver, as specified here +// https://github.com/grpc/grpc/blob/master/doc/service_config.md +type ServiceConfig struct { + // LB is the load balancer the service providers recommends. The balancer specified + // via grpc.WithBalancer will override this. + LB *string + // Methods contains a map for the methods in this service. + // If there is an exact match for a method (i.e. /service/method) in the map, use the corresponding MethodConfig. + // If there's no exact match, look for the default config for the service (/service/) and use the corresponding MethodConfig if it exists. + // Otherwise, the method has no MethodConfig to use. + Methods map[string]MethodConfig + + stickinessMetadataKey *string +} + +func parseDuration(s *string) (*time.Duration, error) { + if s == nil { + return nil, nil + } + if !strings.HasSuffix(*s, "s") { + return nil, fmt.Errorf("malformed duration %q", *s) + } + ss := strings.SplitN((*s)[:len(*s)-1], ".", 3) + if len(ss) > 2 { + return nil, fmt.Errorf("malformed duration %q", *s) + } + // hasDigits is set if either the whole or fractional part of the number is + // present, since both are optional but one is required. + hasDigits := false + var d time.Duration + if len(ss[0]) > 0 { + i, err := strconv.ParseInt(ss[0], 10, 32) + if err != nil { + return nil, fmt.Errorf("malformed duration %q: %v", *s, err) + } + d = time.Duration(i) * time.Second + hasDigits = true + } + if len(ss) == 2 && len(ss[1]) > 0 { + if len(ss[1]) > 9 { + return nil, fmt.Errorf("malformed duration %q", *s) + } + f, err := strconv.ParseInt(ss[1], 10, 64) + if err != nil { + return nil, fmt.Errorf("malformed duration %q: %v", *s, err) + } + for i := 9; i > len(ss[1]); i-- { + f *= 10 + } + d += time.Duration(f) + hasDigits = true + } + if !hasDigits { + return nil, fmt.Errorf("malformed duration %q", *s) + } + + return &d, nil +} + +type jsonName struct { + Service *string + Method *string +} + +func (j jsonName) generatePath() (string, bool) { + if j.Service == nil { + return "", false + } + res := "/" + *j.Service + "/" + if j.Method != nil { + res += *j.Method + } + return res, true +} + +// TODO(lyuxuan): delete this struct after cleaning up old service config implementation. +type jsonMC struct { + Name *[]jsonName + WaitForReady *bool + Timeout *string + MaxRequestMessageBytes *int64 + MaxResponseMessageBytes *int64 +} + +// TODO(lyuxuan): delete this struct after cleaning up old service config implementation. +type jsonSC struct { + LoadBalancingPolicy *string + StickinessMetadataKey *string + MethodConfig *[]jsonMC +} + +func parseServiceConfig(js string) (ServiceConfig, error) { + var rsc jsonSC + err := json.Unmarshal([]byte(js), &rsc) + if err != nil { + grpclog.Warningf("grpc: parseServiceConfig error unmarshaling %s due to %v", js, err) + return ServiceConfig{}, err + } + sc := ServiceConfig{ + LB: rsc.LoadBalancingPolicy, + Methods: make(map[string]MethodConfig), + + stickinessMetadataKey: rsc.StickinessMetadataKey, + } + if rsc.MethodConfig == nil { + return sc, nil + } + + for _, m := range *rsc.MethodConfig { + if m.Name == nil { + continue + } + d, err := parseDuration(m.Timeout) + if err != nil { + grpclog.Warningf("grpc: parseServiceConfig error unmarshaling %s due to %v", js, err) + return ServiceConfig{}, err + } + + mc := MethodConfig{ + WaitForReady: m.WaitForReady, + Timeout: d, + } + if m.MaxRequestMessageBytes != nil { + if *m.MaxRequestMessageBytes > int64(maxInt) { + mc.MaxReqSize = newInt(maxInt) + } else { + mc.MaxReqSize = newInt(int(*m.MaxRequestMessageBytes)) + } + } + if m.MaxResponseMessageBytes != nil { + if *m.MaxResponseMessageBytes > int64(maxInt) { + mc.MaxRespSize = newInt(maxInt) + } else { + mc.MaxRespSize = newInt(int(*m.MaxResponseMessageBytes)) + } + } + for _, n := range *m.Name { + if path, valid := n.generatePath(); valid { + sc.Methods[path] = mc + } + } + } + + return sc, nil +} + +func min(a, b *int) *int { + if *a < *b { + return a + } + return b +} + +func getMaxSize(mcMax, doptMax *int, defaultVal int) *int { + if mcMax == nil && doptMax == nil { + return &defaultVal + } + if mcMax != nil && doptMax != nil { + return min(mcMax, doptMax) + } + if mcMax != nil { + return mcMax + } + return doptMax +} + +func newInt(b int) *int { + return &b +} diff --git a/vendor/google.golang.org/grpc/stats/stats.go b/vendor/google.golang.org/grpc/stats/stats.go index d5aa2f793bf..3f13190a0ac 100644 --- a/vendor/google.golang.org/grpc/stats/stats.go +++ b/vendor/google.golang.org/grpc/stats/stats.go @@ -169,6 +169,8 @@ func (s *OutTrailer) isRPCStats() {} type End struct { // Client is true if this End is from client side. Client bool + // BeginTime is the time when the RPC began. + BeginTime time.Time // EndTime is the time when the RPC ends. EndTime time.Time // Error is the error the RPC ended with. It is an error generated from diff --git a/vendor/google.golang.org/grpc/status/status.go b/vendor/google.golang.org/grpc/status/status.go index 871dc4b31c7..9c61b094508 100644 --- a/vendor/google.golang.org/grpc/status/status.go +++ b/vendor/google.golang.org/grpc/status/status.go @@ -46,7 +46,7 @@ func (se *statusError) Error() string { return fmt.Sprintf("rpc error: code = %s desc = %s", codes.Code(p.GetCode()), p.GetMessage()) } -func (se *statusError) status() *Status { +func (se *statusError) GRPCStatus() *Status { return &Status{s: (*spb.Status)(se)} } @@ -120,15 +120,23 @@ func FromProto(s *spb.Status) *Status { } // FromError returns a Status representing err if it was produced from this -// package, otherwise it returns nil, false. +// package or has a method `GRPCStatus() *Status`. Otherwise, ok is false and a +// Status is returned with codes.Unknown and the original error message. func FromError(err error) (s *Status, ok bool) { if err == nil { return &Status{s: &spb.Status{Code: int32(codes.OK)}}, true } - if s, ok := err.(*statusError); ok { - return s.status(), true + if se, ok := err.(interface{ GRPCStatus() *Status }); ok { + return se.GRPCStatus(), true } - return nil, false + return New(codes.Unknown, err.Error()), false +} + +// Convert is a convenience function which removes the need to handle the +// boolean return value from FromError. +func Convert(err error) *Status { + s, _ := FromError(err) + return s } // WithDetails returns a new status with the provided details messages appended to the status. @@ -166,3 +174,16 @@ func (s *Status) Details() []interface{} { } return details } + +// Code returns the Code of the error if it is a Status error, codes.OK if err +// is nil, or codes.Unknown otherwise. +func Code(err error) codes.Code { + // Don't use FromError to avoid allocation of OK status. + if err == nil { + return codes.OK + } + if se, ok := err.(interface{ GRPCStatus() *Status }); ok { + return se.GRPCStatus().Code() + } + return codes.Unknown +} diff --git a/vendor/google.golang.org/grpc/stream.go b/vendor/google.golang.org/grpc/stream.go index 75eab40b109..82921a15a3a 100644 --- a/vendor/google.golang.org/grpc/stream.go +++ b/vendor/google.golang.org/grpc/stream.go @@ -19,7 +19,6 @@ package grpc import ( - "bytes" "errors" "io" "sync" @@ -28,16 +27,20 @@ import ( "golang.org/x/net/context" "golang.org/x/net/trace" "google.golang.org/grpc/balancer" + "google.golang.org/grpc/channelz" "google.golang.org/grpc/codes" + "google.golang.org/grpc/encoding" "google.golang.org/grpc/metadata" - "google.golang.org/grpc/peer" "google.golang.org/grpc/stats" "google.golang.org/grpc/status" "google.golang.org/grpc/transport" ) // StreamHandler defines the handler called by gRPC server to complete the -// execution of a streaming RPC. +// execution of a streaming RPC. If a StreamHandler returns an error, it +// should be produced by the status package, or else gRPC will use +// codes.Unknown as the status code and err.Error() as the status message +// of the RPC. type StreamHandler func(srv interface{}, stream ServerStream) error // StreamDesc represents a streaming RPC service's method specification. @@ -51,6 +54,8 @@ type StreamDesc struct { } // Stream defines the common interface a client or server stream has to satisfy. +// +// All errors returned from Stream are compatible with the status package. type Stream interface { // Context returns the context for this stream. Context() context.Context @@ -89,43 +94,65 @@ type ClientStream interface { // Stream.SendMsg() may return a non-nil error when something wrong happens sending // the request. The returned error indicates the status of this sending, not the final // status of the RPC. - // Always call Stream.RecvMsg() to get the final status if you care about the status of - // the RPC. + // + // Always call Stream.RecvMsg() to drain the stream and get the final + // status, otherwise there could be leaked resources. Stream } -// NewClientStream creates a new Stream for the client side. This is called -// by generated code. -func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, opts ...CallOption) (_ ClientStream, err error) { +// NewStream creates a new Stream for the client side. This is typically +// called by generated code. +func (cc *ClientConn) NewStream(ctx context.Context, desc *StreamDesc, method string, opts ...CallOption) (ClientStream, error) { + // allow interceptor to see all applicable call options, which means those + // configured as defaults from dial option as well as per-call options + opts = combine(cc.dopts.callOptions, opts) + if cc.dopts.streamInt != nil { return cc.dopts.streamInt(ctx, desc, cc, method, newClientStream, opts...) } return newClientStream(ctx, desc, cc, method, opts...) } +// NewClientStream creates a new Stream for the client side. This is typically +// called by generated code. +// +// DEPRECATED: Use ClientConn.NewStream instead. +func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, opts ...CallOption) (ClientStream, error) { + return cc.NewStream(ctx, desc, method, opts...) +} + func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, opts ...CallOption) (_ ClientStream, err error) { - var ( - t transport.ClientTransport - s *transport.Stream - done func(balancer.DoneInfo) - cancel context.CancelFunc - ) + if channelz.IsOn() { + cc.incrCallsStarted() + defer func() { + if err != nil { + cc.incrCallsFailed() + } + }() + } c := defaultCallInfo() mc := cc.GetMethodConfig(method) if mc.WaitForReady != nil { c.failFast = !*mc.WaitForReady } - if mc.Timeout != nil { + // Possible context leak: + // The cancel function for the child context we create will only be called + // when RecvMsg returns a non-nil error, if the ClientConn is closed, or if + // an error is generated by SendMsg. + // https://github.com/grpc/grpc-go/issues/1818. + var cancel context.CancelFunc + if mc.Timeout != nil && *mc.Timeout >= 0 { ctx, cancel = context.WithTimeout(ctx, *mc.Timeout) - defer func() { - if err != nil { - cancel() - } - }() + } else { + ctx, cancel = context.WithCancel(ctx) } + defer func() { + if err != nil { + cancel() + } + }() - opts = append(cc.dopts.callOptions, opts...) for _, o := range opts { if err := o.before(c); err != nil { return nil, toRPCErr(err) @@ -133,6 +160,9 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth } c.maxSendMessageSize = getMaxSize(mc.MaxReqSize, c.maxSendMessageSize, defaultClientMaxSendMessageSize) c.maxReceiveMessageSize = getMaxSize(mc.MaxRespSize, c.maxReceiveMessageSize, defaultClientMaxReceiveMessageSize) + if err := setCallInfoCodec(c); err != nil { + return nil, err + } callHdr := &transport.CallHdr{ Host: cc.authority, @@ -141,10 +171,27 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth // so we don't flush the header. // If it's client streaming, the user may never send a request or send it any // time soon, so we ask the transport to flush the header. - Flush: desc.ClientStreams, - } - if cc.dopts.cp != nil { + Flush: desc.ClientStreams, + ContentSubtype: c.contentSubtype, + } + + // Set our outgoing compression according to the UseCompressor CallOption, if + // set. In that case, also find the compressor from the encoding package. + // Otherwise, use the compressor configured by the WithCompressor DialOption, + // if set. + var cp Compressor + var comp encoding.Compressor + if ct := c.compressorType; ct != "" { + callHdr.SendCompress = ct + if ct != encoding.Identity { + comp = encoding.GetCompressor(ct) + if comp == nil { + return nil, status.Errorf(codes.Internal, "grpc: Compressor is not installed for requested grpc-encoding %q", ct) + } + } + } else if cc.dopts.cp != nil { callHdr.SendCompress = cc.dopts.cp.Type() + cp = cc.dopts.cp } if c.creds != nil { callHdr.Creds = c.creds @@ -170,11 +217,13 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth } ctx = newContextWithRPCInfo(ctx, c.failFast) sh := cc.dopts.copts.StatsHandler + var beginTime time.Time if sh != nil { ctx = sh.TagRPC(ctx, &stats.RPCTagInfo{FullMethodName: method, FailFast: c.failFast}) + beginTime = time.Now() begin := &stats.Begin{ Client: true, - BeginTime: time.Now(), + BeginTime: beginTime, FailFast: c.failFast, } sh.HandleRPC(ctx, begin) @@ -182,341 +231,384 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth if err != nil { // Only handle end stats if err != nil. end := &stats.End{ - Client: true, - Error: err, + Client: true, + Error: err, + BeginTime: beginTime, + EndTime: time.Now(), } sh.HandleRPC(ctx, end) } }() } + + var ( + t transport.ClientTransport + s *transport.Stream + done func(balancer.DoneInfo) + ) for { + // Check to make sure the context has expired. This will prevent us from + // looping forever if an error occurs for wait-for-ready RPCs where no data + // is sent on the wire. + select { + case <-ctx.Done(): + return nil, toRPCErr(ctx.Err()) + default: + } + t, done, err = cc.getTransport(ctx, c.failFast) if err != nil { - // TODO(zhaoq): Probably revisit the error handling. - if _, ok := status.FromError(err); ok { - return nil, err - } - if err == errConnClosing || err == errConnUnavailable { - if c.failFast { - return nil, Errorf(codes.Unavailable, "%v", err) - } - continue - } - // All the other errors are treated as Internal errors. - return nil, Errorf(codes.Internal, "%v", err) + return nil, err } s, err = t.NewStream(ctx, callHdr) if err != 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 done != nil { done(balancer.DoneInfo{Err: err}) done = nil } - if _, ok := err.(transport.ConnectionError); (ok || err == transport.ErrStreamDrain) && !c.failFast { + // In the event of any error from NewStream, we never attempted to write + // anything to the wire, so we can retry indefinitely for non-fail-fast + // RPCs. + if !c.failFast { continue } return nil, toRPCErr(err) } break } - // Set callInfo.peer object from stream's context. - if peer, ok := peer.FromContext(s.Context()); ok { - c.peer = peer - } + cs := &clientStream{ opts: opts, c: c, + cc: cc, desc: desc, - codec: cc.dopts.codec, - cp: cc.dopts.cp, - dc: cc.dopts.dc, + codec: c.codec, + cp: cp, + comp: comp, cancel: cancel, - - done: done, - t: t, - s: s, - p: &parser{r: s}, - - tracing: EnableTracing, - trInfo: trInfo, - - statsCtx: ctx, - statsHandler: cc.dopts.copts.StatsHandler, + attempt: &csAttempt{ + t: t, + s: s, + p: &parser{r: s}, + done: done, + dc: cc.dopts.dc, + ctx: ctx, + trInfo: trInfo, + statsHandler: sh, + beginTime: beginTime, + }, + } + cs.c.stream = cs + cs.attempt.cs = cs + if desc != unaryStreamDesc { + // Listen on cc and stream contexts to cleanup when the user closes the + // ClientConn or cancels the stream context. In all other cases, an error + // should already be injected into the recv buffer by the transport, which + // the client will eventually receive, and then we will cancel the stream's + // context in clientStream.finish. + go func() { + select { + case <-cc.ctx.Done(): + cs.finish(ErrClientConnClosing) + case <-ctx.Done(): + cs.finish(toRPCErr(ctx.Err())) + } + }() } - // 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() { - select { - case <-t.Error(): - // Incur transport error, simply exit. - case <-cc.ctx.Done(): - cs.finish(ErrClientConnClosing) - cs.closeTransportStream(ErrClientConnClosing) - case <-s.Done(): - // TODO: The trace of the RPC is terminated here when there is no pending - // I/O, which is probably not the optimal solution. - cs.finish(s.Status().Err()) - cs.closeTransportStream(nil) - case <-s.GoAway(): - cs.finish(errConnDrain) - cs.closeTransportStream(errConnDrain) - case <-s.Context().Done(): - err := s.Context().Err() - cs.finish(err) - cs.closeTransportStream(transport.ContextErr(err)) - } - }() return cs, nil } // clientStream implements a client side Stream. type clientStream struct { - opts []CallOption - c *callInfo - t transport.ClientTransport - s *transport.Stream - p *parser - desc *StreamDesc - codec Codec - cp Compressor - dc Decompressor - cancel context.CancelFunc + opts []CallOption + c *callInfo + cc *ClientConn + desc *StreamDesc + + codec baseCodec + cp Compressor + comp encoding.Compressor + + cancel context.CancelFunc // cancels all attempts - tracing bool // set to EnableTracing when the clientStream is created. + sentLast bool // sent an end stream + + mu sync.Mutex // guards finished + finished bool // TODO: replace with atomic cmpxchg or sync.Once? + + attempt *csAttempt // the active client stream attempt + // TODO(hedging): hedging will have multiple attempts simultaneously. +} - mu sync.Mutex - done func(balancer.DoneInfo) - closed bool - finished bool - // trInfo.tr is set when the clientStream is created (if EnableTracing is true), - // and is set to nil when the clientStream's finish method is called. +// csAttempt implements a single transport stream attempt within a +// clientStream. +type csAttempt struct { + cs *clientStream + t transport.ClientTransport + s *transport.Stream + p *parser + done func(balancer.DoneInfo) + + dc Decompressor + decomp encoding.Compressor + decompSet bool + + ctx context.Context // the application's context, wrapped by stats/tracing + + mu sync.Mutex // guards trInfo.tr + // trInfo.tr is set when created (if EnableTracing is true), + // and cleared when the finish method is called. trInfo traceInfo - // statsCtx keeps the user context for stats handling. - // All stats collection should use the statsCtx (instead of the stream context) - // so that all the generated stats for a particular RPC can be associated in the processing phase. - statsCtx context.Context statsHandler stats.Handler + beginTime time.Time } func (cs *clientStream) Context() context.Context { - return cs.s.Context() + // TODO(retry): commit the current attempt (the context has peer-aware data). + return cs.attempt.context() } func (cs *clientStream) Header() (metadata.MD, error) { - m, err := cs.s.Header() + m, err := cs.attempt.header() if err != nil { - if _, ok := err.(transport.ConnectionError); !ok { - cs.closeTransportStream(err) - } + // TODO(retry): maybe retry on error or commit attempt on success. + err = toRPCErr(err) + cs.finish(err) } return m, err } func (cs *clientStream) Trailer() metadata.MD { - return cs.s.Trailer() + // TODO(retry): on error, maybe retry (trailers-only). + return cs.attempt.trailer() } func (cs *clientStream) SendMsg(m interface{}) (err error) { - if cs.tracing { - cs.mu.Lock() - if cs.trInfo.tr != nil { - cs.trInfo.tr.LazyLog(&payload{sent: true, msg: m}, true) - } + // TODO(retry): buffer message for replaying if not committed. + return cs.attempt.sendMsg(m) +} + +func (cs *clientStream) RecvMsg(m interface{}) (err error) { + // TODO(retry): maybe retry on error or commit attempt on success. + return cs.attempt.recvMsg(m) +} + +func (cs *clientStream) CloseSend() error { + cs.attempt.closeSend() + return nil +} + +func (cs *clientStream) finish(err error) { + if err == io.EOF { + // Ending a stream with EOF indicates a success. + err = nil + } + cs.mu.Lock() + if cs.finished { cs.mu.Unlock() + return + } + cs.finished = true + cs.mu.Unlock() + if channelz.IsOn() { + if err != nil { + cs.cc.incrCallsFailed() + } else { + cs.cc.incrCallsSucceeded() + } + } + // TODO(retry): commit current attempt if necessary. + cs.attempt.finish(err) + for _, o := range cs.opts { + o.after(cs.c) } + cs.cancel() +} + +func (a *csAttempt) context() context.Context { + return a.s.Context() +} + +func (a *csAttempt) header() (metadata.MD, error) { + return a.s.Header() +} + +func (a *csAttempt) trailer() metadata.MD { + return a.s.Trailer() +} + +func (a *csAttempt) sendMsg(m interface{}) (err error) { // TODO Investigate how to signal the stats handling party. // generate error stats if err != nil && err != io.EOF? + cs := a.cs defer func() { - if err != nil { - cs.finish(err) - } - if err == nil { - return - } - if err == io.EOF { - // Specialize the process for server streaming. SendMsg is only called - // once when creating the stream object. io.EOF needs to be skipped when - // the rpc is early finished (before the stream object is created.). - // TODO: It is probably better to move this into the generated code. - if !cs.desc.ClientStreams && cs.desc.ServerStreams { - err = nil - } - return + // For non-client-streaming RPCs, we return nil instead of EOF on success + // because the generated code requires it. finish is not called; RecvMsg() + // will call it with the stream's status independently. + if err == io.EOF && !cs.desc.ClientStreams { + err = nil } - if _, ok := err.(transport.ConnectionError); !ok { - cs.closeTransportStream(err) + if err != nil && err != io.EOF { + // Call finish on the client stream for errors generated by this SendMsg + // call, as these indicate problems created by this client. (Transport + // errors are converted to an io.EOF error below; the real error will be + // returned from RecvMsg eventually in that case, or be retried.) + cs.finish(err) } - err = toRPCErr(err) }() + // TODO: Check cs.sentLast and error if we already ended the stream. + if EnableTracing { + a.mu.Lock() + if a.trInfo.tr != nil { + a.trInfo.tr.LazyLog(&payload{sent: true, msg: m}, true) + } + a.mu.Unlock() + } var outPayload *stats.OutPayload - if cs.statsHandler != nil { + if a.statsHandler != nil { outPayload = &stats.OutPayload{ Client: true, } } - hdr, data, err := encode(cs.codec, m, cs.cp, bytes.NewBuffer([]byte{}), outPayload) + hdr, data, err := encode(cs.codec, m, cs.cp, outPayload, cs.comp) if err != nil { return err } - if cs.c.maxSendMessageSize == nil { - return Errorf(codes.Internal, "callInfo maxSendMessageSize field uninitialized(nil)") - } if len(data) > *cs.c.maxSendMessageSize { - return Errorf(codes.ResourceExhausted, "trying to send message larger than max (%d vs. %d)", len(data), *cs.c.maxSendMessageSize) + return status.Errorf(codes.ResourceExhausted, "trying to send message larger than max (%d vs. %d)", len(data), *cs.c.maxSendMessageSize) } - err = cs.t.Write(cs.s, hdr, data, &transport.Options{Last: false}) - if err == nil && outPayload != nil { - outPayload.SentTime = time.Now() - cs.statsHandler.HandleRPC(cs.statsCtx, outPayload) + if !cs.desc.ClientStreams { + cs.sentLast = true } - return err + err = a.t.Write(a.s, hdr, data, &transport.Options{Last: !cs.desc.ClientStreams}) + if err == nil { + if outPayload != nil { + outPayload.SentTime = time.Now() + a.statsHandler.HandleRPC(a.ctx, outPayload) + } + if channelz.IsOn() { + a.t.IncrMsgSent() + } + return nil + } + return io.EOF } -func (cs *clientStream) RecvMsg(m interface{}) (err error) { +func (a *csAttempt) recvMsg(m interface{}) (err error) { + cs := a.cs + defer func() { + if err != nil || !cs.desc.ServerStreams { + // err != nil or non-server-streaming indicates end of stream. + cs.finish(err) + } + }() var inPayload *stats.InPayload - if cs.statsHandler != nil { + if a.statsHandler != nil { inPayload = &stats.InPayload{ Client: true, } } - if cs.c.maxReceiveMessageSize == nil { - return Errorf(codes.Internal, "callInfo maxReceiveMessageSize field uninitialized(nil)") - } - err = recv(cs.p, cs.codec, cs.s, cs.dc, m, *cs.c.maxReceiveMessageSize, inPayload) - defer func() { - // err != nil indicates the termination of the stream. - if err != nil { - cs.finish(err) - } - }() - if err == nil { - if cs.tracing { - cs.mu.Lock() - if cs.trInfo.tr != nil { - cs.trInfo.tr.LazyLog(&payload{sent: false, msg: m}, true) + if !a.decompSet { + // Block until we receive headers containing received message encoding. + if ct := a.s.RecvCompress(); ct != "" && ct != encoding.Identity { + if a.dc == nil || a.dc.Type() != ct { + // No configured decompressor, or it does not match the incoming + // message encoding; attempt to find a registered compressor that does. + a.dc = nil + a.decomp = encoding.GetCompressor(ct) } - cs.mu.Unlock() - } - if inPayload != nil { - cs.statsHandler.HandleRPC(cs.statsCtx, inPayload) - } - if !cs.desc.ClientStreams || cs.desc.ServerStreams { - return - } - // Special handling for client streaming rpc. - // This recv expects EOF or errors, so we don't collect inPayload. - if cs.c.maxReceiveMessageSize == nil { - return Errorf(codes.Internal, "callInfo maxReceiveMessageSize field uninitialized(nil)") - } - err = recv(cs.p, cs.codec, cs.s, cs.dc, m, *cs.c.maxReceiveMessageSize, nil) - cs.closeTransportStream(err) - if err == nil { - return toRPCErr(errors.New("grpc: client streaming protocol violation: get , want ")) + } else { + // No compression is used; disable our decompressor. + a.dc = nil } + // Only initialize this state once per stream. + a.decompSet = true + } + err = recv(a.p, cs.codec, a.s, a.dc, m, *cs.c.maxReceiveMessageSize, inPayload, a.decomp) + if err != nil { if err == io.EOF { - if se := cs.s.Status().Err(); se != nil { - return se + if statusErr := a.s.Status().Err(); statusErr != nil { + return statusErr } - cs.finish(err) - return nil + return io.EOF // indicates successful end of stream. } return toRPCErr(err) } - if _, ok := err.(transport.ConnectionError); !ok { - cs.closeTransportStream(err) - } - if err == io.EOF { - if statusErr := cs.s.Status().Err(); statusErr != nil { - return statusErr + if EnableTracing { + a.mu.Lock() + if a.trInfo.tr != nil { + a.trInfo.tr.LazyLog(&payload{sent: false, msg: m}, true) } - // Returns io.EOF to indicate the end of the stream. - return + a.mu.Unlock() } - return toRPCErr(err) -} - -func (cs *clientStream) CloseSend() (err error) { - err = cs.t.Write(cs.s, nil, nil, &transport.Options{Last: true}) - defer func() { - if err != nil { - cs.finish(err) - } - }() - if err == nil || err == io.EOF { + if inPayload != nil { + a.statsHandler.HandleRPC(a.ctx, inPayload) + } + if channelz.IsOn() { + a.t.IncrMsgRecv() + } + if cs.desc.ServerStreams { + // Subsequent messages should be received by subsequent RecvMsg calls. return nil } - if _, ok := err.(transport.ConnectionError); !ok { - cs.closeTransportStream(err) + + // Special handling for non-server-stream rpcs. + // This recv expects EOF or errors, so we don't collect inPayload. + err = recv(a.p, cs.codec, a.s, a.dc, m, *cs.c.maxReceiveMessageSize, nil, a.decomp) + if err == nil { + return toRPCErr(errors.New("grpc: client streaming protocol violation: get , want ")) + } + if err == io.EOF { + return a.s.Status().Err() // non-server streaming Recv returns nil on success } - err = toRPCErr(err) - return + return toRPCErr(err) } -func (cs *clientStream) closeTransportStream(err error) { - cs.mu.Lock() - if cs.closed { - cs.mu.Unlock() +func (a *csAttempt) closeSend() { + cs := a.cs + if cs.sentLast { return } - cs.closed = true - cs.mu.Unlock() - cs.t.CloseStream(cs.s, err) + cs.sentLast = true + cs.attempt.t.Write(cs.attempt.s, nil, nil, &transport.Options{Last: true}) + // We ignore errors from Write. Any error it would return would also be + // returned by a subsequent RecvMsg call, and the user is supposed to always + // finish the stream by calling RecvMsg until it returns err != nil. } -func (cs *clientStream) finish(err error) { - cs.mu.Lock() - defer cs.mu.Unlock() - if cs.finished { - return - } - cs.finished = true - defer func() { - if cs.cancel != nil { - cs.cancel() - } - }() - for _, o := range cs.opts { - o.after(cs.c) - } - if cs.done != nil { - updateRPCInfoInContext(cs.s.Context(), rpcInfo{ - bytesSent: cs.s.BytesSent(), - bytesReceived: cs.s.BytesReceived(), +func (a *csAttempt) finish(err error) { + a.mu.Lock() + a.t.CloseStream(a.s, err) + + if a.done != nil { + a.done(balancer.DoneInfo{ + Err: err, + BytesSent: true, + BytesReceived: a.s.BytesReceived(), }) - cs.done(balancer.DoneInfo{Err: err}) - cs.done = nil } - if cs.statsHandler != nil { + if a.statsHandler != nil { end := &stats.End{ - Client: true, - EndTime: time.Now(), - } - if err != io.EOF { - // end.Error is nil if the RPC finished successfully. - end.Error = toRPCErr(err) + Client: true, + BeginTime: a.beginTime, + EndTime: time.Now(), + Error: err, } - cs.statsHandler.HandleRPC(cs.statsCtx, end) + a.statsHandler.HandleRPC(a.ctx, end) } - if !cs.tracing { - return - } - if cs.trInfo.tr != nil { - if err == nil || err == io.EOF { - cs.trInfo.tr.LazyPrintf("RPC: [OK]") + if a.trInfo.tr != nil { + if err == nil { + a.trInfo.tr.LazyPrintf("RPC: [OK]") } else { - cs.trInfo.tr.LazyPrintf("RPC: [%v]", err) - cs.trInfo.tr.SetError() + a.trInfo.tr.LazyPrintf("RPC: [%v]", err) + a.trInfo.tr.SetError() } - cs.trInfo.tr.Finish() - cs.trInfo.tr = nil + a.trInfo.tr.Finish() + a.trInfo.tr = nil } + a.mu.Unlock() } // ServerStream defines the interface a server stream has to satisfy. @@ -540,12 +632,17 @@ type ServerStream interface { // serverStream implements a server side Stream. type serverStream struct { - t transport.ServerTransport - s *transport.Stream - p *parser - codec Codec - cp Compressor - dc Decompressor + ctx context.Context + t transport.ServerTransport + s *transport.Stream + p *parser + codec baseCodec + + cp Compressor + dc Decompressor + comp encoding.Compressor + decomp encoding.Compressor + maxReceiveMessageSize int maxSendMessageSize int trInfo *traceInfo @@ -556,7 +653,7 @@ type serverStream struct { } func (ss *serverStream) Context() context.Context { - return ss.s.Context() + return ss.ctx } func (ss *serverStream) SetHeader(md metadata.MD) error { @@ -575,7 +672,6 @@ func (ss *serverStream) SetTrailer(md metadata.MD) { return } ss.s.SetTrailer(md) - return } func (ss *serverStream) SendMsg(m interface{}) (err error) { @@ -596,17 +692,20 @@ func (ss *serverStream) SendMsg(m interface{}) (err error) { st, _ := status.FromError(toRPCErr(err)) ss.t.WriteStatus(ss.s, st) } + if channelz.IsOn() && err == nil { + ss.t.IncrMsgSent() + } }() var outPayload *stats.OutPayload if ss.statsHandler != nil { outPayload = &stats.OutPayload{} } - hdr, data, err := encode(ss.codec, m, ss.cp, bytes.NewBuffer([]byte{}), outPayload) + hdr, data, err := encode(ss.codec, m, ss.cp, outPayload, ss.comp) if err != nil { return err } if len(data) > ss.maxSendMessageSize { - return Errorf(codes.ResourceExhausted, "trying to send message larger than max (%d vs. %d)", len(data), ss.maxSendMessageSize) + return status.Errorf(codes.ResourceExhausted, "trying to send message larger than max (%d vs. %d)", len(data), ss.maxSendMessageSize) } if err := ss.t.Write(ss.s, hdr, data, &transport.Options{Last: false}); err != nil { return toRPCErr(err) @@ -636,17 +735,20 @@ func (ss *serverStream) RecvMsg(m interface{}) (err error) { st, _ := status.FromError(toRPCErr(err)) ss.t.WriteStatus(ss.s, st) } + if channelz.IsOn() && err == nil { + ss.t.IncrMsgRecv() + } }() var inPayload *stats.InPayload if ss.statsHandler != nil { inPayload = &stats.InPayload{} } - if err := recv(ss.p, ss.codec, ss.s, ss.dc, m, ss.maxReceiveMessageSize, inPayload); err != nil { + if err := recv(ss.p, ss.codec, ss.s, ss.dc, m, ss.maxReceiveMessageSize, inPayload, ss.decomp); err != nil { if err == io.EOF { return err } if err == io.ErrUnexpectedEOF { - err = Errorf(codes.Internal, io.ErrUnexpectedEOF.Error()) + err = status.Errorf(codes.Internal, io.ErrUnexpectedEOF.Error()) } return toRPCErr(err) } @@ -655,3 +757,9 @@ func (ss *serverStream) RecvMsg(m interface{}) (err error) { } return nil } + +// MethodFromServerStream returns the method string for the input stream. +// The returned string is in the format of "/service/method". +func MethodFromServerStream(stream ServerStream) (string, bool) { + return Method(stream.Context()) +} diff --git a/vendor/google.golang.org/grpc/transport/bdp_estimator.go b/vendor/google.golang.org/grpc/transport/bdp_estimator.go index 8dd2ed42792..63cd2627c87 100644 --- a/vendor/google.golang.org/grpc/transport/bdp_estimator.go +++ b/vendor/google.golang.org/grpc/transport/bdp_estimator.go @@ -41,12 +41,9 @@ const ( gamma = 2 ) -var ( - // Adding arbitrary data to ping so that its ack can be - // identified. - // Easter-egg: what does the ping message say? - bdpPing = &ping{data: [8]byte{2, 4, 16, 16, 9, 14, 7, 7}} -) +// Adding arbitrary data to ping so that its ack can be identified. +// Easter-egg: what does the ping message say? +var bdpPing = &ping{data: [8]byte{2, 4, 16, 16, 9, 14, 7, 7}} type bdpEstimator struct { // sentAt is the time when the ping was sent. diff --git a/vendor/google.golang.org/grpc/transport/controlbuf.go b/vendor/google.golang.org/grpc/transport/controlbuf.go new file mode 100644 index 00000000000..e147cd51bf1 --- /dev/null +++ b/vendor/google.golang.org/grpc/transport/controlbuf.go @@ -0,0 +1,769 @@ +/* + * + * Copyright 2014 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 ( + "bytes" + "fmt" + "runtime" + "sync" + + "golang.org/x/net/http2" + "golang.org/x/net/http2/hpack" +) + +type itemNode struct { + it interface{} + next *itemNode +} + +type itemList struct { + head *itemNode + tail *itemNode +} + +func (il *itemList) enqueue(i interface{}) { + n := &itemNode{it: i} + if il.tail == nil { + il.head, il.tail = n, n + return + } + il.tail.next = n + il.tail = n +} + +// peek returns the first item in the list without removing it from the +// list. +func (il *itemList) peek() interface{} { + return il.head.it +} + +func (il *itemList) dequeue() interface{} { + if il.head == nil { + return nil + } + i := il.head.it + il.head = il.head.next + if il.head == nil { + il.tail = nil + } + return i +} + +func (il *itemList) dequeueAll() *itemNode { + h := il.head + il.head, il.tail = nil, nil + return h +} + +func (il *itemList) isEmpty() bool { + return il.head == nil +} + +// 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 // Valid on server side. + initStream func(uint32) (bool, error) // Used only on the client side. + onWrite func() + wq *writeQuota // write quota for the stream created. + cleanup *cleanupStream // Valid on the server side. + onOrphaned func(error) // Valid on client-side +} + +type cleanupStream struct { + streamID uint32 + idPtr *uint32 + rst bool + rstCode http2.ErrCode + onWrite func() +} + +type dataFrame struct { + streamID uint32 + endStream bool + h []byte + d []byte + // onEachWrite is called every time + // a part of d is written out. + onEachWrite func() +} + +type incomingWindowUpdate struct { + streamID uint32 + increment uint32 +} + +type outgoingWindowUpdate struct { + streamID uint32 + increment uint32 +} + +type incomingSettings struct { + ss []http2.Setting +} + +type outgoingSettings struct { + ss []http2.Setting +} + +type settingsAck struct { +} + +type incomingGoAway struct { +} + +type goAway struct { + code http2.ErrCode + debugData []byte + headsUp bool + closeConn bool +} + +type ping struct { + ack bool + data [8]byte +} + +type outFlowControlSizeRequest struct { + resp chan uint32 +} + +type outStreamState int + +const ( + active outStreamState = iota + empty + waitingOnStreamQuota +) + +type outStream struct { + id uint32 + state outStreamState + itl *itemList + bytesOutStanding int + wq *writeQuota + + next *outStream + prev *outStream +} + +func (s *outStream) deleteSelf() { + if s.prev != nil { + s.prev.next = s.next + } + if s.next != nil { + s.next.prev = s.prev + } + s.next, s.prev = nil, nil +} + +type outStreamList struct { + // Following are sentinel objects that mark the + // beginning and end of the list. They do not + // contain any item lists. All valid objects are + // inserted in between them. + // This is needed so that an outStream object can + // deleteSelf() in O(1) time without knowing which + // list it belongs to. + head *outStream + tail *outStream +} + +func newOutStreamList() *outStreamList { + head, tail := new(outStream), new(outStream) + head.next = tail + tail.prev = head + return &outStreamList{ + head: head, + tail: tail, + } +} + +func (l *outStreamList) enqueue(s *outStream) { + e := l.tail.prev + e.next = s + s.prev = e + s.next = l.tail + l.tail.prev = s +} + +// remove from the beginning of the list. +func (l *outStreamList) dequeue() *outStream { + b := l.head.next + if b == l.tail { + return nil + } + b.deleteSelf() + return b +} + +type controlBuffer struct { + ch chan struct{} + done <-chan struct{} + mu sync.Mutex + consumerWaiting bool + list *itemList + err error +} + +func newControlBuffer(done <-chan struct{}) *controlBuffer { + return &controlBuffer{ + ch: make(chan struct{}, 1), + list: &itemList{}, + done: done, + } +} + +func (c *controlBuffer) put(it interface{}) error { + _, err := c.executeAndPut(nil, it) + return err +} + +func (c *controlBuffer) executeAndPut(f func(it interface{}) bool, it interface{}) (bool, error) { + var wakeUp bool + c.mu.Lock() + if c.err != nil { + c.mu.Unlock() + return false, c.err + } + if f != nil { + if !f(it) { // f wasn't successful + c.mu.Unlock() + return false, nil + } + } + if c.consumerWaiting { + wakeUp = true + c.consumerWaiting = false + } + c.list.enqueue(it) + c.mu.Unlock() + if wakeUp { + select { + case c.ch <- struct{}{}: + default: + } + } + return true, nil +} + +func (c *controlBuffer) get(block bool) (interface{}, error) { + for { + c.mu.Lock() + if c.err != nil { + c.mu.Unlock() + return nil, c.err + } + if !c.list.isEmpty() { + h := c.list.dequeue() + c.mu.Unlock() + return h, nil + } + if !block { + c.mu.Unlock() + return nil, nil + } + c.consumerWaiting = true + c.mu.Unlock() + select { + case <-c.ch: + case <-c.done: + c.finish() + return nil, ErrConnClosing + } + } +} + +func (c *controlBuffer) finish() { + c.mu.Lock() + if c.err != nil { + c.mu.Unlock() + return + } + c.err = ErrConnClosing + // There may be headers for streams in the control buffer. + // These streams need to be cleaned out since the transport + // is still not aware of these yet. + for head := c.list.dequeueAll(); head != nil; head = head.next { + hdr, ok := head.it.(*headerFrame) + if !ok { + continue + } + if hdr.onOrphaned != nil { // It will be nil on the server-side. + hdr.onOrphaned(ErrConnClosing) + } + } + c.mu.Unlock() +} + +type side int + +const ( + clientSide side = iota + serverSide +) + +type loopyWriter struct { + side side + cbuf *controlBuffer + sendQuota uint32 + oiws uint32 // outbound initial window size. + estdStreams map[uint32]*outStream // Established streams. + activeStreams *outStreamList // Streams that are sending data. + framer *framer + hBuf *bytes.Buffer // The buffer for HPACK encoding. + hEnc *hpack.Encoder // HPACK encoder. + bdpEst *bdpEstimator + draining bool + + // Side-specific handlers + ssGoAwayHandler func(*goAway) (bool, error) +} + +func newLoopyWriter(s side, fr *framer, cbuf *controlBuffer, bdpEst *bdpEstimator) *loopyWriter { + var buf bytes.Buffer + l := &loopyWriter{ + side: s, + cbuf: cbuf, + sendQuota: defaultWindowSize, + oiws: defaultWindowSize, + estdStreams: make(map[uint32]*outStream), + activeStreams: newOutStreamList(), + framer: fr, + hBuf: &buf, + hEnc: hpack.NewEncoder(&buf), + bdpEst: bdpEst, + } + return l +} + +const minBatchSize = 1000 + +// run should be run in a separate goroutine. +func (l *loopyWriter) run() { + var ( + it interface{} + err error + isEmpty bool + ) + defer func() { + errorf("transport: loopyWriter.run returning. Err: %v", err) + }() + for { + it, err = l.cbuf.get(true) + if err != nil { + return + } + if err = l.handle(it); err != nil { + return + } + if _, err = l.processData(); err != nil { + return + } + gosched := true + hasdata: + for { + it, err = l.cbuf.get(false) + if err != nil { + return + } + if it != nil { + if err = l.handle(it); err != nil { + return + } + if _, err = l.processData(); err != nil { + return + } + continue hasdata + } + if isEmpty, err = l.processData(); err != nil { + return + } + if !isEmpty { + continue hasdata + } + if gosched { + gosched = false + if l.framer.writer.offset < minBatchSize { + runtime.Gosched() + continue hasdata + } + } + l.framer.writer.Flush() + break hasdata + + } + } +} + +func (l *loopyWriter) outgoingWindowUpdateHandler(w *outgoingWindowUpdate) error { + return l.framer.fr.WriteWindowUpdate(w.streamID, w.increment) +} + +func (l *loopyWriter) incomingWindowUpdateHandler(w *incomingWindowUpdate) error { + // Otherwise update the quota. + if w.streamID == 0 { + l.sendQuota += w.increment + return nil + } + // Find the stream and update it. + if str, ok := l.estdStreams[w.streamID]; ok { + str.bytesOutStanding -= int(w.increment) + if strQuota := int(l.oiws) - str.bytesOutStanding; strQuota > 0 && str.state == waitingOnStreamQuota { + str.state = active + l.activeStreams.enqueue(str) + return nil + } + } + return nil +} + +func (l *loopyWriter) outgoingSettingsHandler(s *outgoingSettings) error { + return l.framer.fr.WriteSettings(s.ss...) +} + +func (l *loopyWriter) incomingSettingsHandler(s *incomingSettings) error { + if err := l.applySettings(s.ss); err != nil { + return err + } + return l.framer.fr.WriteSettingsAck() +} + +func (l *loopyWriter) headerHandler(h *headerFrame) error { + if l.side == serverSide { + if h.endStream { // Case 1.A: Server wants to close stream. + // Make sure it's not a trailers only response. + if str, ok := l.estdStreams[h.streamID]; ok { + if str.state != empty { // either active or waiting on stream quota. + // add it str's list of items. + str.itl.enqueue(h) + return nil + } + } + if err := l.writeHeader(h.streamID, h.endStream, h.hf, h.onWrite); err != nil { + return err + } + return l.cleanupStreamHandler(h.cleanup) + } + // Case 1.B: Server is responding back with headers. + str := &outStream{ + state: empty, + itl: &itemList{}, + wq: h.wq, + } + l.estdStreams[h.streamID] = str + return l.writeHeader(h.streamID, h.endStream, h.hf, h.onWrite) + } + // Case 2: Client wants to originate stream. + str := &outStream{ + id: h.streamID, + state: empty, + itl: &itemList{}, + wq: h.wq, + } + str.itl.enqueue(h) + return l.originateStream(str) +} + +func (l *loopyWriter) originateStream(str *outStream) error { + hdr := str.itl.dequeue().(*headerFrame) + sendPing, err := hdr.initStream(str.id) + if err != nil { + if err == ErrConnClosing { + return err + } + // Other errors(errStreamDrain) need not close transport. + return nil + } + if err = l.writeHeader(str.id, hdr.endStream, hdr.hf, hdr.onWrite); err != nil { + return err + } + l.estdStreams[str.id] = str + if sendPing { + return l.pingHandler(&ping{data: [8]byte{}}) + } + return nil +} + +func (l *loopyWriter) writeHeader(streamID uint32, endStream bool, hf []hpack.HeaderField, onWrite func()) error { + if onWrite != nil { + onWrite() + } + l.hBuf.Reset() + for _, f := range hf { + if err := l.hEnc.WriteField(f); err != nil { + warningf("transport: loopyWriter.writeHeader encountered error while encoding headers:", err) + } + } + var ( + err error + endHeaders, first bool + ) + first = true + for !endHeaders { + size := l.hBuf.Len() + if size > http2MaxFrameLen { + size = http2MaxFrameLen + } else { + endHeaders = true + } + if first { + first = false + err = l.framer.fr.WriteHeaders(http2.HeadersFrameParam{ + StreamID: streamID, + BlockFragment: l.hBuf.Next(size), + EndStream: endStream, + EndHeaders: endHeaders, + }) + } else { + err = l.framer.fr.WriteContinuation( + streamID, + endHeaders, + l.hBuf.Next(size), + ) + } + if err != nil { + return err + } + } + return nil +} + +func (l *loopyWriter) preprocessData(df *dataFrame) error { + str, ok := l.estdStreams[df.streamID] + if !ok { + return nil + } + // If we got data for a stream it means that + // stream was originated and the headers were sent out. + str.itl.enqueue(df) + if str.state == empty { + str.state = active + l.activeStreams.enqueue(str) + } + return nil +} + +func (l *loopyWriter) pingHandler(p *ping) error { + if !p.ack { + l.bdpEst.timesnap(p.data) + } + return l.framer.fr.WritePing(p.ack, p.data) + +} + +func (l *loopyWriter) outFlowControlSizeRequestHandler(o *outFlowControlSizeRequest) error { + o.resp <- l.sendQuota + return nil +} + +func (l *loopyWriter) cleanupStreamHandler(c *cleanupStream) error { + c.onWrite() + if str, ok := l.estdStreams[c.streamID]; ok { + // On the server side it could be a trailers-only response or + // a RST_STREAM before stream initialization thus the stream might + // not be established yet. + delete(l.estdStreams, c.streamID) + str.deleteSelf() + } + if c.rst { // If RST_STREAM needs to be sent. + if err := l.framer.fr.WriteRSTStream(c.streamID, c.rstCode); err != nil { + return err + } + } + if l.side == clientSide && l.draining && len(l.estdStreams) == 0 { + return ErrConnClosing + } + return nil +} + +func (l *loopyWriter) incomingGoAwayHandler(*incomingGoAway) error { + if l.side == clientSide { + l.draining = true + if len(l.estdStreams) == 0 { + return ErrConnClosing + } + } + return nil +} + +func (l *loopyWriter) goAwayHandler(g *goAway) error { + // Handling of outgoing GoAway is very specific to side. + if l.ssGoAwayHandler != nil { + draining, err := l.ssGoAwayHandler(g) + if err != nil { + return err + } + l.draining = draining + } + return nil +} + +func (l *loopyWriter) handle(i interface{}) error { + switch i := i.(type) { + case *incomingWindowUpdate: + return l.incomingWindowUpdateHandler(i) + case *outgoingWindowUpdate: + return l.outgoingWindowUpdateHandler(i) + case *incomingSettings: + return l.incomingSettingsHandler(i) + case *outgoingSettings: + return l.outgoingSettingsHandler(i) + case *headerFrame: + return l.headerHandler(i) + case *cleanupStream: + return l.cleanupStreamHandler(i) + case *incomingGoAway: + return l.incomingGoAwayHandler(i) + case *dataFrame: + return l.preprocessData(i) + case *ping: + return l.pingHandler(i) + case *goAway: + return l.goAwayHandler(i) + case *outFlowControlSizeRequest: + return l.outFlowControlSizeRequestHandler(i) + default: + return fmt.Errorf("transport: unknown control message type %T", i) + } +} + +func (l *loopyWriter) applySettings(ss []http2.Setting) error { + for _, s := range ss { + switch s.ID { + case http2.SettingInitialWindowSize: + o := l.oiws + l.oiws = s.Val + if o < l.oiws { + // If the new limit is greater make all depleted streams active. + for _, stream := range l.estdStreams { + if stream.state == waitingOnStreamQuota { + stream.state = active + l.activeStreams.enqueue(stream) + } + } + } + } + } + return nil +} + +func (l *loopyWriter) processData() (bool, error) { + if l.sendQuota == 0 { + return true, nil + } + str := l.activeStreams.dequeue() + if str == nil { + return true, nil + } + dataItem := str.itl.peek().(*dataFrame) + if len(dataItem.h) == 0 && len(dataItem.d) == 0 { + // Client sends out empty data frame with endStream = true + if err := l.framer.fr.WriteData(dataItem.streamID, dataItem.endStream, nil); err != nil { + return false, err + } + str.itl.dequeue() + if str.itl.isEmpty() { + str.state = empty + } else if trailer, ok := str.itl.peek().(*headerFrame); ok { // the next item is trailers. + if err := l.writeHeader(trailer.streamID, trailer.endStream, trailer.hf, trailer.onWrite); err != nil { + return false, err + } + if err := l.cleanupStreamHandler(trailer.cleanup); err != nil { + return false, nil + } + } else { + l.activeStreams.enqueue(str) + } + return false, nil + } + var ( + idx int + buf []byte + ) + if len(dataItem.h) != 0 { // data header has not been written out yet. + buf = dataItem.h + } else { + idx = 1 + buf = dataItem.d + } + size := http2MaxFrameLen + if len(buf) < size { + size = len(buf) + } + if strQuota := int(l.oiws) - str.bytesOutStanding; strQuota <= 0 { + str.state = waitingOnStreamQuota + return false, nil + } else if strQuota < size { + size = strQuota + } + + if l.sendQuota < uint32(size) { + size = int(l.sendQuota) + } + // Now that outgoing flow controls are checked we can replenish str's write quota + str.wq.replenish(size) + var endStream bool + // This last data message on this stream and all + // of it can be written in this go. + if dataItem.endStream && size == len(buf) { + // buf contains either data or it contains header but data is empty. + if idx == 1 || len(dataItem.d) == 0 { + endStream = true + } + } + if dataItem.onEachWrite != nil { + dataItem.onEachWrite() + } + if err := l.framer.fr.WriteData(dataItem.streamID, endStream, buf[:size]); err != nil { + return false, err + } + buf = buf[size:] + str.bytesOutStanding += size + l.sendQuota -= uint32(size) + if idx == 0 { + dataItem.h = buf + } else { + dataItem.d = buf + } + + if len(dataItem.h) == 0 && len(dataItem.d) == 0 { // All the data from that message was written out. + str.itl.dequeue() + } + if str.itl.isEmpty() { + str.state = empty + } else if trailer, ok := str.itl.peek().(*headerFrame); ok { // The next item is trailers. + if err := l.writeHeader(trailer.streamID, trailer.endStream, trailer.hf, trailer.onWrite); err != nil { + return false, err + } + if err := l.cleanupStreamHandler(trailer.cleanup); err != nil { + return false, err + } + } else if int(l.oiws)-str.bytesOutStanding <= 0 { // Ran out of stream quota. + str.state = waitingOnStreamQuota + } else { // Otherwise add it back to the list of active streams. + l.activeStreams.enqueue(str) + } + return false, nil +} diff --git a/vendor/google.golang.org/grpc/transport/control.go b/vendor/google.golang.org/grpc/transport/flowcontrol.go similarity index 51% rename from vendor/google.golang.org/grpc/transport/control.go rename to vendor/google.golang.org/grpc/transport/flowcontrol.go index dd1a8d42e7e..378f5c4502c 100644 --- a/vendor/google.golang.org/grpc/transport/control.go +++ b/vendor/google.golang.org/grpc/transport/flowcontrol.go @@ -24,9 +24,6 @@ import ( "sync" "sync/atomic" "time" - - "golang.org/x/net/http2" - "golang.org/x/net/http2/hpack" ) const ( @@ -36,179 +33,109 @@ const ( initialWindowSize = defaultWindowSize // for an RPC infinity = time.Duration(math.MaxInt64) defaultClientKeepaliveTime = infinity - defaultClientKeepaliveTimeout = time.Duration(20 * time.Second) + defaultClientKeepaliveTimeout = 20 * time.Second defaultMaxStreamsClient = 100 defaultMaxConnectionIdle = infinity defaultMaxConnectionAge = infinity defaultMaxConnectionAgeGrace = infinity - defaultServerKeepaliveTime = time.Duration(2 * time.Hour) - defaultServerKeepaliveTimeout = time.Duration(20 * time.Second) - defaultKeepalivePolicyMinTime = time.Duration(5 * time.Minute) + defaultServerKeepaliveTime = 2 * time.Hour + defaultServerKeepaliveTimeout = 20 * time.Second + defaultKeepalivePolicyMinTime = 5 * time.Minute // max window limit set by HTTP2 Specs. maxWindowSize = math.MaxInt32 - // defaultLocalSendQuota sets is default value for number of data + // defaultWriteQuota is the default value for number of data // bytes that each stream can schedule before some of it being // flushed out. - defaultLocalSendQuota = 64 * 1024 + defaultWriteQuota = 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 -} - -func (*windowUpdate) item() {} - -type settings struct { - ack bool - ss []http2.Setting -} - -func (*settings) item() {} - -type resetStream struct { - streamID uint32 - code http2.ErrCode -} - -func (*resetStream) item() {} - -type goAway struct { - code http2.ErrCode - debugData []byte - headsUp bool - closeConn bool -} - -func (*goAway) item() {} - -type flushIO struct { -} - -func (*flushIO) item() {} - -type ping struct { - ack bool - data [8]byte +// writeQuota is a soft limit on the amount of data a stream can +// schedule before some of it is written out. +type writeQuota struct { + quota int32 + // get waits on read from when quota goes less than or equal to zero. + // replenish writes on it when quota goes positive again. + ch chan struct{} + // done is triggered in error case. + done <-chan struct{} +} + +func newWriteQuota(sz int32, done <-chan struct{}) *writeQuota { + return &writeQuota{ + quota: sz, + ch: make(chan struct{}, 1), + done: done, + } } -func (*ping) item() {} - -// quotaPool is a pool which accumulates the quota and sends it to acquire() -// when it is available. -type quotaPool struct { - c chan int - - mu sync.Mutex - version uint32 - quota int +func (w *writeQuota) get(sz int32) error { + for { + if atomic.LoadInt32(&w.quota) > 0 { + atomic.AddInt32(&w.quota, -sz) + return nil + } + select { + case <-w.ch: + continue + case <-w.done: + return errStreamDone + } + } } -// newQuotaPool creates a quotaPool which has quota q available to consume. -func newQuotaPool(q int) *quotaPool { - qb := "aPool{ - c: make(chan int, 1), - } - if q > 0 { - qb.c <- q - } else { - qb.quota = q +func (w *writeQuota) replenish(n int) { + sz := int32(n) + a := atomic.AddInt32(&w.quota, sz) + b := a - sz + if b <= 0 && a > 0 { + select { + case w.ch <- struct{}{}: + default: + } } - return qb } -// add cancels the pending quota sent on acquired, incremented by v and sends -// it back on acquire. -func (qb *quotaPool) add(v int) { - qb.mu.Lock() - defer qb.mu.Unlock() - qb.lockedAdd(v) +type trInFlow struct { + limit uint32 + unacked uint32 + effectiveWindowSize uint32 } -func (qb *quotaPool) lockedAdd(v int) { - select { - case n := <-qb.c: - qb.quota += n - default: - } - qb.quota += v - if qb.quota <= 0 { - return - } - // After the pool has been created, this is the only place that sends on - // the channel. Since mu is held at this point and any quota that was sent - // on the channel has been retrieved, we know that this code will always - // place any positive quota value on the channel. - select { - case qb.c <- qb.quota: - qb.quota = 0 - default: - } +func (f *trInFlow) newLimit(n uint32) uint32 { + d := n - f.limit + f.limit = n + f.updateEffectiveWindowSize() + return d } -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 (f *trInFlow) onData(n uint32) uint32 { + f.unacked += n + if f.unacked >= f.limit/4 { + w := f.unacked + f.unacked = 0 + f.updateEffectiveWindowSize() + return w + } + f.updateEffectiveWindowSize() + return 0 } -func (qb *quotaPool) acquireWithVersion() (<-chan int, uint32) { - return qb.c, atomic.LoadUint32(&(qb.version)) +func (f *trInFlow) reset() uint32 { + w := f.unacked + f.unacked = 0 + f.updateEffectiveWindowSize() + return w } -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 +func (f *trInFlow) updateEffectiveWindowSize() { + atomic.StoreUint32(&f.effectiveWindowSize, f.limit-f.unacked) } -// acquire returns the channel on which available quota amounts are sent. -func (qb *quotaPool) acquire() <-chan int { - return qb.c +func (f *trInFlow) getSize() uint32 { + return atomic.LoadUint32(&f.effectiveWindowSize) } +// TODO(mmukhi): Simplify this code. // inFlow deals with inbound flow control type inFlow struct { mu sync.Mutex @@ -229,9 +156,9 @@ type inFlow struct { // It assumes that n is always greater than the old limit. func (f *inFlow) newLimit(n uint32) uint32 { f.mu.Lock() - defer f.mu.Unlock() d := n - f.limit f.limit = n + f.mu.Unlock() return d } @@ -240,7 +167,6 @@ func (f *inFlow) maybeAdjust(n uint32) uint32 { n = uint32(math.MaxInt32) } f.mu.Lock() - defer f.mu.Unlock() // estSenderQuota is the receiver's view of the maximum number of bytes the sender // can send without a window update. estSenderQuota := int32(f.limit - (f.pendingData + f.pendingUpdate)) @@ -252,7 +178,7 @@ func (f *inFlow) maybeAdjust(n uint32) uint32 { // for this message. Therefore we must send an update over the limit since there's an active read // request from the application. if estUntransmittedData > estSenderQuota { - // Sender's window shouldn't go more than 2^31 - 1 as speecified in the HTTP spec. + // Sender's window shouldn't go more than 2^31 - 1 as specified in the HTTP spec. if f.limit+n > maxWindowSize { f.delta = maxWindowSize - f.limit } else { @@ -261,19 +187,24 @@ func (f *inFlow) maybeAdjust(n uint32) uint32 { // is padded; We will fallback on the current available window(at least a 1/4th of the limit). f.delta = n } + f.mu.Unlock() return f.delta } + f.mu.Unlock() return 0 } // onData is invoked when some data frame is received. It updates pendingData. func (f *inFlow) onData(n uint32) error { f.mu.Lock() - defer f.mu.Unlock() f.pendingData += n if f.pendingData+f.pendingUpdate > f.limit+f.delta { - return fmt.Errorf("received %d-bytes data exceeding the limit %d bytes", f.pendingData+f.pendingUpdate, f.limit) + limit := f.limit + rcvd := f.pendingData + f.pendingUpdate + f.mu.Unlock() + return fmt.Errorf("received %d-bytes data exceeding the limit %d bytes", rcvd, limit) } + f.mu.Unlock() return nil } @@ -281,8 +212,8 @@ func (f *inFlow) onData(n uint32) error { // to be sent to the peer. func (f *inFlow) onRead(n uint32) uint32 { f.mu.Lock() - defer f.mu.Unlock() if f.pendingData == 0 { + f.mu.Unlock() return 0 } f.pendingData -= n @@ -297,15 +228,9 @@ func (f *inFlow) onRead(n uint32) uint32 { if f.pendingUpdate >= f.limit/4 { wu := f.pendingUpdate f.pendingUpdate = 0 + f.mu.Unlock() return wu } + f.mu.Unlock() return 0 } - -func (f *inFlow) resetPendingUpdate() uint32 { - f.mu.Lock() - defer f.mu.Unlock() - n := f.pendingUpdate - f.pendingUpdate = 0 - return n -} diff --git a/vendor/google.golang.org/grpc/transport/go16.go b/vendor/google.golang.org/grpc/transport/go16.go new file mode 100644 index 00000000000..5babcf9b877 --- /dev/null +++ b/vendor/google.golang.org/grpc/transport/go16.go @@ -0,0 +1,51 @@ +// +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" + "net/http" + + "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) +} + +// contextFromRequest returns a background context. +func contextFromRequest(r *http.Request) context.Context { + return context.Background() +} diff --git a/vendor/google.golang.org/grpc/transport/go17.go b/vendor/google.golang.org/grpc/transport/go17.go new file mode 100644 index 00000000000..b7fa6bdb9ca --- /dev/null +++ b/vendor/google.golang.org/grpc/transport/go17.go @@ -0,0 +1,52 @@ +// +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" + "net/http" + + "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) +} + +// contextFromRequest returns a context from the HTTP Request. +func contextFromRequest(r *http.Request) context.Context { + return r.Context() +} diff --git a/vendor/google.golang.org/grpc/transport/handler_server.go b/vendor/google.golang.org/grpc/transport/handler_server.go index 7e0fdb35938..f71b7482174 100644 --- a/vendor/google.golang.org/grpc/transport/handler_server.go +++ b/vendor/google.golang.org/grpc/transport/handler_server.go @@ -40,20 +40,24 @@ import ( "google.golang.org/grpc/credentials" "google.golang.org/grpc/metadata" "google.golang.org/grpc/peer" + "google.golang.org/grpc/stats" "google.golang.org/grpc/status" ) // NewServerHandlerTransport returns a ServerTransport handling gRPC // from inside an http.Handler. It requires that the http Server // supports HTTP/2. -func NewServerHandlerTransport(w http.ResponseWriter, r *http.Request) (ServerTransport, error) { +func NewServerHandlerTransport(w http.ResponseWriter, r *http.Request, stats stats.Handler) (ServerTransport, error) { if r.ProtoMajor != 2 { return nil, errors.New("gRPC requires HTTP/2") } if r.Method != "POST" { return nil, errors.New("invalid gRPC request method") } - if !validContentType(r.Header.Get("Content-Type")) { + contentType := r.Header.Get("Content-Type") + // TODO: do we assume contentType is lowercase? we did before + contentSubtype, validContentType := contentSubtype(contentType) + if !validContentType { return nil, errors.New("invalid gRPC request content-type") } if _, ok := w.(http.Flusher); !ok { @@ -64,10 +68,13 @@ func NewServerHandlerTransport(w http.ResponseWriter, r *http.Request) (ServerTr } st := &serverHandlerTransport{ - rw: w, - req: r, - closedCh: make(chan struct{}), - writes: make(chan func()), + rw: w, + req: r, + closedCh: make(chan struct{}), + writes: make(chan func()), + contentType: contentType, + contentSubtype: contentSubtype, + stats: stats, } if v := r.Header.Get("grpc-timeout"); v != "" { @@ -79,19 +86,19 @@ func NewServerHandlerTransport(w http.ResponseWriter, r *http.Request) (ServerTr st.timeout = to } - var metakv []string + metakv := []string{"content-type", contentType} if r.Host != "" { metakv = append(metakv, ":authority", r.Host) } for k, vv := range r.Header { k = strings.ToLower(k) - if isReservedHeader(k) && !isWhitelistedPseudoHeader(k) { + if isReservedHeader(k) && !isWhitelistedHeader(k) { continue } for _, v := range vv { v, err := decodeMetadataHeader(k, v) if err != nil { - return nil, streamErrorf(codes.InvalidArgument, "malformed binary metadata: %v", err) + return nil, streamErrorf(codes.Internal, "malformed binary metadata: %v", err) } metakv = append(metakv, k, v) } @@ -126,6 +133,14 @@ type serverHandlerTransport struct { // block concurrent WriteStatus calls // e.g. grpc/(*serverStream).SendMsg/RecvMsg writeStatusMu sync.Mutex + + // we just mirror the request content-type + contentType string + // we store both contentType and contentSubtype so we don't keep recreating them + // TODO make sure this is consistent across handler_server and http2_server + contentSubtype string + + stats stats.Handler } func (ht *serverHandlerTransport) Close() error { @@ -219,6 +234,9 @@ func (ht *serverHandlerTransport) WriteStatus(s *Stream, st *status.Status) erro }) if err == nil { // transport has not been closed + if ht.stats != nil { + ht.stats.HandleRPC(s.Context(), &stats.OutTrailer{}) + } ht.Close() close(ht.writes) } @@ -235,7 +253,7 @@ func (ht *serverHandlerTransport) writeCommonHeaders(s *Stream) { h := ht.rw.Header() h["Date"] = nil // suppress Date to make tests happy; TODO: restore - h.Set("Content-Type", "application/grpc") + h.Set("Content-Type", ht.contentType) // Predeclare trailers we'll set later in WriteStatus (after the body). // This is a SHOULD in the HTTP RFC, and the way you add (known) @@ -263,7 +281,7 @@ func (ht *serverHandlerTransport) Write(s *Stream, hdr []byte, data []byte, opts } func (ht *serverHandlerTransport) WriteHeader(s *Stream, md metadata.MD) error { - return ht.do(func() { + err := ht.do(func() { ht.writeCommonHeaders(s) h := ht.rw.Header() for k, vv := range md { @@ -279,17 +297,24 @@ func (ht *serverHandlerTransport) WriteHeader(s *Stream, md metadata.MD) error { ht.rw.WriteHeader(200) ht.rw.(http.Flusher).Flush() }) + + if err == nil { + if ht.stats != nil { + ht.stats.HandleRPC(s.Context(), &stats.OutHeader{}) + } + } + return err } func (ht *serverHandlerTransport) HandleStreams(startStream func(*Stream), traceCtx func(context.Context, string) context.Context) { // With this transport type there will be exactly 1 stream: this HTTP request. - var ctx context.Context + ctx := contextFromRequest(ht.req) var cancel context.CancelFunc if ht.timeoutSet { - ctx, cancel = context.WithTimeout(context.Background(), ht.timeout) + ctx, cancel = context.WithTimeout(ctx, ht.timeout) } else { - ctx, cancel = context.WithCancel(context.Background()) + ctx, cancel = context.WithCancel(ctx) } // requestOver is closed when either the request's context is done @@ -313,13 +338,14 @@ func (ht *serverHandlerTransport) HandleStreams(startStream func(*Stream), trace req := ht.req s := &Stream{ - id: 0, // irrelevant - requestRead: func(int) {}, - cancel: cancel, - buf: newRecvBuffer(), - st: ht, - method: req.URL.Path, - recvCompress: req.Header.Get("grpc-encoding"), + id: 0, // irrelevant + requestRead: func(int) {}, + cancel: cancel, + buf: newRecvBuffer(), + st: ht, + method: req.URL.Path, + recvCompress: req.Header.Get("grpc-encoding"), + contentSubtype: ht.contentSubtype, } pr := &peer.Peer{ Addr: ht.RemoteAddr(), @@ -328,10 +354,18 @@ func (ht *serverHandlerTransport) HandleStreams(startStream func(*Stream), trace pr.AuthInfo = credentials.TLSInfo{State: *req.TLS} } ctx = metadata.NewIncomingContext(ctx, ht.headerMD) - ctx = peer.NewContext(ctx, pr) - s.ctx = newContextWithStream(ctx, s) + s.ctx = peer.NewContext(ctx, pr) + if ht.stats != nil { + s.ctx = ht.stats.TagRPC(s.ctx, &stats.RPCTagInfo{FullMethodName: s.method}) + inHeader := &stats.InHeader{ + FullMethod: s.method, + RemoteAddr: ht.RemoteAddr(), + Compression: s.recvCompress, + } + ht.stats.HandleRPC(s.ctx, inHeader) + } s.trReader = &transportReader{ - reader: &recvBufferReader{ctx: s.ctx, recv: s.buf}, + reader: &recvBufferReader{ctx: s.ctx, ctxDone: s.ctx.Done(), recv: s.buf}, windowHandler: func(int) {}, } @@ -386,6 +420,10 @@ func (ht *serverHandlerTransport) runStream() { } } +func (ht *serverHandlerTransport) IncrMsgSent() {} + +func (ht *serverHandlerTransport) IncrMsgRecv() {} + func (ht *serverHandlerTransport) Drain() { panic("Drain() is not implemented") } diff --git a/vendor/google.golang.org/grpc/transport/http2_client.go b/vendor/google.golang.org/grpc/transport/http2_client.go index 1abb62e6df4..1fdabd954ef 100644 --- a/vendor/google.golang.org/grpc/transport/http2_client.go +++ b/vendor/google.golang.org/grpc/transport/http2_client.go @@ -19,7 +19,6 @@ package transport import ( - "bytes" "io" "math" "net" @@ -31,6 +30,8 @@ import ( "golang.org/x/net/context" "golang.org/x/net/http2" "golang.org/x/net/http2/hpack" + + "google.golang.org/grpc/channelz" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" "google.golang.org/grpc/keepalive" @@ -44,15 +45,17 @@ import ( type http2Client struct { ctx context.Context cancel context.CancelFunc - target string // server name/addr + ctxDone <-chan struct{} // Cache the ctx.Done() chan. userAgent string md interface{} conn net.Conn // underlying communication channel + loopy *loopyWriter remoteAddr net.Addr localAddr net.Addr authInfo credentials.AuthInfo // auth info about the connection - nextID uint32 // the next stream ID to be used + readerDone chan struct{} // sync point to enable testing. + writerDone chan struct{} // sync point to enable testing. // goAway is closed to notify the upper layer (i.e., addrConn.transportMonitor) // that the server sent GoAway on this transport. goAway chan struct{} @@ -60,18 +63,10 @@ type http2Client struct { awakenKeepalive chan struct{} framer *framer - hBuf *bytes.Buffer // the buffer for HPACK encoding - hEnc *hpack.Encoder // HPACK encoder - // controlBuf delivers all the control related tasks (e.g., window // updates, reset streams, and various settings) to the controller. controlBuf *controlBuffer - fc *inFlow - // sendQuotaPool provides flow control to outbound message. - sendQuotaPool *quotaPool - // streamsQuota limits the max number of concurrent streams. - streamsQuota *quotaPool - + fc *trInFlow // The scheme used: https if TLS is on, http otherwise. scheme string @@ -88,43 +83,52 @@ type http2Client struct { initialWindowSize int32 - bdpEst *bdpEstimator - outQuotaVersion uint32 + bdpEst *bdpEstimator + // onSuccess is a callback that client transport calls upon + // receiving server preface to signal that a succefull HTTP2 + // connection was established. + onSuccess func() - mu sync.Mutex // guard the following variables - state transportState // the state of underlying connection + maxConcurrentStreams uint32 + streamQuota int64 + streamsQuotaAvailable chan struct{} + waitingStreams uint32 + nextID uint32 + + mu sync.Mutex // guard the following variables + state transportState activeStreams map[uint32]*Stream - // The max number of concurrent streams - maxStreams int - // the per-stream outbound flow control window size set by the peer. - streamSendQuota uint32 // prevGoAway ID records the Last-Stream-ID in the previous GOAway frame. prevGoAwayID uint32 // goAwayReason records the http2.ErrCode and debug data received with the // GoAway frame. goAwayReason GoAwayReason + + // Fields below are for channelz metric collection. + channelzID int64 // channelz unique identification number + czmu sync.RWMutex + kpCount int64 + // The number of streams that have started, including already finished ones. + streamsStarted int64 + // The number of streams that have ended successfully by receiving EoS bit set + // frame from server. + streamsSucceeded int64 + streamsFailed int64 + lastStreamCreated time.Time + msgSent int64 + msgRecv int64 + lastMsgSent time.Time + lastMsgRecv time.Time } func dial(ctx context.Context, fn func(context.Context, string) (net.Conn, error), addr string) (net.Conn, error) { if fn != nil { return fn(ctx, addr) } - return (&net.Dialer{}).DialContext(ctx, "tcp", addr) + return dialContext(ctx, "tcp", addr) } func isTemporary(err error) bool { - switch err { - case io.EOF: - // Connection closures may be resolved upon retry, and are thus - // treated as temporary. - return true - case context.DeadlineExceeded: - // In Go 1.7, context.DeadlineExceeded implements Timeout(), and this - // special case is not needed. Until then, we need to keep this - // clause. - return true - } - switch err := err.(type) { case interface { Temporary() bool @@ -137,18 +141,16 @@ func isTemporary(err error) bool { // temporary. return err.Timeout() } - return false + return true } // 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, timeout time.Duration) (_ ClientTransport, err error) { +func newHTTP2Client(connectCtx, ctx context.Context, addr TargetInfo, opts ConnectOptions, onSuccess func()) (_ ClientTransport, err error) { scheme := "http" ctx, cancel := context.WithCancel(ctx) - connectCtx, connectCancel := context.WithTimeout(ctx, timeout) defer func() { - connectCancel() if err != nil { cancel() } @@ -173,12 +175,9 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions, t ) if creds := opts.TransportCredentials; creds != nil { scheme = "https" - conn, authInfo, err = creds.ClientHandshake(connectCtx, addr.Addr, conn) + conn, authInfo, err = creds.ClientHandshake(connectCtx, addr.Authority, conn) if err != nil { - // Credentials handshake errors are typically considered permanent - // to avoid retrying on e.g. bad certificates. - temp := isTemporary(err) - return nil, connectionErrorf(temp, err, "transport: authentication handshake failed: %v", err) + return nil, connectionErrorf(isTemporary(err), err, "transport: authentication handshake failed: %v", err) } isSecure = true } @@ -196,7 +195,6 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions, t icwz = opts.InitialConnWindowSize dynamicWindow = false } - var buf bytes.Buffer writeBufSize := defaultWriteBufSize if opts.WriteBufferSize > 0 { writeBufSize = opts.WriteBufferSize @@ -206,37 +204,35 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions, t readBufSize = opts.ReadBufferSize } t := &http2Client{ - ctx: ctx, - cancel: cancel, - target: addr.Addr, - userAgent: opts.UserAgent, - md: addr.Metadata, - conn: conn, - remoteAddr: conn.RemoteAddr(), - localAddr: conn.LocalAddr(), - authInfo: authInfo, - // The client initiated stream id is odd starting from 1. - nextID: 1, - goAway: make(chan struct{}), - awakenKeepalive: make(chan struct{}, 1), - hBuf: &buf, - hEnc: hpack.NewEncoder(&buf), - framer: newFramer(conn, writeBufSize, readBufSize), - controlBuf: newControlBuffer(), - fc: &inFlow{limit: uint32(icwz)}, - sendQuotaPool: newQuotaPool(defaultWindowSize), - scheme: scheme, - state: reachable, - activeStreams: make(map[uint32]*Stream), - isSecure: isSecure, - creds: opts.PerRPCCredentials, - maxStreams: defaultMaxStreamsClient, - streamsQuota: newQuotaPool(defaultMaxStreamsClient), - streamSendQuota: defaultWindowSize, - kp: kp, - statsHandler: opts.StatsHandler, - initialWindowSize: initialWindowSize, - } + ctx: ctx, + ctxDone: ctx.Done(), // Cache Done chan. + cancel: cancel, + userAgent: opts.UserAgent, + md: addr.Metadata, + conn: conn, + remoteAddr: conn.RemoteAddr(), + localAddr: conn.LocalAddr(), + authInfo: authInfo, + readerDone: make(chan struct{}), + writerDone: make(chan struct{}), + goAway: make(chan struct{}), + awakenKeepalive: make(chan struct{}, 1), + framer: newFramer(conn, writeBufSize, readBufSize), + fc: &trInFlow{limit: uint32(icwz)}, + scheme: scheme, + activeStreams: make(map[uint32]*Stream), + isSecure: isSecure, + creds: opts.PerRPCCredentials, + kp: kp, + statsHandler: opts.StatsHandler, + initialWindowSize: initialWindowSize, + onSuccess: onSuccess, + nextID: 1, + maxConcurrentStreams: defaultMaxStreamsClient, + streamQuota: defaultMaxStreamsClient, + streamsQuotaAvailable: make(chan struct{}, 1), + } + t.controlBuf = newControlBuffer(t.ctxDone) if opts.InitialWindowSize >= defaultWindowSize { t.initialWindowSize = opts.InitialWindowSize dynamicWindow = false @@ -260,6 +256,9 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions, t } t.statsHandler.HandleConn(t.ctx, connBegin) } + if channelz.IsOn() { + t.channelzID = channelz.RegisterNormalSocket(t, opts.ChannelzParentID, "") + } // Start the reader goroutine for incoming message. Each transport has // a dedicated goroutine which reads HTTP2 frame from network. Then it // dispatches the frame to the corresponding stream entity. @@ -295,8 +294,10 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions, t } t.framer.writer.Flush() go func() { - loopyWriter(t.ctx, t.controlBuf, t.itemHandler) - t.Close() + t.loopy = newLoopyWriter(clientSide, t.framer, t.controlBuf, t.bdpEst) + t.loopy.run() + t.conn.Close() + close(t.writerDone) }() if t.kp.Time != infinity { go t.keepalive() @@ -307,18 +308,14 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions, t 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)), - localSendQuota: newQuotaPool(defaultLocalSendQuota), headerChan: make(chan struct{}), + contentSubtype: callHdr.ContentSubtype, } - t.nextID += 2 + s.wq = newWriteQuota(defaultWriteQuota, s.done) s.requestRead = func(n int) { t.adjustWindow(s, uint32(n)) } @@ -328,21 +325,18 @@ func (t *http2Client) newStream(ctx context.Context, callHdr *CallHdr) *Stream { s.ctx = ctx s.trReader = &transportReader{ reader: &recvBufferReader{ - ctx: s.ctx, - goAway: s.goAway, - recv: s.buf, + ctx: s.ctx, + ctxDone: s.ctx.Done(), + recv: s.buf, }, windowHandler: func(n int) { t.updateWindow(s, uint32(n)) }, } - return s } -// NewStream creates a stream and registers it into the transport as "active" -// streams. -func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Stream, err error) { +func (t *http2Client) getPeer() *peer.Peer { pr := &peer.Peer{ Addr: t.remoteAddr, } @@ -350,74 +344,20 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea if t.authInfo != nil { pr.AuthInfo = t.authInfo } - ctx = peer.NewContext(ctx, pr) - var ( - authData = make(map[string]string) - audience string - ) - // Create an audience string only if needed. - if len(t.creds) > 0 || callHdr.Creds != nil { - // Construct URI required to get auth request metadata. - // Omit port if it is the default one. - host := strings.TrimSuffix(callHdr.Host, ":443") - pos := strings.LastIndex(callHdr.Method, "/") - if pos == -1 { - pos = len(callHdr.Method) - } - audience = "https://" + host + callHdr.Method[:pos] - } - for _, c := range t.creds { - data, err := c.GetRequestMetadata(ctx, audience) - if err != nil { - return nil, streamErrorf(codes.Internal, "transport: %v", err) - } - for k, v := range data { - // Capital header names are illegal in HTTP/2. - k = strings.ToLower(k) - authData[k] = v - } - } - 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 connection") - } - data, err := callCreds.GetRequestMetadata(ctx, audience) - if err != nil { - return nil, streamErrorf(codes.Internal, "transport: %v", err) - } - for k, v := range data { - // Capital header names are illegal in HTTP/2 - k = strings.ToLower(k) - callAuthData[k] = v - } - } - t.mu.Lock() - if t.activeStreams == nil { - t.mu.Unlock() - return nil, ErrConnClosing - } - if t.state == draining { - t.mu.Unlock() - return nil, ErrStreamDrain - } - if t.state != reachable { - t.mu.Unlock() - return nil, ErrConnClosing - } - t.mu.Unlock() - sq, err := wait(ctx, t.ctx, nil, nil, t.streamsQuota.acquire()) + return pr +} + +func (t *http2Client) createHeaderFields(ctx context.Context, callHdr *CallHdr) ([]hpack.HeaderField, error) { + aud := t.createAudience(callHdr) + authData, err := t.getTrAuthData(ctx, aud) if err != nil { return nil, err } - // Returns the quota balance back. - if sq > 1 { - t.streamsQuota.add(sq - 1) + callAuthData, err := t.getCallAuthData(ctx, aud, callHdr) + if err != nil { + return nil, err } - // TODO(mmukhi): Benchmark if the perfomance gets better if count the metadata and other header fields + // TODO(mmukhi): Benchmark if the performance 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 @@ -427,7 +367,7 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea 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: "content-type", Value: contentType(callHdr.ContentSubtype)}) headerFields = append(headerFields, hpack.HeaderField{Name: "user-agent", Value: t.userAgent}) headerFields = append(headerFields, hpack.HeaderField{Name: "te", Value: "trailers"}) @@ -452,7 +392,22 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea 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 { + + if md, added, ok := metadata.FromOutgoingContextRaw(ctx); ok { + var k string + for _, vv := range added { + for i, v := range vv { + if i%2 == 0 { + k = v + continue + } + // HTTP doesn't allow you to set pseudoheaders after non pseudoheaders were set. + if isReservedHeader(k) { + continue + } + headerFields = append(headerFields, hpack.HeaderField{Name: strings.ToLower(k), Value: encodeMetadataHeader(k, v)}) + } + } for k, vv := range md { // HTTP doesn't allow you to set pseudoheaders after non pseudoheaders were set. if isReservedHeader(k) { @@ -473,42 +428,178 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea } } } - t.mu.Lock() - if t.state == draining { - t.mu.Unlock() - t.streamsQuota.add(1) - return nil, ErrStreamDrain + return headerFields, nil +} + +func (t *http2Client) createAudience(callHdr *CallHdr) string { + // Create an audience string only if needed. + if len(t.creds) == 0 && callHdr.Creds == nil { + return "" } - if t.state != reachable { - t.mu.Unlock() - return nil, ErrConnClosing + // Construct URI required to get auth request metadata. + // Omit port if it is the default one. + host := strings.TrimSuffix(callHdr.Host, ":443") + pos := strings.LastIndex(callHdr.Method, "/") + if pos == -1 { + pos = len(callHdr.Method) + } + return "https://" + host + callHdr.Method[:pos] +} + +func (t *http2Client) getTrAuthData(ctx context.Context, audience string) (map[string]string, error) { + authData := map[string]string{} + for _, c := range t.creds { + data, err := c.GetRequestMetadata(ctx, audience) + if err != nil { + if _, ok := status.FromError(err); ok { + return nil, err + } + + return nil, streamErrorf(codes.Unauthenticated, "transport: %v", err) + } + for k, v := range data { + // Capital header names are illegal in HTTP/2. + k = strings.ToLower(k) + authData[k] = v + } + } + return authData, nil +} + +func (t *http2Client) getCallAuthData(ctx context.Context, audience string, callHdr *CallHdr) (map[string]string, error) { + 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 connection") + } + data, err := callCreds.GetRequestMetadata(ctx, audience) + if err != nil { + return nil, streamErrorf(codes.Internal, "transport: %v", err) + } + for k, v := range data { + // Capital header names are illegal in HTTP/2 + k = strings.ToLower(k) + callAuthData[k] = v + } + } + return callAuthData, nil +} + +// NewStream creates a stream and registers it into the transport as "active" +// streams. +func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Stream, err error) { + ctx = peer.NewContext(ctx, t.getPeer()) + headerFields, err := t.createHeaderFields(ctx, callHdr) + if err != nil { + return nil, err } s := t.newStream(ctx, callHdr) - t.activeStreams[s.id] = s - // If the number of active streams change from 0 to 1, then check if keepalive - // has gone dormant. If so, wake it up. - if len(t.activeStreams) == 1 { - select { - case t.awakenKeepalive <- struct{}{}: - 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. - t.awakenKeepalive <- struct{}{} - default: + cleanup := func(err error) { + if s.swapState(streamDone) == streamDone { + // If it was already done, return. + return + } + // The stream was unprocessed by the server. + atomic.StoreUint32(&s.unprocessed, 1) + s.write(recvMsg{err: err}) + close(s.done) + // If headerChan isn't closed, then close it. + if atomic.SwapUint32(&s.headerDone, 1) == 0 { + close(s.headerChan) } + } - t.controlBuf.put(&headerFrame{ - streamID: s.id, + hdr := &headerFrame{ hf: headerFields, endStream: false, - }) - t.mu.Unlock() - - s.mu.Lock() - s.bytesSent = true - s.mu.Unlock() - + initStream: func(id uint32) (bool, error) { + t.mu.Lock() + if state := t.state; state != reachable { + t.mu.Unlock() + // Do a quick cleanup. + err := error(errStreamDrain) + if state == closing { + err = ErrConnClosing + } + cleanup(err) + return false, err + } + t.activeStreams[id] = s + if channelz.IsOn() { + t.czmu.Lock() + t.streamsStarted++ + t.lastStreamCreated = time.Now() + t.czmu.Unlock() + } + var sendPing bool + // If the number of active streams change from 0 to 1, then check if keepalive + // has gone dormant. If so, wake it up. + if len(t.activeStreams) == 1 { + select { + case t.awakenKeepalive <- struct{}{}: + sendPing = true + // 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. + t.awakenKeepalive <- struct{}{} + default: + } + } + t.mu.Unlock() + return sendPing, nil + }, + onOrphaned: cleanup, + wq: s.wq, + } + firstTry := true + var ch chan struct{} + checkForStreamQuota := func(it interface{}) bool { + if t.streamQuota <= 0 { // Can go negative if server decreases it. + if firstTry { + t.waitingStreams++ + } + ch = t.streamsQuotaAvailable + return false + } + if !firstTry { + t.waitingStreams-- + } + t.streamQuota-- + h := it.(*headerFrame) + h.streamID = t.nextID + t.nextID += 2 + s.id = h.streamID + s.fc = &inFlow{limit: uint32(t.initialWindowSize)} + if t.streamQuota > 0 && t.waitingStreams > 0 { + select { + case t.streamsQuotaAvailable <- struct{}{}: + default: + } + } + return true + } + for { + success, err := t.controlBuf.executeAndPut(checkForStreamQuota, hdr) + if err != nil { + return nil, err + } + if success { + break + } + firstTry = false + select { + case <-ch: + case <-s.ctx.Done(): + return nil, ContextErr(s.ctx.Err()) + case <-t.goAway: + return nil, errStreamDrain + case <-t.ctx.Done(): + return nil, ErrConnClosing + } + } if t.statsHandler != nil { outHeader := &stats.OutHeader{ Client: true, @@ -525,86 +616,97 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea // CloseStream clears the footprint of a stream when the stream is not needed any more. // This must not be executed in reader's goroutine. func (t *http2Client) CloseStream(s *Stream, err error) { - t.mu.Lock() - if t.activeStreams == nil { - t.mu.Unlock() - return - } + var ( + rst bool + rstCode http2.ErrCode + ) if err != nil { - // notify in-flight streams, before the deletion - s.write(recvMsg{err: err}) + rst = true + rstCode = http2.ErrCodeCancel } - delete(t.activeStreams, s.id) - if t.state == draining && len(t.activeStreams) == 0 { - // The transport is draining and s is the last live stream on t. - t.mu.Unlock() - t.Close() + t.closeStream(s, err, rst, rstCode, nil, nil, false) +} + +func (t *http2Client) closeStream(s *Stream, err error, rst bool, rstCode http2.ErrCode, st *status.Status, mdata map[string][]string, eosReceived bool) { + // Set stream status to done. + if s.swapState(streamDone) == streamDone { + // If it was already done, return. return } - t.mu.Unlock() - // rstStream is true in case the stream is being closed at the client-side - // and the server needs to be intimated about it by sending a RST_STREAM - // frame. - // To make sure this frame is written to the wire before the headers of the - // next stream waiting for streamsQuota, we add to streamsQuota pool only - // after having acquired the writableChan to send RST_STREAM out (look at - // the controller() routine). - var rstStream bool - var rstError http2.ErrCode - defer func() { - // In case, the client doesn't have to send RST_STREAM to server - // we can safely add back to streamsQuota pool now. - if !rstStream { - t.streamsQuota.add(1) - return - } - t.controlBuf.put(&resetStream{s.id, rstError}) - }() - s.mu.Lock() - rstStream = s.rstStream - rstError = s.rstError - if s.state == streamDone { - s.mu.Unlock() - return + // status and trailers can be updated here without any synchronization because the stream goroutine will + // only read it after it sees an io.EOF error from read or write and we'll write those errors + // only after updating this. + s.status = st + if len(mdata) > 0 { + s.trailer = mdata + } + if err != nil { + // This will unblock reads eventually. + s.write(recvMsg{err: err}) } - if !s.headerDone { + // This will unblock write. + close(s.done) + // If headerChan isn't closed, then close it. + if atomic.SwapUint32(&s.headerDone, 1) == 0 { close(s.headerChan) - s.headerDone = true } - s.state = streamDone - s.mu.Unlock() - if _, ok := err.(StreamError); ok { - rstStream = true - rstError = http2.ErrCodeCancel + cleanup := &cleanupStream{ + streamID: s.id, + onWrite: func() { + t.mu.Lock() + if t.activeStreams != nil { + delete(t.activeStreams, s.id) + } + t.mu.Unlock() + if channelz.IsOn() { + t.czmu.Lock() + if eosReceived { + t.streamsSucceeded++ + } else { + t.streamsFailed++ + } + t.czmu.Unlock() + } + }, + rst: rst, + rstCode: rstCode, } + addBackStreamQuota := func(interface{}) bool { + t.streamQuota++ + if t.streamQuota > 0 && t.waitingStreams > 0 { + select { + case t.streamsQuotaAvailable <- struct{}{}: + default: + } + } + return true + } + t.controlBuf.executeAndPut(addBackStreamQuota, cleanup) } // Close kicks off the shutdown process of the transport. This should be called // only once on a transport. Once it is called, the transport should not be // accessed any more. -func (t *http2Client) Close() (err error) { +func (t *http2Client) Close() error { t.mu.Lock() + // Make sure we only Close once. if t.state == closing { t.mu.Unlock() - return + return nil } t.state = closing - t.mu.Unlock() - t.cancel() - err = t.conn.Close() - t.mu.Lock() streams := t.activeStreams t.activeStreams = nil t.mu.Unlock() + t.controlBuf.finish() + t.cancel() + err := t.conn.Close() + if channelz.IsOn() { + channelz.RemoveEntry(t.channelzID) + } // Notify all active streams. for _, s := range streams { - s.mu.Lock() - if !s.headerDone { - close(s.headerChan) - s.headerDone = true - } - s.mu.Unlock() - s.write(recvMsg{err: ErrConnClosing}) + t.closeStream(s, ErrConnClosing, false, http2.ErrCodeNo, nil, nil, false) } if t.statsHandler != nil { connEnd := &stats.ConnEnd{ @@ -622,8 +724,8 @@ func (t *http2Client) Close() (err error) { // closing. func (t *http2Client) GracefulClose() error { t.mu.Lock() - switch t.state { - case closing, draining: + // Make sure we move to draining only from active. + if t.state == draining || t.state == closing { t.mu.Unlock() return nil } @@ -639,102 +741,34 @@ 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. func (t *http2Client) Write(s *Stream, hdr []byte, data []byte, opts *Options) error { - select { - case <-s.ctx.Done(): - return ContextErr(s.ctx.Err()) - case <-t.ctx.Done(): - return ErrConnClosing - default: - } - - 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. - 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, t.ctx, s.done, s.goAway, t.sendQuotaPool.acquire()) - if err != nil { - return err - } - if sq < size { - size = sq - } - if tq < size { - size = tq - } - 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, s.done, s.goAway, s.localSendQuota.acquire()) - if err != nil { - if _, ok := err.(ConnectionError); !ok { - t.sendQuotaPool.add(ps) - } - 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 - } - } - } - 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 { + // If it's the last message, update stream state. + if !s.compareAndSwapState(streamActive, streamWriteDone) { + return errStreamDone } + } else if s.getState() != streamActive { + return errStreamDone } - if !opts.Last { - return nil - } - s.mu.Lock() - if s.state != streamDone { - s.state = streamWriteDone + df := &dataFrame{ + streamID: s.id, + endStream: opts.Last, + } + if hdr != nil || data != nil { // If it's not an empty data frame. + // Add some data to grpc message header so that we can equally + // distribute bytes across frames. + emptyLen := http2MaxFrameLen - len(hdr) + if emptyLen > len(data) { + emptyLen = len(data) + } + hdr = append(hdr, data[:emptyLen]...) + data = data[emptyLen:] + df.h, df.d = hdr, data + // TODO(mmukhi): The above logic in this if can be moved to loopyWriter's data handler. + if err := s.wq.get(int32(len(hdr) + len(data))); err != nil { + return err + } } - s.mu.Unlock() - return nil + return t.controlBuf.put(df) } func (t *http2Client) getStream(f http2.Frame) (*Stream, bool) { @@ -748,34 +782,17 @@ func (t *http2Client) getStream(f http2.Frame) (*Stream, bool) { // of stream if the application is requesting data larger in size than // the window. func (t *http2Client) adjustWindow(s *Stream, n uint32) { - s.mu.Lock() - defer s.mu.Unlock() - if s.state == streamDone { - return - } if w := s.fc.maybeAdjust(n); w > 0 { - // Piggyback connection's window update along. - if cw := t.fc.resetPendingUpdate(); cw > 0 { - t.controlBuf.put(&windowUpdate{0, cw}) - } - t.controlBuf.put(&windowUpdate{s.id, w}) + t.controlBuf.put(&outgoingWindowUpdate{streamID: s.id, increment: w}) } } -// updateWindow adjusts the inbound quota for the stream and the transport. -// Window updates will deliver to the controller for sending when -// the cumulative quota exceeds the corresponding threshold. +// updateWindow adjusts the inbound quota for the stream. +// Window updates will be sent out when the cumulative quota +// exceeds the corresponding threshold. func (t *http2Client) updateWindow(s *Stream, n uint32) { - s.mu.Lock() - defer s.mu.Unlock() - if s.state == streamDone { - return - } if w := s.fc.onRead(n); w > 0 { - if cw := t.fc.resetPendingUpdate(); cw > 0 { - t.controlBuf.put(&windowUpdate{0, cw}) - } - t.controlBuf.put(&windowUpdate{s.id, w}) + t.controlBuf.put(&outgoingWindowUpdate{streamID: s.id, increment: w}) } } @@ -787,15 +804,17 @@ func (t *http2Client) updateFlowControl(n uint32) { for _, s := range t.activeStreams { s.fc.newLimit(n) } - t.initialWindowSize = int32(n) t.mu.Unlock() - t.controlBuf.put(&windowUpdate{0, t.fc.newLimit(n)}) - t.controlBuf.put(&settings{ - ack: false, + updateIWS := func(interface{}) bool { + t.initialWindowSize = int32(n) + return true + } + t.controlBuf.executeAndPut(updateIWS, &outgoingWindowUpdate{streamID: 0, increment: t.fc.newLimit(n)}) + t.controlBuf.put(&outgoingSettings{ ss: []http2.Setting{ { ID: http2.SettingInitialWindowSize, - Val: uint32(n), + Val: n, }, }, }) @@ -805,7 +824,7 @@ func (t *http2Client) handleData(f *http2.DataFrame) { size := f.Header().Length var sendBDPPing bool if t.bdpEst != nil { - sendBDPPing = t.bdpEst.add(uint32(size)) + sendBDPPing = t.bdpEst.add(size) } // Decouple connection's flow control from application's read. // An update on connection's flow control should not depend on @@ -816,21 +835,24 @@ func (t *http2Client) handleData(f *http2.DataFrame) { // active(fast) streams from starving in presence of slow or // inactive streams. // - // Furthermore, if a bdpPing is being sent out we can piggyback - // connection's window update for the bytes we just received. + if w := t.fc.onData(size); w > 0 { + t.controlBuf.put(&outgoingWindowUpdate{ + streamID: 0, + increment: w, + }) + } if sendBDPPing { - if size != 0 { // Could've been an empty data frame. - t.controlBuf.put(&windowUpdate{0, uint32(size)}) + // Avoid excessive ping detection (e.g. in an L7 proxy) + // by sending a window update prior to the BDP ping. + + if w := t.fc.reset(); w > 0 { + t.controlBuf.put(&outgoingWindowUpdate{ + streamID: 0, + increment: w, + }) } + t.controlBuf.put(bdpPing) - } else { - if err := t.fc.onData(uint32(size)); err != nil { - t.Close() - return - } - if w := t.fc.onRead(uint32(size)); w > 0 { - t.controlBuf.put(&windowUpdate{0, w}) - } } // Select the right stream to dispatch. s, ok := t.getStream(f) @@ -838,25 +860,15 @@ func (t *http2Client) handleData(f *http2.DataFrame) { return } if size > 0 { - s.mu.Lock() - if s.state == streamDone { - s.mu.Unlock() - return - } - if err := s.fc.onData(uint32(size)); err != nil { - s.rstStream = true - s.rstError = http2.ErrCodeFlowControl - s.finish(status.New(codes.Internal, err.Error())) - s.mu.Unlock() - s.write(recvMsg{err: io.EOF}) + if err := s.fc.onData(size); err != nil { + t.closeStream(s, io.EOF, true, http2.ErrCodeFlowControl, status.New(codes.Internal, err.Error()), nil, false) return } 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}) + if w := s.fc.onRead(size - uint32(len(f.Data()))); w > 0 { + t.controlBuf.put(&outgoingWindowUpdate{s.id, w}) } } - s.mu.Unlock() // TODO(bradfitz, zhaoq): A copy is required here because there is no // guarantee f.Data() is consumed before the arrival of next frame. // Can this copy be eliminated? @@ -869,14 +881,7 @@ func (t *http2Client) handleData(f *http2.DataFrame) { // The server has closed the stream without sending trailers. Record that // the read direction is closed, and set the status appropriately. if f.FrameHeader.Flags.Has(http2.FlagDataEndStream) { - s.mu.Lock() - if s.state == streamDone { - s.mu.Unlock() - return - } - s.finish(status.New(codes.Internal, "server closed the stream without sending trailers")) - s.mu.Unlock() - s.write(recvMsg{err: io.EOF}) + t.closeStream(s, io.EOF, false, http2.ErrCodeNo, status.New(codes.Internal, "server closed the stream without sending trailers"), nil, true) } } @@ -885,36 +890,55 @@ func (t *http2Client) handleRSTStream(f *http2.RSTStreamFrame) { if !ok { return } - s.mu.Lock() - if s.state == streamDone { - s.mu.Unlock() - return - } - if !s.headerDone { - close(s.headerChan) - s.headerDone = true + if f.ErrCode == http2.ErrCodeRefusedStream { + // The stream was unprocessed by the server. + atomic.StoreUint32(&s.unprocessed, 1) } - statusCode, ok := http2ErrConvTab[http2.ErrCode(f.ErrCode)] + statusCode, ok := http2ErrConvTab[f.ErrCode] if !ok { warningf("transport: http2Client.handleRSTStream found no mapped gRPC status for the received http2 error %v", f.ErrCode) statusCode = codes.Unknown } - s.finish(status.Newf(statusCode, "stream terminated by RST_STREAM with error code: %v", f.ErrCode)) - s.mu.Unlock() - s.write(recvMsg{err: io.EOF}) + t.closeStream(s, io.EOF, false, http2.ErrCodeNo, status.Newf(statusCode, "stream terminated by RST_STREAM with error code: %v", f.ErrCode), nil, false) } -func (t *http2Client) handleSettings(f *http2.SettingsFrame) { +func (t *http2Client) handleSettings(f *http2.SettingsFrame, isFirst bool) { if f.IsAck() { return } + var maxStreams *uint32 var ss []http2.Setting f.ForeachSetting(func(s http2.Setting) error { + if s.ID == http2.SettingMaxConcurrentStreams { + maxStreams = new(uint32) + *maxStreams = s.Val + return nil + } ss = append(ss, s) return nil }) - // The settings will be applied once the ack is sent. - t.controlBuf.put(&settings{ack: true, ss: ss}) + if isFirst && maxStreams == nil { + maxStreams = new(uint32) + *maxStreams = math.MaxUint32 + } + sf := &incomingSettings{ + ss: ss, + } + if maxStreams == nil { + t.controlBuf.put(sf) + return + } + updateStreamQuota := func(interface{}) bool { + delta := int64(*maxStreams) - int64(t.maxConcurrentStreams) + t.maxConcurrentStreams = *maxStreams + t.streamQuota += delta + if delta > 0 && t.waitingStreams > 0 { + close(t.streamsQuotaAvailable) // wake all of them up. + t.streamsQuotaAvailable = make(chan struct{}, 1) + } + return true + } + t.controlBuf.executeAndPut(updateStreamQuota, sf) } func (t *http2Client) handlePing(f *http2.PingFrame) { @@ -932,7 +956,7 @@ func (t *http2Client) handlePing(f *http2.PingFrame) { func (t *http2Client) handleGoAway(f *http2.GoAwayFrame) { t.mu.Lock() - if t.state != reachable && t.state != draining { + if t.state == closing { t.mu.Unlock() return } @@ -945,12 +969,16 @@ func (t *http2Client) handleGoAway(f *http2.GoAwayFrame) { t.Close() return } - // 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 - // close all streams created after the second GoAwayId. This way streams that were in-flight while the GoAway from server - // was being sent don't get killed. + // A client can receive multiple GoAways from the server (see + // 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 necessarily close any + // streams. While in case of second GoAway we close all streams created after + // the GoAwayId. This way streams that were in-flight while the GoAway from + // server was being sent don't get killed. select { case <-t.goAway: // t.goAway has been closed (i.e.,multiple GoAways). // If there are multiple GoAways the first one should always have an ID greater than the following ones. @@ -963,6 +991,7 @@ func (t *http2Client) handleGoAway(f *http2.GoAwayFrame) { t.setGoAwayReason(f) close(t.goAway) t.state = draining + t.controlBuf.put(&incomingGoAway{}) } // All streams with IDs greater than the GoAwayId // and smaller than the previous GoAway ID should be killed. @@ -972,7 +1001,9 @@ func (t *http2Client) handleGoAway(f *http2.GoAwayFrame) { } for streamID, stream := range t.activeStreams { if streamID > id && streamID <= upperLimit { - close(stream.goAway) + // The stream was unprocessed by the server. + atomic.StoreUint32(&stream.unprocessed, 1) + t.closeStream(stream, errStreamDrain, false, http2.ErrCodeNo, statusGoAway, nil, false) } } t.prevGoAwayID = id @@ -988,11 +1019,11 @@ func (t *http2Client) handleGoAway(f *http2.GoAwayFrame) { // It expects a lock on transport's mutext to be held by // the caller. func (t *http2Client) setGoAwayReason(f *http2.GoAwayFrame) { - t.goAwayReason = NoReason + t.goAwayReason = GoAwayNoReason switch f.ErrCode { case http2.ErrCodeEnhanceYourCalm: if string(f.DebugData()) == "too_many_pings" { - t.goAwayReason = TooManyPings + t.goAwayReason = GoAwayTooManyPings } } } @@ -1004,15 +1035,10 @@ func (t *http2Client) GetGoAwayReason() GoAwayReason { } func (t *http2Client) handleWindowUpdate(f *http2.WindowUpdateFrame) { - id := f.Header().StreamID - incr := f.Increment - if id == 0 { - t.sendQuotaPool.add(int(incr)) - return - } - if s, ok := t.getStream(f); ok { - s.sendQuotaPool.add(int(incr)) - } + t.controlBuf.put(&incomingWindowUpdate{ + streamID: f.Header().StreamID, + increment: f.Increment, + }) } // operateHeaders takes action on the decoded headers. @@ -1021,18 +1047,10 @@ func (t *http2Client) operateHeaders(frame *http2.MetaHeadersFrame) { if !ok { return } - s.mu.Lock() - s.bytesReceived = true - s.mu.Unlock() + atomic.StoreUint32(&s.bytesReceived, 1) var state decodeState if err := state.decodeResponseHeader(frame); err != nil { - s.mu.Lock() - if !s.headerDone { - close(s.headerChan) - s.headerDone = true - } - s.mu.Unlock() - s.write(recvMsg{err: err}) + t.closeStream(s, err, true, http2.ErrCodeProtocol, nil, nil, false) // Something wrong. Stops reading even when there is remaining. return } @@ -1056,40 +1074,25 @@ func (t *http2Client) operateHeaders(frame *http2.MetaHeadersFrame) { } } }() - - s.mu.Lock() - if !endStream { - s.recvCompress = state.encoding - } - if !s.headerDone { - if !endStream && len(state.mdata) > 0 { - s.header = state.mdata + // If headers haven't been received yet. + if atomic.SwapUint32(&s.headerDone, 1) == 0 { + if !endStream { + // Headers frame is not actually a trailers-only frame. + isHeader = true + // These values can be set without any synchronization because + // stream goroutine will read it only after seeing a closed + // headerChan which we'll close after setting this. + s.recvCompress = state.encoding + if len(state.mdata) > 0 { + s.header = state.mdata + } } close(s.headerChan) - s.headerDone = true - isHeader = true } - if !endStream || s.state == streamDone { - s.mu.Unlock() + if !endStream { return } - - if len(state.mdata) > 0 { - s.trailer = state.mdata - } - s.finish(state.status()) - s.mu.Unlock() - s.write(recvMsg{err: io.EOF}) -} - -func handleMalformedHTTP2(s *Stream, err error) { - s.mu.Lock() - if !s.headerDone { - close(s.headerChan) - s.headerDone = true - } - s.mu.Unlock() - s.write(recvMsg{err: err}) + t.closeStream(s, io.EOF, false, http2.ErrCodeNo, state.status(), state.mdata, true) } // reader runs as a separate goroutine in charge of reading data from network @@ -1099,6 +1102,7 @@ func handleMalformedHTTP2(s *Stream, err error) { // optimal. // TODO(zhaoq): Check the validity of the incoming frame sequence. func (t *http2Client) reader() { + defer close(t.readerDone) // Check the validity of server preface. frame, err := t.framer.fr.ReadFrame() if err != nil { @@ -1111,7 +1115,8 @@ func (t *http2Client) reader() { t.Close() return } - t.handleSettings(sf) + t.onSuccess() + t.handleSettings(sf, true) // loop to keep reading incoming messages on this transport. for { @@ -1127,7 +1132,7 @@ 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.fr.ErrorDetail())) + t.closeStream(s, streamErrorf(http2ErrConvTab[se.Code], "%v", t.framer.fr.ErrorDetail()), true, http2.ErrCodeProtocol, nil, nil, false) } continue } else { @@ -1144,7 +1149,7 @@ func (t *http2Client) reader() { case *http2.RSTStreamFrame: t.handleRSTStream(frame) case *http2.SettingsFrame: - t.handleSettings(frame) + t.handleSettings(frame, false) case *http2.PingFrame: t.handlePing(frame) case *http2.GoAwayFrame: @@ -1157,107 +1162,6 @@ func (t *http2Client) reader() { } } -func (t *http2Client) applySettings(ss []http2.Setting) { - for _, s := range ss { - switch s.ID { - case http2.SettingMaxConcurrentStreams: - // TODO(zhaoq): This is a hack to avoid significant refactoring of the - // code to deal with the unrealistic int32 overflow. Probably will try - // to find a better way to handle this later. - if s.Val > math.MaxInt32 { - s.Val = math.MaxInt32 - } - t.mu.Lock() - ms := t.maxStreams - t.maxStreams = int(s.Val) - t.mu.Unlock() - t.streamsQuota.add(int(s.Val) - ms) - case http2.SettingInitialWindowSize: - t.mu.Lock() - for _, stream := range t.activeStreams { - // Adjust the sending quota for each stream. - stream.sendQuotaPool.addAndUpdate(int(s.Val) - int(t.streamSendQuota)) - } - t.streamSendQuota = s.Val - t.mu.Unlock() - } - } -} - -// 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. func (t *http2Client) keepalive() { p := &ping{data: [8]byte{}} @@ -1284,6 +1188,11 @@ func (t *http2Client) keepalive() { } } else { t.mu.Unlock() + if channelz.IsOn() { + t.czmu.Lock() + t.kpCount++ + t.czmu.Unlock() + } // Send ping. t.controlBuf.put(p) } @@ -1320,3 +1229,56 @@ func (t *http2Client) Error() <-chan struct{} { func (t *http2Client) GoAway() <-chan struct{} { return t.goAway } + +func (t *http2Client) ChannelzMetric() *channelz.SocketInternalMetric { + t.czmu.RLock() + s := channelz.SocketInternalMetric{ + StreamsStarted: t.streamsStarted, + StreamsSucceeded: t.streamsSucceeded, + StreamsFailed: t.streamsFailed, + MessagesSent: t.msgSent, + MessagesReceived: t.msgRecv, + KeepAlivesSent: t.kpCount, + LastLocalStreamCreatedTimestamp: t.lastStreamCreated, + LastMessageSentTimestamp: t.lastMsgSent, + LastMessageReceivedTimestamp: t.lastMsgRecv, + LocalFlowControlWindow: int64(t.fc.getSize()), + //socket options + LocalAddr: t.localAddr, + RemoteAddr: t.remoteAddr, + // Security + // RemoteName : + } + t.czmu.RUnlock() + s.RemoteFlowControlWindow = t.getOutFlowWindow() + return &s +} + +func (t *http2Client) IncrMsgSent() { + t.czmu.Lock() + t.msgSent++ + t.lastMsgSent = time.Now() + t.czmu.Unlock() +} + +func (t *http2Client) IncrMsgRecv() { + t.czmu.Lock() + t.msgRecv++ + t.lastMsgRecv = time.Now() + t.czmu.Unlock() +} + +func (t *http2Client) getOutFlowWindow() int64 { + resp := make(chan uint32, 1) + timer := time.NewTimer(time.Second) + defer timer.Stop() + t.controlBuf.put(&outFlowControlSizeRequest{resp}) + select { + case sz := <-resp: + return int64(sz) + case <-t.ctxDone: + return -1 + case <-timer.C: + return -2 + } +} diff --git a/vendor/google.golang.org/grpc/transport/http2_server.go b/vendor/google.golang.org/grpc/transport/http2_server.go index 00df8eed0fd..ab356189050 100644 --- a/vendor/google.golang.org/grpc/transport/http2_server.go +++ b/vendor/google.golang.org/grpc/transport/http2_server.go @@ -35,6 +35,8 @@ import ( "golang.org/x/net/context" "golang.org/x/net/http2" "golang.org/x/net/http2/hpack" + + "google.golang.org/grpc/channelz" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" "google.golang.org/grpc/keepalive" @@ -52,25 +54,25 @@ var ErrIllegalHeaderWrite = errors.New("transport: the stream is done or WriteHe // http2Server implements the ServerTransport interface with HTTP2. type http2Server struct { ctx context.Context + ctxDone <-chan struct{} // Cache the context.Done() chan cancel context.CancelFunc conn net.Conn + loopy *loopyWriter + readerDone chan struct{} // sync point to enable testing. + writerDone chan struct{} // sync point to enable testing. remoteAddr net.Addr localAddr net.Addr maxStreamID uint32 // max stream ID ever seen authInfo credentials.AuthInfo // auth info about the connection inTapHandle tap.ServerInHandle 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 // updates, reset streams, and various settings) to the controller. controlBuf *controlBuffer - fc *inFlow - // sendQuotaPool provides flow control to outbound message. - sendQuotaPool *quotaPool - stats stats.Handler + fc *trInFlow + stats stats.Handler // Flag to keep track of reading activity on transport. // 1 is true and 0 is false. activity uint32 // Accessed atomically. @@ -101,13 +103,27 @@ type http2Server struct { drainChan chan struct{} state transportState activeStreams map[uint32]*Stream - // 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 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 + + // Fields below are for channelz metric collection. + channelzID int64 // channelz unique identification number + czmu sync.RWMutex + kpCount int64 + // The number of streams that have started, including already finished ones. + streamsStarted int64 + // The number of streams that have ended successfully by sending frame with + // EoS bit set. + streamsSucceeded int64 + streamsFailed int64 + lastStreamCreated time.Time + msgSent int64 + msgRecv int64 + lastMsgSent time.Time + lastMsgRecv time.Time } // newHTTP2Server constructs a ServerTransport based on HTTP2. ConnectionError is @@ -182,32 +198,30 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err if kep.MinTime == 0 { kep.MinTime = defaultKeepalivePolicyMinTime } - var buf bytes.Buffer ctx, cancel := context.WithCancel(context.Background()) t := &http2Server{ ctx: ctx, cancel: cancel, + ctxDone: ctx.Done(), conn: conn, remoteAddr: conn.RemoteAddr(), localAddr: conn.LocalAddr(), authInfo: config.AuthInfo, framer: framer, - hBuf: &buf, - hEnc: hpack.NewEncoder(&buf), + readerDone: make(chan struct{}), + writerDone: make(chan struct{}), maxStreams: maxStreams, inTapHandle: config.InTapHandle, - controlBuf: newControlBuffer(), - fc: &inFlow{limit: uint32(icwz)}, - sendQuotaPool: newQuotaPool(defaultWindowSize), + fc: &trInFlow{limit: uint32(icwz)}, state: reachable, activeStreams: make(map[uint32]*Stream), - streamSendQuota: defaultWindowSize, stats: config.StatsHandler, kp: kp, idle: time.Now(), kep: kep, initialWindowSize: iwz, } + t.controlBuf = newControlBuffer(t.ctxDone) if dynamicWindow { t.bdpEst = &bdpEstimator{ bdp: initialWindowSize, @@ -222,8 +236,17 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err connBegin := &stats.ConnBegin{} t.stats.HandleConn(t.ctx, connBegin) } + if channelz.IsOn() { + t.channelzID = channelz.RegisterNormalSocket(t, config.ChannelzParentID, "") + } t.framer.writer.Flush() + defer func() { + if err != nil { + t.Close() + } + }() + // Check the validity of client preface. preface := make([]byte, len(clientPreface)) if _, err := io.ReadFull(t.conn, preface); err != nil { @@ -235,8 +258,7 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err frame, err := t.framer.fr.ReadFrame() if err == io.EOF || err == io.ErrUnexpectedEOF { - t.Close() - return + return nil, err } if err != nil { return nil, connectionErrorf(false, err, "transport: http2Server.HandleStreams failed to read initial settings frame: %v", err) @@ -249,8 +271,11 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err t.handleSettings(sf) go func() { - loopyWriter(t.ctx, t.controlBuf, t.itemHandler) - t.Close() + t.loopy = newLoopyWriter(serverSide, t.framer, t.controlBuf, t.bdpEst) + t.loopy.ssGoAwayHandler = t.outgoingGoAwayHandler + t.loopy.run() + t.conn.Close() + close(t.writerDone) }() go t.keepalive() return t, nil @@ -259,12 +284,16 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err // operateHeader takes action on the decoded headers. func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(*Stream), traceCtx func(context.Context, string) context.Context) (close bool) { streamID := frame.Header().StreamID - var state decodeState for _, hf := range frame.Fields { if err := state.processHeaderField(hf); err != nil { if se, ok := err.(StreamError); ok { - t.controlBuf.put(&resetStream{streamID, statusCodeConvTab[se.Code]}) + t.controlBuf.put(&cleanupStream{ + streamID: streamID, + rst: true, + rstCode: statusCodeConvTab[se.Code], + onWrite: func() {}, + }) } return } @@ -272,14 +301,14 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func( buf := newRecvBuffer() s := &Stream{ - id: streamID, - st: t, - buf: buf, - fc: &inFlow{limit: uint32(t.initialWindowSize)}, - recvCompress: state.encoding, - method: state.method, + id: streamID, + st: t, + buf: buf, + fc: &inFlow{limit: uint32(t.initialWindowSize)}, + recvCompress: state.encoding, + method: state.method, + contentSubtype: state.contentSubtype, } - if frame.StreamEnded() { // s is just created by the caller. No lock needed. s.state = streamReadDone @@ -297,10 +326,6 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func( pr.AuthInfo = t.authInfo } s.ctx = peer.NewContext(s.ctx, pr) - // Cache the current stream to the context so that the server application - // can find out. Required when the server wants to send some metadata - // back to the client (unary call only). - s.ctx = newContextWithStream(s.ctx, s) // Attach the received metadata to the context. if len(state.mdata) > 0 { s.ctx = metadata.NewIncomingContext(s.ctx, state.mdata) @@ -319,7 +344,12 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func( s.ctx, err = t.inTapHandle(s.ctx, info) if err != nil { warningf("transport: http2Server.operateHeaders got an error from InTapHandle: %v", err) - t.controlBuf.put(&resetStream{s.id, http2.ErrCodeRefusedStream}) + t.controlBuf.put(&cleanupStream{ + streamID: s.id, + rst: true, + rstCode: http2.ErrCodeRefusedStream, + onWrite: func() {}, + }) return } } @@ -330,7 +360,12 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func( } if uint32(len(t.activeStreams)) >= t.maxStreams { t.mu.Unlock() - t.controlBuf.put(&resetStream{streamID, http2.ErrCodeRefusedStream}) + t.controlBuf.put(&cleanupStream{ + streamID: streamID, + rst: true, + rstCode: http2.ErrCodeRefusedStream, + onWrite: func() {}, + }) return } if streamID%2 != 1 || streamID <= t.maxStreamID { @@ -340,13 +375,17 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func( return true } 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{} } t.mu.Unlock() + if channelz.IsOn() { + t.czmu.Lock() + t.streamsStarted++ + t.lastStreamCreated = time.Now() + t.czmu.Unlock() + } s.requestRead = func(n int) { t.adjustWindow(s, uint32(n)) } @@ -362,10 +401,13 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func( } t.stats.HandleRPC(s.ctx, inHeader) } + s.ctxDone = s.ctx.Done() + s.wq = newWriteQuota(defaultWriteQuota, s.ctxDone) s.trReader = &transportReader{ reader: &recvBufferReader{ - ctx: s.ctx, - recv: s.buf, + ctx: s.ctx, + ctxDone: s.ctxDone, + recv: s.buf, }, windowHandler: func(n int) { t.updateWindow(s, uint32(n)) @@ -379,18 +421,26 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func( // typically run in a separate goroutine. // traceCtx attaches trace to ctx and returns the new context. func (t *http2Server) HandleStreams(handle func(*Stream), traceCtx func(context.Context, string) context.Context) { + defer close(t.readerDone) for { frame, err := t.framer.fr.ReadFrame() atomic.StoreUint32(&t.activity, 1) if err != nil { if se, ok := err.(http2.StreamError); ok { + warningf("transport: http2Server.HandleStreams encountered http2.StreamError: %v", se) t.mu.Lock() s := t.activeStreams[se.StreamID] t.mu.Unlock() if s != nil { - t.closeStream(s) + t.closeStream(s, true, se.Code, nil, false) + } else { + t.controlBuf.put(&cleanupStream{ + streamID: se.StreamID, + rst: true, + rstCode: se.Code, + onWrite: func() {}, + }) } - t.controlBuf.put(&resetStream{se.StreamID, se.Code}) continue } if err == io.EOF || err == io.ErrUnexpectedEOF { @@ -444,33 +494,20 @@ func (t *http2Server) getStream(f http2.Frame) (*Stream, bool) { // of stream if the application is requesting data larger in size than // the window. func (t *http2Server) adjustWindow(s *Stream, n uint32) { - s.mu.Lock() - defer s.mu.Unlock() - if s.state == streamDone { - return - } if w := s.fc.maybeAdjust(n); w > 0 { - if cw := t.fc.resetPendingUpdate(); cw > 0 { - t.controlBuf.put(&windowUpdate{0, cw}) - } - t.controlBuf.put(&windowUpdate{s.id, w}) + t.controlBuf.put(&outgoingWindowUpdate{streamID: s.id, increment: w}) } + } // updateWindow adjusts the inbound quota for the stream and the transport. // Window updates will deliver to the controller for sending when // the cumulative quota exceeds the corresponding threshold. func (t *http2Server) updateWindow(s *Stream, n uint32) { - s.mu.Lock() - defer s.mu.Unlock() - if s.state == streamDone { - return - } if w := s.fc.onRead(n); w > 0 { - if cw := t.fc.resetPendingUpdate(); cw > 0 { - t.controlBuf.put(&windowUpdate{0, cw}) - } - t.controlBuf.put(&windowUpdate{s.id, w}) + t.controlBuf.put(&outgoingWindowUpdate{streamID: s.id, + increment: w, + }) } } @@ -484,13 +521,15 @@ func (t *http2Server) updateFlowControl(n uint32) { } t.initialWindowSize = int32(n) t.mu.Unlock() - t.controlBuf.put(&windowUpdate{0, t.fc.newLimit(n)}) - t.controlBuf.put(&settings{ - ack: false, + t.controlBuf.put(&outgoingWindowUpdate{ + streamID: 0, + increment: t.fc.newLimit(n), + }) + t.controlBuf.put(&outgoingSettings{ ss: []http2.Setting{ { ID: http2.SettingInitialWindowSize, - Val: uint32(n), + Val: n, }, }, }) @@ -501,7 +540,7 @@ func (t *http2Server) handleData(f *http2.DataFrame) { size := f.Header().Length var sendBDPPing bool if t.bdpEst != nil { - sendBDPPing = t.bdpEst.add(uint32(size)) + sendBDPPing = t.bdpEst.add(size) } // Decouple connection's flow control from application's read. // An update on connection's flow control should not depend on @@ -511,23 +550,22 @@ func (t *http2Server) handleData(f *http2.DataFrame) { // Decoupling the connection flow control will prevent other // active(fast) streams from starving in presence of slow or // inactive streams. - // - // Furthermore, if a bdpPing is being sent out we can piggyback - // connection's window update for the bytes we just received. + if w := t.fc.onData(size); w > 0 { + t.controlBuf.put(&outgoingWindowUpdate{ + streamID: 0, + increment: w, + }) + } if sendBDPPing { - if size != 0 { // Could be an empty frame. - t.controlBuf.put(&windowUpdate{0, uint32(size)}) + // Avoid excessive ping detection (e.g. in an L7 proxy) + // by sending a window update prior to the BDP ping. + if w := t.fc.reset(); w > 0 { + t.controlBuf.put(&outgoingWindowUpdate{ + streamID: 0, + increment: w, + }) } t.controlBuf.put(bdpPing) - } else { - if err := t.fc.onData(uint32(size)); err != nil { - errorf("transport: http2Server %v", err) - t.Close() - return - } - if w := t.fc.onRead(uint32(size)); w > 0 { - t.controlBuf.put(&windowUpdate{0, w}) - } } // Select the right stream to dispatch. s, ok := t.getStream(f) @@ -535,23 +573,15 @@ func (t *http2Server) handleData(f *http2.DataFrame) { return } if size > 0 { - s.mu.Lock() - if s.state == streamDone { - s.mu.Unlock() - return - } - if err := s.fc.onData(uint32(size)); err != nil { - s.mu.Unlock() - t.closeStream(s) - t.controlBuf.put(&resetStream{s.id, http2.ErrCodeFlowControl}) + if err := s.fc.onData(size); err != nil { + t.closeStream(s, true, http2.ErrCodeFlowControl, nil, false) return } 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}) + if w := s.fc.onRead(size - uint32(len(f.Data()))); w > 0 { + t.controlBuf.put(&outgoingWindowUpdate{s.id, w}) } } - s.mu.Unlock() // TODO(bradfitz, zhaoq): A copy is required here because there is no // guarantee f.Data() is consumed before the arrival of next frame. // Can this copy be eliminated? @@ -563,11 +593,7 @@ func (t *http2Server) handleData(f *http2.DataFrame) { } if f.Header().Flags.Has(http2.FlagDataEndStream) { // Received the end of stream from the client. - s.mu.Lock() - if s.state != streamDone { - s.state = streamReadDone - } - s.mu.Unlock() + s.compareAndSwapState(streamActive, streamReadDone) s.write(recvMsg{err: io.EOF}) } } @@ -577,7 +603,7 @@ func (t *http2Server) handleRSTStream(f *http2.RSTStreamFrame) { if !ok { return } - t.closeStream(s) + t.closeStream(s, false, 0, nil, false) } func (t *http2Server) handleSettings(f *http2.SettingsFrame) { @@ -589,21 +615,9 @@ func (t *http2Server) handleSettings(f *http2.SettingsFrame) { ss = append(ss, s) return nil }) - 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() - } - - } + t.controlBuf.put(&incomingSettings{ + ss: ss, + }) } const ( @@ -656,39 +670,37 @@ 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.") + errorf("transport: Got too many pings from the client, closing the connection.") t.controlBuf.put(&goAway{code: http2.ErrCodeEnhanceYourCalm, debugData: []byte("too_many_pings"), closeConn: true}) } } func (t *http2Server) handleWindowUpdate(f *http2.WindowUpdateFrame) { - id := f.Header().StreamID - incr := f.Increment - if id == 0 { - t.sendQuotaPool.add(int(incr)) - return - } - if s, ok := t.getStream(f); ok { - s.sendQuotaPool.add(int(incr)) + t.controlBuf.put(&incomingWindowUpdate{ + streamID: f.Header().StreamID, + increment: f.Increment, + }) +} + +func appendHeaderFieldsFromMD(headerFields []hpack.HeaderField, md metadata.MD) []hpack.HeaderField { + for k, vv := range md { + if isReservedHeader(k) { + // Clients don't tolerate reading restricted headers after some non restricted ones were sent. + continue + } + for _, v := range vv { + headerFields = append(headerFields, hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)}) + } } + return headerFields } // 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() + if s.updateHeaderSent() || s.getState() == streamDone { return ErrIllegalHeaderWrite } - s.headerOk = true + s.hdrMu.Lock() if md.Len() > 0 { if s.header.Len() > 0 { s.header = metadata.Join(s.header, md) @@ -696,37 +708,36 @@ func (t *http2Server) WriteHeader(s *Stream, md metadata.MD) error { s.header = md } } - md = s.header - s.mu.Unlock() - // TODO(mmukhi): Benchmark if the perfomance gets better if count the metadata and other header fields + t.writeHeaderLocked(s) + s.hdrMu.Unlock() + return nil +} + +func (t *http2Server) writeHeaderLocked(s *Stream) { + // TODO(mmukhi): Benchmark if the performance 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"}) + headerFields = append(headerFields, hpack.HeaderField{Name: "content-type", Value: contentType(s.contentSubtype)}) if s.sendCompress != "" { headerFields = append(headerFields, hpack.HeaderField{Name: "grpc-encoding", Value: s.sendCompress}) } - for k, vv := range md { - if isReservedHeader(k) { - // Clients don't tolerate reading restricted headers after some non restricted ones were sent. - continue - } - for _, v := range vv { - headerFields = append(headerFields, hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)}) - } - } + headerFields = appendHeaderFieldsFromMD(headerFields, s.header) t.controlBuf.put(&headerFrame{ streamID: s.id, hf: headerFields, endStream: false, + onWrite: func() { + atomic.StoreUint32(&t.resetPingStrikes, 1) + }, + wq: s.wq, }) if t.stats != nil { - outHeader := &stats.OutHeader{ - //WireLength: // TODO(mmukhi): Revisit this later, if needed. - } + // Note: WireLength is not set in outHeader. + // TODO(mmukhi): Revisit this later, if needed. + outHeader := &stats.OutHeader{} t.stats.HandleRPC(s.Context(), outHeader) } - return nil } // WriteStatus sends stream status to the client and terminates the stream. @@ -734,37 +745,20 @@ 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 { - s.mu.Unlock() + if s.getState() == streamDone { return nil } - if s.headerOk { - headersSent = true - } - if s.header.Len() > 0 { - hasHeader = true - } - s.mu.Unlock() - - if !headersSent && hasHeader { - t.WriteHeader(s, nil) - headersSent = true - } - - // TODO(mmukhi): Benchmark if the perfomance gets better if count the metadata and other header fields + s.hdrMu.Lock() + // TODO(mmukhi): Benchmark if the performance 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 { - headerFields = append(headerFields, hpack.HeaderField{Name: ":status", Value: "200"}) - headerFields = append(headerFields, hpack.HeaderField{Name: "content-type", Value: "application/grpc"}) + if !s.updateHeaderSent() { // No headers have been sent. + if len(s.header) > 0 { // Send a separate header frame. + t.writeHeaderLocked(s) + } else { // Send a trailer only response. + headerFields = append(headerFields, hpack.HeaderField{Name: ":status", Value: "200"}) + headerFields = append(headerFields, hpack.HeaderField{Name: "content-type", Value: contentType(s.contentSubtype)}) + } } 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())}) @@ -780,119 +774,68 @@ func (t *http2Server) WriteStatus(s *Stream, st *status.Status) error { } // Attach the trailer metadata. - for k, vv := range s.trailer { - // Clients don't tolerate reading restricted headers after some non restricted ones were sent. - if isReservedHeader(k) { - continue - } - for _, v := range vv { - headerFields = append(headerFields, hpack.HeaderField{Name: k, Value: encodeMetadataHeader(k, v)}) - } - } - t.controlBuf.put(&headerFrame{ + headerFields = appendHeaderFieldsFromMD(headerFields, s.trailer) + trailingHeader := &headerFrame{ streamID: s.id, hf: headerFields, endStream: true, - }) + onWrite: func() { + atomic.StoreUint32(&t.resetPingStrikes, 1) + }, + } + s.hdrMu.Unlock() + t.closeStream(s, false, 0, trailingHeader, true) if t.stats != nil { t.stats.HandleRPC(s.Context(), &stats.OutTrailer{}) } - t.closeStream(s) 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) { - select { - case <-s.ctx.Done(): - return ContextErr(s.ctx.Err()) - case <-t.ctx.Done(): - return ErrConnClosing - default: - } - - var writeHeaderFrame bool - s.mu.Lock() - if s.state == streamDone { - s.mu.Unlock() - return streamErrorf(codes.Unknown, "the stream has been done") - } - if !s.headerOk { - writeHeaderFrame = true - } - s.mu.Unlock() - if writeHeaderFrame { - t.WriteHeader(s, nil) +func (t *http2Server) Write(s *Stream, hdr []byte, data []byte, opts *Options) error { + if !s.isHeaderSent() { // Headers haven't been written yet. + if err := t.WriteHeader(s, nil); err != nil { + // TODO(mmukhi, dfawley): Make sure this is the right code to return. + return streamErrorf(codes.Internal, "transport: %v", err) + } + } else { + // Writing headers checks for this condition. + if s.getState() == streamDone { + // TODO(mmukhi, dfawley): Should the server write also return io.EOF? + s.cancel() + select { + case <-t.ctx.Done(): + return ErrConnClosing + default: + } + return ContextErr(s.ctx.Err()) + } } - // Add data to header frame so that we can equally distribute data across frames. + // Add some data to header frame so that we can equally distribute bytes 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 - } - // 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 - } - if sq < size { - size = sq - } - if tq < size { - size = tq - } - 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) - } - 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. + df := &dataFrame{ + streamID: s.id, + h: hdr, + d: data, + onEachWrite: func() { 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) - } + }, + } + if err := s.wq.get(int32(len(hdr) + len(data))); err != nil { + select { + case <-t.ctx.Done(): + return ErrConnClosing + default: } + return ContextErr(s.ctx.Err()) } - return nil + return t.controlBuf.put(df) } // keepalive running in a separate goroutine does the following: @@ -937,7 +880,7 @@ func (t *http2Server) keepalive() { // The connection has been idle for a duration of keepalive.MaxConnectionIdle or more. // Gracefully close the connection. t.drain(http2.ErrCodeNo, []byte{}) - // Reseting the timer so that the clean-up doesn't deadlock. + // Resetting the timer so that the clean-up doesn't deadlock. maxIdle.Reset(infinity) return } @@ -949,7 +892,7 @@ func (t *http2Server) keepalive() { case <-maxAge.C: // Close the connection after grace period. t.Close() - // Reseting the timer so that the clean-up doesn't deadlock. + // Resetting the timer so that the clean-up doesn't deadlock. maxAge.Reset(infinity) case <-t.ctx.Done(): } @@ -962,11 +905,16 @@ func (t *http2Server) keepalive() { } if pingSent { t.Close() - // Reseting the timer so that the clean-up doesn't deadlock. + // Resetting the timer so that the clean-up doesn't deadlock. keepalive.Reset(infinity) return } pingSent = true + if channelz.IsOn() { + t.czmu.Lock() + t.kpCount++ + t.czmu.Unlock() + } t.controlBuf.put(p) keepalive.Reset(t.kp.Timeout) case <-t.ctx.Done(): @@ -975,127 +923,6 @@ func (t *http2Server) keepalive() { } } -var goAwayPing = &ping{data: [8]byte{1, 6, 1, 8, 0, 3, 3, 9}} - -// 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.drainChan: - case <-timer.C: - case <-t.ctx.Done(): - 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. @@ -1109,8 +936,12 @@ func (t *http2Server) Close() error { streams := t.activeStreams t.activeStreams = nil t.mu.Unlock() + t.controlBuf.finish() t.cancel() err := t.conn.Close() + if channelz.IsOn() { + channelz.RemoveEntry(t.channelzID) + } // Cancel all active streams. for _, s := range streams { s.cancel() @@ -1124,27 +955,45 @@ func (t *http2Server) Close() error { // closeStream clears the footprint of a stream when the stream is not needed // any more. -func (t *http2Server) closeStream(s *Stream) { - t.mu.Lock() - delete(t.activeStreams, s.id) - if len(t.activeStreams) == 0 { - t.idle = time.Now() - } - if t.state == draining && len(t.activeStreams) == 0 { - defer t.Close() +func (t *http2Server) closeStream(s *Stream, rst bool, rstCode http2.ErrCode, hdr *headerFrame, eosReceived bool) { + if s.swapState(streamDone) == streamDone { + // If the stream was already done, return. + return } - t.mu.Unlock() // In case stream sending and receiving are invoked in separate // goroutines (e.g., bi-directional streaming), cancel needs to be // called to interrupt the potential blocking on other goroutines. s.cancel() - s.mu.Lock() - if s.state == streamDone { - s.mu.Unlock() - return + cleanup := &cleanupStream{ + streamID: s.id, + rst: rst, + rstCode: rstCode, + onWrite: func() { + t.mu.Lock() + if t.activeStreams != nil { + delete(t.activeStreams, s.id) + if len(t.activeStreams) == 0 { + t.idle = time.Now() + } + } + t.mu.Unlock() + if channelz.IsOn() { + t.czmu.Lock() + if eosReceived { + t.streamsSucceeded++ + } else { + t.streamsFailed++ + } + t.czmu.Unlock() + } + }, + } + if hdr != nil { + hdr.cleanup = cleanup + t.controlBuf.put(hdr) + } else { + t.controlBuf.put(cleanup) } - s.state = streamDone - s.mu.Unlock() } func (t *http2Server) RemoteAddr() net.Addr { @@ -1165,6 +1014,116 @@ func (t *http2Server) drain(code http2.ErrCode, debugData []byte) { t.controlBuf.put(&goAway{code: code, debugData: debugData, headsUp: true}) } +var goAwayPing = &ping{data: [8]byte{1, 6, 1, 8, 0, 3, 3, 9}} + +// Handles outgoing GoAway and returns true if loopy needs to put itself +// in draining mode. +func (t *http2Server) outgoingGoAwayHandler(g *goAway) (bool, error) { + t.mu.Lock() + if t.state == closing { // TODO(mmukhi): This seems unnecessary. + t.mu.Unlock() + // The transport is closing. + return false, ErrConnClosing + } + sid := t.maxStreamID + if !g.headsUp { + // Stop accepting more streams now. + t.state = draining + if len(t.activeStreams) == 0 { + g.closeConn = true + } + t.mu.Unlock() + if err := t.framer.fr.WriteGoAway(sid, g.code, g.debugData); err != nil { + return false, err + } + if g.closeConn { + // Abruptly close the connection following the GoAway (via + // loopywriter). But flush out what's inside the buffer first. + t.framer.writer.Flush() + return false, fmt.Errorf("transport: Connection closing") + } + return true, 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 false, err + } + if err := t.framer.fr.WritePing(false, goAwayPing.data); err != nil { + return false, err + } + go func() { + timer := time.NewTimer(time.Minute) + defer timer.Stop() + select { + case <-t.drainChan: + case <-timer.C: + case <-t.ctx.Done(): + return + } + t.controlBuf.put(&goAway{code: g.code, debugData: g.debugData}) + }() + return false, nil +} + +func (t *http2Server) ChannelzMetric() *channelz.SocketInternalMetric { + t.czmu.RLock() + s := channelz.SocketInternalMetric{ + StreamsStarted: t.streamsStarted, + StreamsSucceeded: t.streamsSucceeded, + StreamsFailed: t.streamsFailed, + MessagesSent: t.msgSent, + MessagesReceived: t.msgRecv, + KeepAlivesSent: t.kpCount, + LastRemoteStreamCreatedTimestamp: t.lastStreamCreated, + LastMessageSentTimestamp: t.lastMsgSent, + LastMessageReceivedTimestamp: t.lastMsgRecv, + LocalFlowControlWindow: int64(t.fc.getSize()), + //socket options + LocalAddr: t.localAddr, + RemoteAddr: t.remoteAddr, + // Security + // RemoteName : + } + t.czmu.RUnlock() + s.RemoteFlowControlWindow = t.getOutFlowWindow() + return &s +} + +func (t *http2Server) IncrMsgSent() { + t.czmu.Lock() + t.msgSent++ + t.lastMsgSent = time.Now() + t.czmu.Unlock() +} + +func (t *http2Server) IncrMsgRecv() { + t.czmu.Lock() + t.msgRecv++ + t.lastMsgRecv = time.Now() + t.czmu.Unlock() +} + +func (t *http2Server) getOutFlowWindow() int64 { + resp := make(chan uint32) + timer := time.NewTimer(time.Second) + defer timer.Stop() + t.controlBuf.put(&outFlowControlSizeRequest{resp}) + select { + case sz := <-resp: + return int64(sz) + case <-t.ctxDone: + return -1 + case <-timer.C: + return -2 + } +} + var rgen = rand.New(rand.NewSource(time.Now().UnixNano())) func getJitter(v time.Duration) time.Duration { diff --git a/vendor/google.golang.org/grpc/transport/http_util.go b/vendor/google.golang.org/grpc/transport/http_util.go index 39f878cfd5b..835c8126946 100644 --- a/vendor/google.golang.org/grpc/transport/http_util.go +++ b/vendor/google.golang.org/grpc/transport/http_util.go @@ -23,7 +23,6 @@ import ( "bytes" "encoding/base64" "fmt" - "io" "net" "net/http" "strconv" @@ -46,6 +45,12 @@ const ( // http2IOBufSize specifies the buffer size for sending frames. defaultWriteBufSize = 32 * 1024 defaultReadBufSize = 32 * 1024 + // baseContentType is the base content-type for gRPC. This is a valid + // content-type on it's own, but can also include a content-subtype such as + // "proto" as a suffix after "+" or ";". See + // https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#requests + // for more details. + baseContentType = "application/grpc" ) var ( @@ -64,7 +69,7 @@ var ( http2.ErrCodeConnect: codes.Internal, http2.ErrCodeEnhanceYourCalm: codes.ResourceExhausted, http2.ErrCodeInadequateSecurity: codes.PermissionDenied, - http2.ErrCodeHTTP11Required: codes.FailedPrecondition, + http2.ErrCodeHTTP11Required: codes.Internal, } statusCodeConvTab = map[codes.Code]http2.ErrCode{ codes.Internal: http2.ErrCodeInternal, @@ -111,9 +116,10 @@ type decodeState struct { timeout time.Duration method string // key-value metadata map from the peer. - mdata map[string][]string - statsTags []byte - statsTrace []byte + mdata map[string][]string + statsTags []byte + statsTrace []byte + contentSubtype string } // isReservedHeader checks whether hdr belongs to HTTP2 headers @@ -125,6 +131,7 @@ func isReservedHeader(hdr string) bool { } switch hdr { case "content-type", + "user-agent", "grpc-message-type", "grpc-encoding", "grpc-message", @@ -138,28 +145,55 @@ func isReservedHeader(hdr string) bool { } } -// isWhitelistedPseudoHeader checks whether hdr belongs to HTTP2 pseudoheaders -// that should be propagated into metadata visible to users. -func isWhitelistedPseudoHeader(hdr string) bool { +// isWhitelistedHeader checks whether hdr should be propagated +// into metadata visible to users. +func isWhitelistedHeader(hdr string) bool { switch hdr { - case ":authority": + case ":authority", "user-agent": return true default: return false } } -func validContentType(t string) bool { - e := "application/grpc" - if !strings.HasPrefix(t, e) { - return false +// contentSubtype returns the content-subtype for the given content-type. The +// given content-type must be a valid content-type that starts with +// "application/grpc". A content-subtype will follow "application/grpc" after a +// "+" or ";". See +// https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#requests for +// more details. +// +// If contentType is not a valid content-type for gRPC, the boolean +// will be false, otherwise true. If content-type == "application/grpc", +// "application/grpc+", or "application/grpc;", the boolean will be true, +// but no content-subtype will be returned. +// +// contentType is assumed to be lowercase already. +func contentSubtype(contentType string) (string, bool) { + if contentType == baseContentType { + return "", true + } + if !strings.HasPrefix(contentType, baseContentType) { + return "", false + } + // guaranteed since != baseContentType and has baseContentType prefix + switch contentType[len(baseContentType)] { + case '+', ';': + // this will return true for "application/grpc+" or "application/grpc;" + // which the previous validContentType function tested to be valid, so we + // just say that no content-subtype is specified in this case + return contentType[len(baseContentType)+1:], true + default: + return "", false } - // Support variations on the content-type - // (e.g. "application/grpc+blah", "application/grpc;blah"). - if len(t) > len(e) && t[len(e)] != '+' && t[len(e)] != ';' { - return false +} + +// contentSubtype is assumed to be lowercase +func contentType(contentSubtype string) string { + if contentSubtype == "" { + return baseContentType } - return true + return baseContentType + "+" + contentSubtype } func (d *decodeState) status() *status.Status { @@ -228,9 +262,9 @@ func (d *decodeState) decodeResponseHeader(frame *http2.MetaHeadersFrame) error // gRPC status doesn't exist and http status is OK. // Set rawStatusCode to be unknown and return nil error. // So that, if the stream has ended this Unknown status - // will be propogated to the user. + // will be propagated to the user. // Otherwise, it will be ignored. In which case, status from - // a later trailer, that has StreamEnded flag set, is propogated. + // a later trailer, that has StreamEnded flag set, is propagated. code := int(codes.Unknown) d.rawStatusCode = &code return nil @@ -247,9 +281,16 @@ func (d *decodeState) addMetadata(k, v string) { func (d *decodeState) processHeaderField(f hpack.HeaderField) error { switch f.Name { case "content-type": - if !validContentType(f.Value) { - return streamErrorf(codes.FailedPrecondition, "transport: received the unexpected content-type %q", f.Value) + contentSubtype, validContentType := contentSubtype(f.Value) + if !validContentType { + return streamErrorf(codes.Internal, "transport: received the unexpected content-type %q", f.Value) } + d.contentSubtype = contentSubtype + // TODO: do we want to propagate the whole content-type in the metadata, + // or come up with a way to just propagate the content-subtype if it was set? + // ie {"content-type": "application/grpc+proto"} or {"content-subtype": "proto"} + // in the metadata? + d.addMetadata(f.Name, f.Value) case "grpc-encoding": d.encoding = f.Value case "grpc-status": @@ -299,7 +340,7 @@ func (d *decodeState) processHeaderField(f hpack.HeaderField) error { d.statsTrace = v d.addMetadata(f.Name, string(v)) default: - if isReservedHeader(f.Name) && !isWhitelistedPseudoHeader(f.Name) { + if isReservedHeader(f.Name) && !isWhitelistedHeader(f.Name) { break } v, err := decodeMetadataHeader(f.Name, f.Value) @@ -307,7 +348,7 @@ func (d *decodeState) processHeaderField(f hpack.HeaderField) error { errorf("Failed to decode metadata header (%q, %q): %v", f.Name, f.Value, err) return nil } - d.addMetadata(f.Name, string(v)) + d.addMetadata(f.Name, v) } return nil } @@ -468,19 +509,67 @@ func decodeGrpcMessageUnchecked(msg string) string { return buf.String() } +type bufWriter struct { + buf []byte + offset int + batchSize int + conn net.Conn + err error + + onFlush func() +} + +func newBufWriter(conn net.Conn, batchSize int) *bufWriter { + return &bufWriter{ + buf: make([]byte, batchSize*2), + batchSize: batchSize, + conn: conn, + } +} + +func (w *bufWriter) Write(b []byte) (n int, err error) { + if w.err != nil { + return 0, w.err + } + for len(b) > 0 { + nn := copy(w.buf[w.offset:], b) + b = b[nn:] + w.offset += nn + n += nn + if w.offset >= w.batchSize { + err = w.Flush() + } + } + return n, err +} + +func (w *bufWriter) Flush() error { + if w.err != nil { + return w.err + } + if w.offset == 0 { + return nil + } + if w.onFlush != nil { + w.onFlush() + } + _, w.err = w.conn.Write(w.buf[:w.offset]) + w.offset = 0 + return w.err +} + type framer struct { - numWriters int32 - reader io.Reader - writer *bufio.Writer - fr *http2.Framer + writer *bufWriter + fr *http2.Framer } func newFramer(conn net.Conn, writeBufferSize, readBufferSize int) *framer { + r := bufio.NewReaderSize(conn, readBufferSize) + w := newBufWriter(conn, writeBufferSize) f := &framer{ - reader: bufio.NewReaderSize(conn, readBufferSize), - writer: bufio.NewWriterSize(conn, writeBufferSize), + writer: w, + fr: http2.NewFramer(w, r), } - f.fr = http2.NewFramer(f.writer, f.reader) // Opt-in to Frame reuse API on framer to reduce garbage. // Frames aren't safe to read from after a subsequent call to ReadFrame. f.fr.SetReuseFrames() diff --git a/vendor/google.golang.org/grpc/transport/transport.go b/vendor/google.golang.org/grpc/transport/transport.go index bde8fa5c3a2..f51f878884d 100644 --- a/vendor/google.golang.org/grpc/transport/transport.go +++ b/vendor/google.golang.org/grpc/transport/transport.go @@ -17,19 +17,19 @@ */ // Package transport defines and implements message oriented communication -// channel to complete various transactions (e.g., an RPC). -package transport // import "google.golang.org/grpc/transport" +// channel to complete various transactions (e.g., an RPC). It is meant for +// grpc-internal usage and is not intended to be imported directly by users. +package transport // externally used as import "google.golang.org/grpc/transport" import ( - stdctx "context" + "errors" "fmt" "io" "net" "sync" - "time" + "sync/atomic" "golang.org/x/net/context" - "golang.org/x/net/http2" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" "google.golang.org/grpc/keepalive" @@ -58,6 +58,7 @@ type recvBuffer struct { c chan recvMsg mu sync.Mutex backlog []recvMsg + err error } func newRecvBuffer() *recvBuffer { @@ -69,6 +70,13 @@ func newRecvBuffer() *recvBuffer { func (b *recvBuffer) put(r recvMsg) { b.mu.Lock() + if b.err != nil { + b.mu.Unlock() + // An error had occurred earlier, don't accept more + // data or errors. + return + } + b.err = r.err if len(b.backlog) == 0 { select { case b.c <- r: @@ -102,14 +110,15 @@ func (b *recvBuffer) get() <-chan recvMsg { return b.c } +// // recvBufferReader implements io.Reader interface to read the data from // recvBuffer. type recvBufferReader struct { - ctx context.Context - goAway chan struct{} - recv *recvBuffer - last []byte // Stores the remaining data in the previous calls. - err error + ctx context.Context + ctxDone <-chan struct{} // cache of ctx.Done() (for performance). + recv *recvBuffer + last []byte // Stores the remaining data in the previous calls. + err error } // Read reads the next len(p) bytes from last. If last is drained, it tries to @@ -131,10 +140,8 @@ func (r *recvBufferReader) read(p []byte) (n int, err error) { return copied, nil } select { - case <-r.ctx.Done(): + case <-r.ctxDone: return 0, ContextErr(r.ctx.Err()) - case <-r.goAway: - return 0, ErrStreamDrain case m := <-r.recv.get(): r.recv.load() if m.err != nil { @@ -146,61 +153,7 @@ func (r *recvBufferReader) read(p []byte) (n int, err error) { } } -// All items in an out of a controlBuffer should be the same type. -type item interface { - item() -} - -// controlBuffer is an unbounded channel of item. -type controlBuffer struct { - c chan item - mu sync.Mutex - backlog []item -} - -func newControlBuffer() *controlBuffer { - b := &controlBuffer{ - c: make(chan item, 1), - } - return b -} - -func (b *controlBuffer) put(r item) { - b.mu.Lock() - 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() - if len(b.backlog) > 0 { - select { - case b.c <- b.backlog[0]: - b.backlog[0] = nil - b.backlog = b.backlog[1:] - default: - } - } - b.mu.Unlock() -} - -// get returns the channel that receives an item in the buffer. -// -// Upon receipt of an item, the caller should call load to send another -// item onto the channel if there is any. -func (b *controlBuffer) get() <-chan item { - return b.c -} - -type streamState uint8 +type streamState uint32 const ( streamActive streamState = iota @@ -211,66 +164,93 @@ const ( // Stream represents an RPC in the transport layer. type Stream struct { - id uint32 - // nil for client side Stream. - st ServerTransport - // ctx is the associated context of the stream. - ctx context.Context - // cancel is always nil for client side Stream. - cancel context.CancelFunc - // done is closed when the final status arrives. - done chan struct{} - // goAway is closed when the server sent GoAways signal before this stream was initiated. - goAway chan struct{} - // method records the associated RPC method of the stream. - method string + id uint32 + st ServerTransport // nil for client side Stream + ctx context.Context // the associated context of the stream + cancel context.CancelFunc // always nil for client side Stream + done chan struct{} // closed at the end of stream to unblock writers. On the client side. + ctxDone <-chan struct{} // same as done chan but for server side. Cache of ctx.Done() (for performance) + method string // the associated RPC method of the stream recvCompress string sendCompress string buf *recvBuffer trReader io.Reader fc *inFlow recvQuota uint32 - - // TODO: Remote this unused variable. - // The accumulated inbound quota pending for window update. - updateQuota uint32 + wq *writeQuota // Callback to state application's intentions to read data. This - // is used to adjust flow control, if need be. + // is used to adjust flow control, if needed. requestRead func(int) - sendQuotaPool *quotaPool - localSendQuota *quotaPool - // Close headerChan to indicate the end of reception of header metadata. - headerChan chan struct{} - // header caches the received header metadata. - header metadata.MD - // The key-value map of trailer metadata. - trailer metadata.MD - - mu sync.RWMutex // guard the following - // headerOK becomes true from the first header is about to send. - headerOk bool - state streamState - // true iff headerChan is closed. Used to avoid closing headerChan - // multiple times. - headerDone bool - // the status error received from the server. + headerChan chan struct{} // closed to indicate the end of header metadata. + headerDone uint32 // set when headerChan is closed. Used to avoid closing headerChan multiple times. + + // hdrMu protects header and trailer metadata on the server-side. + hdrMu sync.Mutex + header metadata.MD // the received header metadata. + trailer metadata.MD // the key-value map of trailer metadata. + + // On the server-side, headerSent is atomically set to 1 when the headers are sent out. + headerSent uint32 + + state streamState + + // On client-side it is the status error received from the server. + // On server-side it is unused. status *status.Status - // rstStream indicates whether a RST_STREAM frame needs to be sent - // to the server to signify that this stream is closing. - rstStream bool - // rstError is the error that needs to be sent along with the RST_STREAM frame. - rstError http2.ErrCode - // bytesSent and bytesReceived indicates whether any bytes have been sent or - // received on this stream. - bytesSent bool - bytesReceived bool + + bytesReceived uint32 // indicates whether any bytes have been received on this stream + unprocessed uint32 // set if the server sends a refused stream or GOAWAY including this stream + + // contentSubtype is the content-subtype for requests. + // this must be lowercase or the behavior is undefined. + contentSubtype string +} + +// isHeaderSent is only valid on the server-side. +func (s *Stream) isHeaderSent() bool { + return atomic.LoadUint32(&s.headerSent) == 1 +} + +// updateHeaderSent updates headerSent and returns true +// if it was alreay set. It is valid only on server-side. +func (s *Stream) updateHeaderSent() bool { + return atomic.SwapUint32(&s.headerSent, 1) == 1 +} + +func (s *Stream) swapState(st streamState) streamState { + return streamState(atomic.SwapUint32((*uint32)(&s.state), uint32(st))) +} + +func (s *Stream) compareAndSwapState(oldState, newState streamState) bool { + return atomic.CompareAndSwapUint32((*uint32)(&s.state), uint32(oldState), uint32(newState)) +} + +func (s *Stream) getState() streamState { + return streamState(atomic.LoadUint32((*uint32)(&s.state))) +} + +func (s *Stream) waitOnHeader() error { + if s.headerChan == nil { + // On the server headerChan is always nil since a stream originates + // only after having received headers. + return nil + } + select { + case <-s.ctx.Done(): + return ContextErr(s.ctx.Err()) + case <-s.headerChan: + return nil + } } // RecvCompress returns the compression algorithm applied to the inbound // message. It is empty string if there is no compression applied. func (s *Stream) RecvCompress() string { + if err := s.waitOnHeader(); err != nil { + return "" + } return s.recvCompress } @@ -285,28 +265,17 @@ func (s *Stream) Done() <-chan struct{} { return s.done } -// GoAway returns a channel which is closed when the server sent GoAways signal -// before this stream was initiated. -func (s *Stream) GoAway() <-chan struct{} { - return s.goAway -} - // Header acquires the key-value pairs of header metadata once it // is available. It blocks until i) the metadata is ready or ii) there is no // header metadata or iii) the stream is canceled/expired. func (s *Stream) Header() (metadata.MD, error) { - var err error - select { - case <-s.ctx.Done(): - err = ContextErr(s.ctx.Err()) - case <-s.goAway: - err = ErrStreamDrain - case <-s.headerChan: - return s.header.Copy(), nil - } + err := s.waitOnHeader() // Even if the stream is closed, header is returned if available. select { case <-s.headerChan: + if s.header == nil { + return nil, nil + } return s.header.Copy(), nil default: } @@ -316,10 +285,10 @@ func (s *Stream) Header() (metadata.MD, error) { // Trailer returns the cached trailer metedata. Note that if it is not called // after the entire stream is done, it could return an empty MD. Client // side only. +// It can be safely read only after stream has ended that is either read +// or write have returned io.EOF. func (s *Stream) Trailer() metadata.MD { - s.mu.RLock() c := s.trailer.Copy() - s.mu.RUnlock() return c } @@ -329,6 +298,15 @@ func (s *Stream) ServerTransport() ServerTransport { return s.st } +// ContentSubtype returns the content-subtype for a request. For example, a +// content-subtype of "proto" will result in a content-type of +// "application/grpc+proto". This will always be lowercase. See +// https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#requests for +// more details. +func (s *Stream) ContentSubtype() string { + return s.contentSubtype +} + // Context returns the context of the stream. func (s *Stream) Context() context.Context { return s.ctx @@ -340,36 +318,49 @@ func (s *Stream) Method() string { } // Status returns the status received from the server. +// Status can be read safely only after the stream has ended, +// that is, read or write has returned io.EOF. func (s *Stream) Status() *status.Status { return s.status } // SetHeader sets the header metadata. This can be called multiple times. // Server side only. +// This should not be called in parallel to other data writes. func (s *Stream) SetHeader(md metadata.MD) error { - s.mu.Lock() - if s.headerOk || s.state == streamDone { - s.mu.Unlock() - return ErrIllegalHeaderWrite - } if md.Len() == 0 { - s.mu.Unlock() return nil } + if s.isHeaderSent() || s.getState() == streamDone { + return ErrIllegalHeaderWrite + } + s.hdrMu.Lock() s.header = metadata.Join(s.header, md) - s.mu.Unlock() + s.hdrMu.Unlock() return nil } +// SendHeader sends the given header metadata. The given metadata is +// combined with any metadata set by previous calls to SetHeader and +// then written to the transport stream. +func (s *Stream) SendHeader(md metadata.MD) error { + t := s.ServerTransport() + return t.WriteHeader(s, md) +} + // SetTrailer sets the trailer metadata which will be sent with the RPC status // by the server. This can be called multiple times. Server side only. +// This should not be called parallel to other data writes. func (s *Stream) SetTrailer(md metadata.MD) error { if md.Len() == 0 { return nil } - s.mu.Lock() + if s.getState() == streamDone { + return ErrIllegalHeaderWrite + } + s.hdrMu.Lock() s.trailer = metadata.Join(s.trailer, md) - s.mu.Unlock() + s.hdrMu.Unlock() return nil } @@ -409,28 +400,15 @@ func (t *transportReader) Read(p []byte) (n int, err error) { return } -// finish sets the stream's state and status, and closes the done channel. -// s.mu must be held by the caller. st must always be non-nil. -func (s *Stream) finish(st *status.Status) { - s.status = st - s.state = streamDone - close(s.done) -} - -// BytesSent indicates whether any bytes have been sent on this stream. -func (s *Stream) BytesSent() bool { - s.mu.Lock() - 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() - br := s.bytesReceived - s.mu.Unlock() - return br + return atomic.LoadUint32(&s.bytesReceived) == 1 +} + +// Unprocessed indicates whether the server did not process this stream -- +// i.e. it sent a refused stream or GOAWAY including this stream ID. +func (s *Stream) Unprocessed() bool { + return atomic.LoadUint32(&s.unprocessed) == 1 } // GoString is implemented by Stream so context.String() won't @@ -439,21 +417,6 @@ func (s *Stream) GoString() string { return fmt.Sprintf("", s, s.method) } -// The key to save transport.Stream in the context. -type streamKey struct{} - -// newContextWithStream creates a new context from ctx and attaches stream -// to it. -func newContextWithStream(ctx context.Context, stream *Stream) context.Context { - return context.WithValue(ctx, streamKey{}, stream) -} - -// StreamFromContext returns the stream saved in ctx. -func StreamFromContext(ctx context.Context) (s *Stream, ok bool) { - s, ok = ctx.Value(streamKey{}).(*Stream) - return -} - // state of transport type transportState int @@ -475,6 +438,7 @@ type ServerConfig struct { InitialConnWindowSize int32 WriteBufferSize int ReadBufferSize int + ChannelzParentID int64 } // NewServerTransport creates a ServerTransport with conn or non-nil error @@ -510,18 +474,21 @@ type ConnectOptions struct { 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 + // ChannelzParentID sets the addrConn id which initiate the creation of this client transport. + ChannelzParentID int64 } // TargetInfo contains the information of the target such as network address and metadata. type TargetInfo struct { - Addr string - Metadata interface{} + Addr string + Metadata interface{} + Authority string } // NewClientTransport establishes the transport with the required ConnectOptions // and returns it to the caller. -func NewClientTransport(ctx context.Context, target TargetInfo, opts ConnectOptions, timeout time.Duration) (ClientTransport, error) { - return newHTTP2Client(ctx, target, opts, timeout) +func NewClientTransport(connectCtx, ctx context.Context, target TargetInfo, opts ConnectOptions, onSuccess func()) (ClientTransport, error) { + return newHTTP2Client(connectCtx, ctx, target, opts, onSuccess) } // Options provides additional hints and information for message @@ -545,10 +512,6 @@ type CallHdr struct { // Method specifies the operation to perform. Method string - // RecvCompress specifies the compression algorithm applied on - // inbound messages. - RecvCompress string - // SendCompress specifies the compression algorithm applied on // outbound message. SendCompress string @@ -563,6 +526,14 @@ type CallHdr struct { // for performance purposes. // If it's false, new stream will never be flushed. Flush bool + + // ContentSubtype specifies the content-subtype for a request. For example, a + // content-subtype of "proto" will result in a content-type of + // "application/grpc+proto". The value of ContentSubtype must be all + // lowercase, otherwise the behavior is undefined. See + // https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#requests + // for more details. + ContentSubtype string } // ClientTransport is the common interface for all gRPC client-side transport @@ -604,6 +575,12 @@ type ClientTransport interface { // GetGoAwayReason returns the reason why GoAway frame was received. GetGoAwayReason() GoAwayReason + + // IncrMsgSent increments the number of message sent through this transport. + IncrMsgSent() + + // IncrMsgRecv increments the number of message received through this transport. + IncrMsgRecv() } // ServerTransport is the common interface for all gRPC server-side transport @@ -637,6 +614,12 @@ type ServerTransport interface { // Drain notifies the client this ServerTransport stops accepting new RPCs. Drain() + + // IncrMsgSent increments the number of message sent through this transport. + IncrMsgSent() + + // IncrMsgRecv increments the number of message received through this transport. + IncrMsgRecv() } // streamErrorf creates an StreamError with the specified error code and description. @@ -686,9 +669,16 @@ func (e ConnectionError) Origin() error { var ( // ErrConnClosing indicates that the transport is closing. ErrConnClosing = connectionErrorf(true, nil, "transport is closing") - // ErrStreamDrain indicates that the stream is rejected by the server because - // the server stops accepting new RPCs. - ErrStreamDrain = streamErrorf(codes.Unavailable, "the server stops accepting new RPCs") + // errStreamDrain indicates that the stream is rejected because the + // connection is draining. This could be caused by goaway or balancer + // removing the address. + errStreamDrain = streamErrorf(codes.Unavailable, "the connection is draining") + // errStreamDone is returned from write at the client side to indiacte application + // layer of an error. + errStreamDone = errors.New("the stream is done") + // StatusGoAway indicates that the server sent a GOAWAY that included this + // stream's ID in unprocessed RPCs. + statusGoAway = status.New(codes.Unavailable, "the stream is rejected because server is draining the connection") ) // TODO: See if we can replace StreamError with status package errors. @@ -703,75 +693,16 @@ 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 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: - return 0, io.EOF - case <-goAway: - return 0, ErrStreamDrain - 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 const ( - // Invalid indicates that no GoAway frame is received. - Invalid GoAwayReason = 0 - // NoReason is the default value when GoAway frame is received. - NoReason GoAwayReason = 1 - // TooManyPings indicates that a GoAway frame with ErrCodeEnhanceYourCalm - // was received and that the debug data said "too_many_pings". - TooManyPings GoAwayReason = 2 + // GoAwayInvalid indicates that no GoAway frame is received. + GoAwayInvalid GoAwayReason = 0 + // GoAwayNoReason is the default value when GoAway frame is received. + GoAwayNoReason GoAwayReason = 1 + // GoAwayTooManyPings indicates that a GoAway frame with + // ErrCodeEnhanceYourCalm was received and that the debug data said + // "too_many_pings". + GoAwayTooManyPings 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 - } - } - } -}