Make window size configurable. (#1210)

* Make window size configurable.
This commit is contained in:
MakMukhi
2017-05-05 13:26:56 -07:00
committed by GitHub
parent 07f4e6bf4e
commit f3b5bf53ce
7 changed files with 407 additions and 91 deletions

View File

@ -105,6 +105,22 @@ const defaultClientMaxMsgSize = math.MaxInt32
// DialOption configures how we set up the connection. // DialOption configures how we set up the connection.
type DialOption func(*dialOptions) type DialOption func(*dialOptions)
// WithInitialWindowSize returns a DialOption which sets the value for initial window size on a stream.
// The lower bound for window size is 64K and any value smaller than that will be ignored.
func WithInitialWindowSize(s int32) DialOption {
return func(o *dialOptions) {
o.copts.InitialWindowSize = s
}
}
// WithInitialConnWindowSize returns a DialOption which sets the value for initial window size on a connection.
// The lower bound for window size is 64K and any value smaller than that will be ignored.
func WithInitialConnWindowSize(s int32) DialOption {
return func(o *dialOptions) {
o.copts.InitialConnWindowSize = s
}
}
// 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.
func WithMaxMsgSize(s int) DialOption { func WithMaxMsgSize(s int) DialOption {
return func(o *dialOptions) { return func(o *dialOptions) {

View File

@ -107,20 +107,22 @@ type Server struct {
} }
type options struct { type options struct {
creds credentials.TransportCredentials creds credentials.TransportCredentials
codec Codec codec Codec
cp Compressor cp Compressor
dc Decompressor dc Decompressor
maxMsgSize int maxMsgSize int
unaryInt UnaryServerInterceptor unaryInt UnaryServerInterceptor
streamInt StreamServerInterceptor streamInt StreamServerInterceptor
inTapHandle tap.ServerInHandle inTapHandle tap.ServerInHandle
statsHandler stats.Handler statsHandler stats.Handler
maxConcurrentStreams uint32 maxConcurrentStreams uint32
useHandlerImpl bool // use http.Handler-based server useHandlerImpl bool // use http.Handler-based server
unknownStreamDesc *StreamDesc unknownStreamDesc *StreamDesc
keepaliveParams keepalive.ServerParameters keepaliveParams keepalive.ServerParameters
keepalivePolicy keepalive.EnforcementPolicy keepalivePolicy keepalive.EnforcementPolicy
initialWindowSize int32
initialConnWindowSize int32
} }
var defaultMaxMsgSize = 1024 * 1024 * 4 // use 4MB as the default message size limit var defaultMaxMsgSize = 1024 * 1024 * 4 // use 4MB as the default message size limit
@ -128,6 +130,22 @@ var defaultMaxMsgSize = 1024 * 1024 * 4 // use 4MB as the default message size l
// A ServerOption sets options such as credentials, codec and keepalive parameters, etc. // A ServerOption sets options such as credentials, codec and keepalive parameters, etc.
type ServerOption func(*options) type ServerOption func(*options)
// InitialWindowSize returns a ServerOption that sets window size for stream.
// The lower bound for window size is 64K and any value smaller than that will be ignored.
func InitialWindowSize(s int32) ServerOption {
return func(o *options) {
o.initialWindowSize = s
}
}
// InitialConnWindowSize returns a ServerOption that sets window size for a connection.
// The lower bound for window size is 64K and any value smaller than that will be ignored.
func InitialConnWindowSize(s int32) ServerOption {
return func(o *options) {
o.initialConnWindowSize = s
}
}
// KeepaliveParams returns a ServerOption that sets keepalive and max-age parameters for the server. // KeepaliveParams returns a ServerOption that sets keepalive and max-age parameters for the server.
func KeepaliveParams(kp keepalive.ServerParameters) ServerOption { func KeepaliveParams(kp keepalive.ServerParameters) ServerOption {
return func(o *options) { return func(o *options) {
@ -483,12 +501,14 @@ func (s *Server) handleRawConn(rawConn net.Conn) {
// transport.NewServerTransport). // transport.NewServerTransport).
func (s *Server) serveHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo) { func (s *Server) serveHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo) {
config := &transport.ServerConfig{ config := &transport.ServerConfig{
MaxStreams: s.opts.maxConcurrentStreams, MaxStreams: s.opts.maxConcurrentStreams,
AuthInfo: authInfo, AuthInfo: authInfo,
InTapHandle: s.opts.inTapHandle, InTapHandle: s.opts.inTapHandle,
StatsHandler: s.opts.statsHandler, StatsHandler: s.opts.statsHandler,
KeepaliveParams: s.opts.keepaliveParams, KeepaliveParams: s.opts.keepaliveParams,
KeepalivePolicy: s.opts.keepalivePolicy, KeepalivePolicy: s.opts.keepalivePolicy,
InitialWindowSize: s.opts.initialWindowSize,
InitialConnWindowSize: s.opts.initialConnWindowSize,
} }
st, err := transport.NewServerTransport("http2", c, config) st, err := transport.NewServerTransport("http2", c, config)
if err != nil { if err != nil {

View File

@ -431,20 +431,24 @@ type test struct {
cancel context.CancelFunc cancel context.CancelFunc
// Configurable knobs, after newTest returns: // Configurable knobs, after newTest returns:
testServer testpb.TestServiceServer // nil means none testServer testpb.TestServiceServer // nil means none
healthServer *health.Server // nil means disabled healthServer *health.Server // nil means disabled
maxStream uint32 maxStream uint32
tapHandle tap.ServerInHandle tapHandle tap.ServerInHandle
maxMsgSize int maxMsgSize int
userAgent string userAgent string
clientCompression bool clientCompression bool
serverCompression bool serverCompression bool
unaryClientInt grpc.UnaryClientInterceptor unaryClientInt grpc.UnaryClientInterceptor
streamClientInt grpc.StreamClientInterceptor streamClientInt grpc.StreamClientInterceptor
unaryServerInt grpc.UnaryServerInterceptor unaryServerInt grpc.UnaryServerInterceptor
streamServerInt grpc.StreamServerInterceptor streamServerInt grpc.StreamServerInterceptor
unknownHandler grpc.StreamHandler unknownHandler grpc.StreamHandler
sc <-chan grpc.ServiceConfig sc <-chan grpc.ServiceConfig
serverInitialWindowSize int32
serverInitialConnWindowSize int32
clientInitialWindowSize int32
clientInitialConnWindowSize int32
// srv and srvAddr are set once startServer is called. // srv and srvAddr are set once startServer is called.
srv *grpc.Server srv *grpc.Server
@ -512,6 +516,12 @@ func (te *test) startServer(ts testpb.TestServiceServer) {
if te.unknownHandler != nil { if te.unknownHandler != nil {
sopts = append(sopts, grpc.UnknownServiceHandler(te.unknownHandler)) sopts = append(sopts, grpc.UnknownServiceHandler(te.unknownHandler))
} }
if te.serverInitialWindowSize > 0 {
sopts = append(sopts, grpc.InitialWindowSize(te.serverInitialWindowSize))
}
if te.serverInitialConnWindowSize > 0 {
sopts = append(sopts, grpc.InitialConnWindowSize(te.serverInitialConnWindowSize))
}
la := "localhost:0" la := "localhost:0"
switch te.e.network { switch te.e.network {
case "unix": case "unix":
@ -605,6 +615,12 @@ func (te *test) clientConn() *grpc.ClientConn {
if te.e.balancer { if te.e.balancer {
opts = append(opts, grpc.WithBalancer(grpc.RoundRobin(nil))) opts = append(opts, grpc.WithBalancer(grpc.RoundRobin(nil)))
} }
if te.clientInitialWindowSize > 0 {
opts = append(opts, grpc.WithInitialWindowSize(te.clientInitialWindowSize))
}
if te.clientInitialConnWindowSize > 0 {
opts = append(opts, grpc.WithInitialConnWindowSize(te.clientInitialConnWindowSize))
}
var err error var err error
te.cc, err = grpc.Dial(te.srvAddr, opts...) te.cc, err = grpc.Dial(te.srvAddr, opts...)
if err != nil { if err != nil {
@ -3881,3 +3897,90 @@ func TestStreamingProxyDoesNotForwardMetadata(t *testing.T) {
t.Fatalf("doFDC(_, proxy.client) = %v; want nil", err) t.Fatalf("doFDC(_, proxy.client) = %v; want nil", err)
} }
} }
type windowSizeConfig struct {
serverStream int32
serverConn int32
clientStream int32
clientConn int32
}
func max(a, b int32) int32 {
if a > b {
return a
}
return b
}
func TestConfigurableWindowSizeWithLargeWindow(t *testing.T) {
defer leakCheck(t)()
wc := windowSizeConfig{
serverStream: 8 * 1024 * 1024,
serverConn: 12 * 1024 * 1024,
clientStream: 6 * 1024 * 1024,
clientConn: 8 * 1024 * 1024,
}
for _, e := range listTestEnv() {
testConfigurableWindowSize(t, e, wc)
}
}
func TestConfigurableWindowSizeWithSmallWindow(t *testing.T) {
defer leakCheck(t)()
wc := windowSizeConfig{
serverStream: 1,
serverConn: 1,
clientStream: 1,
clientConn: 1,
}
for _, e := range listTestEnv() {
testConfigurableWindowSize(t, e, wc)
}
}
func testConfigurableWindowSize(t *testing.T, e env, wc windowSizeConfig) {
te := newTest(t, e)
te.serverInitialWindowSize = wc.serverStream
te.serverInitialConnWindowSize = wc.serverConn
te.clientInitialWindowSize = wc.clientStream
te.clientInitialConnWindowSize = wc.clientConn
te.startServer(&testServer{security: e.security})
defer te.tearDown()
cc := te.clientConn()
tc := testpb.NewTestServiceClient(cc)
stream, err := tc.FullDuplexCall(context.Background())
if err != nil {
t.Fatalf("%v.FullDuplexCall(_) = _, %v, want <nil>", tc, err)
}
numOfIter := 11
// Set message size to exhaust largest of window sizes.
messageSize := max(max(wc.serverStream, wc.serverConn), max(wc.clientStream, wc.clientConn)) / int32(numOfIter-1)
messageSize = max(messageSize, 64*1024)
payload, err := newPayload(testpb.PayloadType_COMPRESSABLE, messageSize)
if err != nil {
t.Fatal(err)
}
respParams := []*testpb.ResponseParameters{
{
Size: proto.Int32(messageSize),
},
}
req := &testpb.StreamingOutputCallRequest{
ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(),
ResponseParameters: respParams,
Payload: payload,
}
for i := 0; i < numOfIter; i++ {
if err := stream.Send(req); err != nil {
t.Fatalf("%v.Send(%v) = %v, want <nil>", stream, req, err)
}
if _, err := stream.Recv(); err != nil {
t.Fatalf("%v.Recv() = _, %v, want _, <nil>", stream, err)
}
}
if err := stream.CloseSend(); err != nil {
t.Fatalf("%v.CloseSend() = %v, want <nil>", stream, err)
}
}

View File

@ -110,6 +110,8 @@ type http2Client struct {
statsHandler stats.Handler statsHandler stats.Handler
initialWindowSize int32
mu sync.Mutex // guard the following variables mu sync.Mutex // guard the following variables
state transportState // the state of underlying connection state transportState // the state of underlying connection
activeStreams map[uint32]*Stream activeStreams map[uint32]*Stream
@ -198,6 +200,10 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (
if kp.Timeout == 0 { if kp.Timeout == 0 {
kp.Timeout = defaultClientKeepaliveTimeout kp.Timeout = defaultClientKeepaliveTimeout
} }
icwz := int32(initialConnWindowSize)
if opts.InitialConnWindowSize >= defaultWindowSize {
icwz = opts.InitialConnWindowSize
}
var buf bytes.Buffer var buf bytes.Buffer
t := &http2Client{ t := &http2Client{
ctx: ctx, ctx: ctx,
@ -209,27 +215,31 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (
localAddr: conn.LocalAddr(), localAddr: conn.LocalAddr(),
authInfo: authInfo, authInfo: authInfo,
// The client initiated stream id is odd starting from 1. // The client initiated stream id is odd starting from 1.
nextID: 1, nextID: 1,
writableChan: make(chan int, 1), writableChan: make(chan int, 1),
shutdownChan: make(chan struct{}), shutdownChan: make(chan struct{}),
errorChan: make(chan struct{}), errorChan: make(chan struct{}),
goAway: make(chan struct{}), goAway: make(chan struct{}),
awakenKeepalive: make(chan struct{}, 1), awakenKeepalive: make(chan struct{}, 1),
framer: newFramer(conn), framer: newFramer(conn),
hBuf: &buf, hBuf: &buf,
hEnc: hpack.NewEncoder(&buf), hEnc: hpack.NewEncoder(&buf),
controlBuf: newRecvBuffer(), controlBuf: newRecvBuffer(),
fc: &inFlow{limit: initialConnWindowSize}, fc: &inFlow{limit: uint32(icwz)},
sendQuotaPool: newQuotaPool(defaultWindowSize), sendQuotaPool: newQuotaPool(defaultWindowSize),
scheme: scheme, scheme: scheme,
state: reachable, state: reachable,
activeStreams: make(map[uint32]*Stream), activeStreams: make(map[uint32]*Stream),
creds: opts.PerRPCCredentials, creds: opts.PerRPCCredentials,
maxStreams: defaultMaxStreamsClient, maxStreams: defaultMaxStreamsClient,
streamsQuota: newQuotaPool(defaultMaxStreamsClient), streamsQuota: newQuotaPool(defaultMaxStreamsClient),
streamSendQuota: defaultWindowSize, streamSendQuota: defaultWindowSize,
kp: kp, kp: kp,
statsHandler: opts.StatsHandler, statsHandler: opts.StatsHandler,
initialWindowSize: initialWindowSize,
}
if opts.InitialWindowSize >= defaultWindowSize {
t.initialWindowSize = opts.InitialWindowSize
} }
// Make sure awakenKeepalive can't be written upon. // Make sure awakenKeepalive can't be written upon.
// keepalive routine will make it writable, if need be. // keepalive routine will make it writable, if need be.
@ -258,10 +268,10 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (
t.Close() t.Close()
return nil, connectionErrorf(true, err, "transport: preface mismatch, wrote %d bytes; want %d", n, len(clientPreface)) return nil, connectionErrorf(true, err, "transport: preface mismatch, wrote %d bytes; want %d", n, len(clientPreface))
} }
if initialWindowSize != defaultWindowSize { if t.initialWindowSize != defaultWindowSize {
err = t.framer.writeSettings(true, http2.Setting{ err = t.framer.writeSettings(true, http2.Setting{
ID: http2.SettingInitialWindowSize, ID: http2.SettingInitialWindowSize,
Val: uint32(initialWindowSize), Val: uint32(t.initialWindowSize),
}) })
} else { } else {
err = t.framer.writeSettings(true) err = t.framer.writeSettings(true)
@ -271,7 +281,7 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (
return nil, connectionErrorf(true, err, "transport: %v", err) return nil, connectionErrorf(true, err, "transport: %v", err)
} }
// Adjust the connection flow control window if needed. // Adjust the connection flow control window if needed.
if delta := uint32(initialConnWindowSize - defaultWindowSize); delta > 0 { if delta := uint32(icwz - defaultWindowSize); delta > 0 {
if err := t.framer.writeWindowUpdate(true, 0, delta); err != nil { if err := t.framer.writeWindowUpdate(true, 0, delta); err != nil {
t.Close() t.Close()
return nil, connectionErrorf(true, err, "transport: %v", err) return nil, connectionErrorf(true, err, "transport: %v", err)
@ -294,7 +304,7 @@ func (t *http2Client) newStream(ctx context.Context, callHdr *CallHdr) *Stream {
method: callHdr.Method, method: callHdr.Method,
sendCompress: callHdr.SendCompress, sendCompress: callHdr.SendCompress,
buf: newRecvBuffer(), buf: newRecvBuffer(),
fc: &inFlow{limit: initialWindowSize}, fc: &inFlow{limit: uint32(t.initialWindowSize)},
sendQuotaPool: newQuotaPool(int(t.streamSendQuota)), sendQuotaPool: newQuotaPool(int(t.streamSendQuota)),
headerChan: make(chan struct{}), headerChan: make(chan struct{}),
} }

View File

@ -113,6 +113,8 @@ type http2Server struct {
// 1 means yes. // 1 means yes.
resetPingStrikes uint32 // Accessed atomically. resetPingStrikes uint32 // Accessed atomically.
initialWindowSize int32
mu sync.Mutex // guard the following mu sync.Mutex // guard the following
state transportState state transportState
activeStreams map[uint32]*Stream activeStreams map[uint32]*Stream
@ -142,16 +144,24 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
Val: maxStreams, Val: maxStreams,
}) })
} }
if initialWindowSize != defaultWindowSize { iwz := int32(initialWindowSize)
if config.InitialWindowSize >= defaultWindowSize {
iwz = config.InitialWindowSize
}
icwz := int32(initialConnWindowSize)
if config.InitialConnWindowSize >= defaultWindowSize {
icwz = config.InitialConnWindowSize
}
if iwz != defaultWindowSize {
settings = append(settings, http2.Setting{ settings = append(settings, http2.Setting{
ID: http2.SettingInitialWindowSize, ID: http2.SettingInitialWindowSize,
Val: uint32(initialWindowSize)}) Val: uint32(iwz)})
} }
if err := framer.writeSettings(true, settings...); err != nil { if err := framer.writeSettings(true, settings...); err != nil {
return nil, connectionErrorf(true, err, "transport: %v", err) return nil, connectionErrorf(true, err, "transport: %v", err)
} }
// Adjust the connection flow control window if needed. // Adjust the connection flow control window if needed.
if delta := uint32(initialConnWindowSize - defaultWindowSize); delta > 0 { if delta := uint32(icwz - defaultWindowSize); delta > 0 {
if err := framer.writeWindowUpdate(true, 0, delta); err != nil { if err := framer.writeWindowUpdate(true, 0, delta); err != nil {
return nil, connectionErrorf(true, err, "transport: %v", err) return nil, connectionErrorf(true, err, "transport: %v", err)
} }
@ -180,28 +190,29 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
} }
var buf bytes.Buffer var buf bytes.Buffer
t := &http2Server{ t := &http2Server{
ctx: context.Background(), ctx: context.Background(),
conn: conn, conn: conn,
remoteAddr: conn.RemoteAddr(), remoteAddr: conn.RemoteAddr(),
localAddr: conn.LocalAddr(), localAddr: conn.LocalAddr(),
authInfo: config.AuthInfo, authInfo: config.AuthInfo,
framer: framer, framer: framer,
hBuf: &buf, hBuf: &buf,
hEnc: hpack.NewEncoder(&buf), hEnc: hpack.NewEncoder(&buf),
maxStreams: maxStreams, maxStreams: maxStreams,
inTapHandle: config.InTapHandle, inTapHandle: config.InTapHandle,
controlBuf: newRecvBuffer(), controlBuf: newRecvBuffer(),
fc: &inFlow{limit: initialConnWindowSize}, fc: &inFlow{limit: uint32(icwz)},
sendQuotaPool: newQuotaPool(defaultWindowSize), sendQuotaPool: newQuotaPool(defaultWindowSize),
state: reachable, state: reachable,
writableChan: make(chan int, 1), writableChan: make(chan int, 1),
shutdownChan: make(chan struct{}), shutdownChan: make(chan struct{}),
activeStreams: make(map[uint32]*Stream), activeStreams: make(map[uint32]*Stream),
streamSendQuota: defaultWindowSize, streamSendQuota: defaultWindowSize,
stats: config.StatsHandler, stats: config.StatsHandler,
kp: kp, kp: kp,
idle: time.Now(), idle: time.Now(),
kep: kep, kep: kep,
initialWindowSize: iwz,
} }
if t.stats != nil { if t.stats != nil {
t.ctx = t.stats.TagConn(t.ctx, &stats.ConnTagInfo{ t.ctx = t.stats.TagConn(t.ctx, &stats.ConnTagInfo{
@ -224,7 +235,7 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(
id: frame.Header().StreamID, id: frame.Header().StreamID,
st: t, st: t,
buf: buf, buf: buf,
fc: &inFlow{limit: initialWindowSize}, fc: &inFlow{limit: uint32(t.initialWindowSize)},
} }
var state decodeState var state decodeState

View File

@ -392,12 +392,14 @@ const (
// ServerConfig consists of all the configurations to establish a server transport. // ServerConfig consists of all the configurations to establish a server transport.
type ServerConfig struct { type ServerConfig struct {
MaxStreams uint32 MaxStreams uint32
AuthInfo credentials.AuthInfo AuthInfo credentials.AuthInfo
InTapHandle tap.ServerInHandle InTapHandle tap.ServerInHandle
StatsHandler stats.Handler StatsHandler stats.Handler
KeepaliveParams keepalive.ServerParameters KeepaliveParams keepalive.ServerParameters
KeepalivePolicy keepalive.EnforcementPolicy KeepalivePolicy keepalive.EnforcementPolicy
InitialWindowSize int32
InitialConnWindowSize int32
} }
// NewServerTransport creates a ServerTransport with conn or non-nil error // NewServerTransport creates a ServerTransport with conn or non-nil error
@ -425,6 +427,10 @@ type ConnectOptions struct {
KeepaliveParams keepalive.ClientParameters KeepaliveParams keepalive.ClientParameters
// StatsHandler stores the handler for stats. // StatsHandler stores the handler for stats.
StatsHandler stats.Handler StatsHandler stats.Handler
// InitialWindowSize sets the intial window size for a stream.
InitialWindowSize int32
// InitialConnWindowSize sets the intial window size for a connection.
InitialConnWindowSize int32
} }
// TargetInfo contains the information of the target such as network address and metadata. // TargetInfo contains the information of the target such as network address and metadata.

View File

@ -1266,3 +1266,153 @@ func TestContextErr(t *testing.T) {
} }
} }
} }
type windowSizeConfig struct {
serverStream int32
serverConn int32
clientStream int32
clientConn int32
}
func TestAccountCheckWindowSizeWithLargeWindow(t *testing.T) {
wc := windowSizeConfig{
serverStream: 10 * 1024 * 1024,
serverConn: 12 * 1024 * 1024,
clientStream: 6 * 1024 * 1024,
clientConn: 8 * 1024 * 1024,
}
testAccountCheckWindowSize(t, wc)
}
func TestAccountCheckWindowSizeWithSmallWindow(t *testing.T) {
wc := windowSizeConfig{
serverStream: defaultWindowSize,
// Note this is smaller than initialConnWindowSize which is the current default.
serverConn: defaultWindowSize,
clientStream: defaultWindowSize,
clientConn: defaultWindowSize,
}
testAccountCheckWindowSize(t, wc)
}
func testAccountCheckWindowSize(t *testing.T, wc windowSizeConfig) {
serverConfig := &ServerConfig{
InitialWindowSize: wc.serverStream,
InitialConnWindowSize: wc.serverConn,
}
connectOptions := ConnectOptions{
InitialWindowSize: wc.clientStream,
InitialConnWindowSize: wc.clientConn,
}
server, client := setUpWithOptions(t, 0, serverConfig, suspended, connectOptions)
defer server.stop()
defer client.Close()
// Wait for server conns to be populated with new server transport.
waitWhileTrue(t, func() (bool, error) {
server.mu.Lock()
defer server.mu.Unlock()
if len(server.conns) == 0 {
return true, fmt.Errorf("timed out waiting for server transport to be created")
}
return false, nil
})
var st *http2Server
server.mu.Lock()
for k := range server.conns {
st = k.(*http2Server)
}
server.mu.Unlock()
ct := client.(*http2Client)
cstream, err := client.NewStream(context.Background(), &CallHdr{Flush: true})
if err != nil {
t.Fatalf("Failed to create stream. Err: %v", err)
}
// Wait for server to receive headers.
waitWhileTrue(t, func() (bool, error) {
st.mu.Lock()
defer st.mu.Unlock()
if len(st.activeStreams) == 0 {
return true, fmt.Errorf("timed out waiting for server to receive headers")
}
return false, nil
})
// Sleeping to make sure the settings are applied in case of negative test.
time.Sleep(time.Second)
waitWhileTrue(t, func() (bool, error) {
if lim := st.fc.limit; lim != uint32(serverConfig.InitialConnWindowSize) {
return true, fmt.Errorf("Server transport flow control window size: got %v, want %v", lim, serverConfig.InitialConnWindowSize)
}
return false, nil
})
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
serverSendQuota, err := wait(ctx, nil, nil, nil, st.sendQuotaPool.acquire())
if err != nil {
t.Fatalf("Error while acquiring sendQuota on server. Err: %v", err)
}
cancel()
st.sendQuotaPool.add(serverSendQuota)
if serverSendQuota != int(connectOptions.InitialConnWindowSize) {
t.Fatalf("Server send quota(%v) not equal to client's window size(%v) on conn.", serverSendQuota, connectOptions.InitialConnWindowSize)
}
st.mu.Lock()
if st.streamSendQuota != uint32(connectOptions.InitialWindowSize) {
t.Fatalf("Server stream send quota(%v) not equal to client's window size(%v) on stream.", ct.streamSendQuota, connectOptions.InitialWindowSize)
}
st.mu.Unlock()
if ct.fc.limit != uint32(connectOptions.InitialConnWindowSize) {
t.Fatalf("Client transport flow control window size is %v, want %v", ct.fc.limit, connectOptions.InitialConnWindowSize)
}
ctx, cancel = context.WithTimeout(context.Background(), time.Second)
clientSendQuota, err := wait(ctx, nil, nil, nil, ct.sendQuotaPool.acquire())
if err != nil {
t.Fatalf("Error while acquiring sendQuota on client. Err: %v", err)
}
cancel()
ct.sendQuotaPool.add(clientSendQuota)
if clientSendQuota != int(serverConfig.InitialConnWindowSize) {
t.Fatalf("Client send quota(%v) not equal to server's window size(%v) on conn.", clientSendQuota, serverConfig.InitialConnWindowSize)
}
ct.mu.Lock()
if ct.streamSendQuota != uint32(serverConfig.InitialWindowSize) {
t.Fatalf("Client stream send quota(%v) not equal to server's window size(%v) on stream.", ct.streamSendQuota, serverConfig.InitialWindowSize)
}
ct.mu.Unlock()
if cstream.fc.limit != uint32(connectOptions.InitialWindowSize) {
t.Fatalf("Client stream flow control window size is %v, want %v", cstream.fc.limit, connectOptions.InitialWindowSize)
}
var sstream *Stream
st.mu.Lock()
for _, v := range st.activeStreams {
sstream = v
}
st.mu.Unlock()
if sstream.fc.limit != uint32(serverConfig.InitialWindowSize) {
t.Fatalf("Server stream flow control window size is %v, want %v", sstream.fc.limit, serverConfig.InitialWindowSize)
}
}
func waitWhileTrue(t *testing.T, condition func() (bool, error)) {
var (
wait bool
err error
)
timer := time.NewTimer(time.Second * 5)
for {
wait, err = condition()
if wait {
select {
case <-timer.C:
t.Fatalf(err.Error())
default:
time.Sleep(50 * time.Millisecond)
continue
}
}
if !timer.Stop() {
<-timer.C
}
break
}
}