Browse Source

vendor: upgrade grpc/grpc-go to v1.7.3

Signed-off-by: Gyu-Ho Lee <gyuhox@gmail.com>
Gyu-Ho Lee 8 years ago
parent
commit
f0497de216

+ 4 - 0
cmd/vendor/google.golang.org/grpc/balancer_conn_wrappers.go

@@ -171,7 +171,9 @@ func (ccb *ccBalancerWrapper) NewSubConn(addrs []resolver.Address, opts balancer
 		return nil, err
 	}
 	acbw := &acBalancerWrapper{ac: ac}
+	ac.mu.Lock()
 	ac.acbw = acbw
+	ac.mu.Unlock()
 	return acbw, nil
 }
 
@@ -228,7 +230,9 @@ func (acbw *acBalancerWrapper) UpdateAddresses(addrs []resolver.Address) {
 			return
 		}
 		acbw.ac = ac
+		ac.mu.Lock()
 		ac.acbw = acbw
+		ac.mu.Unlock()
 		if acState != connectivity.Idle {
 			ac.connect(false)
 		}

+ 10 - 0
cmd/vendor/google.golang.org/grpc/clientconn.go

@@ -929,6 +929,16 @@ func (ac *addrConn) resetTransport() error {
 			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)

+ 4 - 0
cmd/vendor/google.golang.org/grpc/credentials/credentials.go

@@ -91,10 +91,14 @@ type TransportCredentials interface {
 	// (io.EOF, context.DeadlineExceeded or err.Temporary() == true).
 	// If the returned error is a wrapper error, implementations should make sure that
 	// the error implements Temporary() to have the correct retry behaviors.
+	//
+	// If the returned net.Conn is closed, it MUST close the net.Conn provided.
 	ClientHandshake(context.Context, string, net.Conn) (net.Conn, AuthInfo, error)
 	// ServerHandshake does the authentication handshake for servers. It returns
 	// the authenticated connection and the corresponding auth information about
 	// the connection.
+	//
+	// If the returned net.Conn is closed, it MUST close the net.Conn provided.
 	ServerHandshake(net.Conn) (net.Conn, AuthInfo, error)
 	// Info provides the ProtocolInfo of this TransportCredentials.
 	Info() ProtocolInfo

+ 1 - 1
cmd/vendor/google.golang.org/grpc/rpc_util.go

@@ -567,6 +567,6 @@ const SupportPackageIsVersion3 = true
 const SupportPackageIsVersion4 = true
 
 // Version is the current grpc version.
-const Version = "1.7.2"
+const Version = "1.7.3"
 
 const grpcUA = "grpc-go/" + Version

+ 28 - 11
cmd/vendor/google.golang.org/grpc/server.go

@@ -118,11 +118,13 @@ type options struct {
 	initialConnWindowSize int32
 	writeBufferSize       int
 	readBufferSize        int
+	connectionTimeout     time.Duration
 }
 
 var defaultServerOptions = options{
 	maxReceiveMessageSize: defaultServerMaxReceiveMessageSize,
 	maxSendMessageSize:    defaultServerMaxSendMessageSize,
+	connectionTimeout:     120 * time.Second,
 }
 
 // A ServerOption sets options such as credentials, codec and keepalive parameters, etc.
@@ -291,6 +293,16 @@ func UnknownServiceHandler(streamHandler StreamHandler) ServerOption {
 	}
 }
 
+// ConnectionTimeout returns a ServerOption that sets the timeout for
+// 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.
+func ConnectionTimeout(d time.Duration) ServerOption {
+	return func(o *options) {
+		o.connectionTimeout = d
+	}
+}
+
 // NewServer creates a gRPC server which has no service registered and has not
 // started to accept requests yet.
 func NewServer(opt ...ServerOption) *Server {
@@ -499,16 +511,18 @@ func (s *Server) Serve(lis net.Listener) error {
 // handleRawConn is run in its own goroutine and handles 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)
 	if err != nil {
 		s.mu.Lock()
 		s.errorf("ServerHandshake(%q) failed: %v", rawConn.RemoteAddr(), err)
 		s.mu.Unlock()
 		grpclog.Warningf("grpc: Server.Serve failed to complete security handshake from %q: %v", rawConn.RemoteAddr(), err)
-		// If serverHandShake returns ErrConnDispatched, keep rawConn open.
+		// If serverHandshake returns ErrConnDispatched, keep rawConn open.
 		if err != credentials.ErrConnDispatched {
 			rawConn.Close()
 		}
+		rawConn.SetDeadline(time.Time{})
 		return
 	}
 
@@ -521,18 +535,21 @@ func (s *Server) handleRawConn(rawConn net.Conn) {
 	s.mu.Unlock()
 
 	if s.opts.useHandlerImpl {
+		rawConn.SetDeadline(time.Time{})
 		s.serveUsingHandler(conn)
 	} else {
-		s.serveHTTP2Transport(conn, authInfo)
+		st := s.newHTTP2Transport(conn, authInfo)
+		if st == nil {
+			return
+		}
+		rawConn.SetDeadline(time.Time{})
+		s.serveStreams(st)
 	}
 }
 
-// serveHTTP2Transport sets up a http/2 transport (using the
-// gRPC http2 server transport in transport/http2_server.go) and
-// serves streams on it.
-// This is run in its own goroutine (it does network I/O in
-// transport.NewServerTransport).
-func (s *Server) serveHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo) {
+// newHTTP2Transport sets up a http/2 transport (using the
+// gRPC http2 server transport in transport/http2_server.go).
+func (s *Server) newHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo) transport.ServerTransport {
 	config := &transport.ServerConfig{
 		MaxStreams:            s.opts.maxConcurrentStreams,
 		AuthInfo:              authInfo,
@@ -552,13 +569,13 @@ func (s *Server) serveHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo)
 		s.mu.Unlock()
 		c.Close()
 		grpclog.Warningln("grpc: Server.Serve failed to create ServerTransport: ", err)
-		return
+		return nil
 	}
 	if !s.addConn(st) {
 		st.Close()
-		return
+		return nil
 	}
-	s.serveStreams(st)
+	return st
 }
 
 func (s *Server) serveStreams(st transport.ServerTransport) {

+ 27 - 37
cmd/vendor/google.golang.org/grpc/transport/http2_server.go

@@ -152,12 +152,12 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
 			Val: uint32(iwz)})
 	}
 	if err := framer.fr.WriteSettings(isettings...); err != nil {
-		return nil, connectionErrorf(true, err, "transport: %v", err)
+		return nil, connectionErrorf(false, err, "transport: %v", err)
 	}
 	// Adjust the connection flow control window if needed.
 	if delta := uint32(icwz - defaultWindowSize); delta > 0 {
 		if err := framer.fr.WriteWindowUpdate(0, delta); err != nil {
-			return nil, connectionErrorf(true, err, "transport: %v", err)
+			return nil, connectionErrorf(false, err, "transport: %v", err)
 		}
 	}
 	kp := config.KeepaliveParams
@@ -223,6 +223,31 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
 		t.stats.HandleConn(t.ctx, connBegin)
 	}
 	t.framer.writer.Flush()
+
+	// Check the validity of client preface.
+	preface := make([]byte, len(clientPreface))
+	if _, err := io.ReadFull(t.conn, preface); err != nil {
+		return nil, connectionErrorf(false, err, "transport: http2Server.HandleStreams failed to receive the preface from client: %v", err)
+	}
+	if !bytes.Equal(preface, clientPreface) {
+		return nil, connectionErrorf(false, nil, "transport: http2Server.HandleStreams received bogus greeting from client: %q", preface)
+	}
+
+	frame, err := t.framer.fr.ReadFrame()
+	if err == io.EOF || err == io.ErrUnexpectedEOF {
+		t.Close()
+		return
+	}
+	if err != nil {
+		return nil, connectionErrorf(false, err, "transport: http2Server.HandleStreams failed to read initial settings frame: %v", err)
+	}
+	atomic.StoreUint32(&t.activity, 1)
+	sf, ok := frame.(*http2.SettingsFrame)
+	if !ok {
+		return nil, connectionErrorf(false, nil, "transport: http2Server.HandleStreams saw invalid preface type %T from client", frame)
+	}
+	t.handleSettings(sf)
+
 	go func() {
 		loopyWriter(t.ctx, t.controlBuf, t.itemHandler)
 		t.Close()
@@ -354,41 +379,6 @@ 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) {
-	// Check the validity of client preface.
-	preface := make([]byte, len(clientPreface))
-	if _, err := io.ReadFull(t.conn, preface); err != nil {
-		// Only log if it isn't a simple tcp accept check (ie: tcp balancer doing open/close socket)
-		if err != io.EOF {
-			errorf("transport: http2Server.HandleStreams failed to receive the preface from client: %v", err)
-		}
-		t.Close()
-		return
-	}
-	if !bytes.Equal(preface, clientPreface) {
-		errorf("transport: http2Server.HandleStreams received bogus greeting from client: %q", preface)
-		t.Close()
-		return
-	}
-
-	frame, err := t.framer.fr.ReadFrame()
-	if err == io.EOF || err == io.ErrUnexpectedEOF {
-		t.Close()
-		return
-	}
-	if err != nil {
-		errorf("transport: http2Server.HandleStreams failed to read initial settings frame: %v", err)
-		t.Close()
-		return
-	}
-	atomic.StoreUint32(&t.activity, 1)
-	sf, ok := frame.(*http2.SettingsFrame)
-	if !ok {
-		errorf("transport: http2Server.HandleStreams saw invalid preface type %T from client", frame)
-		t.Close()
-		return
-	}
-	t.handleSettings(sf)
-
 	for {
 		frame, err := t.framer.fr.ReadFrame()
 		atomic.StoreUint32(&t.activity, 1)

+ 3 - 3
glide.lock

@@ -1,5 +1,5 @@
-hash: 8d556efcf8d917aba74445bd98eb9de3abcf981ca70adb53276026c4c3ecfcfd
-updated: 2017-11-10T12:07:01.187305-08:00
+hash: 0d3f3703d96e2e3e92b8488d93429062ad0c9d5034b12001e73746c7eee434a3
+updated: 2017-11-14T13:55:48.257041-08:00
 imports:
 - name: github.com/beorn7/perks
   version: 4c0e84591b9aa9e6dcfdf3e020114cd81f89d5f9
@@ -158,7 +158,7 @@ imports:
   subpackages:
   - googleapis/rpc/status
 - name: google.golang.org/grpc
-  version: 5ffe3083946d5603a0578721101dc8165b1d5b5f
+  version: 401e0e00e4bb830a10496d64cd95e068c5bf50de
   subpackages:
   - balancer
   - codes

+ 1 - 1
glide.yaml

@@ -108,7 +108,7 @@ import:
   subpackages:
   - rate
 - package: google.golang.org/grpc
-  version: v1.7.2
+  version: v1.7.3
   subpackages:
   - codes
   - credentials