|
@@ -140,29 +140,6 @@ func newHTTP2Client(addr string, opts *ConnectOptions) (_ ClientTransport, err e
|
|
|
conn.Close()
|
|
conn.Close()
|
|
|
}
|
|
}
|
|
|
}()
|
|
}()
|
|
|
- // Send connection preface to server.
|
|
|
|
|
- n, err := conn.Write(clientPreface)
|
|
|
|
|
- if err != nil {
|
|
|
|
|
- return nil, ConnectionErrorf("transport: %v", err)
|
|
|
|
|
- }
|
|
|
|
|
- if n != len(clientPreface) {
|
|
|
|
|
- return nil, ConnectionErrorf("transport: preface mismatch, wrote %d bytes; want %d", n, len(clientPreface))
|
|
|
|
|
- }
|
|
|
|
|
- framer := newFramer(conn)
|
|
|
|
|
- if initialWindowSize != defaultWindowSize {
|
|
|
|
|
- err = framer.writeSettings(true, http2.Setting{http2.SettingInitialWindowSize, uint32(initialWindowSize)})
|
|
|
|
|
- } else {
|
|
|
|
|
- err = framer.writeSettings(true)
|
|
|
|
|
- }
|
|
|
|
|
- if err != nil {
|
|
|
|
|
- return nil, ConnectionErrorf("transport: %v", err)
|
|
|
|
|
- }
|
|
|
|
|
- // Adjust the connection flow control window if needed.
|
|
|
|
|
- if delta := uint32(initialConnWindowSize - defaultWindowSize); delta > 0 {
|
|
|
|
|
- if err := framer.writeWindowUpdate(true, 0, delta); err != nil {
|
|
|
|
|
- return nil, ConnectionErrorf("transport: %v", err)
|
|
|
|
|
- }
|
|
|
|
|
- }
|
|
|
|
|
ua := primaryUA
|
|
ua := primaryUA
|
|
|
if opts.UserAgent != "" {
|
|
if opts.UserAgent != "" {
|
|
|
ua = opts.UserAgent + " " + ua
|
|
ua = opts.UserAgent + " " + ua
|
|
@@ -178,7 +155,7 @@ func newHTTP2Client(addr string, opts *ConnectOptions) (_ ClientTransport, err e
|
|
|
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{}),
|
|
|
- framer: framer,
|
|
|
|
|
|
|
+ framer: newFramer(conn),
|
|
|
hBuf: &buf,
|
|
hBuf: &buf,
|
|
|
hEnc: hpack.NewEncoder(&buf),
|
|
hEnc: hpack.NewEncoder(&buf),
|
|
|
controlBuf: newRecvBuffer(),
|
|
controlBuf: newRecvBuffer(),
|
|
@@ -191,28 +168,49 @@ func newHTTP2Client(addr string, opts *ConnectOptions) (_ ClientTransport, err e
|
|
|
maxStreams: math.MaxInt32,
|
|
maxStreams: math.MaxInt32,
|
|
|
streamSendQuota: defaultWindowSize,
|
|
streamSendQuota: defaultWindowSize,
|
|
|
}
|
|
}
|
|
|
|
|
+ // Start the reader goroutine for incoming message. Each transport has
|
|
|
|
|
+ // a dedicated goroutine which reads HTTP2 frame from network. Then it
|
|
|
|
|
+ // dispatches the frame to the corresponding stream entity.
|
|
|
|
|
+ go t.reader()
|
|
|
|
|
+ // Send connection preface to server.
|
|
|
|
|
+ n, err := t.conn.Write(clientPreface)
|
|
|
|
|
+ if err != nil {
|
|
|
|
|
+ t.Close()
|
|
|
|
|
+ return nil, ConnectionErrorf("transport: %v", err)
|
|
|
|
|
+ }
|
|
|
|
|
+ if n != len(clientPreface) {
|
|
|
|
|
+ t.Close()
|
|
|
|
|
+ return nil, ConnectionErrorf("transport: preface mismatch, wrote %d bytes; want %d", n, len(clientPreface))
|
|
|
|
|
+ }
|
|
|
|
|
+ if initialWindowSize != defaultWindowSize {
|
|
|
|
|
+ err = t.framer.writeSettings(true, http2.Setting{http2.SettingInitialWindowSize, uint32(initialWindowSize)})
|
|
|
|
|
+ } else {
|
|
|
|
|
+ err = t.framer.writeSettings(true)
|
|
|
|
|
+ }
|
|
|
|
|
+ if err != nil {
|
|
|
|
|
+ t.Close()
|
|
|
|
|
+ return nil, ConnectionErrorf("transport: %v", err)
|
|
|
|
|
+ }
|
|
|
|
|
+ // Adjust the connection flow control window if needed.
|
|
|
|
|
+ if delta := uint32(initialConnWindowSize - defaultWindowSize); delta > 0 {
|
|
|
|
|
+ if err := t.framer.writeWindowUpdate(true, 0, delta); err != nil {
|
|
|
|
|
+ t.Close()
|
|
|
|
|
+ return nil, ConnectionErrorf("transport: %v", err)
|
|
|
|
|
+ }
|
|
|
|
|
+ }
|
|
|
go t.controller()
|
|
go t.controller()
|
|
|
t.writableChan <- 0
|
|
t.writableChan <- 0
|
|
|
- // Start the reader goroutine for incoming message. The threading model
|
|
|
|
|
- // on receiving is that each transport has a dedicated goroutine which
|
|
|
|
|
- // reads HTTP2 frame from network. Then it dispatches the frame to the
|
|
|
|
|
- // corresponding stream entity.
|
|
|
|
|
- go t.reader()
|
|
|
|
|
return t, nil
|
|
return t, nil
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
func (t *http2Client) newStream(ctx context.Context, callHdr *CallHdr) *Stream {
|
|
func (t *http2Client) newStream(ctx context.Context, callHdr *CallHdr) *Stream {
|
|
|
- fc := &inFlow{
|
|
|
|
|
- limit: initialWindowSize,
|
|
|
|
|
- conn: t.fc,
|
|
|
|
|
- }
|
|
|
|
|
// TODO(zhaoq): Handle uint32 overflow of Stream.id.
|
|
// TODO(zhaoq): Handle uint32 overflow of Stream.id.
|
|
|
s := &Stream{
|
|
s := &Stream{
|
|
|
id: t.nextID,
|
|
id: t.nextID,
|
|
|
method: callHdr.Method,
|
|
method: callHdr.Method,
|
|
|
sendCompress: callHdr.SendCompress,
|
|
sendCompress: callHdr.SendCompress,
|
|
|
buf: newRecvBuffer(),
|
|
buf: newRecvBuffer(),
|
|
|
- fc: fc,
|
|
|
|
|
|
|
+ fc: &inFlow{limit: initialWindowSize},
|
|
|
sendQuotaPool: newQuotaPool(int(t.streamSendQuota)),
|
|
sendQuotaPool: newQuotaPool(int(t.streamSendQuota)),
|
|
|
headerChan: make(chan struct{}),
|
|
headerChan: make(chan struct{}),
|
|
|
}
|
|
}
|
|
@@ -237,8 +235,10 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (_ *Strea
|
|
|
if dl, ok := ctx.Deadline(); ok {
|
|
if dl, ok := ctx.Deadline(); ok {
|
|
|
timeout = dl.Sub(time.Now())
|
|
timeout = dl.Sub(time.Now())
|
|
|
}
|
|
}
|
|
|
- if err := ctx.Err(); err != nil {
|
|
|
|
|
- return nil, ContextErr(err)
|
|
|
|
|
|
|
+ select {
|
|
|
|
|
+ case <-ctx.Done():
|
|
|
|
|
+ return nil, ContextErr(ctx.Err())
|
|
|
|
|
+ default:
|
|
|
}
|
|
}
|
|
|
pr := &peer.Peer{
|
|
pr := &peer.Peer{
|
|
|
Addr: t.conn.RemoteAddr(),
|
|
Addr: t.conn.RemoteAddr(),
|
|
@@ -404,8 +404,10 @@ func (t *http2Client) CloseStream(s *Stream, err error) {
|
|
|
// other goroutines.
|
|
// other goroutines.
|
|
|
s.cancel()
|
|
s.cancel()
|
|
|
s.mu.Lock()
|
|
s.mu.Lock()
|
|
|
- if q := s.fc.restoreConn(); q > 0 {
|
|
|
|
|
- t.controlBuf.put(&windowUpdate{0, q})
|
|
|
|
|
|
|
+ if q := s.fc.resetPendingData(); q > 0 {
|
|
|
|
|
+ if n := t.fc.onRead(q); n > 0 {
|
|
|
|
|
+ t.controlBuf.put(&windowUpdate{0, n})
|
|
|
|
|
+ }
|
|
|
}
|
|
}
|
|
|
if s.state == streamDone {
|
|
if s.state == streamDone {
|
|
|
s.mu.Unlock()
|
|
s.mu.Unlock()
|
|
@@ -427,6 +429,9 @@ func (t *http2Client) CloseStream(s *Stream, err error) {
|
|
|
// accessed any more.
|
|
// accessed any more.
|
|
|
func (t *http2Client) Close() (err error) {
|
|
func (t *http2Client) Close() (err error) {
|
|
|
t.mu.Lock()
|
|
t.mu.Lock()
|
|
|
|
|
+ if t.state == reachable {
|
|
|
|
|
+ close(t.errorChan)
|
|
|
|
|
+ }
|
|
|
if t.state == closing {
|
|
if t.state == closing {
|
|
|
t.mu.Unlock()
|
|
t.mu.Unlock()
|
|
|
return errors.New("transport: Close() was already called")
|
|
return errors.New("transport: Close() was already called")
|
|
@@ -505,6 +510,10 @@ func (t *http2Client) Write(s *Stream, data []byte, opts *Options) error {
|
|
|
t.framer.adjustNumWriters(1)
|
|
t.framer.adjustNumWriters(1)
|
|
|
// Got some quota. Try to acquire writing privilege on the transport.
|
|
// Got some quota. Try to acquire writing privilege on the transport.
|
|
|
if _, err := wait(s.ctx, t.shutdownChan, t.writableChan); err != nil {
|
|
if _, err := wait(s.ctx, t.shutdownChan, t.writableChan); err != nil {
|
|
|
|
|
+ if _, ok := err.(StreamError); ok {
|
|
|
|
|
+ // Return the connection quota back.
|
|
|
|
|
+ t.sendQuotaPool.add(len(p))
|
|
|
|
|
+ }
|
|
|
if t.framer.adjustNumWriters(-1) == 0 {
|
|
if t.framer.adjustNumWriters(-1) == 0 {
|
|
|
// This writer is the last one in this batch and has the
|
|
// This writer is the last one in this batch and has the
|
|
|
// responsibility to flush the buffered frames. It queues
|
|
// responsibility to flush the buffered frames. It queues
|
|
@@ -514,6 +523,16 @@ func (t *http2Client) Write(s *Stream, data []byte, opts *Options) error {
|
|
|
}
|
|
}
|
|
|
return err
|
|
return err
|
|
|
}
|
|
}
|
|
|
|
|
+ select {
|
|
|
|
|
+ case <-s.ctx.Done():
|
|
|
|
|
+ t.sendQuotaPool.add(len(p))
|
|
|
|
|
+ if t.framer.adjustNumWriters(-1) == 0 {
|
|
|
|
|
+ t.controlBuf.put(&flushIO{})
|
|
|
|
|
+ }
|
|
|
|
|
+ t.writableChan <- 0
|
|
|
|
|
+ return ContextErr(s.ctx.Err())
|
|
|
|
|
+ default:
|
|
|
|
|
+ }
|
|
|
if r.Len() == 0 && t.framer.adjustNumWriters(0) == 1 {
|
|
if r.Len() == 0 && t.framer.adjustNumWriters(0) == 1 {
|
|
|
// Do a force flush iff this is last frame for the entire gRPC message
|
|
// Do a force flush iff this is last frame for the entire gRPC message
|
|
|
// and the caller is the only writer at this moment.
|
|
// and the caller is the only writer at this moment.
|
|
@@ -560,41 +579,39 @@ func (t *http2Client) getStream(f http2.Frame) (*Stream, bool) {
|
|
|
// Window updates will deliver to the controller for sending when
|
|
// Window updates will deliver to the controller for sending when
|
|
|
// the cumulative quota exceeds the corresponding threshold.
|
|
// the cumulative quota exceeds the corresponding threshold.
|
|
|
func (t *http2Client) updateWindow(s *Stream, n uint32) {
|
|
func (t *http2Client) updateWindow(s *Stream, n uint32) {
|
|
|
- swu, cwu := s.fc.onRead(n)
|
|
|
|
|
- if swu > 0 {
|
|
|
|
|
- t.controlBuf.put(&windowUpdate{s.id, swu})
|
|
|
|
|
|
|
+ if w := t.fc.onRead(n); w > 0 {
|
|
|
|
|
+ t.controlBuf.put(&windowUpdate{0, w})
|
|
|
}
|
|
}
|
|
|
- if cwu > 0 {
|
|
|
|
|
- t.controlBuf.put(&windowUpdate{0, cwu})
|
|
|
|
|
|
|
+ if w := s.fc.onRead(n); w > 0 {
|
|
|
|
|
+ t.controlBuf.put(&windowUpdate{s.id, w})
|
|
|
}
|
|
}
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
func (t *http2Client) handleData(f *http2.DataFrame) {
|
|
func (t *http2Client) handleData(f *http2.DataFrame) {
|
|
|
- // Select the right stream to dispatch.
|
|
|
|
|
size := len(f.Data())
|
|
size := len(f.Data())
|
|
|
|
|
+ if err := t.fc.onData(uint32(size)); err != nil {
|
|
|
|
|
+ t.notifyError(ConnectionErrorf("%v", err))
|
|
|
|
|
+ return
|
|
|
|
|
+ }
|
|
|
|
|
+ // Select the right stream to dispatch.
|
|
|
s, ok := t.getStream(f)
|
|
s, ok := t.getStream(f)
|
|
|
if !ok {
|
|
if !ok {
|
|
|
- cwu, err := t.fc.adjustConnPendingUpdate(uint32(size))
|
|
|
|
|
- if err != nil {
|
|
|
|
|
- t.notifyError(err)
|
|
|
|
|
- return
|
|
|
|
|
- }
|
|
|
|
|
- if cwu > 0 {
|
|
|
|
|
- t.controlBuf.put(&windowUpdate{0, cwu})
|
|
|
|
|
|
|
+ if w := t.fc.onRead(uint32(size)); w > 0 {
|
|
|
|
|
+ t.controlBuf.put(&windowUpdate{0, w})
|
|
|
}
|
|
}
|
|
|
return
|
|
return
|
|
|
}
|
|
}
|
|
|
if size > 0 {
|
|
if size > 0 {
|
|
|
- if err := s.fc.onData(uint32(size)); err != nil {
|
|
|
|
|
- if _, ok := err.(ConnectionError); ok {
|
|
|
|
|
- t.notifyError(err)
|
|
|
|
|
- return
|
|
|
|
|
- }
|
|
|
|
|
- s.mu.Lock()
|
|
|
|
|
- if s.state == streamDone {
|
|
|
|
|
- s.mu.Unlock()
|
|
|
|
|
- return
|
|
|
|
|
|
|
+ s.mu.Lock()
|
|
|
|
|
+ if s.state == streamDone {
|
|
|
|
|
+ s.mu.Unlock()
|
|
|
|
|
+ // The stream has been closed. Release the corresponding quota.
|
|
|
|
|
+ if w := t.fc.onRead(uint32(size)); w > 0 {
|
|
|
|
|
+ t.controlBuf.put(&windowUpdate{0, w})
|
|
|
}
|
|
}
|
|
|
|
|
+ return
|
|
|
|
|
+ }
|
|
|
|
|
+ if err := s.fc.onData(uint32(size)); err != nil {
|
|
|
s.state = streamDone
|
|
s.state = streamDone
|
|
|
s.statusCode = codes.Internal
|
|
s.statusCode = codes.Internal
|
|
|
s.statusDesc = err.Error()
|
|
s.statusDesc = err.Error()
|
|
@@ -603,6 +620,7 @@ func (t *http2Client) handleData(f *http2.DataFrame) {
|
|
|
t.controlBuf.put(&resetStream{s.id, http2.ErrCodeFlowControl})
|
|
t.controlBuf.put(&resetStream{s.id, http2.ErrCodeFlowControl})
|
|
|
return
|
|
return
|
|
|
}
|
|
}
|
|
|
|
|
+ s.mu.Unlock()
|
|
|
// TODO(bradfitz, zhaoq): A copy is required here because there is no
|
|
// TODO(bradfitz, zhaoq): A copy is required here because there is no
|
|
|
// guarantee f.Data() is consumed before the arrival of next frame.
|
|
// guarantee f.Data() is consumed before the arrival of next frame.
|
|
|
// Can this copy be eliminated?
|
|
// Can this copy be eliminated?
|