* Export changes to OSS.

* First commit.

* Cherry-pick.

* Documentation.

* Post review updates.
This commit is contained in:
mmukhi
2018-04-30 09:54:33 -07:00
committed by GitHub
parent fc37cf1364
commit 7a8c989507
15 changed files with 1080 additions and 1023 deletions

View File

@ -39,6 +39,7 @@ import (
"google.golang.org/grpc/channelz"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/internal/msgdecoder"
"google.golang.org/grpc/keepalive"
"google.golang.org/grpc/metadata"
"google.golang.org/grpc/peer"
@ -212,7 +213,6 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
writerDone: make(chan struct{}),
maxStreams: maxStreams,
inTapHandle: config.InTapHandle,
fc: &trInFlow{limit: uint32(icwz)},
state: reachable,
activeStreams: make(map[uint32]*Stream),
stats: config.StatsHandler,
@ -222,6 +222,15 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
initialWindowSize: iwz,
}
t.controlBuf = newControlBuffer(t.ctxDone)
t.fc = &trInFlow{
limit: uint32(icwz),
scheduleWU: func(w uint32) {
t.controlBuf.put(&outgoingWindowUpdate{
streamID: 0,
increment: w,
})
},
}
if dynamicWindow {
t.bdpEst = &bdpEstimator{
bdp: initialWindowSize,
@ -298,25 +307,14 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(
return
}
}
buf := newRecvBuffer()
s := &Stream{
id: streamID,
st: t,
buf: buf,
fc: &inFlow{limit: uint32(t.initialWindowSize)},
recvCompress: state.encoding,
method: state.method,
contentSubtype: state.contentSubtype,
}
if frame.StreamEnded() {
// s is just created by the caller. No lock needed.
s.state = streamReadDone
}
var (
ctx context.Context
cancel func()
)
if state.timeoutSet {
s.ctx, s.cancel = context.WithTimeout(t.ctx, state.timeout)
ctx, cancel = context.WithTimeout(t.ctx, state.timeout)
} else {
s.ctx, s.cancel = context.WithCancel(t.ctx)
ctx, cancel = context.WithCancel(t.ctx)
}
pr := &peer.Peer{
Addr: t.remoteAddr,
@ -325,34 +323,55 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(
if t.authInfo != nil {
pr.AuthInfo = t.authInfo
}
s.ctx = peer.NewContext(s.ctx, pr)
ctx = peer.NewContext(ctx, pr)
// Attach the received metadata to the context.
if len(state.mdata) > 0 {
s.ctx = metadata.NewIncomingContext(s.ctx, state.mdata)
ctx = metadata.NewIncomingContext(ctx, state.mdata)
}
if state.statsTags != nil {
s.ctx = stats.SetIncomingTags(s.ctx, state.statsTags)
ctx = stats.SetIncomingTags(ctx, state.statsTags)
}
if state.statsTrace != nil {
s.ctx = stats.SetIncomingTrace(s.ctx, state.statsTrace)
ctx = stats.SetIncomingTrace(ctx, state.statsTrace)
}
if t.inTapHandle != nil {
var err error
info := &tap.Info{
FullMethodName: state.method,
}
s.ctx, err = t.inTapHandle(s.ctx, info)
ctx, err = t.inTapHandle(ctx, info)
if err != nil {
warningf("transport: http2Server.operateHeaders got an error from InTapHandle: %v", err)
t.controlBuf.put(&cleanupStream{
streamID: s.id,
streamID: streamID,
rst: true,
rstCode: http2.ErrCodeRefusedStream,
onWrite: func() {},
})
cancel()
return
}
}
ctx = traceCtx(ctx, state.method)
s := newStream(ctx)
// Initialize s with server-side specific fields.
s.cancel = cancel
s.id = streamID
s.st = t
s.fc = &stInFlow{
limit: uint32(t.initialWindowSize),
scheduleWU: func(w uint32) {
t.controlBuf.put(&outgoingWindowUpdate{streamID: streamID, increment: w})
},
wuThreshold: uint32(t.initialWindowSize / 4),
}
s.recvCompress = state.encoding
s.method = state.method
s.contentSubtype = state.contentSubtype
if frame.StreamEnded() {
s.state = streamReadDone
}
s.wq = newWriteQuota(defaultWriteQuota, s.ctxDone)
t.mu.Lock()
if t.state != reachable {
t.mu.Unlock()
@ -386,10 +405,6 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(
t.lastStreamCreated = time.Now()
t.czmu.Unlock()
}
s.requestRead = func(n int) {
t.adjustWindow(s, uint32(n))
}
s.ctx = traceCtx(s.ctx, s.method)
if t.stats != nil {
s.ctx = t.stats.TagRPC(s.ctx, &stats.RPCTagInfo{FullMethodName: s.method})
inHeader := &stats.InHeader{
@ -401,18 +416,6 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(
}
t.stats.HandleRPC(s.ctx, inHeader)
}
s.ctxDone = s.ctx.Done()
s.wq = newWriteQuota(defaultWriteQuota, s.ctxDone)
s.trReader = &transportReader{
reader: &recvBufferReader{
ctx: s.ctx,
ctxDone: s.ctxDone,
recv: s.buf,
},
windowHandler: func(n int) {
t.updateWindow(s, uint32(n))
},
}
handle(s)
return
}
@ -490,41 +493,20 @@ func (t *http2Server) getStream(f http2.Frame) (*Stream, bool) {
return s, true
}
// adjustWindow sends out extra window update over the initial window size
// of stream if the application is requesting data larger in size than
// the window.
func (t *http2Server) adjustWindow(s *Stream, n uint32) {
if w := s.fc.maybeAdjust(n); w > 0 {
t.controlBuf.put(&outgoingWindowUpdate{streamID: s.id, increment: w})
}
}
// updateWindow adjusts the inbound quota for the stream and the transport.
// Window updates will deliver to the controller for sending when
// the cumulative quota exceeds the corresponding threshold.
func (t *http2Server) updateWindow(s *Stream, n uint32) {
if w := s.fc.onRead(n); w > 0 {
t.controlBuf.put(&outgoingWindowUpdate{streamID: s.id,
increment: w,
})
}
}
// updateFlowControl updates the incoming flow control windows
// for the transport and the stream based on the current bdp
// estimation.
func (t *http2Server) updateFlowControl(n uint32) {
t.mu.Lock()
// Update all the current streams' window.
for _, s := range t.activeStreams {
s.fc.newLimit(n)
}
// Update all the future streams' window.
t.initialWindowSize = int32(n)
t.mu.Unlock()
t.controlBuf.put(&outgoingWindowUpdate{
streamID: 0,
increment: t.fc.newLimit(n),
})
t.fc.newLimit(n) // Update transport's window.
// Notify the other side of the updated value.
t.controlBuf.put(&outgoingSettings{
ss: []http2.Setting{
{
@ -538,9 +520,15 @@ func (t *http2Server) updateFlowControl(n uint32) {
func (t *http2Server) handleData(f *http2.DataFrame) {
size := f.Header().Length
var sendBDPPing bool
if t.bdpEst != nil {
sendBDPPing = t.bdpEst.add(size)
if size == 0 {
if f.StreamEnded() {
if s, ok := t.getStream(f); ok {
// Received the end of stream from the client.
s.compareAndSwapState(streamActive, streamReadDone)
s.notifyErr(io.EOF)
}
}
return
}
// Decouple connection's flow control from application's read.
// An update on connection's flow control should not depend on
@ -550,51 +538,30 @@ func (t *http2Server) handleData(f *http2.DataFrame) {
// Decoupling the connection flow control will prevent other
// active(fast) streams from starving in presence of slow or
// inactive streams.
if w := t.fc.onData(size); w > 0 {
t.controlBuf.put(&outgoingWindowUpdate{
streamID: 0,
increment: w,
})
}
if sendBDPPing {
t.fc.onData(size)
if t.bdpEst != nil && t.bdpEst.add(size) {
// Avoid excessive ping detection (e.g. in an L7 proxy)
// by sending a window update prior to the BDP ping.
if w := t.fc.reset(); w > 0 {
t.controlBuf.put(&outgoingWindowUpdate{
streamID: 0,
increment: w,
})
}
t.fc.reset()
t.controlBuf.put(bdpPing)
}
// Select the right stream to dispatch.
s, ok := t.getStream(f)
if !ok {
return
}
if size > 0 {
if err := s.fc.onData(size); err != nil {
if s, ok := t.getStream(f); ok {
d := f.Data()
padding := 0
if f.Header().Flags.Has(http2.FlagDataPadded) {
padding = int(size) - len(d)
}
if err := s.consume(d, padding); err != nil {
errorf("transport: flow control error on server: %v", err)
t.closeStream(s, true, http2.ErrCodeFlowControl, nil, false)
return
}
if f.Header().Flags.Has(http2.FlagDataPadded) {
if w := s.fc.onRead(size - uint32(len(f.Data()))); w > 0 {
t.controlBuf.put(&outgoingWindowUpdate{s.id, w})
}
if f.StreamEnded() {
// Received the end of stream from the client.
s.compareAndSwapState(streamActive, streamReadDone)
s.notifyErr(io.EOF)
}
// TODO(bradfitz, zhaoq): A copy is required here because there is no
// guarantee f.Data() is consumed before the arrival of next frame.
// Can this copy be eliminated?
if len(f.Data()) > 0 {
data := make([]byte, len(f.Data()))
copy(data, f.Data())
s.write(recvMsg{data: data})
}
}
if f.Header().Flags.Has(http2.FlagDataEndStream) {
// Received the end of stream from the client.
s.compareAndSwapState(streamActive, streamReadDone)
s.write(recvMsg{err: io.EOF})
}
}
@ -792,7 +759,7 @@ func (t *http2Server) WriteStatus(s *Stream, st *status.Status) error {
// Write converts the data into HTTP2 data frame and sends it out. Non-nil error
// is returns if it fails (e.g., framing error, transport error).
func (t *http2Server) Write(s *Stream, hdr []byte, data []byte, opts *Options) error {
func (t *http2Server) Write(s *Stream, data []byte, opts *Options) error {
if !s.headerOk { // Headers haven't been written yet.
if err := t.WriteHeader(s, nil); err != nil {
// TODO(mmukhi, dfawley): Make sure this is the right code to return.
@ -811,6 +778,8 @@ func (t *http2Server) Write(s *Stream, hdr []byte, data []byte, opts *Options) e
return ContextErr(s.ctx.Err())
}
}
// Get a gRPC-specific header for this message.
hdr := msgdecoder.CreateMessageHeader(len(data), opts.IsCompressed)
// Add some data to header frame so that we can equally distribute bytes across frames.
emptyLen := http2MaxFrameLen - len(hdr)
if emptyLen > len(data) {