From 50d4175072941195bca4f18fedd25d04132185e0 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Mon, 3 Apr 2017 15:03:24 -0700 Subject: [PATCH 01/23] Add client and service side apis for limiting the send/recv msg size. Update MethodConfig struct --- call.go | 55 +++- clientconn.go | 58 ++-- rpc_util.go | 24 +- server.go | 84 ++++-- stream.go | 113 ++++--- test/end2end_test.go | 684 ++++++++++++++++++++++++++++++++++++++++--- 6 files changed, 868 insertions(+), 150 deletions(-) diff --git a/call.go b/call.go index c1588c6375c7..e92a4bc958be 100644 --- a/call.go +++ b/call.go @@ -51,7 +51,7 @@ import ( // // 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) { +func recvResponse(ctx context.Context, dopts dialOptions, msgSizeLimit int, 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 { @@ -72,7 +72,7 @@ func recvResponse(ctx context.Context, dopts dialOptions, t transport.ClientTran } } for { - if err = recv(p, dopts.codec, stream, dopts.dc, reply, dopts.maxMsgSize, inPayload); err != nil { + if err = recv(p, dopts.codec, stream, dopts.dc, reply, msgSizeLimit, inPayload); err != nil { if err == io.EOF { break } @@ -92,7 +92,7 @@ func recvResponse(ctx context.Context, dopts dialOptions, t transport.ClientTran } // sendRequest writes out various information of an RPC such as Context and Message. -func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, callHdr *transport.CallHdr, t transport.ClientTransport, args interface{}, opts *transport.Options) (_ *transport.Stream, err error) { +func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, msgSizeLimit int, callHdr *transport.CallHdr, t transport.ClientTransport, args interface{}, opts *transport.Options) (_ *transport.Stream, err error) { stream, err := t.NewStream(ctx, callHdr) if err != nil { return nil, err @@ -121,6 +121,9 @@ func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, if err != nil { return nil, Errorf(codes.Internal, "grpc: %v", err) } + if len(outBuf) > msgSizeLimit { + return nil, Errorf(codes.InvalidArgument, "Sent message larger than max (%d vs. %d)", len(outBuf), msgSizeLimit) + } err = t.Write(stream, outBuf, opts) if err == nil && outPayload != nil { outPayload.SentTime = time.Now() @@ -146,15 +149,49 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli return invoke(ctx, method, args, reply, cc, opts...) } +const defaultClientMaxReceiveMessageSize = 1024 * 1024 * 4 +const defaultClientMaxSendMessageSize = 1024 * 1024 * 4 + +func min(a, b int) int { + if a < b { + return a + } + return b +} + func invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) (e error) { c := defaultCallInfo - if mc, ok := cc.getMethodConfig(method); ok { - c.failFast = !mc.WaitForReady - if mc.Timeout > 0 { + maxReceiveMessageSize := defaultClientMaxReceiveMessageSize + maxSendMessageSize := defaultClientMaxSendMessageSize + if mc, ok := cc.GetMethodConfig(method); ok { + 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) + ctx, cancel = context.WithTimeout(ctx, *mc.Timeout) defer cancel() } + + if mc.MaxReqSize != nil && cc.dopts.maxSendMessageSize >= 0 { + maxSendMessageSize = min(*mc.MaxReqSize, cc.dopts.maxSendMessageSize) + } else if mc.MaxReqSize != nil { + maxSendMessageSize = *mc.MaxReqSize + } + + if mc.MaxRespSize != nil && cc.dopts.maxReceiveMessageSize >= 0 { + maxReceiveMessageSize = min(*mc.MaxRespSize, cc.dopts.maxReceiveMessageSize) + } else if mc.MaxRespSize != nil { + maxReceiveMessageSize = *mc.MaxRespSize + } + } else { + if cc.dopts.maxSendMessageSize >= 0 { + maxSendMessageSize = cc.dopts.maxSendMessageSize + } + if cc.dopts.maxReceiveMessageSize >= 0 { + maxReceiveMessageSize = cc.dopts.maxReceiveMessageSize + } } for _, o := range opts { if err := o.before(&c); err != nil { @@ -245,7 +282,7 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli if c.traceInfo.tr != nil { c.traceInfo.tr.LazyLog(&payload{sent: true, msg: args}, true) } - stream, err = sendRequest(ctx, cc.dopts, cc.dopts.cp, callHdr, t, args, topts) + stream, err = sendRequest(ctx, cc.dopts, cc.dopts.cp, maxSendMessageSize, callHdr, t, args, topts) if err != nil { if put != nil { put() @@ -262,7 +299,7 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli } return toRPCErr(err) } - err = recvResponse(ctx, cc.dopts, t, &c, stream, reply) + err = recvResponse(ctx, cc.dopts, maxReceiveMessageSize, t, &c, stream, reply) if err != nil { if put != nil { put() diff --git a/clientconn.go b/clientconn.go index 1ba592c500cf..c257284fa546 100644 --- a/clientconn.go +++ b/clientconn.go @@ -88,19 +88,20 @@ var ( // dialOptions configure a Dial call. dialOptions are set by the DialOption // values passed to Dial. type dialOptions struct { - unaryInt UnaryClientInterceptor - streamInt StreamClientInterceptor - codec Codec - cp Compressor - dc Decompressor - bs backoffStrategy - balancer Balancer - block bool - insecure bool - timeout time.Duration - scChan <-chan ServiceConfig - copts transport.ConnectOptions - maxMsgSize int + unaryInt UnaryClientInterceptor + streamInt StreamClientInterceptor + codec Codec + cp Compressor + dc Decompressor + bs backoffStrategy + balancer Balancer + block bool + insecure bool + timeout time.Duration + scChan <-chan ServiceConfig + copts transport.ConnectOptions + maxReceiveMessageSize int + maxSendMessageSize int } const defaultClientMaxMsgSize = math.MaxInt32 @@ -108,10 +109,24 @@ const defaultClientMaxMsgSize = math.MaxInt32 // DialOption configures how we set up the connection. type DialOption func(*dialOptions) -// WithMaxMsgSize returns a DialOption which sets the maximum message size the client can receive. +// WithMaxMsgSize returns a DialOption which sets the maximum message size the client can receive. This function is for backward API compatibility. It has essentially the same functionality as WithMaxReceiveMessageSize. func WithMaxMsgSize(s int) DialOption { return func(o *dialOptions) { - o.maxMsgSize = s + o.maxReceiveMessageSize = s + } +} + +// WithMaxReceiveMessageSize returns a DialOption which sets the maximum message size the client can receive. Negative input is invalid and has the same effect as not setting the field. +func WithMaxReceiveMessageSize(s int) DialOption { + return func(o *dialOptions) { + o.maxReceiveMessageSize = s + } +} + +// WithMaxSendMessageSize returns a DialOption which sets the maximum message size the client can send. Negative input is invalid and has the same effect as not seeting the field. +func WithMaxSendMessageSize(s int) DialOption { + return func(o *dialOptions) { + o.maxSendMessageSize = s } } @@ -307,7 +322,11 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * conns: make(map[Address]*addrConn), } cc.ctx, cc.cancel = context.WithCancel(context.Background()) - cc.dopts.maxMsgSize = defaultClientMaxMsgSize + + // initialize maxReceiveMessageSize and maxSendMessageSize to -1 before applying DialOption functions to distinguish whether the user set the message limit or not. + cc.dopts.maxReceiveMessageSize = -1 + cc.dopts.maxSendMessageSize = -1 + for _, opt := range opts { opt(&cc.dopts) } @@ -609,11 +628,16 @@ func (cc *ClientConn) resetAddrConn(addr Address, skipWait bool, tearDownErr err return nil } +// GetMethodConfig gets the method config of the input method. If there's no exact match for the input method (i.e. /service/method), we will return the default config for all methods under the service (/service/). // TODO: Avoid the locking here. -func (cc *ClientConn) getMethodConfig(method string) (m MethodConfig, ok bool) { +func (cc *ClientConn) GetMethodConfig(method string) (m MethodConfig, ok bool) { cc.mu.RLock() defer cc.mu.RUnlock() m, ok = cc.sc.Methods[method] + if !ok { + i := strings.LastIndex(method, "/") + m, ok = cc.sc.Methods[method[:i+1]] + } return } diff --git a/rpc_util.go b/rpc_util.go index 73c3a96655d3..df232af1914a 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -239,7 +239,7 @@ type parser struct { // No other error values or types must be returned, which also means // that the underlying io.Reader must not return an incompatible // error. -func (p *parser) recvMsg(maxMsgSize int) (pf payloadFormat, msg []byte, err error) { +func (p *parser) recvMsg(maxReceiveMessageSize int) (pf payloadFormat, msg []byte, err error) { if _, err := io.ReadFull(p.r, p.header[:]); err != nil { return 0, nil, err } @@ -250,8 +250,8 @@ func (p *parser) recvMsg(maxMsgSize int) (pf payloadFormat, msg []byte, err erro if length == 0 { return pf, nil, nil } - if length > uint32(maxMsgSize) { - return 0, nil, Errorf(codes.Internal, "grpc: received message length %d exceeding the max size %d", length, maxMsgSize) + if length > uint32(maxReceiveMessageSize) { + return 0, nil, Errorf(codes.InvalidArgument, "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: @@ -335,8 +335,8 @@ func checkRecvPayload(pf payloadFormat, recvCompress string, dc Decompressor) er return nil } -func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{}, maxMsgSize int, inPayload *stats.InPayload) error { - pf, d, err := p.recvMsg(maxMsgSize) +func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{}, maxReceiveMessageSize int, inPayload *stats.InPayload) error { + pf, d, err := p.recvMsg(maxReceiveMessageSize) if err != nil { return err } @@ -352,10 +352,10 @@ func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{ return Errorf(codes.Internal, "grpc: failed to decompress the received message %v", err) } } - if len(d) > maxMsgSize { + if len(d) > maxReceiveMessageSize { // TODO: Revisit the error code. Currently keep it consistent with java // implementation. - return Errorf(codes.Internal, "grpc: received a message of %d bytes exceeding %d limit", len(d), maxMsgSize) + return Errorf(codes.InvalidArgument, "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) @@ -489,24 +489,22 @@ 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 + 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 + 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 minumum 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. - // TODO: support this. - MaxReqSize uint32 + MaxReqSize *int // MaxRespSize is the maximum allowed payload size for an individual response in a // stream (server->client) in bytes. - // TODO: support this. - MaxRespSize uint32 + MaxRespSize *int } // ServiceConfig is provided by the service provider and contains parameters for how diff --git a/server.go b/server.go index 157f35eea502..af0b93183a3f 100644 --- a/server.go +++ b/server.go @@ -105,21 +105,23 @@ type Server struct { } type options struct { - creds credentials.TransportCredentials - codec Codec - cp Compressor - dc Decompressor - maxMsgSize int - unaryInt UnaryServerInterceptor - streamInt StreamServerInterceptor - inTapHandle tap.ServerInHandle - statsHandler stats.Handler - maxConcurrentStreams uint32 - useHandlerImpl bool // use http.Handler-based server - unknownStreamDesc *StreamDesc + creds credentials.TransportCredentials + codec Codec + cp Compressor + dc Decompressor + maxReceiveMessageSize int + maxSendMessageSize int + unaryInt UnaryServerInterceptor + streamInt StreamServerInterceptor + inTapHandle tap.ServerInHandle + statsHandler stats.Handler + maxConcurrentStreams uint32 + useHandlerImpl bool // use http.Handler-based server + unknownStreamDesc *StreamDesc } -var defaultMaxMsgSize = 1024 * 1024 * 4 // use 4MB as the default message size limit +const defaultServerMaxReceiveMessageSize = 1024 * 1024 * 4 // Use 4MB as the default receive message size limit. +const defaultServerMaxSendMessageSize = 1024 * 1024 * 4 // Use 4MB as the default send message size limit. // A ServerOption sets options. type ServerOption func(*options) @@ -146,10 +148,26 @@ func RPCDecompressor(dc Decompressor) ServerOption { } // MaxMsgSize returns a ServerOption to set the max message size in bytes for inbound mesages. -// If this is not set, gRPC uses the default 4MB. +// If this is not set, gRPC uses the default 4MB. This function is for backward compatability. It has essentially the same functionality as MaxReceiveMessageSize. func MaxMsgSize(m int) ServerOption { return func(o *options) { - o.maxMsgSize = m + o.maxReceiveMessageSize = m + } +} + +// MaxReceiveMessageSize returns a ServerOption to set the max message size in bytes for inbound mesages. +// If this is not set, gRPC uses the default 4MB. +func MaxReceiveMessageSize(m int) ServerOption { + return func(o *options) { + o.maxReceiveMessageSize = m + } +} + +// MaxSendMessageSize returns a ServerOption to set the max message size in bytes for outbound mesages. +// If this is not set, gRPC uses the default 4MB. +func MaxSendMessageSize(m int) ServerOption { + return func(o *options) { + o.maxSendMessageSize = m } } @@ -211,7 +229,7 @@ func StatsHandler(h stats.Handler) ServerOption { // UnknownServiceHandler returns a ServerOption that allows for adding a custom // unknown service handler. The provided method is a bidi-streaming RPC service -// handler that will be invoked instead of returning the the "unimplemented" gRPC +// handler that will be invoked instead of returning the "unimplemented" gRPC // error whenever a request is received for an unregistered service or method. // The handling function has full access to the Context of the request and the // stream, and the invocation passes through interceptors. @@ -231,7 +249,8 @@ func UnknownServiceHandler(streamHandler StreamHandler) ServerOption { // started to accept requests yet. func NewServer(opt ...ServerOption) *Server { var opts options - opts.maxMsgSize = defaultMaxMsgSize + opts.maxReceiveMessageSize = defaultServerMaxReceiveMessageSize + opts.maxSendMessageSize = defaultServerMaxSendMessageSize for _, o := range opt { o(&opts) } @@ -609,6 +628,9 @@ func (s *Server) sendResponse(t transport.ServerTransport, stream *transport.Str // the optimal option. grpclog.Fatalf("grpc: Server failed to encode response %v", err) } + if len(p) > s.opts.maxSendMessageSize { + return Errorf(codes.InvalidArgument, "Sent message larger than max (%d vs. %d)", len(p), s.opts.maxSendMessageSize) + } err = t.Write(stream, p, opts) if err == nil && outPayload != nil { outPayload.SentTime = time.Now() @@ -653,7 +675,7 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. } p := &parser{r: stream} for { - pf, req, err := p.recvMsg(s.opts.maxMsgSize) + pf, req, err := p.recvMsg(s.opts.maxReceiveMessageSize) if err == io.EOF { // The entire stream is done (for unary RPC only). return err @@ -715,11 +737,11 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. return Errorf(codes.Internal, err.Error()) } } - if len(req) > s.opts.maxMsgSize { + if len(req) > s.opts.maxReceiveMessageSize { // TODO: Revisit the error code. Currently keep it consistent with // java implementation. - statusCode = codes.Internal - statusDesc = fmt.Sprintf("grpc: server received a message of %d bytes exceeding %d limit", len(req), s.opts.maxMsgSize) + statusCode = codes.InvalidArgument + statusDesc = fmt.Sprintf("grpc: server received a message of %d bytes exceeding %d limit", len(req), s.opts.maxReceiveMessageSize) } if err := s.opts.codec.Unmarshal(req, v); err != nil { return err @@ -771,7 +793,6 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. statusCode = codes.Unknown statusDesc = err.Error() } - return err } if trInfo != nil { trInfo.tr.LazyLog(&payload{sent: true, msg: reply}, true) @@ -807,15 +828,16 @@ func (s *Server) processStreamingRPC(t transport.ServerTransport, stream *transp stream.SetSendCompress(s.opts.cp.Type()) } ss := &serverStream{ - t: t, - s: stream, - p: &parser{r: stream}, - codec: s.opts.codec, - cp: s.opts.cp, - dc: s.opts.dc, - maxMsgSize: s.opts.maxMsgSize, - trInfo: trInfo, - statsHandler: sh, + t: t, + s: stream, + p: &parser{r: stream}, + codec: s.opts.codec, + cp: s.opts.cp, + dc: s.opts.dc, + maxReceiveMessageSize: s.opts.maxReceiveMessageSize, + maxSendMessageSize: s.opts.maxSendMessageSize, + trInfo: trInfo, + statsHandler: sh, } if ss.cp != nil { ss.cbuf = new(bytes.Buffer) diff --git a/stream.go b/stream.go index 0ef2077ce254..034bcf7e39ad 100644 --- a/stream.go +++ b/stream.go @@ -112,12 +112,40 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth cancel context.CancelFunc ) c := defaultCallInfo - if mc, ok := cc.getMethodConfig(method); ok { - c.failFast = !mc.WaitForReady - if mc.Timeout > 0 { - ctx, cancel = context.WithTimeout(ctx, mc.Timeout) + maxReceiveMessageSize := defaultClientMaxReceiveMessageSize + maxSendMessageSize := defaultClientMaxSendMessageSize + + if mc, ok := cc.GetMethodConfig(method); ok { + 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() + } + + if mc.MaxReqSize != nil && cc.dopts.maxSendMessageSize >= 0 { + maxSendMessageSize = min(*mc.MaxReqSize, cc.dopts.maxSendMessageSize) + } else if mc.MaxReqSize != nil { + maxSendMessageSize = *mc.MaxReqSize + } + + if mc.MaxRespSize != nil && cc.dopts.maxReceiveMessageSize >= 0 { + maxReceiveMessageSize = min(*mc.MaxRespSize, cc.dopts.maxReceiveMessageSize) + } else if mc.MaxRespSize != nil { + maxReceiveMessageSize = *mc.MaxRespSize + } + } else { + if cc.dopts.maxSendMessageSize >= 0 { + maxSendMessageSize = cc.dopts.maxSendMessageSize + } + if cc.dopts.maxReceiveMessageSize >= 0 { + maxReceiveMessageSize = cc.dopts.maxReceiveMessageSize } } + for _, o := range opts { if err := o.before(&c); err != nil { return nil, toRPCErr(err) @@ -207,14 +235,15 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth break } cs := &clientStream{ - opts: opts, - c: c, - desc: desc, - codec: cc.dopts.codec, - cp: cc.dopts.cp, - dc: cc.dopts.dc, - maxMsgSize: cc.dopts.maxMsgSize, - cancel: cancel, + opts: opts, + c: c, + desc: desc, + codec: cc.dopts.codec, + cp: cc.dopts.cp, + dc: cc.dopts.dc, + maxReceiveMessageSize: maxReceiveMessageSize, + maxSendMessageSize: maxSendMessageSize, + cancel: cancel, put: put, t: t, @@ -259,18 +288,19 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth // 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 - cbuf *bytes.Buffer - dc Decompressor - maxMsgSize int - cancel context.CancelFunc + opts []CallOption + c callInfo + t transport.ClientTransport + s *transport.Stream + p *parser + desc *StreamDesc + codec Codec + cp Compressor + cbuf *bytes.Buffer + dc Decompressor + maxReceiveMessageSize int + maxSendMessageSize int + cancel context.CancelFunc tracing bool // set to EnableTracing when the clientStream is created. @@ -353,6 +383,9 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) { if err != nil { return Errorf(codes.Internal, "grpc: %v", err) } + if len(out) > cs.maxSendMessageSize { + return Errorf(codes.InvalidArgument, "Sent message larger than max (%d vs. %d)", len(out), cs.maxSendMessageSize) + } err = cs.t.Write(cs.s, out, &transport.Options{Last: false}) if err == nil && outPayload != nil { outPayload.SentTime = time.Now() @@ -383,7 +416,7 @@ func (cs *clientStream) RecvMsg(m interface{}) (err error) { Client: true, } } - err = recv(cs.p, cs.codec, cs.s, cs.dc, m, cs.maxMsgSize, inPayload) + err = recv(cs.p, cs.codec, cs.s, cs.dc, m, cs.maxReceiveMessageSize, inPayload) defer func() { // err != nil indicates the termination of the stream. if err != nil { @@ -406,7 +439,7 @@ func (cs *clientStream) RecvMsg(m interface{}) (err error) { } // Special handling for client streaming rpc. // This recv expects EOF or errors, so we don't collect inPayload. - err = recv(cs.p, cs.codec, cs.s, cs.dc, m, cs.maxMsgSize, nil) + err = recv(cs.p, cs.codec, cs.s, cs.dc, m, cs.maxReceiveMessageSize, nil) cs.closeTransportStream(err) if err == nil { return toRPCErr(errors.New("grpc: client streaming protocol violation: get , want ")) @@ -512,17 +545,18 @@ 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 - cbuf *bytes.Buffer - maxMsgSize int - statusCode codes.Code - statusDesc string - trInfo *traceInfo + t transport.ServerTransport + s *transport.Stream + p *parser + codec Codec + cp Compressor + dc Decompressor + cbuf *bytes.Buffer + maxReceiveMessageSize int + maxSendMessageSize int + statusCode codes.Code + statusDesc string + trInfo *traceInfo statsHandler stats.Handler @@ -581,6 +615,9 @@ func (ss *serverStream) SendMsg(m interface{}) (err error) { err = Errorf(codes.Internal, "grpc: %v", err) return err } + if len(out) > ss.maxSendMessageSize { + return Errorf(codes.InvalidArgument, "Sent message larger than max (%d vs. %d)", len(out), ss.maxSendMessageSize) + } if err := ss.t.Write(ss.s, out, &transport.Options{Last: false}); err != nil { return toRPCErr(err) } @@ -610,7 +647,7 @@ func (ss *serverStream) RecvMsg(m interface{}) (err error) { if ss.statsHandler != nil { inPayload = &stats.InPayload{} } - if err := recv(ss.p, ss.codec, ss.s, ss.dc, m, ss.maxMsgSize, inPayload); err != nil { + if err := recv(ss.p, ss.codec, ss.s, ss.dc, m, ss.maxReceiveMessageSize, inPayload); err != nil { if err == io.EOF { return err } diff --git a/test/end2end_test.go b/test/end2end_test.go index 98d590e9ea7f..910a3113209c 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -416,20 +416,25 @@ type test struct { cancel context.CancelFunc // Configurable knobs, after newTest returns: - testServer testpb.TestServiceServer // nil means none - healthServer *health.Server // nil means disabled - maxStream uint32 - tapHandle tap.ServerInHandle - maxMsgSize int - userAgent string - clientCompression bool - serverCompression bool - unaryClientInt grpc.UnaryClientInterceptor - streamClientInt grpc.StreamClientInterceptor - unaryServerInt grpc.UnaryServerInterceptor - streamServerInt grpc.StreamServerInterceptor - unknownHandler grpc.StreamHandler - sc <-chan grpc.ServiceConfig + testServer testpb.TestServiceServer // nil means none + healthServer *health.Server // nil means disabled + maxStream uint32 + tapHandle tap.ServerInHandle + maxMsgSize int + maxClientReceiveMsgSize int + maxClientSendMsgSize int + maxServerReceiveMsgSize int + maxServerSendMsgSize int + userAgent string + clientCompression bool + serverCompression bool + timeout time.Duration + unaryClientInt grpc.UnaryClientInterceptor + streamClientInt grpc.StreamClientInterceptor + unaryServerInt grpc.UnaryServerInterceptor + streamServerInt grpc.StreamServerInterceptor + unknownHandler grpc.StreamHandler + sc <-chan grpc.ServiceConfig // srv and srvAddr are set once startServer is called. srv *grpc.Server @@ -465,6 +470,12 @@ func newTest(t *testing.T, e env) *test { t: t, e: e, maxStream: math.MaxUint32, + // Default value 0 is meaningful (0 byte msg size limit), thus using -1 to indiciate the field is unset. + maxClientReceiveMsgSize: -1, + maxClientSendMsgSize: -1, + maxServerReceiveMsgSize: -1, + maxServerSendMsgSize: -1, + maxMsgSize: -1, } te.ctx, te.cancel = context.WithCancel(context.Background()) return te @@ -476,9 +487,15 @@ func (te *test) startServer(ts testpb.TestServiceServer) { te.testServer = ts te.t.Logf("Running test in %s environment...", te.e.name) sopts := []grpc.ServerOption{grpc.MaxConcurrentStreams(te.maxStream)} - if te.maxMsgSize > 0 { + if te.maxMsgSize >= 0 { sopts = append(sopts, grpc.MaxMsgSize(te.maxMsgSize)) } + if te.maxServerReceiveMsgSize >= 0 { + sopts = append(sopts, grpc.MaxReceiveMessageSize(te.maxServerReceiveMsgSize)) + } + if te.maxServerSendMsgSize >= 0 { + sopts = append(sopts, grpc.MaxSendMessageSize(te.maxServerSendMsgSize)) + } if te.tapHandle != nil { sopts = append(sopts, grpc.InTapHandle(te.tapHandle)) } @@ -570,9 +587,18 @@ func (te *test) clientConn() *grpc.ClientConn { if te.streamClientInt != nil { opts = append(opts, grpc.WithStreamInterceptor(te.streamClientInt)) } - if te.maxMsgSize > 0 { + if te.maxMsgSize >= 0 { opts = append(opts, grpc.WithMaxMsgSize(te.maxMsgSize)) } + if te.maxClientReceiveMsgSize >= 0 { + opts = append(opts, grpc.WithMaxReceiveMessageSize(te.maxClientReceiveMsgSize)) + } + if te.maxClientSendMsgSize >= 0 { + opts = append(opts, grpc.WithMaxSendMessageSize(te.maxClientSendMsgSize)) + } + if te.timeout > 0 { + opts = append(opts, grpc.WithTimeout(te.timeout)) + } switch te.e.security { case "tls": creds, err := credentials.NewClientTLSFromFile(tlsDir+"ca.pem", "x.test.youtube.com") @@ -1030,11 +1056,14 @@ func testFailFast(t *testing.T, e env) { func TestServiceConfig(t *testing.T) { defer leakCheck(t)() for _, e := range listTestEnv() { - testServiceConfig(t, e) + testGetMethodConfig(t, e) + testServiceConfigWaitForReady(t, e) + // Timeout logic (min of service config and client API) is implemented implicitly in context. WithTimeout(). No need to test here. + testServiceConfigMaxMsgSize(t, e) } } -func testServiceConfig(t *testing.T, e env) { +func testServiceConfigSetup(t *testing.T, e env) (*test, chan grpc.ServiceConfig) { te := newTest(t, e) ch := make(chan grpc.ServiceConfig) te.sc = ch @@ -1045,15 +1074,93 @@ func testServiceConfig(t *testing.T, e env) { "grpc: addrConn.resetTransport failed to create client transport: connection error", "Failed to dial : context canceled; please retry.", ) + return te, ch +} + +func testGetMethodConfig(t *testing.T, e env) { + te, ch := testServiceConfigSetup(t, e) + defer te.tearDown() + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + w1 := new(bool) + *w1 = true + to := new(time.Duration) + *to = time.Millisecond + mc1 := grpc.MethodConfig{ + WaitForReady: w1, + Timeout: to, + } + w2 := new(bool) + *w2 = false + mc2 := grpc.MethodConfig{ + WaitForReady: w2, + } + m := make(map[string]grpc.MethodConfig) + m["/grpc.testing.TestService/EmptyCall"] = mc1 + m["/grpc.testing.TestService"] = mc2 + sc := grpc.ServiceConfig{ + Methods: m, + } + ch <- sc + }() + cc := te.clientConn() + tc := testpb.NewTestServiceClient(cc) + // The following RPCs are expected to become non-fail-fast ones with 1ms deadline. + if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); grpc.Code(err) != codes.DeadlineExceeded { + t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want _, %s", err, codes.DeadlineExceeded) + } + wg.Wait() + w1 := new(bool) + *w1 = true + to := new(time.Duration) + *to = time.Millisecond + mc1 := grpc.MethodConfig{ + WaitForReady: w1, + Timeout: to, + } + w2 := new(bool) + *w2 = false + mc2 := grpc.MethodConfig{ + WaitForReady: w2, + } + m := make(map[string]grpc.MethodConfig) + m["/grpc.testing.TestService/UnaryCall"] = mc1 + m["/grpc.testing.TestService/"] = mc2 + sc := grpc.ServiceConfig{ + Methods: m, + } + ch <- sc + // Wait for the new service config to propagate. + for { + if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); grpc.Code(err) == codes.DeadlineExceeded { + continue + } + break + } + if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); grpc.Code(err) != codes.Unavailable { + t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want _, %s", err, codes.Unavailable) + } +} + +func testServiceConfigWaitForReady(t *testing.T, e env) { + te, ch := testServiceConfigSetup(t, e) defer te.tearDown() + // Case1: Client API set failfast to be false, and service config set wait_for_ready to be false, Client API should win, and the rpc will wait until deadline exceeds. var wg sync.WaitGroup wg.Add(1) go func() { defer wg.Done() + w := new(bool) + *w = false + to := new(time.Duration) + *to = time.Millisecond mc := grpc.MethodConfig{ - WaitForReady: true, - Timeout: time.Millisecond, + WaitForReady: w, + Timeout: to, } m := make(map[string]grpc.MethodConfig) m["/grpc.testing.TestService/EmptyCall"] = mc @@ -1066,16 +1173,23 @@ func testServiceConfig(t *testing.T, e env) { cc := te.clientConn() tc := testpb.NewTestServiceClient(cc) // The following RPCs are expected to become non-fail-fast ones with 1ms deadline. - if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); grpc.Code(err) != codes.DeadlineExceeded { + if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}, grpc.FailFast(false)); grpc.Code(err) != codes.DeadlineExceeded { t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want _, %s", err, codes.DeadlineExceeded) } - if _, err := tc.FullDuplexCall(context.Background()); grpc.Code(err) != codes.DeadlineExceeded { + if _, err := tc.FullDuplexCall(context.Background(), grpc.FailFast(false)); grpc.Code(err) != codes.DeadlineExceeded { t.Fatalf("TestService/FullDuplexCall(_) = _, %v, want %s", err, codes.DeadlineExceeded) } wg.Wait() + // Generate a service config update. + // Case2: Client API does not set failfast, and service config set wait_for_ready to be true, and the rpc will wait until deadline exceeds. + w := new(bool) + *w = true + to := new(time.Duration) + *to = time.Millisecond mc := grpc.MethodConfig{ - WaitForReady: false, + WaitForReady: w, + Timeout: to, } m := make(map[string]grpc.MethodConfig) m["/grpc.testing.TestService/EmptyCall"] = mc @@ -1084,19 +1198,504 @@ func testServiceConfig(t *testing.T, e env) { Methods: m, } ch <- sc - // Loop until the new update becomes effective. + + // Wait for the new service config to take effect. + mc, ok := cc.GetMethodConfig("/grpc.testing.TestService/EmptyCall") for { - if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); grpc.Code(err) != codes.Unavailable { + if ok && !*mc.WaitForReady { + time.Sleep(100 * time.Millisecond) + mc, ok = cc.GetMethodConfig("/grpc.testing.TestService/EmptyCall") continue } break } - // The following RPCs are expected to become fail-fast. - if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); grpc.Code(err) != codes.Unavailable { - t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want _, %s", err, codes.Unavailable) + // The following RPCs are expected to become non-fail-fast ones with 1ms deadline. + if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); grpc.Code(err) != codes.DeadlineExceeded { + t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want _, %s", err, codes.DeadlineExceeded) + } + if _, err := tc.FullDuplexCall(context.Background()); grpc.Code(err) != codes.DeadlineExceeded { + t.Fatalf("TestService/FullDuplexCall(_) = _, %v, want %s", err, codes.DeadlineExceeded) + } +} + +func testServiceConfigMaxMsgSize(t *testing.T, e env) { + // Setting up values and objects shared across all test cases. + const smallSize = 1 + const largeSize = 1024 + const extraLargeSize = 2048 + + smallPayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, smallSize) + if err != nil { + t.Fatal(err) + } + largePayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, largeSize) + if err != nil { + t.Fatal(err) + } + extraLargePayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, extraLargeSize) + if err != nil { + t.Fatal(err) + } + + mreq := new(int) + *mreq = extraLargeSize + mresp := new(int) + *mresp = extraLargeSize + mc := grpc.MethodConfig{ + MaxReqSize: mreq, + MaxRespSize: mresp, + } + m := make(map[string]grpc.MethodConfig) + m["/grpc.testing.TestService/UnaryCall"] = mc + m["/grpc.testing.TestService/FullDuplexCall"] = mc + sc := grpc.ServiceConfig{ + Methods: m, + } + // Case1: sc set maxReqSize to 2048 (send), maxRespSize to 2048 (recv). + te1, ch1 := testServiceConfigSetup(t, e) + te1.startServer(&testServer{security: e.security}) + defer te1.tearDown() + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + ch1 <- sc + }() + tc := testpb.NewTestServiceClient(te1.clientConn()) + + req := &testpb.SimpleRequest{ + ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), + ResponseSize: proto.Int32(int32(extraLargeSize)), + Payload: smallPayload, + } + // test for unary RPC recv + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + // test for unary RPC send + req.Payload = extraLargePayload + req.ResponseSize = proto.Int32(int32(smallSize)) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + // test for streaming RPC recv + respParam := []*testpb.ResponseParameters{ + { + Size: proto.Int32(int32(extraLargeSize)), + }, + } + sreq := &testpb.StreamingOutputCallRequest{ + ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), + ResponseParameters: respParam, + Payload: smallPayload, + } + stream, err := tc.FullDuplexCall(te1.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err != nil { + t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) + } + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + } + + // test for streaming RPC send + respParam[0].Size = proto.Int32(int32(smallSize)) + sreq.Payload = extraLargePayload + stream, err = tc.FullDuplexCall(te1.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) + } + + wg.Wait() + // Case2: Client API set maxReqSize to 1024 (send), maxRespSize to 1024 (recv). Sc sets maxReqSize to 2048 (send), maxRespSize to 2048 (recv). + te2, ch2 := testServiceConfigSetup(t, e) + te2.maxClientReceiveMsgSize = 1024 + te2.maxClientSendMsgSize = 1024 + te2.startServer(&testServer{security: e.security}) + defer te2.tearDown() + wg.Add(1) + go func() { + defer wg.Done() + ch2 <- sc + }() + tc = testpb.NewTestServiceClient(te2.clientConn()) + + // Test for unary RPC recv. + req.Payload = smallPayload + req.ResponseSize = proto.Int32(int32(largeSize)) + + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + // Test for unary RPC send. + req.Payload = largePayload + req.ResponseSize = proto.Int32(int32(smallSize)) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + // Test for streaming RPC recv. + stream, err = tc.FullDuplexCall(te2.ctx) + respParam[0].Size = proto.Int32(int32(largeSize)) + sreq.Payload = smallPayload + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err != nil { + t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) + } + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + } + + // Test for streaming RPC send. + respParam[0].Size = proto.Int32(int32(smallSize)) + sreq.Payload = largePayload + stream, err = tc.FullDuplexCall(te2.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) + } + wg.Wait() + + // Case3: Client API set maxReqSize to 4096 (send), maxRespSize to 4096 (recv). Sc sets maxReqSize to 2048 (send), maxRespSize to 2048 (recv). + te3, ch3 := testServiceConfigSetup(t, e) + te3.maxClientReceiveMsgSize = 4096 + te3.maxClientSendMsgSize = 4096 + te3.startServer(&testServer{security: e.security}) + defer te3.tearDown() + wg.Add(1) + go func() { + defer wg.Done() + ch3 <- sc + }() + tc = testpb.NewTestServiceClient(te3.clientConn()) + + // Test for unary RPC recv. + req.Payload = smallPayload + req.ResponseSize = proto.Int32(int32(largeSize)) + + if _, err := tc.UnaryCall(context.Background(), req); err != nil { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want ", err) + } + + req.ResponseSize = proto.Int32(int32(extraLargeSize)) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + // Test for unary RPC send. + req.Payload = largePayload + req.ResponseSize = proto.Int32(int32(smallSize)) + if _, err := tc.UnaryCall(context.Background(), req); err != nil { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want ", err) + } + + req.Payload = extraLargePayload + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + // Test for streaming RPC recv. + stream, err = tc.FullDuplexCall(te3.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + respParam[0].Size = proto.Int32(int32(largeSize)) + sreq.Payload = smallPayload + + if err := stream.Send(sreq); err != nil { + t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) + } + if _, err := stream.Recv(); err != nil { + t.Fatalf("%v.Recv() = _, %v, want ", stream, err) + } + + respParam[0].Size = proto.Int32(int32(extraLargeSize)) + + if err := stream.Send(sreq); err != nil { + t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) + } + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + } + + // Test for streaming RPC send. + respParam[0].Size = proto.Int32(int32(smallSize)) + sreq.Payload = largePayload + stream, err = tc.FullDuplexCall(te3.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err != nil { + t.Fatalf("%v.Send(%v) = %v, want ", stream, err) + } + sreq.Payload = extraLargePayload + if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) + } + wg.Wait() +} + +func TestMsgSizeDefaultAndApi(t *testing.T) { + defer leakCheck(t)() + for _, e := range listTestEnv() { + testMaxMsgSizeClientDefault(t, e) + testMaxMsgSizeClientApi(t, e) + testMaxMsgSizeServerApi(t, e) + } +} + +func testMaxMsgSizeClientDefault(t *testing.T, e env) { + te := newTest(t, e) + te.userAgent = testAppUA + // To avoid error on server side. + te.maxServerSendMsgSize = 5 * 1024 * 1024 + te.declareLogNoise( + "transport: http2Client.notifyError got notified that the client transport was broken EOF", + "grpc: addrConn.transportMonitor exits due to: grpc: the connection is closing", + "grpc: addrConn.resetTransport failed to create client transport: connection error", + "Failed to dial : context canceled; please retry.", + ) + te.startServer(&testServer{security: e.security}) + + defer te.tearDown() + tc := testpb.NewTestServiceClient(te.clientConn()) + + const smallSize = 1 + const largeSize = 4 * 1024 * 1024 + smallPayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, smallSize) + if err != nil { + t.Fatal(err) + } + + largePayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, largeSize) + if err != nil { + t.Fatal(err) + } + req := &testpb.SimpleRequest{ + ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), + ResponseSize: proto.Int32(int32(largeSize)), + Payload: smallPayload, + } + // Test for unary RPC recv. + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + // Test for unary RPC send. + req.Payload = largePayload + req.ResponseSize = proto.Int32(int32(smallSize)) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + respParam := []*testpb.ResponseParameters{ + { + Size: proto.Int32(int32(largeSize)), + }, + } + sreq := &testpb.StreamingOutputCallRequest{ + ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), + ResponseParameters: respParam, + Payload: smallPayload, + } + + // Test for streaming RPC recv. + stream, err := tc.FullDuplexCall(te.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err != nil { + t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) + } + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + } + + // Test for streaming RPC send. + respParam[0].Size = proto.Int32(int32(smallSize)) + sreq.Payload = largePayload + stream, err = tc.FullDuplexCall(te.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Send(%v) = %v, want _, error codes: %s", stream, sreq, err, codes.InvalidArgument) + } +} + +func testMaxMsgSizeClientApi(t *testing.T, e env) { + te := newTest(t, e) + te.userAgent = testAppUA + // To avoid error on server side. + te.maxServerSendMsgSize = 5 * 1024 * 1024 + te.maxClientReceiveMsgSize = 1024 + te.maxClientSendMsgSize = 1024 + te.declareLogNoise( + "transport: http2Client.notifyError got notified that the client transport was broken EOF", + "grpc: addrConn.transportMonitor exits due to: grpc: the connection is closing", + "grpc: addrConn.resetTransport failed to create client transport: connection error", + "Failed to dial : context canceled; please retry.", + ) + te.startServer(&testServer{security: e.security}) + + defer te.tearDown() + tc := testpb.NewTestServiceClient(te.clientConn()) + + const smallSize = 1 + const largeSize = 1024 + smallPayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, smallSize) + if err != nil { + t.Fatal(err) + } + + largePayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, largeSize) + if err != nil { + t.Fatal(err) + } + req := &testpb.SimpleRequest{ + ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), + ResponseSize: proto.Int32(int32(largeSize)), + Payload: smallPayload, + } + // Test for unary RPC recv. + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + // Test for unary RPC send. + req.Payload = largePayload + req.ResponseSize = proto.Int32(int32(smallSize)) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + respParam := []*testpb.ResponseParameters{ + { + Size: proto.Int32(int32(largeSize)), + }, + } + sreq := &testpb.StreamingOutputCallRequest{ + ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), + ResponseParameters: respParam, + Payload: smallPayload, + } + + // Test for streaming RPC recv. + stream, err := tc.FullDuplexCall(te.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err != nil { + t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) + } + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + } + + // Test for streaming RPC send. + respParam[0].Size = proto.Int32(int32(smallSize)) + sreq.Payload = largePayload + stream, err = tc.FullDuplexCall(te.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) + } +} + +func testMaxMsgSizeServerApi(t *testing.T, e env) { + te := newTest(t, e) + te.userAgent = testAppUA + te.maxServerReceiveMsgSize = 1024 + te.maxServerSendMsgSize = 1024 + te.declareLogNoise( + "transport: http2Client.notifyError got notified that the client transport was broken EOF", + "grpc: addrConn.transportMonitor exits due to: grpc: the connection is closing", + "grpc: addrConn.resetTransport failed to create client transport: connection error", + "Failed to dial : context canceled; please retry.", + ) + te.startServer(&testServer{security: e.security}) + + defer te.tearDown() + tc := testpb.NewTestServiceClient(te.clientConn()) + + const smallSize = 1 + const largeSize = 1024 + smallPayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, smallSize) + if err != nil { + t.Fatal(err) + } + + largePayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, largeSize) + if err != nil { + t.Fatal(err) + } + req := &testpb.SimpleRequest{ + ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), + ResponseSize: proto.Int32(int32(largeSize)), + Payload: smallPayload, + } + // Test for unary RPC send. + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.Unknown { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + // Test for unary RPC recv. + req.Payload = largePayload + req.ResponseSize = proto.Int32(int32(smallSize)) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + } + + respParam := []*testpb.ResponseParameters{ + { + Size: proto.Int32(int32(largeSize)), + }, + } + sreq := &testpb.StreamingOutputCallRequest{ + ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), + ResponseParameters: respParam, + Payload: smallPayload, + } + + // Test for streaming RPC send. + stream, err := tc.FullDuplexCall(te.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err != nil { + t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) + } + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + } + + // Test for streaming RPC recv. + respParam[0].Size = proto.Int32(int32(smallSize)) + sreq.Payload = largePayload + stream, err = tc.FullDuplexCall(te.ctx) + if err != nil { + t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) + } + if err := stream.Send(sreq); err != nil { + t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := tc.FullDuplexCall(context.Background()); grpc.Code(err) != codes.Unavailable { - t.Fatalf("TestService/FullDuplexCall(_) = _, %v, want %s", err, codes.Unavailable) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) } } @@ -1415,6 +2014,7 @@ func testLargeUnary(t *testing.T, e env) { } } +// Test backward-compatability API for setting msg size limit. func TestExceedMsgLimit(t *testing.T) { defer leakCheck(t)() for _, e := range listTestEnv() { @@ -1441,23 +2041,23 @@ func testExceedMsgLimit(t *testing.T, e env) { t.Fatal(err) } - // test on server side for unary RPC + // Test on server side for unary RPC. req := &testpb.SimpleRequest{ ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(), ResponseSize: proto.Int32(smallSize), Payload: payload, } - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.Internal { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.Internal) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) } - // test on client side for unary RPC + // Test on client side for unary RPC. req.ResponseSize = proto.Int32(int32(te.maxMsgSize) + 1) req.Payload = smallPayload - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.Internal { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.Internal) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) } - // test on server side for streaming RPC + // Test on server side for streaming RPC. stream, err := tc.FullDuplexCall(te.ctx) if err != nil { t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) @@ -1481,11 +2081,11 @@ func testExceedMsgLimit(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.Internal { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.Internal) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) } - // test on client side for streaming RPC + // Test on client side for streaming RPC. stream, err = tc.FullDuplexCall(te.ctx) if err != nil { t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) @@ -1495,8 +2095,8 @@ func testExceedMsgLimit(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.Internal { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.Internal) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) } } From a0b902a11d9e9b741aead5ec7335c11fb2951e22 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Mon, 3 Apr 2017 15:29:39 -0700 Subject: [PATCH 02/23] fix missing import --- clientconn.go | 1 + 1 file changed, 1 insertion(+) diff --git a/clientconn.go b/clientconn.go index 4eec42b35381..30d8d7b2e88f 100644 --- a/clientconn.go +++ b/clientconn.go @@ -40,6 +40,7 @@ import ( "net" "sync" "time" + "strings" "golang.org/x/net/context" "golang.org/x/net/trace" From ad16b942fb94281ff038d9bad2af5aca69d10c9b Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Mon, 3 Apr 2017 16:03:05 -0700 Subject: [PATCH 03/23] fix gofmt goimports go tool vet errors --- clientconn.go | 2 +- server.go | 28 ++++++++++++++-------------- test/end2end_test.go | 2 +- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/clientconn.go b/clientconn.go index 30d8d7b2e88f..12b7818628ff 100644 --- a/clientconn.go +++ b/clientconn.go @@ -38,9 +38,9 @@ import ( "fmt" "math" "net" + "strings" "sync" "time" - "strings" "golang.org/x/net/context" "golang.org/x/net/trace" diff --git a/server.go b/server.go index 9dc795615b33..5049763dfc6d 100644 --- a/server.go +++ b/server.go @@ -106,22 +106,22 @@ type Server struct { } type options struct { - creds credentials.TransportCredentials - codec Codec - cp Compressor - dc Decompressor - maxMsgSize int - unaryInt UnaryServerInterceptor - streamInt StreamServerInterceptor - inTapHandle tap.ServerInHandle - statsHandler stats.Handler - maxConcurrentStreams uint32 + creds credentials.TransportCredentials + codec Codec + cp Compressor + dc Decompressor + maxMsgSize int + unaryInt UnaryServerInterceptor + streamInt StreamServerInterceptor + inTapHandle tap.ServerInHandle + statsHandler stats.Handler + maxConcurrentStreams uint32 maxReceiveMessageSize int maxSendMessageSize int - useHandlerImpl bool // use http.Handler-based server - unknownStreamDesc *StreamDesc - keepaliveParams keepalive.ServerParameters - keepalivePolicy keepalive.EnforcementPolicy + useHandlerImpl bool // use http.Handler-based server + unknownStreamDesc *StreamDesc + keepaliveParams keepalive.ServerParameters + keepalivePolicy keepalive.EnforcementPolicy } const defaultServerMaxReceiveMessageSize = 1024 * 1024 * 4 // Use 4MB as the default receive message size limit. diff --git a/test/end2end_test.go b/test/end2end_test.go index 20fbdca95dd9..d0898baffc55 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -1441,7 +1441,7 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) } if err := stream.Send(sreq); err != nil { - t.Fatalf("%v.Send(%v) = %v, want ", stream, err) + t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } sreq.Payload = extraLargePayload if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { From a66f9231a8560e6831b96f2de8193a59a3bc24ce Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Wed, 5 Apr 2017 11:08:50 -0700 Subject: [PATCH 04/23] Make initial service config non-blocking in Dial() --- clientconn.go | 5 ++--- test/end2end_test.go | 10 +++++----- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/clientconn.go b/clientconn.go index 12b7818628ff..3b4f182f8b6d 100644 --- a/clientconn.go +++ b/clientconn.go @@ -356,14 +356,13 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * }() if cc.dopts.scChan != nil { - // Wait for the initial service config. + // Try to get an initial service config. select { case sc, ok := <-cc.dopts.scChan: if ok { cc.sc = sc } - case <-ctx.Done(): - return nil, ctx.Err() + default: } } // Set defaults. diff --git a/test/end2end_test.go b/test/end2end_test.go index d0898baffc55..b857bb623149 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -1450,12 +1450,12 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { wg.Wait() } -func TestMsgSizeDefaultAndApi(t *testing.T) { +func TestMsgSizeDefaultAndAPI(t *testing.T) { defer leakCheck(t)() for _, e := range listTestEnv() { testMaxMsgSizeClientDefault(t, e) - testMaxMsgSizeClientApi(t, e) - testMaxMsgSizeServerApi(t, e) + testMaxMsgSizeClientAPI(t, e) + testMaxMsgSizeServerAPI(t, e) } } @@ -1538,7 +1538,7 @@ func testMaxMsgSizeClientDefault(t *testing.T, e env) { } } -func testMaxMsgSizeClientApi(t *testing.T, e env) { +func testMaxMsgSizeClientAPI(t *testing.T, e env) { te := newTest(t, e) te.userAgent = testAppUA // To avoid error on server side. @@ -1619,7 +1619,7 @@ func testMaxMsgSizeClientApi(t *testing.T, e env) { } } -func testMaxMsgSizeServerApi(t *testing.T, e env) { +func testMaxMsgSizeServerAPI(t *testing.T, e env) { te := newTest(t, e) te.userAgent = testAppUA te.maxServerReceiveMsgSize = 1024 From f1bb70facf983daef40d337ac5741085c9ef5611 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Wed, 5 Apr 2017 15:08:25 -0700 Subject: [PATCH 05/23] gofmt --- stream.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/stream.go b/stream.go index 2f0548675852..20c7aeb2f090 100644 --- a/stream.go +++ b/stream.go @@ -542,16 +542,16 @@ 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 - cbuf *bytes.Buffer + t transport.ServerTransport + s *transport.Stream + p *parser + codec Codec + cp Compressor + dc Decompressor + cbuf *bytes.Buffer maxReceiveMessageSize int maxSendMessageSize int - trInfo *traceInfo + trInfo *traceInfo statsHandler stats.Handler From 6f8b55318ad2c3645b1096f11cd27996bad2add3 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Thu, 6 Apr 2017 14:08:04 -0700 Subject: [PATCH 06/23] fix the testMaxMsgSizeServerAPI failure --- call.go | 14 ++-- clientconn.go | 14 ++-- rpc_util.go | 10 +++ server.go | 21 +++--- stream.go | 6 +- test/end2end_test.go | 161 ++++++++++++++++--------------------------- 6 files changed, 97 insertions(+), 129 deletions(-) diff --git a/call.go b/call.go index e370aeec119b..46cf064204a3 100644 --- a/call.go +++ b/call.go @@ -150,16 +150,6 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli return invoke(ctx, method, args, reply, cc, opts...) } -const defaultClientMaxReceiveMessageSize = 1024 * 1024 * 4 -const defaultClientMaxSendMessageSize = 1024 * 1024 * 4 - -func min(a, b int) int { - if a < b { - return a - } - return b -} - func invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) (e error) { c := defaultCallInfo maxReceiveMessageSize := defaultClientMaxReceiveMessageSize @@ -179,12 +169,16 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli maxSendMessageSize = min(*mc.MaxReqSize, cc.dopts.maxSendMessageSize) } else if mc.MaxReqSize != nil { maxSendMessageSize = *mc.MaxReqSize + } else if mc.MaxReqSize == nil && cc.dopts.maxSendMessageSize >= 0 { + maxSendMessageSize = cc.dopts.maxSendMessageSize } if mc.MaxRespSize != nil && cc.dopts.maxReceiveMessageSize >= 0 { maxReceiveMessageSize = min(*mc.MaxRespSize, cc.dopts.maxReceiveMessageSize) } else if mc.MaxRespSize != nil { maxReceiveMessageSize = *mc.MaxRespSize + } else if mc.MaxRespSize == nil && cc.dopts.maxReceiveMessageSize >= 0 { + maxReceiveMessageSize = cc.dopts.maxReceiveMessageSize } } else { if cc.dopts.maxSendMessageSize >= 0 { diff --git a/clientconn.go b/clientconn.go index 3b4f182f8b6d..3ee29d4a03b9 100644 --- a/clientconn.go +++ b/clientconn.go @@ -36,7 +36,6 @@ package grpc import ( "errors" "fmt" - "math" "net" "strings" "sync" @@ -103,16 +102,19 @@ type dialOptions struct { maxSendMessageSize int } -const defaultClientMaxMsgSize = math.MaxInt32 +const ( + defaultClientMaxReceiveMessageSize = 1024 * 1024 * 4 + defaultClientMaxSendMessageSize = 1024 * 1024 * 4 + defaultServerMaxReceiveMessageSize = 1024 * 1024 * 4 + defaultServerMaxSendMessageSize = 1024 * 1024 * 4 +) // DialOption configures how we set up the connection. type DialOption func(*dialOptions) -// WithMaxMsgSize returns a DialOption which sets the maximum message size the client can receive. This function is for backward API compatibility. It has essentially the same functionality as WithMaxReceiveMessageSize. +// WithMaxMsgSize Deprecated: use WithMaxReceiveMessageSize instead. func WithMaxMsgSize(s int) DialOption { - return func(o *dialOptions) { - o.maxReceiveMessageSize = s - } + return WithMaxReceiveMessageSize(s) } // WithMaxReceiveMessageSize returns a DialOption which sets the maximum message size the client can receive. Negative input is invalid and has the same effect as not setting the field. diff --git a/rpc_util.go b/rpc_util.go index 1fad9d47682c..41c4be1a18e2 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -485,6 +485,9 @@ type ServiceConfig struct { // 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 all methods under the service (/service/) and use the corresponding MethodConfig. + // Otherwise, the method has no MethodConfig to use. Methods map[string]MethodConfig } @@ -498,3 +501,10 @@ const SupportPackageIsVersion4 = true // Version is the current grpc version. const Version = "1.3.0-dev" + +func min(a, b int) int { + if a < b { + return a + } + return b +} diff --git a/server.go b/server.go index e2f46831b550..a93e86476ef7 100644 --- a/server.go +++ b/server.go @@ -125,9 +125,6 @@ type options struct { keepalivePolicy keepalive.EnforcementPolicy } -const defaultServerMaxReceiveMessageSize = 1024 * 1024 * 4 // Use 4MB as the default receive message size limit. -const defaultServerMaxSendMessageSize = 1024 * 1024 * 4 // Use 4MB as the default send message size limit. - // A ServerOption sets options. type ServerOption func(*options) @@ -166,12 +163,9 @@ func RPCDecompressor(dc Decompressor) ServerOption { } } -// MaxMsgSize returns a ServerOption to set the max message size in bytes for inbound mesages. -// If this is not set, gRPC uses the default 4MB. This function is for backward compatability. It has essentially the same functionality as MaxReceiveMessageSize. +// MaxMsgSize Deprecated: use MaxReceiveMessageSize instead. func MaxMsgSize(m int) ServerOption { - return func(o *options) { - o.maxReceiveMessageSize = m - } + return MaxReceiveMessageSize(m) } // MaxReceiveMessageSize returns a ServerOption to set the max message size in bytes for inbound mesages. @@ -650,7 +644,7 @@ func (s *Server) sendResponse(t transport.ServerTransport, stream *transport.Str grpclog.Fatalf("grpc: Server failed to encode response %v", err) } if len(p) > s.opts.maxSendMessageSize { - return Errorf(codes.InvalidArgument, "Sent message larger than max (%d vs. %d)", len(p), s.opts.maxSendMessageSize) + return status.Errorf(codes.InvalidArgument, "Sent message larger than max (%d vs. %d)", len(p), s.opts.maxSendMessageSize) } err = t.Write(stream, p, opts) if err == nil && outPayload != nil { @@ -755,7 +749,7 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. if len(req) > s.opts.maxReceiveMessageSize { // TODO: Revisit the error code. Currently keep it consistent with // java implementation. - return status.Errorf(codes.InvalidArgument, "grpc: server received a message of %d bytes exceeding %d limit", len(req), s.opts.maxReceiveMessageSize) + return status.Errorf(codes.InvalidArgument, "Received message larger than max (%d vs. %d)", len(req), s.opts.maxReceiveMessageSize) } if err := s.opts.codec.Unmarshal(req, v); err != nil { return status.Errorf(codes.Internal, "grpc: error unmarshalling request: %v", err) @@ -798,6 +792,13 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. if err := s.sendResponse(t, stream, reply, s.opts.cp, opts); err != nil { // TODO: Translate error into a status.Status error if necessary? // TODO: Write status when appropriate. + switch e := err.(type) { + case status.Status: + if se := t.WriteStatus(stream, e); e != nil { + grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", se) + } + default: + } return err } if trInfo != nil { diff --git a/stream.go b/stream.go index 20c7aeb2f090..7e2adb7fb56b 100644 --- a/stream.go +++ b/stream.go @@ -115,7 +115,6 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth c := defaultCallInfo maxReceiveMessageSize := defaultClientMaxReceiveMessageSize maxSendMessageSize := defaultClientMaxSendMessageSize - if mc, ok := cc.GetMethodConfig(method); ok { if mc.WaitForReady != nil { c.failFast = !*mc.WaitForReady @@ -131,12 +130,16 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth maxSendMessageSize = min(*mc.MaxReqSize, cc.dopts.maxSendMessageSize) } else if mc.MaxReqSize != nil { maxSendMessageSize = *mc.MaxReqSize + } else if mc.MaxReqSize == nil && cc.dopts.maxSendMessageSize >= 0 { + maxSendMessageSize = cc.dopts.maxSendMessageSize } if mc.MaxRespSize != nil && cc.dopts.maxReceiveMessageSize >= 0 { maxReceiveMessageSize = min(*mc.MaxRespSize, cc.dopts.maxReceiveMessageSize) } else if mc.MaxRespSize != nil { maxReceiveMessageSize = *mc.MaxRespSize + } else if mc.MaxRespSize == nil && cc.dopts.maxReceiveMessageSize >= 0 { + maxReceiveMessageSize = cc.dopts.maxReceiveMessageSize } } else { if cc.dopts.maxSendMessageSize >= 0 { @@ -146,7 +149,6 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth maxReceiveMessageSize = cc.dopts.maxReceiveMessageSize } } - for _, o := range opts { if err := o.before(&c); err != nil { return nil, toRPCErr(err) diff --git a/test/end2end_test.go b/test/end2end_test.go index 32f20c76c247..c8773c6031c3 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -1078,7 +1078,8 @@ func TestServiceConfig(t *testing.T) { func testServiceConfigSetup(t *testing.T, e env) (*test, chan grpc.ServiceConfig) { te := newTest(t, e) - ch := make(chan grpc.ServiceConfig) + // We write before read. + ch := make(chan grpc.ServiceConfig, 1) te.sc = ch te.userAgent = testAppUA te.declareLogNoise( @@ -1090,59 +1091,52 @@ func testServiceConfigSetup(t *testing.T, e env) (*test, chan grpc.ServiceConfig return te, ch } +func newBool(b bool) (a *bool) { + a = new(bool) + *a = b + return +} + +func newInt(b int) (a *int) { + a = new(int) + *a = b + return +} + +func newDuration(b time.Duration) (a *time.Duration) { + a = new(time.Duration) + *a = b + return +} + func testGetMethodConfig(t *testing.T, e env) { te, ch := testServiceConfigSetup(t, e) defer te.tearDown() - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - w1 := new(bool) - *w1 = true - to := new(time.Duration) - *to = time.Millisecond - mc1 := grpc.MethodConfig{ - WaitForReady: w1, - Timeout: to, - } - w2 := new(bool) - *w2 = false - mc2 := grpc.MethodConfig{ - WaitForReady: w2, - } - m := make(map[string]grpc.MethodConfig) - m["/grpc.testing.TestService/EmptyCall"] = mc1 - m["/grpc.testing.TestService"] = mc2 - sc := grpc.ServiceConfig{ - Methods: m, - } - ch <- sc - }() + mc1 := grpc.MethodConfig{ + WaitForReady: newBool(true), + Timeout: newDuration(time.Millisecond), + } + mc2 := grpc.MethodConfig{WaitForReady: newBool(false)} + m := make(map[string]grpc.MethodConfig) + m["/grpc.testing.TestService/EmptyCall"] = mc1 + m["/grpc.testing.TestService/"] = mc2 + sc := grpc.ServiceConfig{ + Methods: m, + } + ch <- sc + cc := te.clientConn() tc := testpb.NewTestServiceClient(cc) // The following RPCs are expected to become non-fail-fast ones with 1ms deadline. if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); grpc.Code(err) != codes.DeadlineExceeded { t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want _, %s", err, codes.DeadlineExceeded) } - wg.Wait() - w1 := new(bool) - *w1 = true - to := new(time.Duration) - *to = time.Millisecond - mc1 := grpc.MethodConfig{ - WaitForReady: w1, - Timeout: to, - } - w2 := new(bool) - *w2 = false - mc2 := grpc.MethodConfig{ - WaitForReady: w2, - } - m := make(map[string]grpc.MethodConfig) + + m = make(map[string]grpc.MethodConfig) m["/grpc.testing.TestService/UnaryCall"] = mc1 m["/grpc.testing.TestService/"] = mc2 - sc := grpc.ServiceConfig{ + sc = grpc.ServiceConfig{ Methods: m, } ch <- sc @@ -1163,26 +1157,18 @@ func testServiceConfigWaitForReady(t *testing.T, e env) { defer te.tearDown() // Case1: Client API set failfast to be false, and service config set wait_for_ready to be false, Client API should win, and the rpc will wait until deadline exceeds. - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - w := new(bool) - *w = false - to := new(time.Duration) - *to = time.Millisecond - mc := grpc.MethodConfig{ - WaitForReady: w, - Timeout: to, - } - m := make(map[string]grpc.MethodConfig) - m["/grpc.testing.TestService/EmptyCall"] = mc - m["/grpc.testing.TestService/FullDuplexCall"] = mc - sc := grpc.ServiceConfig{ - Methods: m, - } - ch <- sc - }() + mc := grpc.MethodConfig{ + WaitForReady: newBool(false), + Timeout: newDuration(time.Millisecond), + } + m := make(map[string]grpc.MethodConfig) + m["/grpc.testing.TestService/EmptyCall"] = mc + m["/grpc.testing.TestService/FullDuplexCall"] = mc + sc := grpc.ServiceConfig{ + Methods: m, + } + ch <- sc + cc := te.clientConn() tc := testpb.NewTestServiceClient(cc) // The following RPCs are expected to become non-fail-fast ones with 1ms deadline. @@ -1192,22 +1178,14 @@ func testServiceConfigWaitForReady(t *testing.T, e env) { if _, err := tc.FullDuplexCall(context.Background(), grpc.FailFast(false)); grpc.Code(err) != codes.DeadlineExceeded { t.Fatalf("TestService/FullDuplexCall(_) = _, %v, want %s", err, codes.DeadlineExceeded) } - wg.Wait() // Generate a service config update. // Case2: Client API does not set failfast, and service config set wait_for_ready to be true, and the rpc will wait until deadline exceeds. - w := new(bool) - *w = true - to := new(time.Duration) - *to = time.Millisecond - mc := grpc.MethodConfig{ - WaitForReady: w, - Timeout: to, - } - m := make(map[string]grpc.MethodConfig) + mc.WaitForReady = newBool(true) + m = make(map[string]grpc.MethodConfig) m["/grpc.testing.TestService/EmptyCall"] = mc m["/grpc.testing.TestService/FullDuplexCall"] = mc - sc := grpc.ServiceConfig{ + sc = grpc.ServiceConfig{ Methods: m, } ch <- sc @@ -1250,14 +1228,11 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { t.Fatal(err) } - mreq := new(int) - *mreq = extraLargeSize - mresp := new(int) - *mresp = extraLargeSize mc := grpc.MethodConfig{ - MaxReqSize: mreq, - MaxRespSize: mresp, + MaxReqSize: newInt(extraLargeSize), + MaxRespSize: newInt(extraLargeSize), } + m := make(map[string]grpc.MethodConfig) m["/grpc.testing.TestService/UnaryCall"] = mc m["/grpc.testing.TestService/FullDuplexCall"] = mc @@ -1269,12 +1244,7 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { te1.startServer(&testServer{security: e.security}) defer te1.tearDown() - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - ch1 <- sc - }() + ch1 <- sc tc := testpb.NewTestServiceClient(te1.clientConn()) req := &testpb.SimpleRequest{ @@ -1327,18 +1297,13 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) } - wg.Wait() // Case2: Client API set maxReqSize to 1024 (send), maxRespSize to 1024 (recv). Sc sets maxReqSize to 2048 (send), maxRespSize to 2048 (recv). te2, ch2 := testServiceConfigSetup(t, e) te2.maxClientReceiveMsgSize = 1024 te2.maxClientSendMsgSize = 1024 te2.startServer(&testServer{security: e.security}) defer te2.tearDown() - wg.Add(1) - go func() { - defer wg.Done() - ch2 <- sc - }() + ch2 <- sc tc = testpb.NewTestServiceClient(te2.clientConn()) // Test for unary RPC recv. @@ -1380,7 +1345,6 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) } - wg.Wait() // Case3: Client API set maxReqSize to 4096 (send), maxRespSize to 4096 (recv). Sc sets maxReqSize to 2048 (send), maxRespSize to 2048 (recv). te3, ch3 := testServiceConfigSetup(t, e) @@ -1388,11 +1352,7 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { te3.maxClientSendMsgSize = 4096 te3.startServer(&testServer{security: e.security}) defer te3.tearDown() - wg.Add(1) - go func() { - defer wg.Done() - ch3 <- sc - }() + ch3 <- sc tc = testpb.NewTestServiceClient(te3.clientConn()) // Test for unary RPC recv. @@ -1458,14 +1418,13 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) } - wg.Wait() } func TestMsgSizeDefaultAndAPI(t *testing.T) { defer leakCheck(t)() for _, e := range listTestEnv() { - testMaxMsgSizeClientDefault(t, e) - testMaxMsgSizeClientAPI(t, e) + // testMaxMsgSizeClientDefault(t, e) + // testMaxMsgSizeClientAPI(t, e) testMaxMsgSizeServerAPI(t, e) } } @@ -1663,7 +1622,7 @@ func testMaxMsgSizeServerAPI(t *testing.T, e env) { Payload: smallPayload, } // Test for unary RPC send. - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.Unknown { + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) } From c6a393703304a3d55a37b11ad5feee5fe15cbe01 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Thu, 6 Apr 2017 14:30:59 -0700 Subject: [PATCH 07/23] fix send response error case --- server.go | 16 +++++++++++----- test/end2end_test.go | 4 ++-- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/server.go b/server.go index aa9d5c226a05..e457cdc96e7d 100644 --- a/server.go +++ b/server.go @@ -794,12 +794,18 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. if err := s.sendResponse(t, stream, reply, s.opts.cp, opts); err != nil { // TODO: Translate error into a status.Status error if necessary? // TODO: Write status when appropriate. - switch e := err.(type) { - case status.Status: - if se := t.WriteStatus(stream, e); e != nil { - grpclog.Printf("grpc: Server.processUnaryRPC failed to write status %v", se) + s, ok := status.FromError(err) + if !ok { + // TODO: Parse possible non-status error + } else { + switch s.Code() { + case codes.InvalidArgument: + if e := t.WriteStatus(stream, s); e != nil { + grpclog.Printf("grpc: Server.processUnaryRPC failed to write status: %v", e) + } + // TODO: Add cases if needed + default: } - default: } return err } diff --git a/test/end2end_test.go b/test/end2end_test.go index c8773c6031c3..eb3de446a1c8 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -1423,8 +1423,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { func TestMsgSizeDefaultAndAPI(t *testing.T) { defer leakCheck(t)() for _, e := range listTestEnv() { - // testMaxMsgSizeClientDefault(t, e) - // testMaxMsgSizeClientAPI(t, e) + testMaxMsgSizeClientDefault(t, e) + testMaxMsgSizeClientAPI(t, e) testMaxMsgSizeServerAPI(t, e) } } From cb02ab4d25bc7f9209238fb50ace3a0eef6bb9ba Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Thu, 13 Apr 2017 16:51:56 -0700 Subject: [PATCH 08/23] change error message from InvalidArgument to ResourceExhausted --- call.go | 2 +- rpc_util.go | 6 +-- server.go | 6 +-- stream.go | 4 +- test/end2end_test.go | 112 +++++++++++++++++++++---------------------- 5 files changed, 65 insertions(+), 65 deletions(-) diff --git a/call.go b/call.go index 3f0549f501af..8c56e5a1c53c 100644 --- a/call.go +++ b/call.go @@ -123,7 +123,7 @@ func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, return nil, Errorf(codes.Internal, "grpc: %v", err) } if len(outBuf) > msgSizeLimit { - return nil, Errorf(codes.InvalidArgument, "Sent message larger than max (%d vs. %d)", len(outBuf), msgSizeLimit) + return nil, Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(outBuf), msgSizeLimit) } err = t.Write(stream, outBuf, opts) if err == nil && outPayload != nil { diff --git a/rpc_util.go b/rpc_util.go index 18d1f0dfc845..e45e93b4226a 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -226,7 +226,7 @@ func (p *parser) recvMsg(maxReceiveMessageSize int) (pf payloadFormat, msg []byt return pf, nil, nil } if length > uint32(maxReceiveMessageSize) { - return 0, nil, Errorf(codes.InvalidArgument, "grpc: Received message larger than max (%d vs. %d)", length, maxReceiveMessageSize) + return 0, nil, 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: @@ -269,7 +269,7 @@ func encode(c Codec, msg interface{}, cp Compressor, cbuf *bytes.Buffer, outPayl length = uint(len(b)) } if length > math.MaxUint32 { - return nil, Errorf(codes.InvalidArgument, "grpc: message too large (%d bytes)", length) + return nil, Errorf(codes.ResourceExhausted, "grpc: message too large (%d bytes)", length) } const ( @@ -330,7 +330,7 @@ func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{ if len(d) > maxReceiveMessageSize { // TODO: Revisit the error code. Currently keep it consistent with java // implementation. - return Errorf(codes.InvalidArgument, "grpc: Received message larger than max (%d vs. %d)", len(d), maxReceiveMessageSize) + return 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) diff --git a/server.go b/server.go index 10fa50d4868e..e4433cf39543 100644 --- a/server.go +++ b/server.go @@ -644,7 +644,7 @@ func (s *Server) sendResponse(t transport.ServerTransport, stream *transport.Str grpclog.Fatalf("grpc: Server failed to encode response %v", err) } if len(p) > s.opts.maxSendMessageSize { - return status.Errorf(codes.InvalidArgument, "Sent message larger than max (%d vs. %d)", len(p), s.opts.maxSendMessageSize) + return status.Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(p), s.opts.maxSendMessageSize) } err = t.Write(stream, p, opts) if err == nil && outPayload != nil { @@ -751,7 +751,7 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. if len(req) > s.opts.maxReceiveMessageSize { // TODO: Revisit the error code. Currently keep it consistent with // java implementation. - return status.Errorf(codes.InvalidArgument, "Received message larger than max (%d vs. %d)", len(req), s.opts.maxReceiveMessageSize) + return status.Errorf(codes.ResourceExhausted, "Received message larger than max (%d vs. %d)", len(req), s.opts.maxReceiveMessageSize) } if err := s.opts.codec.Unmarshal(req, v); err != nil { return status.Errorf(codes.Internal, "grpc: error unmarshalling request: %v", err) @@ -931,7 +931,7 @@ func (s *Server) handleStream(t transport.ServerTransport, stream *transport.Str trInfo.tr.SetError() } errDesc := fmt.Sprintf("malformed method name: %q", stream.Method()) - if err := t.WriteStatus(stream, status.New(codes.InvalidArgument, errDesc)); err != nil { + if err := t.WriteStatus(stream, status.New(codes.ResourceExhausted, errDesc)); err != nil { if trInfo != nil { trInfo.tr.LazyLog(&fmtStringer{"%v", []interface{}{err}}, true) trInfo.tr.SetError() diff --git a/stream.go b/stream.go index 02ec28ff4d89..1b83fc452907 100644 --- a/stream.go +++ b/stream.go @@ -384,7 +384,7 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) { return Errorf(codes.Internal, "grpc: %v", err) } if len(out) > cs.maxSendMessageSize { - return Errorf(codes.InvalidArgument, "Sent message larger than max (%d vs. %d)", len(out), cs.maxSendMessageSize) + return Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(out), cs.maxSendMessageSize) } err = cs.t.Write(cs.s, out, &transport.Options{Last: false}) if err == nil && outPayload != nil { @@ -614,7 +614,7 @@ func (ss *serverStream) SendMsg(m interface{}) (err error) { return err } if len(out) > ss.maxSendMessageSize { - return Errorf(codes.InvalidArgument, "Sent message larger than max (%d vs. %d)", len(out), ss.maxSendMessageSize) + return Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(out), ss.maxSendMessageSize) } if err := ss.t.Write(ss.s, out, &transport.Options{Last: false}); err != nil { return toRPCErr(err) diff --git a/test/end2end_test.go b/test/end2end_test.go index 881b1a495d1d..3504423852e8 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -1253,15 +1253,15 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { Payload: smallPayload, } // test for unary RPC recv - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // test for unary RPC send req.Payload = extraLargePayload req.ResponseSize = proto.Int32(int32(smallSize)) - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // test for streaming RPC recv @@ -1282,8 +1282,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } // test for streaming RPC send @@ -1293,8 +1293,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { if err != nil { t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) } - if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) + if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.ResourceExhausted) } // Case2: Client API set maxReqSize to 1024 (send), maxRespSize to 1024 (recv). Sc sets maxReqSize to 2048 (send), maxRespSize to 2048 (recv). @@ -1310,15 +1310,15 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { req.Payload = smallPayload req.ResponseSize = proto.Int32(int32(largeSize)) - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // Test for unary RPC send. req.Payload = largePayload req.ResponseSize = proto.Int32(int32(smallSize)) - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // Test for streaming RPC recv. @@ -1331,8 +1331,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } // Test for streaming RPC send. @@ -1342,8 +1342,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { if err != nil { t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) } - if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) + if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.ResourceExhausted) } // Case3: Client API set maxReqSize to 4096 (send), maxRespSize to 4096 (recv). Sc sets maxReqSize to 2048 (send), maxRespSize to 2048 (recv). @@ -1364,8 +1364,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { } req.ResponseSize = proto.Int32(int32(extraLargeSize)) - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // Test for unary RPC send. @@ -1376,8 +1376,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { } req.Payload = extraLargePayload - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // Test for streaming RPC recv. @@ -1400,8 +1400,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } // Test for streaming RPC send. @@ -1415,8 +1415,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } sreq.Payload = extraLargePayload - if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) + if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.ResourceExhausted) } } @@ -1462,15 +1462,15 @@ func testMaxMsgSizeClientDefault(t *testing.T, e env) { Payload: smallPayload, } // Test for unary RPC recv. - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // Test for unary RPC send. req.Payload = largePayload req.ResponseSize = proto.Int32(int32(smallSize)) - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } respParam := []*testpb.ResponseParameters{ @@ -1492,8 +1492,8 @@ func testMaxMsgSizeClientDefault(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } // Test for streaming RPC send. @@ -1503,8 +1503,8 @@ func testMaxMsgSizeClientDefault(t *testing.T, e env) { if err != nil { t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) } - if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Send(%v) = %v, want _, error codes: %s", stream, sreq, err, codes.InvalidArgument) + if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Send(%v) = %v, want _, error codes: %s", stream, sreq, err, codes.ResourceExhausted) } } @@ -1543,15 +1543,15 @@ func testMaxMsgSizeClientAPI(t *testing.T, e env) { Payload: smallPayload, } // Test for unary RPC recv. - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // Test for unary RPC send. req.Payload = largePayload req.ResponseSize = proto.Int32(int32(smallSize)) - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } respParam := []*testpb.ResponseParameters{ @@ -1573,8 +1573,8 @@ func testMaxMsgSizeClientAPI(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } // Test for streaming RPC send. @@ -1584,8 +1584,8 @@ func testMaxMsgSizeClientAPI(t *testing.T, e env) { if err != nil { t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) } - if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.InvalidArgument) + if err := stream.Send(sreq); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Send(%v) = %v, want _, error code: %s", stream, sreq, err, codes.ResourceExhausted) } } @@ -1622,15 +1622,15 @@ func testMaxMsgSizeServerAPI(t *testing.T, e env) { Payload: smallPayload, } // Test for unary RPC send. - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // Test for unary RPC recv. req.Payload = largePayload req.ResponseSize = proto.Int32(int32(smallSize)) - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } respParam := []*testpb.ResponseParameters{ @@ -1652,8 +1652,8 @@ func testMaxMsgSizeServerAPI(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } // Test for streaming RPC recv. @@ -1666,8 +1666,8 @@ func testMaxMsgSizeServerAPI(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } } @@ -2020,14 +2020,14 @@ func testExceedMsgLimit(t *testing.T, e env) { ResponseSize: proto.Int32(smallSize), Payload: payload, } - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // Test on client side for unary RPC. req.ResponseSize = proto.Int32(int32(te.maxMsgSize) + 1) req.Payload = smallPayload - if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.InvalidArgument) + if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // Test on server side for streaming RPC. @@ -2054,8 +2054,8 @@ func testExceedMsgLimit(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } // Test on client side for streaming RPC. @@ -2068,8 +2068,8 @@ func testExceedMsgLimit(t *testing.T, e env) { if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) } - if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.InvalidArgument { - t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.InvalidArgument) + if _, err := stream.Recv(); err == nil || grpc.Code(err) != codes.ResourceExhausted { + t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } } From 983d8372eaff662ddddc96b77a12a7c8b6cd307a Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Fri, 21 Apr 2017 16:18:59 -0700 Subject: [PATCH 09/23] update the merge of client api and sc --- call.go | 44 +++++++++++--------------------------------- clientconn.go | 18 +++++++----------- rpc_util.go | 13 +++++++++++++ stream.go | 45 +++++++++++---------------------------------- 4 files changed, 42 insertions(+), 78 deletions(-) diff --git a/call.go b/call.go index 8c56e5a1c53c..a512060ae1cd 100644 --- a/call.go +++ b/call.go @@ -152,42 +152,20 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli func invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) (e error) { c := defaultCallInfo - maxReceiveMessageSize := defaultClientMaxReceiveMessageSize - maxSendMessageSize := defaultClientMaxSendMessageSize - if mc, ok := cc.GetMethodConfig(method); ok { - if mc.WaitForReady != nil { - c.failFast = !*mc.WaitForReady - } + 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() - } + if mc.Timeout != nil && *mc.Timeout >= 0 { + var cancel context.CancelFunc + ctx, cancel = context.WithTimeout(ctx, *mc.Timeout) + defer cancel() + } - if mc.MaxReqSize != nil && cc.dopts.maxSendMessageSize >= 0 { - maxSendMessageSize = min(*mc.MaxReqSize, cc.dopts.maxSendMessageSize) - } else if mc.MaxReqSize != nil { - maxSendMessageSize = *mc.MaxReqSize - } else if mc.MaxReqSize == nil && cc.dopts.maxSendMessageSize >= 0 { - maxSendMessageSize = cc.dopts.maxSendMessageSize - } + maxSendMessageSize := getMaxSize(mc.MaxReqSize, cc.dopts.maxSendMessageSize, defaultClientMaxSendMessageSize) + maxReceiveMessageSize := getMaxSize(mc.MaxRespSize, cc.dopts.maxReceiveMessageSize, defaultClientMaxReceiveMessageSize) - if mc.MaxRespSize != nil && cc.dopts.maxReceiveMessageSize >= 0 { - maxReceiveMessageSize = min(*mc.MaxRespSize, cc.dopts.maxReceiveMessageSize) - } else if mc.MaxRespSize != nil { - maxReceiveMessageSize = *mc.MaxRespSize - } else if mc.MaxRespSize == nil && cc.dopts.maxReceiveMessageSize >= 0 { - maxReceiveMessageSize = cc.dopts.maxReceiveMessageSize - } - } else { - if cc.dopts.maxSendMessageSize >= 0 { - maxSendMessageSize = cc.dopts.maxSendMessageSize - } - if cc.dopts.maxReceiveMessageSize >= 0 { - maxReceiveMessageSize = cc.dopts.maxReceiveMessageSize - } - } for _, o := range opts { if err := o.before(&c); err != nil { return toRPCErr(err) diff --git a/clientconn.go b/clientconn.go index 8232dfdce444..6556c3a8fec0 100644 --- a/clientconn.go +++ b/clientconn.go @@ -98,8 +98,8 @@ type dialOptions struct { timeout time.Duration scChan <-chan ServiceConfig copts transport.ConnectOptions - maxReceiveMessageSize int - maxSendMessageSize int + maxReceiveMessageSize *int + maxSendMessageSize *int } const ( @@ -120,14 +120,14 @@ func WithMaxMsgSize(s int) DialOption { // WithMaxReceiveMessageSize returns a DialOption which sets the maximum message size the client can receive. Negative input is invalid and has the same effect as not setting the field. func WithMaxReceiveMessageSize(s int) DialOption { return func(o *dialOptions) { - o.maxReceiveMessageSize = s + *o.maxReceiveMessageSize = s } } // WithMaxSendMessageSize returns a DialOption which sets the maximum message size the client can send. Negative input is invalid and has the same effect as not seeting the field. func WithMaxSendMessageSize(s int) DialOption { return func(o *dialOptions) { - o.maxSendMessageSize = s + *o.maxSendMessageSize = s } } @@ -324,10 +324,6 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * } cc.ctx, cc.cancel = context.WithCancel(context.Background()) - // initialize maxReceiveMessageSize and maxSendMessageSize to -1 before applying DialOption functions to distinguish whether the user set the message limit or not. - cc.dopts.maxReceiveMessageSize = -1 - cc.dopts.maxSendMessageSize = -1 - for _, opt := range opts { opt(&cc.dopts) } @@ -646,13 +642,13 @@ func (cc *ClientConn) resetAddrConn(addr Address, block bool, tearDownErr error) // GetMethodConfig gets the method config of the input method. If there's no exact match for the input method (i.e. /service/method), we will return the default config for all methods under the service (/service/). // TODO: Avoid the locking here. -func (cc *ClientConn) GetMethodConfig(method string) (m MethodConfig, ok bool) { +func (cc *ClientConn) GetMethodConfig(method string) (m MethodConfig) { cc.mu.RLock() defer cc.mu.RUnlock() - m, ok = cc.sc.Methods[method] + m, ok := cc.sc.Methods[method] if !ok { i := strings.LastIndex(method, "/") - m, ok = cc.sc.Methods[method[:i+1]] + m, _ = cc.sc.Methods[method[:i+1]] } return } diff --git a/rpc_util.go b/rpc_util.go index f921a559d24b..f33d5043627e 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -483,4 +483,17 @@ func min(a, b int) int { 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 +} + const grpcUA = "grpc-go/" + Version diff --git a/stream.go b/stream.go index 44ee2f25b13a..cfe7b443a059 100644 --- a/stream.go +++ b/stream.go @@ -113,42 +113,19 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth cancel context.CancelFunc ) c := defaultCallInfo - maxReceiveMessageSize := defaultClientMaxReceiveMessageSize - maxSendMessageSize := defaultClientMaxSendMessageSize - if mc, ok := cc.GetMethodConfig(method); ok { - 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() - } - - if mc.MaxReqSize != nil && cc.dopts.maxSendMessageSize >= 0 { - maxSendMessageSize = min(*mc.MaxReqSize, cc.dopts.maxSendMessageSize) - } else if mc.MaxReqSize != nil { - maxSendMessageSize = *mc.MaxReqSize - } else if mc.MaxReqSize == nil && cc.dopts.maxSendMessageSize >= 0 { - maxSendMessageSize = cc.dopts.maxSendMessageSize - } + mc := cc.GetMethodConfig(method) + if mc.WaitForReady != nil { + c.failFast = !*mc.WaitForReady + } - if mc.MaxRespSize != nil && cc.dopts.maxReceiveMessageSize >= 0 { - maxReceiveMessageSize = min(*mc.MaxRespSize, cc.dopts.maxReceiveMessageSize) - } else if mc.MaxRespSize != nil { - maxReceiveMessageSize = *mc.MaxRespSize - } else if mc.MaxRespSize == nil && cc.dopts.maxReceiveMessageSize >= 0 { - maxReceiveMessageSize = cc.dopts.maxReceiveMessageSize - } - } else { - if cc.dopts.maxSendMessageSize >= 0 { - maxSendMessageSize = cc.dopts.maxSendMessageSize - } - if cc.dopts.maxReceiveMessageSize >= 0 { - maxReceiveMessageSize = cc.dopts.maxReceiveMessageSize - } + if mc.Timeout != nil && *mc.Timeout >= 0 { + var cancel context.CancelFunc + ctx, cancel = context.WithTimeout(ctx, *mc.Timeout) + defer cancel() } + + maxSendMessageSize := getMaxSize(mc.MaxReqSize, cc.dopts.maxSendMessageSize, defaultClientMaxSendMessageSize) + maxReceiveMessageSize := getMaxSize(mc.MaxRespSize, cc.dopts.maxReceiveMessageSize, defaultClientMaxReceiveMessageSize) for _, o := range opts { if err := o.before(&c); err != nil { return nil, toRPCErr(err) From 9c5f260e67f83cea6f61b5b7f4b48c68abe60b8d Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Wed, 26 Apr 2017 15:50:58 -0700 Subject: [PATCH 10/23] make max size a pointer type and initialize function a CallOption --- call.go | 24 ++++++++------- clientconn.go | 48 +++++++++++++----------------- rpc_util.go | 42 ++++++++++++++++++-------- stream.go | 12 ++++---- test/end2end_test.go | 70 ++++++++++++++++++++------------------------ 5 files changed, 102 insertions(+), 94 deletions(-) diff --git a/call.go b/call.go index a512060ae1cd..688efed92a0d 100644 --- a/call.go +++ b/call.go @@ -52,7 +52,7 @@ import ( // // 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, msgSizeLimit int, t transport.ClientTransport, c *callInfo, stream *transport.Stream, reply interface{}) (err error) { +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 { @@ -73,7 +73,7 @@ func recvResponse(ctx context.Context, dopts dialOptions, msgSizeLimit int, t tr } } for { - if err = recv(p, dopts.codec, stream, dopts.dc, reply, msgSizeLimit, inPayload); err != nil { + if err = recv(p, dopts.codec, stream, dopts.dc, reply, *c.maxReceiveMessageSize, inPayload); err != nil { if err == io.EOF { break } @@ -93,7 +93,7 @@ func recvResponse(ctx context.Context, dopts dialOptions, msgSizeLimit int, t tr } // sendRequest writes out various information of an RPC such as Context and Message. -func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, msgSizeLimit int, callHdr *transport.CallHdr, t transport.ClientTransport, args interface{}, opts *transport.Options) (_ *transport.Stream, err error) { +func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, c *callInfo, callHdr *transport.CallHdr, t transport.ClientTransport, args interface{}, opts *transport.Options) (_ *transport.Stream, err error) { stream, err := t.NewStream(ctx, callHdr) if err != nil { return nil, err @@ -122,8 +122,8 @@ func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, if err != nil { return nil, Errorf(codes.Internal, "grpc: %v", err) } - if len(outBuf) > msgSizeLimit { - return nil, Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(outBuf), msgSizeLimit) + if len(outBuf) > *c.maxSendMessageSize { + return nil, Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(outBuf), *c.maxSendMessageSize) } err = t.Write(stream, outBuf, opts) if err == nil && outPayload != nil { @@ -152,7 +152,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli func invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) (e error) { c := defaultCallInfo - mc := cc.GetMethodConfig(method) + mc, _ := cc.GetMethodConfig(method) if mc.WaitForReady != nil { c.failFast = !*mc.WaitForReady } @@ -163,9 +163,7 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli defer cancel() } - maxSendMessageSize := getMaxSize(mc.MaxReqSize, cc.dopts.maxSendMessageSize, defaultClientMaxSendMessageSize) - maxReceiveMessageSize := getMaxSize(mc.MaxRespSize, cc.dopts.maxReceiveMessageSize, defaultClientMaxReceiveMessageSize) - + opts = append(cc.dopts.callOptions, opts...) for _, o := range opts { if err := o.before(&c); err != nil { return toRPCErr(err) @@ -176,6 +174,10 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli o.after(&c) } }() + + c.maxSendMessageSize = getMaxSize(mc.MaxReqSize, c.maxSendMessageSize, defaultClientMaxSendMessageSize) + c.maxReceiveMessageSize = getMaxSize(mc.MaxRespSize, c.maxReceiveMessageSize, defaultClientMaxReceiveMessageSize) + if EnableTracing { c.traceInfo.tr = trace.New("grpc.Sent."+methodFamily(method), method) defer c.traceInfo.tr.Finish() @@ -255,7 +257,7 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli if c.traceInfo.tr != nil { c.traceInfo.tr.LazyLog(&payload{sent: true, msg: args}, true) } - stream, err = sendRequest(ctx, cc.dopts, cc.dopts.cp, maxSendMessageSize, callHdr, t, args, topts) + stream, err = sendRequest(ctx, cc.dopts, cc.dopts.cp, &c, callHdr, t, args, topts) if err != nil { if put != nil { put() @@ -272,7 +274,7 @@ func invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli } return toRPCErr(err) } - err = recvResponse(ctx, cc.dopts, maxReceiveMessageSize, t, &c, stream, reply) + err = recvResponse(ctx, cc.dopts, t, &c, stream, reply) if err != nil { if put != nil { put() diff --git a/clientconn.go b/clientconn.go index 6556c3a8fec0..029899f1297e 100644 --- a/clientconn.go +++ b/clientconn.go @@ -86,20 +86,19 @@ var ( // dialOptions configure a Dial call. dialOptions are set by the DialOption // values passed to Dial. type dialOptions struct { - unaryInt UnaryClientInterceptor - streamInt StreamClientInterceptor - codec Codec - cp Compressor - dc Decompressor - bs backoffStrategy - balancer Balancer - block bool - insecure bool - timeout time.Duration - scChan <-chan ServiceConfig - copts transport.ConnectOptions - maxReceiveMessageSize *int - maxSendMessageSize *int + unaryInt UnaryClientInterceptor + streamInt StreamClientInterceptor + codec Codec + cp Compressor + dc Decompressor + bs backoffStrategy + balancer Balancer + block bool + insecure bool + timeout time.Duration + scChan <-chan ServiceConfig + copts transport.ConnectOptions + callOptions []CallOption } const ( @@ -114,20 +113,13 @@ type DialOption func(*dialOptions) // WithMaxMsgSize Deprecated: use WithMaxReceiveMessageSize instead. func WithMaxMsgSize(s int) DialOption { - return WithMaxReceiveMessageSize(s) + return WithDefaultCallOptions(WithMaxReceiveMessageSize(s)) } -// WithMaxReceiveMessageSize returns a DialOption which sets the maximum message size the client can receive. Negative input is invalid and has the same effect as not setting the field. -func WithMaxReceiveMessageSize(s int) DialOption { +// WithDefaultCallOptions returns a DialOption which sets the default CallOptions for calls over the connection. +func WithDefaultCallOptions(cos ...CallOption) DialOption { return func(o *dialOptions) { - *o.maxReceiveMessageSize = s - } -} - -// WithMaxSendMessageSize returns a DialOption which sets the maximum message size the client can send. Negative input is invalid and has the same effect as not seeting the field. -func WithMaxSendMessageSize(s int) DialOption { - return func(o *dialOptions) { - *o.maxSendMessageSize = s + o.callOptions = append(o.callOptions, cos...) } } @@ -642,13 +634,13 @@ func (cc *ClientConn) resetAddrConn(addr Address, block bool, tearDownErr error) // GetMethodConfig gets the method config of the input method. If there's no exact match for the input method (i.e. /service/method), we will return the default config for all methods under the service (/service/). // TODO: Avoid the locking here. -func (cc *ClientConn) GetMethodConfig(method string) (m MethodConfig) { +func (cc *ClientConn) GetMethodConfig(method string) (m MethodConfig, ok bool) { cc.mu.RLock() defer cc.mu.RUnlock() - m, ok := cc.sc.Methods[method] + m, ok = cc.sc.Methods[method] if !ok { i := strings.LastIndex(method, "/") - m, _ = cc.sc.Methods[method[:i+1]] + m, ok = cc.sc.Methods[method[:i+1]] } return } diff --git a/rpc_util.go b/rpc_util.go index f33d5043627e..606294be155e 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -111,11 +111,13 @@ func (d *gzipDecompressor) Type() string { // callInfo contains all related configuration and information about an RPC. type callInfo struct { - failFast bool - headerMD metadata.MD - trailerMD metadata.MD - peer *peer.Peer - traceInfo traceInfo // in trace.go + failFast bool + headerMD metadata.MD + trailerMD metadata.MD + peer *peer.Peer + traceInfo traceInfo // in trace.go + maxReceiveMessageSize *int + maxSendMessageSize *int } var defaultCallInfo = callInfo{failFast: true} @@ -181,6 +183,22 @@ func FailFast(failFast bool) CallOption { }) } +// WithMaxReceiveMessageSize returns a DialOption which sets the maximum message size the client can receive. Negative input is invalid and has the same effect as not setting the field. +func WithMaxReceiveMessageSize(s int) CallOption { + return beforeCall(func(o *callInfo) error { + o.maxReceiveMessageSize = &s + return nil + }) +} + +// WithMaxSendMessageSize returns a DialOption which sets the maximum message size the client can send. Negative input is invalid and has the same effect as not seeting the field. +func WithMaxSendMessageSize(s int) CallOption { + return beforeCall(func(o *callInfo) error { + o.maxSendMessageSize = &s + return nil + }) +} + // The format of the payload: compressed or not? type payloadFormat uint8 @@ -476,24 +494,24 @@ const SupportPackageIsVersion4 = true // Version is the current grpc version. const Version = "1.3.0-dev" -func min(a, b int) int { - if a < b { +func min(a, b *int) *int { + if *a < *b { return a } return b } -func getMaxSize(mcMax, doptMax *int, defaultVal int) int { +func getMaxSize(mcMax, doptMax *int, defaultVal int) *int { if mcMax == nil && doptMax == nil { - return defaultVal + return &defaultVal } if mcMax != nil && doptMax != nil { - return min(*mcMax, *doptMax) + return min(mcMax, doptMax) } if mcMax != nil { - return *mcMax + return mcMax } - return *doptMax + return doptMax } const grpcUA = "grpc-go/" + Version diff --git a/stream.go b/stream.go index cfe7b443a059..8dcd0627a527 100644 --- a/stream.go +++ b/stream.go @@ -113,7 +113,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth cancel context.CancelFunc ) c := defaultCallInfo - mc := cc.GetMethodConfig(method) + mc, _ := cc.GetMethodConfig(method) if mc.WaitForReady != nil { c.failFast = !*mc.WaitForReady } @@ -124,13 +124,15 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth defer cancel() } - maxSendMessageSize := getMaxSize(mc.MaxReqSize, cc.dopts.maxSendMessageSize, defaultClientMaxSendMessageSize) - maxReceiveMessageSize := getMaxSize(mc.MaxRespSize, cc.dopts.maxReceiveMessageSize, defaultClientMaxReceiveMessageSize) + opts = append(cc.dopts.callOptions, opts...) for _, o := range opts { if err := o.before(&c); err != nil { return nil, toRPCErr(err) } } + c.maxSendMessageSize = getMaxSize(mc.MaxReqSize, c.maxSendMessageSize, defaultClientMaxSendMessageSize) + c.maxReceiveMessageSize = getMaxSize(mc.MaxRespSize, c.maxReceiveMessageSize, defaultClientMaxReceiveMessageSize) + callHdr := &transport.CallHdr{ Host: cc.authority, Method: method, @@ -221,8 +223,8 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth codec: cc.dopts.codec, cp: cc.dopts.cp, dc: cc.dopts.dc, - maxReceiveMessageSize: maxReceiveMessageSize, - maxSendMessageSize: maxSendMessageSize, + maxReceiveMessageSize: *c.maxReceiveMessageSize, + maxSendMessageSize: *c.maxSendMessageSize, cancel: cancel, put: put, diff --git a/test/end2end_test.go b/test/end2end_test.go index c85c33dc3035..bc3c8fadc0b7 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -433,11 +433,11 @@ type test struct { healthServer *health.Server // nil means disabled maxStream uint32 tapHandle tap.ServerInHandle - maxMsgSize int - maxClientReceiveMsgSize int - maxClientSendMsgSize int - maxServerReceiveMsgSize int - maxServerSendMsgSize int + maxMsgSize *int + maxClientReceiveMsgSize *int + maxClientSendMsgSize *int + maxServerReceiveMsgSize *int + maxServerSendMsgSize *int userAgent string clientCompression bool serverCompression bool @@ -483,12 +483,6 @@ func newTest(t *testing.T, e env) *test { t: t, e: e, maxStream: math.MaxUint32, - // Default value 0 is meaningful (0 byte msg size limit), thus using -1 to indiciate the field is unset. - maxClientReceiveMsgSize: -1, - maxClientSendMsgSize: -1, - maxServerReceiveMsgSize: -1, - maxServerSendMsgSize: -1, - maxMsgSize: -1, } te.ctx, te.cancel = context.WithCancel(context.Background()) return te @@ -500,14 +494,14 @@ func (te *test) startServer(ts testpb.TestServiceServer) { te.testServer = ts te.t.Logf("Running test in %s environment...", te.e.name) sopts := []grpc.ServerOption{grpc.MaxConcurrentStreams(te.maxStream)} - if te.maxMsgSize >= 0 { - sopts = append(sopts, grpc.MaxMsgSize(te.maxMsgSize)) + if te.maxMsgSize != nil { + sopts = append(sopts, grpc.MaxMsgSize(*te.maxMsgSize)) } - if te.maxServerReceiveMsgSize >= 0 { - sopts = append(sopts, grpc.MaxReceiveMessageSize(te.maxServerReceiveMsgSize)) + if te.maxServerReceiveMsgSize != nil { + sopts = append(sopts, grpc.MaxReceiveMessageSize(*te.maxServerReceiveMsgSize)) } - if te.maxServerSendMsgSize >= 0 { - sopts = append(sopts, grpc.MaxSendMessageSize(te.maxServerSendMsgSize)) + if te.maxServerSendMsgSize != nil { + sopts = append(sopts, grpc.MaxSendMessageSize(*te.maxServerSendMsgSize)) } if te.tapHandle != nil { sopts = append(sopts, grpc.InTapHandle(te.tapHandle)) @@ -600,14 +594,14 @@ func (te *test) clientConn() *grpc.ClientConn { if te.streamClientInt != nil { opts = append(opts, grpc.WithStreamInterceptor(te.streamClientInt)) } - if te.maxMsgSize >= 0 { - opts = append(opts, grpc.WithMaxMsgSize(te.maxMsgSize)) + if te.maxMsgSize != nil { + opts = append(opts, grpc.WithMaxMsgSize(*te.maxMsgSize)) } - if te.maxClientReceiveMsgSize >= 0 { - opts = append(opts, grpc.WithMaxReceiveMessageSize(te.maxClientReceiveMsgSize)) + if te.maxClientReceiveMsgSize != nil { + opts = append(opts, grpc.WithDefaultCallOptions(grpc.WithMaxReceiveMessageSize(*te.maxClientReceiveMsgSize))) } - if te.maxClientSendMsgSize >= 0 { - opts = append(opts, grpc.WithMaxSendMessageSize(te.maxClientSendMsgSize)) + if te.maxClientSendMsgSize != nil { + opts = append(opts, grpc.WithDefaultCallOptions(grpc.WithMaxSendMessageSize(*te.maxClientSendMsgSize))) } if te.timeout > 0 { opts = append(opts, grpc.WithTimeout(te.timeout)) @@ -1334,8 +1328,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { // Case2: Client API set maxReqSize to 1024 (send), maxRespSize to 1024 (recv). Sc sets maxReqSize to 2048 (send), maxRespSize to 2048 (recv). te2, ch2 := testServiceConfigSetup(t, e) - te2.maxClientReceiveMsgSize = 1024 - te2.maxClientSendMsgSize = 1024 + te2.maxClientReceiveMsgSize = newInt(1024) + te2.maxClientSendMsgSize = newInt(1024) te2.startServer(&testServer{security: e.security}) defer te2.tearDown() ch2 <- sc @@ -1383,8 +1377,8 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { // Case3: Client API set maxReqSize to 4096 (send), maxRespSize to 4096 (recv). Sc sets maxReqSize to 2048 (send), maxRespSize to 2048 (recv). te3, ch3 := testServiceConfigSetup(t, e) - te3.maxClientReceiveMsgSize = 4096 - te3.maxClientSendMsgSize = 4096 + te3.maxClientReceiveMsgSize = newInt(4096) + te3.maxClientSendMsgSize = newInt(4096) te3.startServer(&testServer{security: e.security}) defer te3.tearDown() ch3 <- sc @@ -1468,7 +1462,7 @@ func testMaxMsgSizeClientDefault(t *testing.T, e env) { te := newTest(t, e) te.userAgent = testAppUA // To avoid error on server side. - te.maxServerSendMsgSize = 5 * 1024 * 1024 + te.maxServerSendMsgSize = newInt(5 * 1024 * 1024) te.declareLogNoise( "transport: http2Client.notifyError got notified that the client transport was broken EOF", "grpc: addrConn.transportMonitor exits due to: grpc: the connection is closing", @@ -1547,9 +1541,9 @@ func testMaxMsgSizeClientAPI(t *testing.T, e env) { te := newTest(t, e) te.userAgent = testAppUA // To avoid error on server side. - te.maxServerSendMsgSize = 5 * 1024 * 1024 - te.maxClientReceiveMsgSize = 1024 - te.maxClientSendMsgSize = 1024 + te.maxServerSendMsgSize = newInt(5 * 1024 * 1024) + te.maxClientReceiveMsgSize = newInt(1024) + te.maxClientSendMsgSize = newInt(1024) te.declareLogNoise( "transport: http2Client.notifyError got notified that the client transport was broken EOF", "grpc: addrConn.transportMonitor exits due to: grpc: the connection is closing", @@ -1627,8 +1621,8 @@ func testMaxMsgSizeClientAPI(t *testing.T, e env) { func testMaxMsgSizeServerAPI(t *testing.T, e env) { te := newTest(t, e) te.userAgent = testAppUA - te.maxServerReceiveMsgSize = 1024 - te.maxServerSendMsgSize = 1024 + te.maxServerReceiveMsgSize = newInt(1024) + te.maxServerSendMsgSize = newInt(1024) te.declareLogNoise( "transport: http2Client.notifyError got notified that the client transport was broken EOF", "grpc: addrConn.transportMonitor exits due to: grpc: the connection is closing", @@ -2032,12 +2026,12 @@ func TestExceedMsgLimit(t *testing.T) { func testExceedMsgLimit(t *testing.T, e env) { te := newTest(t, e) - te.maxMsgSize = 1024 + te.maxMsgSize = newInt(1024) te.startServer(&testServer{security: e.security}) defer te.tearDown() tc := testpb.NewTestServiceClient(te.clientConn()) - argSize := int32(te.maxMsgSize + 1) + argSize := int32(*te.maxMsgSize + 1) const smallSize = 1 payload, err := newPayload(testpb.PayloadType_COMPRESSABLE, argSize) @@ -2059,7 +2053,7 @@ func testExceedMsgLimit(t *testing.T, e env) { t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } // Test on client side for unary RPC. - req.ResponseSize = proto.Int32(int32(te.maxMsgSize) + 1) + req.ResponseSize = proto.Int32(int32(*te.maxMsgSize) + 1) req.Payload = smallPayload if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) @@ -2076,7 +2070,7 @@ func testExceedMsgLimit(t *testing.T, e env) { }, } - spayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, int32(te.maxMsgSize+1)) + spayload, err := newPayload(testpb.PayloadType_COMPRESSABLE, int32(*te.maxMsgSize+1)) if err != nil { t.Fatal(err) } @@ -2098,7 +2092,7 @@ func testExceedMsgLimit(t *testing.T, e env) { if err != nil { t.Fatalf("%v.FullDuplexCall(_) = _, %v, want ", tc, err) } - respParam[0].Size = proto.Int32(int32(te.maxMsgSize) + 1) + respParam[0].Size = proto.Int32(int32(*te.maxMsgSize) + 1) sreq.Payload = smallPayload if err := stream.Send(sreq); err != nil { t.Fatalf("%v.Send(%v) = %v, want ", stream, sreq, err) From eaa9ccb053a070b22622af8ba922a7a9db868e39 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Wed, 26 Apr 2017 16:23:24 -0700 Subject: [PATCH 11/23] minor comment change --- rpc_util.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rpc_util.go b/rpc_util.go index 606294be155e..36af31f8d719 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -183,7 +183,7 @@ func FailFast(failFast bool) CallOption { }) } -// WithMaxReceiveMessageSize returns a DialOption which sets the maximum message size the client can receive. Negative input is invalid and has the same effect as not setting the field. +// WithMaxReceiveMessageSize returns a DialOption which sets the maximum message size the client can receive. func WithMaxReceiveMessageSize(s int) CallOption { return beforeCall(func(o *callInfo) error { o.maxReceiveMessageSize = &s @@ -191,7 +191,7 @@ func WithMaxReceiveMessageSize(s int) CallOption { }) } -// WithMaxSendMessageSize returns a DialOption which sets the maximum message size the client can send. Negative input is invalid and has the same effect as not seeting the field. +// WithMaxSendMessageSize returns a DialOption which sets the maximum message size the client can send. func WithMaxSendMessageSize(s int) CallOption { return beforeCall(func(o *callInfo) error { o.maxSendMessageSize = &s From ecbc34aacae0c9034de85c044cfcc9600b85cc50 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Wed, 26 Apr 2017 17:39:57 -0700 Subject: [PATCH 12/23] move server defaults, delete defer cancel() in stream.go --- rpc_util.go | 22 +++++++++++----------- server.go | 5 +++++ stream.go | 2 -- 3 files changed, 16 insertions(+), 13 deletions(-) diff --git a/rpc_util.go b/rpc_util.go index 36af31f8d719..35f6dbf33db8 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -483,17 +483,6 @@ type ServiceConfig struct { Methods map[string]MethodConfig } -// SupportPackageIsVersion4 is referenced from generated protocol buffer files -// to assert that that code is compatible with this version of the grpc package. -// -// 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 - -// Version is the current grpc version. -const Version = "1.3.0-dev" - func min(a, b *int) *int { if *a < *b { return a @@ -514,4 +503,15 @@ func getMaxSize(mcMax, doptMax *int, defaultVal int) *int { return doptMax } +// SupportPackageIsVersion4 is referenced from generated protocol buffer files +// to assert that that code is compatible with this version of the grpc package. +// +// 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 + +// Version is the current grpc version. +const Version = "1.3.0-dev" + const grpcUA = "grpc-go/" + Version diff --git a/server.go b/server.go index e4433cf39543..3c8bd69c6bb3 100644 --- a/server.go +++ b/server.go @@ -61,6 +61,11 @@ import ( "google.golang.org/grpc/transport" ) +const ( + defaultServerMaxReceiveMessageSize = 1024 * 1024 * 4 + defaultServerMaxSendMessageSize = 1024 * 1024 * 4 +) + type methodHandler func(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor UnaryServerInterceptor) (interface{}, error) // MethodDesc represents an RPC service's method specification. diff --git a/stream.go b/stream.go index 8dcd0627a527..ee68bbf56b99 100644 --- a/stream.go +++ b/stream.go @@ -119,9 +119,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth } if mc.Timeout != nil && *mc.Timeout >= 0 { - var cancel context.CancelFunc ctx, cancel = context.WithTimeout(ctx, *mc.Timeout) - defer cancel() } opts = append(cc.dopts.callOptions, opts...) From ea230c713e491cc1cc6a847c52d8d7fab8901e86 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Wed, 26 Apr 2017 17:54:25 -0700 Subject: [PATCH 13/23] update --- clientconn.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/clientconn.go b/clientconn.go index 029899f1297e..38c182306673 100644 --- a/clientconn.go +++ b/clientconn.go @@ -104,8 +104,6 @@ type dialOptions struct { const ( defaultClientMaxReceiveMessageSize = 1024 * 1024 * 4 defaultClientMaxSendMessageSize = 1024 * 1024 * 4 - defaultServerMaxReceiveMessageSize = 1024 * 1024 * 4 - defaultServerMaxSendMessageSize = 1024 * 1024 * 4 ) // DialOption configures how we set up the connection. From d9265441dedf101184292acc31f8ab21b411742d Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Mon, 8 May 2017 00:17:01 +0000 Subject: [PATCH 14/23] remove unessary nil in return statement --- call.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/call.go b/call.go index 2b53d9102df9..5822086feb4b 100644 --- a/call.go +++ b/call.go @@ -119,7 +119,7 @@ func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, return Errorf(codes.Internal, "grpc: %v", err) } if len(outBuf) > *c.maxSendMessageSize { - return nil, Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(outBuf), *c.maxSendMessageSize) + return Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(outBuf), *c.maxSendMessageSize) } err = t.Write(stream, outBuf, opts) if err == nil && outPayload != nil { From 59426b3c05f5423a3e57ce65b432fe3710e2c3e2 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Mon, 8 May 2017 00:22:57 +0000 Subject: [PATCH 15/23] gofmt --- test/end2end_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/end2end_test.go b/test/end2end_test.go index 7c7ff41fef9c..2b0ea5dd2c1f 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -443,7 +443,7 @@ type test struct { userAgent string clientCompression bool serverCompression bool - timeout time.Duration + timeout time.Duration unaryClientInt grpc.UnaryClientInterceptor streamClientInt grpc.StreamClientInterceptor unaryServerInt grpc.UnaryServerInterceptor From bdf9a640e4fb00373ad986649c3af5bc594bc59a Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Mon, 15 May 2017 13:51:11 -0700 Subject: [PATCH 16/23] add timeout test, add check or pointer filed in callOption, fix minor issues --- call.go | 10 +++++- clientconn.go | 26 ++++++++++---- rpc_util.go | 4 +-- server.go | 7 ++-- stream.go | 62 ++++++++++++++++++--------------- test/end2end_test.go | 81 ++++++++++++++++++++++++++++++++++++-------- 6 files changed, 136 insertions(+), 54 deletions(-) diff --git a/call.go b/call.go index 5822086feb4b..eaccdad45a5e 100644 --- a/call.go +++ b/call.go @@ -73,6 +73,10 @@ func recvResponse(ctx context.Context, dopts dialOptions, t transport.ClientTran } } for { + if c.maxReceiveMessageSize == nil { + // TODO(lyuxuan): codes.Internal the right error to return here? + 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 @@ -118,6 +122,10 @@ func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, if err != nil { return Errorf(codes.Internal, "grpc: %v", err) } + if c.maxSendMessageSize == nil { + // TODO(lyuxuan): codes.Internal the right error to return here? + return Errorf(codes.Internal, "callInfo maxSendMessageSize field uninitialized(nil)") + } if len(outBuf) > *c.maxSendMessageSize { return Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(outBuf), *c.maxSendMessageSize) } @@ -148,7 +156,7 @@ func Invoke(ctx context.Context, method string, args, reply interface{}, cc *Cli func invoke(ctx context.Context, method string, args, reply interface{}, cc *ClientConn, opts ...CallOption) (e error) { c := defaultCallInfo - mc, _ := cc.GetMethodConfig(method) + mc := cc.GetMethodConfig(method) if mc.WaitForReady != nil { c.failFast = !*mc.WaitForReady } diff --git a/clientconn.go b/clientconn.go index 81258ec2d523..db6ad41d3959 100644 --- a/clientconn.go +++ b/clientconn.go @@ -126,7 +126,7 @@ func WithInitialConnWindowSize(s int32) DialOption { } } -// WithMaxMsgSize Deprecated: use WithMaxReceiveMessageSize instead. +// WithMaxMsgSize returns a DialOption which sets the maximum message size the client can receive. Deprecated: use WithMaxReceiveMessageSize instead. func WithMaxMsgSize(s int) DialOption { return WithDefaultCallOptions(WithMaxReceiveMessageSize(s)) } @@ -367,12 +367,14 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * } }() + scSet := false if cc.dopts.scChan != nil { // Try to get an initial service config. select { case sc, ok := <-cc.dopts.scChan: if ok { cc.sc = sc + scSet = true } default: } @@ -436,7 +438,17 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * return nil, err } } - + if cc.dopts.scChan != nil && !scSet { + // Blocking Wait for the initial service config. + select { + case sc, ok := <-cc.dopts.scChan: + if ok { + cc.sc = sc + } + case <-ctx.Done(): + return nil, ctx.Err() + } + } if cc.dopts.scChan != nil { go cc.scWatcher() } @@ -646,17 +658,19 @@ func (cc *ClientConn) resetAddrConn(addr Address, block bool, tearDownErr error) return nil } -// GetMethodConfig gets the method config of the input method. If there's no exact match for the input method (i.e. /service/method), we will return the default config for all methods under the service (/service/). +// GetMethodConfig gets the method config of the input method. If there's no exact +// match for the input method (i.e. /service/method), we will return the default +// config for all methods under the service (/service/). // TODO: Avoid the locking here. -func (cc *ClientConn) GetMethodConfig(method string) (m MethodConfig, ok bool) { +func (cc *ClientConn) GetMethodConfig(method string) MethodConfig { cc.mu.RLock() defer cc.mu.RUnlock() - m, ok = cc.sc.Methods[method] + m, ok := cc.sc.Methods[method] if !ok { i := strings.LastIndex(method, "/") m, ok = cc.sc.Methods[method[:i+1]] } - return + return m } func (cc *ClientConn) getTransport(ctx context.Context, opts BalancerGetOptions) (transport.ClientTransport, func(), error) { diff --git a/rpc_util.go b/rpc_util.go index 58f4ed9e5608..6d46706d154f 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -184,7 +184,7 @@ func FailFast(failFast bool) CallOption { }) } -// WithMaxReceiveMessageSize returns a DialOption which sets the maximum message size the client can receive. +// WithMaxReceiveMessageSize returns a CallOption which sets the maximum message size the client can receive. func WithMaxReceiveMessageSize(s int) CallOption { return beforeCall(func(o *callInfo) error { o.maxReceiveMessageSize = &s @@ -192,7 +192,7 @@ func WithMaxReceiveMessageSize(s int) CallOption { }) } -// WithMaxSendMessageSize returns a DialOption which sets the maximum message size the client can send. +// WithMaxSendMessageSize returns a CallOption which sets the maximum message size the client can send. func WithMaxSendMessageSize(s int) CallOption { return beforeCall(func(o *callInfo) error { o.maxSendMessageSize = &s diff --git a/server.go b/server.go index ddaf26220bb4..d7eff5f6369e 100644 --- a/server.go +++ b/server.go @@ -116,7 +116,6 @@ type options struct { codec Codec cp Compressor dc Decompressor - maxMsgSize int unaryInt UnaryServerInterceptor streamInt StreamServerInterceptor inTapHandle tap.ServerInHandle @@ -132,6 +131,8 @@ type options struct { initialConnWindowSize int32 } +var defaultServerOptions = options{maxReceiveMessageSize: defaultServerMaxReceiveMessageSize, maxSendMessageSize: defaultServerMaxSendMessageSize} + // A ServerOption sets options such as credentials, codec and keepalive parameters, etc. type ServerOption func(*options) @@ -284,9 +285,7 @@ func UnknownServiceHandler(streamHandler StreamHandler) ServerOption { // NewServer creates a gRPC server which has no service registered and has not // started to accept requests yet. func NewServer(opt ...ServerOption) *Server { - var opts options - opts.maxReceiveMessageSize = defaultServerMaxReceiveMessageSize - opts.maxSendMessageSize = defaultServerMaxSendMessageSize + opts := defaultServerOptions for _, o := range opt { o(&opts) } diff --git a/stream.go b/stream.go index a644f8f8d969..bf33dc38d2c0 100644 --- a/stream.go +++ b/stream.go @@ -113,7 +113,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth cancel context.CancelFunc ) c := defaultCallInfo - mc, _ := cc.GetMethodConfig(method) + mc := cc.GetMethodConfig(method) if mc.WaitForReady != nil { c.failFast = !*mc.WaitForReady } @@ -219,15 +219,13 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth break } cs := &clientStream{ - opts: opts, - c: c, - desc: desc, - codec: cc.dopts.codec, - cp: cc.dopts.cp, - dc: cc.dopts.dc, - maxReceiveMessageSize: *c.maxReceiveMessageSize, - maxSendMessageSize: *c.maxSendMessageSize, - cancel: cancel, + opts: opts, + c: c, + desc: desc, + codec: cc.dopts.codec, + cp: cc.dopts.cp, + dc: cc.dopts.dc, + cancel: cancel, put: put, t: t, @@ -271,19 +269,17 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth // 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 - cbuf *bytes.Buffer - dc Decompressor - maxReceiveMessageSize int - maxSendMessageSize int - cancel context.CancelFunc + opts []CallOption + c callInfo + t transport.ClientTransport + s *transport.Stream + p *parser + desc *StreamDesc + codec Codec + cp Compressor + cbuf *bytes.Buffer + dc Decompressor + cancel context.CancelFunc tracing bool // set to EnableTracing when the clientStream is created. @@ -367,8 +363,12 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) { if err != nil { return Errorf(codes.Internal, "grpc: %v", err) } - if len(out) > cs.maxSendMessageSize { - return Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(out), cs.maxSendMessageSize) + if cs.c.maxSendMessageSize == nil { + // TODO(lyuxuan): codes.Internal the right error to return here? + return Errorf(codes.Internal, "callInfo maxSendMessageSize field uninitialized(nil)") + } + if len(out) > *cs.c.maxSendMessageSize { + return Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(out), *cs.c.maxSendMessageSize) } err = cs.t.Write(cs.s, out, &transport.Options{Last: false}) if err == nil && outPayload != nil { @@ -385,7 +385,11 @@ func (cs *clientStream) RecvMsg(m interface{}) (err error) { Client: true, } } - err = recv(cs.p, cs.codec, cs.s, cs.dc, m, cs.maxReceiveMessageSize, inPayload) + if cs.c.maxReceiveMessageSize == nil { + // TODO(lyuxuan): codes.Internal the right error to return here? + 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 { @@ -408,7 +412,11 @@ func (cs *clientStream) RecvMsg(m interface{}) (err error) { } // Special handling for client streaming rpc. // This recv expects EOF or errors, so we don't collect inPayload. - err = recv(cs.p, cs.codec, cs.s, cs.dc, m, cs.maxReceiveMessageSize, nil) + if cs.c.maxReceiveMessageSize == nil { + // TODO(lyuxuan): codes.Internal the right error to return here? + 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 ")) diff --git a/test/end2end_test.go b/test/end2end_test.go index 2b0ea5dd2c1f..8835a5cd4d69 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -443,7 +443,6 @@ type test struct { userAgent string clientCompression bool serverCompression bool - timeout time.Duration unaryClientInt grpc.UnaryClientInterceptor streamClientInt grpc.StreamClientInterceptor unaryServerInt grpc.UnaryServerInterceptor @@ -615,9 +614,6 @@ func (te *test) clientConn() *grpc.ClientConn { if te.maxClientSendMsgSize != nil { opts = append(opts, grpc.WithDefaultCallOptions(grpc.WithMaxSendMessageSize(*te.maxClientSendMsgSize))) } - if te.timeout > 0 { - opts = append(opts, grpc.WithTimeout(te.timeout)) - } switch te.e.security { case "tls": creds, err := credentials.NewClientTLSFromFile(tlsDir+"ca.pem", "x.test.youtube.com") @@ -1118,7 +1114,7 @@ func TestServiceConfig(t *testing.T) { for _, e := range listTestEnv() { testGetMethodConfig(t, e) testServiceConfigWaitForReady(t, e) - // Timeout logic (min of service config and client API) is implemented implicitly in context. WithTimeout(). No need to test here. + testServiceConfigTimeout(t, e) testServiceConfigMaxMsgSize(t, e) } } @@ -1139,15 +1135,11 @@ func testServiceConfigSetup(t *testing.T, e env) (*test, chan grpc.ServiceConfig } func newBool(b bool) (a *bool) { - a = new(bool) - *a = b - return + return &b } func newInt(b int) (a *int) { - a = new(int) - *a = b - return + return &b } func newDuration(b time.Duration) (a *time.Duration) { @@ -1238,11 +1230,11 @@ func testServiceConfigWaitForReady(t *testing.T, e env) { ch <- sc // Wait for the new service config to take effect. - mc, ok := cc.GetMethodConfig("/grpc.testing.TestService/EmptyCall") + mc = cc.GetMethodConfig("/grpc.testing.TestService/EmptyCall") for { - if ok && !*mc.WaitForReady { + if !*mc.WaitForReady { time.Sleep(100 * time.Millisecond) - mc, ok = cc.GetMethodConfig("/grpc.testing.TestService/EmptyCall") + mc = cc.GetMethodConfig("/grpc.testing.TestService/EmptyCall") continue } break @@ -1256,6 +1248,67 @@ func testServiceConfigWaitForReady(t *testing.T, e env) { } } +func testServiceConfigTimeout(t *testing.T, e env) { + te, ch := testServiceConfigSetup(t, e) + defer te.tearDown() + + // Case1: Client API sets timeout to be 1ns and ServiceConfig sets timeout to be 1hr. Timeout should be 1ns (min of 1ns and 1hr) and the rpc will wait until deadline exceeds. + mc := grpc.MethodConfig{ + Timeout: newDuration(time.Hour), + } + m := make(map[string]grpc.MethodConfig) + m["/grpc.testing.TestService/EmptyCall"] = mc + m["/grpc.testing.TestService/FullDuplexCall"] = mc + sc := grpc.ServiceConfig{ + Methods: m, + } + ch <- sc + + cc := te.clientConn() + tc := testpb.NewTestServiceClient(cc) + // The following RPCs are expected to become non-fail-fast ones with 1ns deadline. + ctx, _ := context.WithTimeout(context.Background(), time.Nanosecond) + if _, err := tc.EmptyCall(ctx, &testpb.Empty{}, grpc.FailFast(false)); grpc.Code(err) != codes.DeadlineExceeded { + t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want _, %s", err, codes.DeadlineExceeded) + } + ctx, _ = context.WithTimeout(context.Background(), time.Nanosecond) + if _, err := tc.FullDuplexCall(ctx, grpc.FailFast(false)); grpc.Code(err) != codes.DeadlineExceeded { + t.Fatalf("TestService/FullDuplexCall(_) = _, %v, want %s", err, codes.DeadlineExceeded) + } + + // Generate a service config update. + // Case2: Client API sets timeout to be 1hr and ServiceConfig sets timeout to be 1ns. Timeout should be 1ns (min of 1ns and 1hr) and the rpc will wait until deadline exceeds. + mc.Timeout = newDuration(time.Nanosecond) + m = make(map[string]grpc.MethodConfig) + m["/grpc.testing.TestService/EmptyCall"] = mc + m["/grpc.testing.TestService/FullDuplexCall"] = mc + sc = grpc.ServiceConfig{ + Methods: m, + } + ch <- sc + + // // Wait for the new service config to take effect. + mc = cc.GetMethodConfig("/grpc.testing.TestService/FullDuplexCall") + for { + if *mc.Timeout != time.Nanosecond { + time.Sleep(100 * time.Millisecond) + mc = cc.GetMethodConfig("/grpc.testing.TestService/FullDuplexCall") + continue + } + break + } + + ctx, _ = context.WithTimeout(context.Background(), time.Hour) + if _, err := tc.EmptyCall(ctx, &testpb.Empty{}, grpc.FailFast(false)); grpc.Code(err) != codes.DeadlineExceeded { + t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want _, %s", err, codes.DeadlineExceeded) + } + + ctx, _ = context.WithTimeout(context.Background(), time.Hour) + if _, err := tc.FullDuplexCall(ctx, grpc.FailFast(false)); grpc.Code(err) != codes.DeadlineExceeded { + t.Fatalf("TestService/FullDuplexCall(_) = _, %v, want %s", err, codes.DeadlineExceeded) + } +} + func testServiceConfigMaxMsgSize(t *testing.T, e env) { // Setting up values and objects shared across all test cases. const smallSize = 1 From 4d2b4b5c51dc002e6b3f15a3e12a64cc9c808a18 Mon Sep 17 00:00:00 2001 From: lyuxuan Date: Mon, 15 May 2017 14:36:20 -0700 Subject: [PATCH 17/23] fix minor typo --- clientconn.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clientconn.go b/clientconn.go index 3949d92a64f4..54c4ca0f7dff 100644 --- a/clientconn.go +++ b/clientconn.go @@ -440,7 +440,7 @@ func DialContext(ctx context.Context, target string, opts ...DialOption) (conn * } } if cc.dopts.scChan != nil && !scSet { - // Blocking Wait for the initial service config. + // Blocking wait for the initial service config. select { case sc, ok := <-cc.dopts.scChan: if ok { From 750548184851b0f015d05b9c3730d95d1c369aee Mon Sep 17 00:00:00 2001 From: lyuxuan Date: Mon, 15 May 2017 14:41:08 -0700 Subject: [PATCH 18/23] comments added --- clientconn.go | 3 ++- server.go | 3 ++- test/end2end_test.go | 15 ++++++++------- 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/clientconn.go b/clientconn.go index 54c4ca0f7dff..a502d338e51e 100644 --- a/clientconn.go +++ b/clientconn.go @@ -661,7 +661,8 @@ func (cc *ClientConn) resetAddrConn(addr Address, block bool, tearDownErr error) // GetMethodConfig gets the method config of the input method. If there's no exact // match for the input method (i.e. /service/method), we will return the default -// config for all methods under the service (/service/). +// config for all methods under the service (/service/). Otherwise, we will return +// an empty MethodConfig. // TODO: Avoid the locking here. func (cc *ClientConn) GetMethodConfig(method string) MethodConfig { cc.mu.RLock() diff --git a/server.go b/server.go index d7eff5f6369e..4ae3372a40b3 100644 --- a/server.go +++ b/server.go @@ -187,7 +187,8 @@ func RPCDecompressor(dc Decompressor) ServerOption { } } -// MaxMsgSize Deprecated: use MaxReceiveMessageSize instead. +// MaxMsgSize returns a ServerOption to set the max message size in bytes for inbound mesages. +// If this is not set, gRPC uses the default limit. Deprecated: use MaxReceiveMessageSize instead. func MaxMsgSize(m int) ServerOption { return MaxReceiveMessageSize(m) } diff --git a/test/end2end_test.go b/test/end2end_test.go index 2ac5047f8b02..241cc90c3da1 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -1190,6 +1190,7 @@ func testGetMethodConfig(t *testing.T, e env) { } break } + // The following RPCs are expected to become fail-fast. if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}); grpc.Code(err) != codes.Unavailable { t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want _, %s", err, codes.Unavailable) } @@ -1291,7 +1292,7 @@ func testServiceConfigTimeout(t *testing.T, e env) { } ch <- sc - // // Wait for the new service config to take effect. + // Wait for the new service config to take effect. mc = cc.GetMethodConfig("/grpc.testing.TestService/FullDuplexCall") for { if *mc.Timeout != time.Nanosecond { @@ -1356,19 +1357,19 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { ResponseSize: proto.Int32(int32(extraLargeSize)), Payload: smallPayload, } - // test for unary RPC recv + // Test for unary RPC recv. if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } - // test for unary RPC send + // Test for unary RPC send. req.Payload = extraLargePayload req.ResponseSize = proto.Int32(int32(smallSize)) if _, err := tc.UnaryCall(context.Background(), req); err == nil || grpc.Code(err) != codes.ResourceExhausted { t.Fatalf("TestService/UnaryCall(_, _) = _, %v, want _, error code: %s", err, codes.ResourceExhausted) } - // test for streaming RPC recv + // Test for streaming RPC recv. respParam := []*testpb.ResponseParameters{ { Size: proto.Int32(int32(extraLargeSize)), @@ -1390,7 +1391,7 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } - // test for streaming RPC send + // Test for streaming RPC send. respParam[0].Size = proto.Int32(int32(smallSize)) sreq.Payload = extraLargePayload stream, err = tc.FullDuplexCall(te1.ctx) @@ -1439,7 +1440,7 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } - // Test for streaming RPC send. + // Test for streaming RPC send. respParam[0].Size = proto.Int32(int32(smallSize)) sreq.Payload = largePayload stream, err = tc.FullDuplexCall(te2.ctx) @@ -1508,7 +1509,7 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { t.Fatalf("%v.Recv() = _, %v, want _, error code: %s", stream, err, codes.ResourceExhausted) } - // Test for streaming RPC send. + // Test for streaming RPC send. respParam[0].Size = proto.Int32(int32(smallSize)) sreq.Payload = largePayload stream, err = tc.FullDuplexCall(te3.ctx) From d19bbe846e809da62d36f3c9df469e6736be9693 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Fri, 19 May 2017 11:08:40 -0700 Subject: [PATCH 19/23] change max message size functions name --- call.go | 2 +- clientconn.go | 4 ++-- rpc_util.go | 12 ++++++------ server.go | 23 +++++++++++++---------- stream.go | 4 ++-- test/end2end_test.go | 8 ++++---- 6 files changed, 28 insertions(+), 25 deletions(-) diff --git a/call.go b/call.go index 73f1e7f7ad8a..f6c6399406b2 100644 --- a/call.go +++ b/call.go @@ -127,7 +127,7 @@ func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, return Errorf(codes.Internal, "callInfo maxSendMessageSize field uninitialized(nil)") } if len(outBuf) > *c.maxSendMessageSize { - return Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(outBuf), *c.maxSendMessageSize) + return Errorf(codes.ResourceExhausted, "grpc: trying to send message larger than max (%d vs. %d)", len(outBuf), *c.maxSendMessageSize) } err = t.Write(stream, outBuf, opts) if err == nil && outPayload != nil { diff --git a/clientconn.go b/clientconn.go index a502d338e51e..9359366dcbb6 100644 --- a/clientconn.go +++ b/clientconn.go @@ -126,9 +126,9 @@ func WithInitialConnWindowSize(s int32) DialOption { } } -// WithMaxMsgSize returns a DialOption which sets the maximum message size the client can receive. Deprecated: use WithMaxReceiveMessageSize 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(WithMaxReceiveMessageSize(s)) + return WithDefaultCallOptions(MaxCallRecvMsgSize(s)) } // WithDefaultCallOptions returns a DialOption which sets the default CallOptions for calls over the connection. diff --git a/rpc_util.go b/rpc_util.go index 4a4ca409f480..00b08337938c 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -219,16 +219,16 @@ func FailFast(failFast bool) CallOption { }) } -// WithMaxReceiveMessageSize returns a CallOption which sets the maximum message size the client can receive. -func WithMaxReceiveMessageSize(s int) CallOption { +// 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 }) } -// WithMaxSendMessageSize returns a CallOption which sets the maximum message size the client can send. -func WithMaxSendMessageSize(s int) CallOption { +// 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 @@ -289,7 +289,7 @@ func (p *parser) recvMsg(maxReceiveMessageSize int) (pf payloadFormat, msg []byt 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) + return 0, nil, 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: @@ -393,7 +393,7 @@ func recv(p *parser, c Codec, s *transport.Stream, dc Decompressor, m interface{ 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 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) diff --git a/server.go b/server.go index 5807daee1d3d..62f4149a0c1e 100644 --- a/server.go +++ b/server.go @@ -131,7 +131,10 @@ type options struct { initialConnWindowSize int32 } -var defaultServerOptions = options{maxReceiveMessageSize: defaultServerMaxReceiveMessageSize, maxSendMessageSize: defaultServerMaxSendMessageSize} +var defaultServerOptions = options{ + maxReceiveMessageSize: defaultServerMaxReceiveMessageSize, + maxSendMessageSize: defaultServerMaxSendMessageSize, +} // A ServerOption sets options such as credentials, codec and keepalive parameters, etc. type ServerOption func(*options) @@ -187,23 +190,23 @@ func RPCDecompressor(dc Decompressor) ServerOption { } } -// MaxMsgSize returns a ServerOption to set the max message size in bytes for inbound mesages. -// If this is not set, gRPC uses the default limit. Deprecated: use MaxReceiveMessageSize instead. +// 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. func MaxMsgSize(m int) ServerOption { - return MaxReceiveMessageSize(m) + return MaxRecvMsgSize(m) } -// MaxReceiveMessageSize returns a ServerOption to set the max message size in bytes for inbound mesages. +// MaxRecvMsgSize returns a ServerOption to set the max message size in bytes the server can receive. // If this is not set, gRPC uses the default 4MB. -func MaxReceiveMessageSize(m int) ServerOption { +func MaxRecvMsgSize(m int) ServerOption { return func(o *options) { o.maxReceiveMessageSize = m } } -// MaxSendMessageSize returns a ServerOption to set the max message size in bytes for outbound mesages. +// MaxSendMsgSize returns a ServerOption to set the max message size in bytes the server can send. // If this is not set, gRPC uses the default 4MB. -func MaxSendMessageSize(m int) ServerOption { +func MaxSendMsgSize(m int) ServerOption { return func(o *options) { o.maxSendMessageSize = m } @@ -669,7 +672,7 @@ func (s *Server) sendResponse(t transport.ServerTransport, stream *transport.Str grpclog.Fatalf("grpc: Server failed to encode response %v", err) } if len(p) > s.opts.maxSendMessageSize { - return status.Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(p), s.opts.maxSendMessageSize) + return status.Errorf(codes.ResourceExhausted, "grpc: trying to send message larger than max (%d vs. %d)", len(p), s.opts.maxSendMessageSize) } err = t.Write(stream, p, opts) if err == nil && outPayload != nil { @@ -773,7 +776,7 @@ func (s *Server) processUnaryRPC(t transport.ServerTransport, stream *transport. if len(req) > s.opts.maxReceiveMessageSize { // TODO: Revisit the error code. Currently keep it consistent with // java implementation. - return status.Errorf(codes.ResourceExhausted, "Received message larger than max (%d vs. %d)", len(req), s.opts.maxReceiveMessageSize) + 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 { return status.Errorf(codes.Internal, "grpc: error unmarshalling request: %v", err) diff --git a/stream.go b/stream.go index 38e03a51c8a4..a999037b39f3 100644 --- a/stream.go +++ b/stream.go @@ -371,7 +371,7 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) { return Errorf(codes.Internal, "callInfo maxSendMessageSize field uninitialized(nil)") } if len(out) > *cs.c.maxSendMessageSize { - return Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(out), *cs.c.maxSendMessageSize) + return Errorf(codes.ResourceExhausted, "trying to send message larger than max (%d vs. %d)", len(out), *cs.c.maxSendMessageSize) } err = cs.t.Write(cs.s, out, &transport.Options{Last: false}) if err == nil && outPayload != nil { @@ -613,7 +613,7 @@ func (ss *serverStream) SendMsg(m interface{}) (err error) { return err } if len(out) > ss.maxSendMessageSize { - return Errorf(codes.ResourceExhausted, "Sent message larger than max (%d vs. %d)", len(out), ss.maxSendMessageSize) + return Errorf(codes.ResourceExhausted, "trying to send message larger than max (%d vs. %d)", len(out), ss.maxSendMessageSize) } if err := ss.t.Write(ss.s, out, &transport.Options{Last: false}); err != nil { return toRPCErr(err) diff --git a/test/end2end_test.go b/test/end2end_test.go index 5f72c79fae22..61ad8ee555de 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -504,10 +504,10 @@ func (te *test) startServer(ts testpb.TestServiceServer) { sopts = append(sopts, grpc.MaxMsgSize(*te.maxMsgSize)) } if te.maxServerReceiveMsgSize != nil { - sopts = append(sopts, grpc.MaxReceiveMessageSize(*te.maxServerReceiveMsgSize)) + sopts = append(sopts, grpc.MaxRecvMsgSize(*te.maxServerReceiveMsgSize)) } if te.maxServerSendMsgSize != nil { - sopts = append(sopts, grpc.MaxSendMessageSize(*te.maxServerSendMsgSize)) + sopts = append(sopts, grpc.MaxSendMsgSize(*te.maxServerSendMsgSize)) } if te.tapHandle != nil { sopts = append(sopts, grpc.InTapHandle(te.tapHandle)) @@ -610,10 +610,10 @@ func (te *test) clientConn() *grpc.ClientConn { opts = append(opts, grpc.WithMaxMsgSize(*te.maxMsgSize)) } if te.maxClientReceiveMsgSize != nil { - opts = append(opts, grpc.WithDefaultCallOptions(grpc.WithMaxReceiveMessageSize(*te.maxClientReceiveMsgSize))) + opts = append(opts, grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(*te.maxClientReceiveMsgSize))) } if te.maxClientSendMsgSize != nil { - opts = append(opts, grpc.WithDefaultCallOptions(grpc.WithMaxSendMessageSize(*te.maxClientSendMsgSize))) + opts = append(opts, grpc.WithDefaultCallOptions(grpc.MaxCallSendMsgSize(*te.maxClientSendMsgSize))) } switch te.e.security { case "tls": From ed64d51c1868620f855c82f82fedb21b738bcfaf Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Fri, 19 May 2017 11:52:09 -0700 Subject: [PATCH 20/23] remove unnecessary ok --- clientconn.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clientconn.go b/clientconn.go index 9359366dcbb6..e69ba464c276 100644 --- a/clientconn.go +++ b/clientconn.go @@ -670,7 +670,7 @@ func (cc *ClientConn) GetMethodConfig(method string) MethodConfig { m, ok := cc.sc.Methods[method] if !ok { i := strings.LastIndex(method, "/") - m, ok = cc.sc.Methods[method[:i+1]] + m, _ = cc.sc.Methods[method[:i+1]] } return m } From 27ae1472a32c23fae99c5386dfee2e954ecf0398 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Fri, 19 May 2017 14:55:35 -0700 Subject: [PATCH 21/23] remove some todo comments --- call.go | 2 -- stream.go | 3 --- 2 files changed, 5 deletions(-) diff --git a/call.go b/call.go index f6c6399406b2..a2b89ac6a36e 100644 --- a/call.go +++ b/call.go @@ -74,7 +74,6 @@ func recvResponse(ctx context.Context, dopts dialOptions, t transport.ClientTran } for { if c.maxReceiveMessageSize == nil { - // TODO(lyuxuan): codes.Internal the right error to return here? return Errorf(codes.Internal, "callInfo maxReceiveMessageSize field uninitialized(nil)") } if err = recv(p, dopts.codec, stream, dopts.dc, reply, *c.maxReceiveMessageSize, inPayload); err != nil { @@ -123,7 +122,6 @@ func sendRequest(ctx context.Context, dopts dialOptions, compressor Compressor, return Errorf(codes.Internal, "grpc: %v", err) } if c.maxSendMessageSize == nil { - // TODO(lyuxuan): codes.Internal the right error to return here? return Errorf(codes.Internal, "callInfo maxSendMessageSize field uninitialized(nil)") } if len(outBuf) > *c.maxSendMessageSize { diff --git a/stream.go b/stream.go index a999037b39f3..f26b96edaa48 100644 --- a/stream.go +++ b/stream.go @@ -367,7 +367,6 @@ func (cs *clientStream) SendMsg(m interface{}) (err error) { return Errorf(codes.Internal, "grpc: %v", err) } if cs.c.maxSendMessageSize == nil { - // TODO(lyuxuan): codes.Internal the right error to return here? return Errorf(codes.Internal, "callInfo maxSendMessageSize field uninitialized(nil)") } if len(out) > *cs.c.maxSendMessageSize { @@ -389,7 +388,6 @@ func (cs *clientStream) RecvMsg(m interface{}) (err error) { } } if cs.c.maxReceiveMessageSize == nil { - // TODO(lyuxuan): codes.Internal the right error to return here? return Errorf(codes.Internal, "callInfo maxReceiveMessageSize field uninitialized(nil)") } err = recv(cs.p, cs.codec, cs.s, cs.dc, m, *cs.c.maxReceiveMessageSize, inPayload) @@ -416,7 +414,6 @@ func (cs *clientStream) RecvMsg(m interface{}) (err error) { // Special handling for client streaming rpc. // This recv expects EOF or errors, so we don't collect inPayload. if cs.c.maxReceiveMessageSize == nil { - // TODO(lyuxuan): codes.Internal the right error to return here? return Errorf(codes.Internal, "callInfo maxReceiveMessageSize field uninitialized(nil)") } err = recv(cs.p, cs.codec, cs.s, cs.dc, m, *cs.c.maxReceiveMessageSize, nil) From cb649383816bf2b57143c15b602f171960f49987 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Fri, 19 May 2017 16:02:02 -0700 Subject: [PATCH 22/23] fix minor issues --- clientconn.go | 13 ++++++---- rpc_util.go | 2 +- test/end2end_test.go | 56 ++++++++++++++++++++++++++++++++++---------- 3 files changed, 52 insertions(+), 19 deletions(-) diff --git a/clientconn.go b/clientconn.go index e69ba464c276..9dca29c37b52 100644 --- a/clientconn.go +++ b/clientconn.go @@ -659,12 +659,15 @@ func (cc *ClientConn) resetAddrConn(addr Address, block bool, tearDownErr error) return nil } -// GetMethodConfig gets the method config of the input method. If there's no exact -// match for the input method (i.e. /service/method), we will return the default -// config for all methods under the service (/service/). Otherwise, we will return -// an empty MethodConfig. -// TODO: Avoid the locking here. +// GetMethodConfig gets the method config of the input method. +// If there's an exact match for input method (i.e. /service/method), we return +// 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. +// Otherwise, we return an empty MethodConfig. func (cc *ClientConn) GetMethodConfig(method string) MethodConfig { + // TODO: Avoid the locking here. cc.mu.RLock() defer cc.mu.RUnlock() m, ok := cc.sc.Methods[method] diff --git a/rpc_util.go b/rpc_util.go index 00b08337938c..11558d701721 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -546,7 +546,7 @@ type ServiceConfig struct { 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 all methods under the service (/service/) and 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 } diff --git a/test/end2end_test.go b/test/end2end_test.go index 61ad8ee555de..b028e83eb9bc 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -1113,16 +1113,6 @@ func testFailFast(t *testing.T, e env) { awaitNewConnLogOutput() } -func TestServiceConfig(t *testing.T) { - defer leakCheck(t)() - for _, e := range listTestEnv() { - testGetMethodConfig(t, e) - testServiceConfigWaitForReady(t, e) - testServiceConfigTimeout(t, e) - testServiceConfigMaxMsgSize(t, e) - } -} - func testServiceConfigSetup(t *testing.T, e env) (*test, chan grpc.ServiceConfig) { te := newTest(t, e) // We write before read. @@ -1152,6 +1142,13 @@ func newDuration(b time.Duration) (a *time.Duration) { return } +func TestServiceConfigGetMethodConfig(t *testing.T) { + defer leakCheck(t)() + for _, e := range listTestEnv() { + testGetMethodConfig(t, e) + } +} + func testGetMethodConfig(t *testing.T, e env) { te, ch := testServiceConfigSetup(t, e) defer te.tearDown() @@ -1196,6 +1193,13 @@ func testGetMethodConfig(t *testing.T, e env) { } } +func TestServiceConfigWaitForReady(t *testing.T) { + defer leakCheck(t)() + for _, e := range listTestEnv() { + testServiceConfigWaitForReady(t, e) + } +} + func testServiceConfigWaitForReady(t *testing.T, e env) { te, ch := testServiceConfigSetup(t, e) defer te.tearDown() @@ -1253,6 +1257,13 @@ func testServiceConfigWaitForReady(t *testing.T, e env) { } } +func TestServiceConfigTimeout(t *testing.T) { + defer leakCheck(t)() + for _, e := range listTestEnv() { + testServiceConfigTimeout(t, e) + } +} + func testServiceConfigTimeout(t *testing.T, e env) { te, ch := testServiceConfigSetup(t, e) defer te.tearDown() @@ -1314,6 +1325,13 @@ func testServiceConfigTimeout(t *testing.T, e env) { } } +func TestServiceConfigMaxMsgSize(t *testing.T) { + defer leakCheck(t)() + for _, e := range listTestEnv() { + testServiceConfigMaxMsgSize(t, e) + } +} + func testServiceConfigMaxMsgSize(t *testing.T, e env) { // Setting up values and objects shared across all test cases. const smallSize = 1 @@ -1525,12 +1543,10 @@ func testServiceConfigMaxMsgSize(t *testing.T, e env) { } } -func TestMsgSizeDefaultAndAPI(t *testing.T) { +func TestMaxMsgSizeClientDefault(t *testing.T) { defer leakCheck(t)() for _, e := range listTestEnv() { testMaxMsgSizeClientDefault(t, e) - testMaxMsgSizeClientAPI(t, e) - testMaxMsgSizeServerAPI(t, e) } } @@ -1613,6 +1629,13 @@ func testMaxMsgSizeClientDefault(t *testing.T, e env) { } } +func TestMaxMsgSizeClientAPI(t *testing.T) { + defer leakCheck(t)() + for _, e := range listTestEnv() { + testMaxMsgSizeClientAPI(t, e) + } +} + func testMaxMsgSizeClientAPI(t *testing.T, e env) { te := newTest(t, e) te.userAgent = testAppUA @@ -1694,6 +1717,13 @@ func testMaxMsgSizeClientAPI(t *testing.T, e env) { } } +func TestMaxMsgSizeServerAPI(t *testing.T) { + defer leakCheck(t)() + for _, e := range listTestEnv() { + testMaxMsgSizeServerAPI(t, e) + } +} + func testMaxMsgSizeServerAPI(t *testing.T, e env) { te := newTest(t, e) te.userAgent = testAppUA From 4a7b4d033a30b8f3a1cf869c82c8b6117fdbf282 Mon Sep 17 00:00:00 2001 From: Yuxuan Li Date: Fri, 19 May 2017 16:38:37 -0700 Subject: [PATCH 23/23] minor fix --- stream.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stream.go b/stream.go index f26b96edaa48..ed0ebe7b73ec 100644 --- a/stream.go +++ b/stream.go @@ -118,7 +118,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth c.failFast = !*mc.WaitForReady } - if mc.Timeout != nil && *mc.Timeout >= 0 { + if mc.Timeout != nil { ctx, cancel = context.WithTimeout(ctx, *mc.Timeout) }