Less mem (#1987)
* Export changes to OSS. * First commit. * Cherry-pick. * Documentation. * Post review updates.
This commit is contained in:
@ -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) {
|
||||
|
Reference in New Issue
Block a user