Commit 4be9b97e authored by Tom Bergan's avatar Tom Bergan Committed by Brad Fitzpatrick

http2: interface to support pluggable schedulers

This adds an interface to support pluggable schedulers. The interface
is defined in writesched.go. The only type needed by this interface is
FrameWriteRequest, which describes a request to write a frame (this used
to be called frameWriteMsg). The scheduler can be configured with a new
field in http2.Server. Two schedulers are implemented:

1) A random scheduler that is essentially equivalent to the existing
   scheduler. This is currently the default scheduler if none is
   configured. The implementation is in writesched_random.go.

2) A scheduler that uses H2 weights and priorities. The H2 priority tree
   is maintained as a tree of priorityNodes. The next frame is chosen by
   walking this tree in topological order, where sibling nodes are ordered
   by their bandwidth usage relative to their H2 weight. Two optional
   features are added to improve performance -- these are configured with
   PriorityWriteSchedulerConfig.

Fixes golang/go#16168

Change-Id: I97ec93e5c58c2efec35455ba2f3c31e849f706af
Reviewed-on: https://go-review.googlesource.com/25366Reviewed-by: 's avatarBrad Fitzpatrick <bradfitz@golang.org>
Run-TryBot: Brad Fitzpatrick <bradfitz@golang.org>
TryBot-Result: Gobot Gobot <gobot@golang.org>
parent 5695785b
// Copyright 2014 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package http2
import (
"testing"
)
func TestPriority(t *testing.T) {
// A -> B
// move A's parent to B
streams := make(map[uint32]*stream)
a := &stream{
parent: nil,
weight: 16,
}
streams[1] = a
b := &stream{
parent: a,
weight: 16,
}
streams[2] = b
adjustStreamPriority(streams, 1, PriorityParam{
Weight: 20,
StreamDep: 2,
})
if a.parent != b {
t.Errorf("Expected A's parent to be B")
}
if a.weight != 20 {
t.Errorf("Expected A's weight to be 20; got %d", a.weight)
}
if b.parent != nil {
t.Errorf("Expected B to have no parent")
}
if b.weight != 16 {
t.Errorf("Expected B's weight to be 16; got %d", b.weight)
}
}
func TestPriorityExclusiveZero(t *testing.T) {
// A B and C are all children of the 0 stream.
// Exclusive reprioritization to any of the streams
// should bring the rest of the streams under the
// reprioritized stream
streams := make(map[uint32]*stream)
a := &stream{
parent: nil,
weight: 16,
}
streams[1] = a
b := &stream{
parent: nil,
weight: 16,
}
streams[2] = b
c := &stream{
parent: nil,
weight: 16,
}
streams[3] = c
adjustStreamPriority(streams, 3, PriorityParam{
Weight: 20,
StreamDep: 0,
Exclusive: true,
})
if a.parent != c {
t.Errorf("Expected A's parent to be C")
}
if a.weight != 16 {
t.Errorf("Expected A's weight to be 16; got %d", a.weight)
}
if b.parent != c {
t.Errorf("Expected B's parent to be C")
}
if b.weight != 16 {
t.Errorf("Expected B's weight to be 16; got %d", b.weight)
}
if c.parent != nil {
t.Errorf("Expected C to have no parent")
}
if c.weight != 20 {
t.Errorf("Expected C's weight to be 20; got %d", b.weight)
}
}
func TestPriorityOwnParent(t *testing.T) {
streams := make(map[uint32]*stream)
a := &stream{
parent: nil,
weight: 16,
}
streams[1] = a
b := &stream{
parent: a,
weight: 16,
}
streams[2] = b
adjustStreamPriority(streams, 1, PriorityParam{
Weight: 20,
StreamDep: 1,
})
if a.parent != nil {
t.Errorf("Expected A's parent to be nil")
}
if a.weight != 20 {
t.Errorf("Expected A's weight to be 20; got %d", a.weight)
}
if b.parent != a {
t.Errorf("Expected B's parent to be A")
}
if b.weight != 16 {
t.Errorf("Expected B's weight to be 16; got %d", b.weight)
}
}
......@@ -108,6 +108,10 @@ type Server struct {
// closed with a GOAWAY frame. PING frames are not considered
// activity for the purposes of IdleTimeout.
IdleTimeout time.Duration
// NewWriteScheduler constructs a write scheduler for a connection.
// If nil, a default scheduler is chosen.
NewWriteScheduler func() WriteScheduler
}
func (s *Server) maxReadFrameSize() uint32 {
......@@ -248,29 +252,33 @@ func (s *Server) ServeConn(c net.Conn, opts *ServeConnOpts) {
defer cancel()
sc := &serverConn{
srv: s,
hs: opts.baseConfig(),
conn: c,
baseCtx: baseCtx,
remoteAddrStr: c.RemoteAddr().String(),
bw: newBufferedWriter(c),
handler: opts.handler(),
streams: make(map[uint32]*stream),
readFrameCh: make(chan readFrameResult),
wantWriteFrameCh: make(chan frameWriteMsg, 8),
wroteFrameCh: make(chan frameWriteResult, 1), // buffered; one send in writeFrameAsync
bodyReadCh: make(chan bodyReadMsg), // buffering doesn't matter either way
doneServing: make(chan struct{}),
advMaxStreams: s.maxConcurrentStreams(),
writeSched: writeScheduler{
maxFrameSize: initialMaxFrameSize,
},
srv: s,
hs: opts.baseConfig(),
conn: c,
baseCtx: baseCtx,
remoteAddrStr: c.RemoteAddr().String(),
bw: newBufferedWriter(c),
handler: opts.handler(),
streams: make(map[uint32]*stream),
readFrameCh: make(chan readFrameResult),
wantWriteFrameCh: make(chan FrameWriteRequest, 8),
wroteFrameCh: make(chan frameWriteResult, 1), // buffered; one send in writeFrameAsync
bodyReadCh: make(chan bodyReadMsg), // buffering doesn't matter either way
doneServing: make(chan struct{}),
advMaxStreams: s.maxConcurrentStreams(),
initialWindowSize: initialWindowSize,
maxFrameSize: initialMaxFrameSize,
headerTableSize: initialHeaderTableSize,
serveG: newGoroutineLock(),
pushEnabled: true,
}
if s.NewWriteScheduler != nil {
sc.writeSched = s.NewWriteScheduler()
} else {
sc.writeSched = NewRandomWriteScheduler()
}
sc.flow.add(initialWindowSize)
sc.inflow.add(initialWindowSize)
sc.hpackEncoder = hpack.NewEncoder(&sc.headerWriteBuf)
......@@ -350,16 +358,17 @@ type serverConn struct {
handler http.Handler
baseCtx contextContext
framer *Framer
doneServing chan struct{} // closed when serverConn.serve ends
readFrameCh chan readFrameResult // written by serverConn.readFrames
wantWriteFrameCh chan frameWriteMsg // from handlers -> serve
wroteFrameCh chan frameWriteResult // from writeFrameAsync -> serve, tickles more frame writes
bodyReadCh chan bodyReadMsg // from handlers -> serve
testHookCh chan func(int) // code to run on the serve loop
flow flow // conn-wide (not stream-specific) outbound flow control
inflow flow // conn-wide inbound flow control
tlsState *tls.ConnectionState // shared by all handlers, like net/http
doneServing chan struct{} // closed when serverConn.serve ends
readFrameCh chan readFrameResult // written by serverConn.readFrames
wantWriteFrameCh chan FrameWriteRequest // from handlers -> serve
wroteFrameCh chan frameWriteResult // from writeFrameAsync -> serve, tickles more frame writes
bodyReadCh chan bodyReadMsg // from handlers -> serve
testHookCh chan func(int) // code to run on the serve loop
flow flow // conn-wide (not stream-specific) outbound flow control
inflow flow // conn-wide inbound flow control
tlsState *tls.ConnectionState // shared by all handlers, like net/http
remoteAddrStr string
writeSched WriteScheduler
// Everything following is owned by the serve loop; use serveG.check():
serveG goroutineLock // used to verify funcs are on serve()
......@@ -373,14 +382,14 @@ type serverConn struct {
maxStreamID uint32 // max ever seen
streams map[uint32]*stream
initialWindowSize int32
maxFrameSize int32
headerTableSize uint32
peerMaxHeaderListSize uint32 // zero means unknown (default)
canonHeader map[string]string // http2-lower-case -> Go-Canonical-Case
writingFrame bool // started write goroutine but haven't heard back on wroteFrameCh
needsFrameFlush bool // last frame write wasn't a flush
writeSched writeScheduler
inGoAway bool // we've started to or sent GOAWAY
needToSendGoAway bool // we need to schedule a GOAWAY frame write
inGoAway bool // we've started to or sent GOAWAY
needToSendGoAway bool // we need to schedule a GOAWAY frame write
goAwayCode ErrCode
shutdownTimerCh <-chan time.Time // nil until used
shutdownTimer *time.Timer // nil until used
......@@ -598,17 +607,17 @@ func (sc *serverConn) readFrames() {
// frameWriteResult is the message passed from writeFrameAsync to the serve goroutine.
type frameWriteResult struct {
wm frameWriteMsg // what was written (or attempted)
err error // result of the writeFrame call
wr FrameWriteRequest // what was written (or attempted)
err error // result of the writeFrame call
}
// writeFrameAsync runs in its own goroutine and writes a single frame
// and then reports when it's done.
// At most one goroutine can be running writeFrameAsync at a time per
// serverConn.
func (sc *serverConn) writeFrameAsync(wm frameWriteMsg) {
err := wm.write.writeFrame(sc)
sc.wroteFrameCh <- frameWriteResult{wm, err}
func (sc *serverConn) writeFrameAsync(wr FrameWriteRequest) {
err := wr.write.writeFrame(sc)
sc.wroteFrameCh <- frameWriteResult{wr, err}
}
func (sc *serverConn) closeAllStreamsOnConnClose() {
......@@ -652,7 +661,7 @@ func (sc *serverConn) serve() {
sc.vlogf("http2: server connection from %v on %p", sc.conn.RemoteAddr(), sc.hs)
}
sc.writeFrame(frameWriteMsg{
sc.writeFrame(FrameWriteRequest{
write: writeSettings{
{SettingMaxFrameSize, sc.srv.maxReadFrameSize()},
{SettingMaxConcurrentStreams, sc.advMaxStreams},
......@@ -690,8 +699,8 @@ func (sc *serverConn) serve() {
for {
loopNum++
select {
case wm := <-sc.wantWriteFrameCh:
sc.writeFrame(wm)
case wr := <-sc.wantWriteFrameCh:
sc.writeFrame(wr)
case res := <-sc.wroteFrameCh:
sc.wroteFrame(res)
case res := <-sc.readFrameCh:
......@@ -764,7 +773,7 @@ func (sc *serverConn) writeDataFromHandler(stream *stream, data []byte, endStrea
ch := errChanPool.Get().(chan error)
writeArg := writeDataPool.Get().(*writeData)
*writeArg = writeData{stream.id, data, endStream}
err := sc.writeFrameFromHandler(frameWriteMsg{
err := sc.writeFrameFromHandler(FrameWriteRequest{
write: writeArg,
stream: stream,
done: ch,
......@@ -800,17 +809,17 @@ func (sc *serverConn) writeDataFromHandler(stream *stream, data []byte, endStrea
return err
}
// writeFrameFromHandler sends wm to sc.wantWriteFrameCh, but aborts
// writeFrameFromHandler sends wr to sc.wantWriteFrameCh, but aborts
// if the connection has gone away.
//
// This must not be run from the serve goroutine itself, else it might
// deadlock writing to sc.wantWriteFrameCh (which is only mildly
// buffered and is read by serve itself). If you're on the serve
// goroutine, call writeFrame instead.
func (sc *serverConn) writeFrameFromHandler(wm frameWriteMsg) error {
func (sc *serverConn) writeFrameFromHandler(wr FrameWriteRequest) error {
sc.serveG.checkNotOn() // NOT
select {
case sc.wantWriteFrameCh <- wm:
case sc.wantWriteFrameCh <- wr:
return nil
case <-sc.doneServing:
// Serve loop is gone.
......@@ -827,38 +836,38 @@ func (sc *serverConn) writeFrameFromHandler(wm frameWriteMsg) error {
// make it onto the wire
//
// If you're not on the serve goroutine, use writeFrameFromHandler instead.
func (sc *serverConn) writeFrame(wm frameWriteMsg) {
func (sc *serverConn) writeFrame(wr FrameWriteRequest) {
sc.serveG.check()
var ignoreWrite bool
// Don't send a 100-continue response if we've already sent headers.
// See golang.org/issue/14030.
switch wm.write.(type) {
switch wr.write.(type) {
case *writeResHeaders:
wm.stream.wroteHeaders = true
wr.stream.wroteHeaders = true
case write100ContinueHeadersFrame:
if wm.stream.wroteHeaders {
if wr.stream.wroteHeaders {
ignoreWrite = true
}
}
if !ignoreWrite {
sc.writeSched.add(wm)
sc.writeSched.Push(wr)
}
sc.scheduleFrameWrite()
}
// startFrameWrite starts a goroutine to write wm (in a separate
// startFrameWrite starts a goroutine to write wr (in a separate
// goroutine since that might block on the network), and updates the
// serve goroutine's state about the world, updated from info in wm.
func (sc *serverConn) startFrameWrite(wm frameWriteMsg) {
// serve goroutine's state about the world, updated from info in wr.
func (sc *serverConn) startFrameWrite(wr FrameWriteRequest) {
sc.serveG.check()
if sc.writingFrame {
panic("internal error: can only be writing one frame at a time")
}
st := wm.stream
st := wr.stream
if st != nil {
switch st.state {
case stateHalfClosedLocal:
......@@ -869,13 +878,13 @@ func (sc *serverConn) startFrameWrite(wm frameWriteMsg) {
sc.scheduleFrameWrite()
return
}
panic(fmt.Sprintf("internal error: attempt to send a write %v on a closed stream", wm))
panic(fmt.Sprintf("internal error: attempt to send a write %v on a closed stream", wr))
}
}
sc.writingFrame = true
sc.needsFrameFlush = true
go sc.writeFrameAsync(wm)
go sc.writeFrameAsync(wr)
}
// errHandlerPanicked is the error given to any callers blocked in a read from
......@@ -892,24 +901,24 @@ func (sc *serverConn) wroteFrame(res frameWriteResult) {
}
sc.writingFrame = false
wm := res.wm
st := wm.stream
wr := res.wr
st := wr.stream
closeStream := endsStream(wm.write)
closeStream := endsStream(wr.write)
if _, ok := wm.write.(handlerPanicRST); ok {
if _, ok := wr.write.(handlerPanicRST); ok {
sc.closeStream(st, errHandlerPanicked)
}
// Reply (if requested) to the blocked ServeHTTP goroutine.
if ch := wm.done; ch != nil {
if ch := wr.done; ch != nil {
select {
case ch <- res.err:
default:
panic(fmt.Sprintf("unbuffered done channel passed in for type %T", wm.write))
panic(fmt.Sprintf("unbuffered done channel passed in for type %T", wr.write))
}
}
wm.write = nil // prevent use (assume it's tainted after wm.done send)
wr.write = nil // prevent use (assume it's tainted after wr.done send)
if closeStream {
if st == nil {
......@@ -955,7 +964,7 @@ func (sc *serverConn) scheduleFrameWrite() {
}
if sc.needToSendGoAway {
sc.needToSendGoAway = false
sc.startFrameWrite(frameWriteMsg{
sc.startFrameWrite(FrameWriteRequest{
write: &writeGoAway{
maxStreamID: sc.maxStreamID,
code: sc.goAwayCode,
......@@ -965,17 +974,17 @@ func (sc *serverConn) scheduleFrameWrite() {
}
if sc.needToSendSettingsAck {
sc.needToSendSettingsAck = false
sc.startFrameWrite(frameWriteMsg{write: writeSettingsAck{}})
sc.startFrameWrite(FrameWriteRequest{write: writeSettingsAck{}})
return
}
if !sc.inGoAway {
if wm, ok := sc.writeSched.take(); ok {
sc.startFrameWrite(wm)
if wr, ok := sc.writeSched.Pop(); ok {
sc.startFrameWrite(wr)
return
}
}
if sc.needsFrameFlush {
sc.startFrameWrite(frameWriteMsg{write: flushFrameWriter{}})
sc.startFrameWrite(FrameWriteRequest{write: flushFrameWriter{}})
sc.needsFrameFlush = false // after startFrameWrite, since it sets this true
return
}
......@@ -1006,7 +1015,7 @@ func (sc *serverConn) shutDownIn(d time.Duration) {
func (sc *serverConn) resetStream(se StreamError) {
sc.serveG.check()
sc.writeFrame(frameWriteMsg{write: se})
sc.writeFrame(FrameWriteRequest{write: se})
if st, ok := sc.streams[se.StreamID]; ok {
st.sentReset = true
sc.closeStream(st, se)
......@@ -1122,7 +1131,7 @@ func (sc *serverConn) processPing(f *PingFrame) error {
if sc.inGoAway {
return nil
}
sc.writeFrame(frameWriteMsg{write: writePingAck{f}})
sc.writeFrame(FrameWriteRequest{write: writePingAck{f}})
return nil
}
......@@ -1206,7 +1215,7 @@ func (sc *serverConn) closeStream(st *stream, err error) {
p.CloseWithError(err)
}
st.cw.Close() // signals Handler's CloseNotifier, unblocks writes, etc
sc.writeSched.forgetStream(st.id)
sc.writeSched.CloseStream(st.id)
}
func (sc *serverConn) processSettings(f *SettingsFrame) error {
......@@ -1251,7 +1260,7 @@ func (sc *serverConn) processSetting(s Setting) error {
case SettingInitialWindowSize:
return sc.processSettingInitialWindowSize(s.Val)
case SettingMaxFrameSize:
sc.writeSched.maxFrameSize = s.Val
sc.maxFrameSize = int32(s.Val) // the maximum valid s.Val is < 2^31
case SettingMaxHeaderListSize:
sc.peerMaxHeaderListSize = s.Val
default:
......@@ -1458,8 +1467,9 @@ func (sc *serverConn) processHeaders(f *MetaHeadersFrame) error {
st.inflow.add(initialWindowSize) // TODO: update this when we send a higher initial window size in the initial settings
sc.streams[id] = st
sc.writeSched.OpenStream(st.id, OpenStreamOptions{})
if f.HasPriority() {
adjustStreamPriority(sc.streams, st.id, f.Priority)
sc.writeSched.AdjustStream(st.id, f.Priority)
}
sc.curOpenStreams++
if sc.curOpenStreams == 1 {
......@@ -1554,46 +1564,10 @@ func (sc *serverConn) processPriority(f *PriorityFrame) error {
if sc.inGoAway {
return nil
}
adjustStreamPriority(sc.streams, f.StreamID, f.PriorityParam)
sc.writeSched.AdjustStream(f.StreamID, f.PriorityParam)
return nil
}
func adjustStreamPriority(streams map[uint32]*stream, streamID uint32, priority PriorityParam) {
st, ok := streams[streamID]
if !ok {
// TODO: not quite correct (this streamID might
// already exist in the dep tree, but be closed), but
// close enough for now.
return
}
st.weight = priority.Weight
parent := streams[priority.StreamDep] // might be nil
if parent == st {
// if client tries to set this stream to be the parent of itself
// ignore and keep going
return
}
// section 5.3.3: If a stream is made dependent on one of its
// own dependencies, the formerly dependent stream is first
// moved to be dependent on the reprioritized stream's previous
// parent. The moved dependency retains its weight.
for piter := parent; piter != nil; piter = piter.parent {
if piter == st {
parent.parent = st.parent
break
}
}
st.parent = parent
if priority.Exclusive && (st.parent != nil || priority.StreamDep == 0) {
for _, openStream := range streams {
if openStream != st && openStream.parent == st.parent {
openStream.parent = st
}
}
}
}
func (sc *serverConn) newWriterAndRequest(st *stream, f *MetaHeadersFrame) (*responseWriter, *http.Request, error) {
sc.serveG.check()
......@@ -1758,7 +1732,7 @@ func (sc *serverConn) runHandler(rw *responseWriter, req *http.Request, handler
const size = 64 << 10
buf := make([]byte, size)
buf = buf[:runtime.Stack(buf, false)]
sc.writeFrameFromHandler(frameWriteMsg{
sc.writeFrameFromHandler(FrameWriteRequest{
write: handlerPanicRST{rw.rws.stream.id},
stream: rw.rws.stream,
})
......@@ -1793,7 +1767,7 @@ func (sc *serverConn) writeHeaders(st *stream, headerData *writeResHeaders) erro
// mutates it.
errc = errChanPool.Get().(chan error)
}
if err := sc.writeFrameFromHandler(frameWriteMsg{
if err := sc.writeFrameFromHandler(FrameWriteRequest{
write: headerData,
stream: st,
done: errc,
......@@ -1816,7 +1790,7 @@ func (sc *serverConn) writeHeaders(st *stream, headerData *writeResHeaders) erro
// called from handler goroutines.
func (sc *serverConn) write100ContinueHeaders(st *stream) {
sc.writeFrameFromHandler(frameWriteMsg{
sc.writeFrameFromHandler(FrameWriteRequest{
write: write100ContinueHeadersFrame{st.id},
stream: st,
})
......@@ -1887,7 +1861,7 @@ func (sc *serverConn) sendWindowUpdate32(st *stream, n int32) {
if st != nil {
streamID = st.id
}
sc.writeFrame(frameWriteMsg{
sc.writeFrame(FrameWriteRequest{
write: writeWindowUpdate{streamID: streamID, n: uint32(n)},
stream: st,
})
......
......@@ -6,14 +6,51 @@ package http2
import "fmt"
// frameWriteMsg is a request to write a frame.
type frameWriteMsg struct {
// WriteScheduler is the interface implemented by HTTP/2 write schedulers.
// Methods are never called concurrently.
type WriteScheduler interface {
// OpenStream opens a new stream in the write scheduler.
// It is illegal to call this with streamID=0 or with a streamID that is
// already open -- the call may panic.
OpenStream(streamID uint32, options OpenStreamOptions)
// CloseStream closes a stream in the write scheduler. Any frames queued on
// this stream should be discarded. It is illegal to call this on a stream
// that is not open -- the call may panic.
CloseStream(streamID uint32)
// AdjustStream adjusts the priority of the given stream. This may be called
// on a stream that has not yet been opened or has been closed. Note that
// RFC 7540 allows PRIORITY frames to be sent on streams in any state. See:
// https://tools.ietf.org/html/rfc7540#section-5.1
AdjustStream(streamID uint32, priority PriorityParam)
// Push queues a frame in the scheduler.
Push(wr FrameWriteRequest)
// Pop dequeues the next frame to write. Returns false if no frames can
// be written. Frames with a given wr.StreamID() are Pop'd in the same
// order they are Push'd.
Pop() (wr FrameWriteRequest, ok bool)
}
// OpenStreamOptions specifies extra options for WriteScheduler.OpenStream.
type OpenStreamOptions struct {
// PusherID is zero if the stream was initiated by the client. Otherwise,
// PusherID names the stream that pushed the newly opened stream.
PusherID uint32
}
// FrameWriteRequest is a request to write a frame.
type FrameWriteRequest struct {
// write is the interface value that does the writing, once the
// writeScheduler (below) has decided to select this frame
// to write. The write functions are all defined in write.go.
// WriteScheduler has selected this frame to write. The write
// functions are all defined in write.go.
write writeFramer
stream *stream // used for prioritization. nil for non-stream frames.
// stream is the stream on which this frame will be written.
// nil for non-stream frames like PING and SETTINGS.
stream *stream
// done, if non-nil, must be a buffered channel with space for
// 1 message and is sent the return value from write (or an
......@@ -21,263 +58,166 @@ type frameWriteMsg struct {
done chan error
}
// for debugging only:
func (wm frameWriteMsg) String() string {
var streamID uint32
if wm.stream != nil {
streamID = wm.stream.id
}
var des string
if s, ok := wm.write.(fmt.Stringer); ok {
des = s.String()
} else {
des = fmt.Sprintf("%T", wm.write)
}
return fmt.Sprintf("[frameWriteMsg stream=%d, ch=%v, type: %v]", streamID, wm.done != nil, des)
}
// writeScheduler tracks pending frames to write, priorities, and decides
// the next one to use. It is not thread-safe.
type writeScheduler struct {
// zero are frames not associated with a specific stream.
// They're sent before any stream-specific freams.
zero writeQueue
// maxFrameSize is the maximum size of a DATA frame
// we'll write. Must be non-zero and between 16K-16M.
maxFrameSize uint32
// sq contains the stream-specific queues, keyed by stream ID.
// when a stream is idle, it's deleted from the map.
sq map[uint32]*writeQueue
// canSend is a slice of memory that's reused between frame
// scheduling decisions to hold the list of writeQueues (from sq)
// which have enough flow control data to send. After canSend is
// built, the best is selected.
canSend []*writeQueue
// pool of empty queues for reuse.
queuePool []*writeQueue
}
func (ws *writeScheduler) putEmptyQueue(q *writeQueue) {
if len(q.s) != 0 {
panic("queue must be empty")
}
ws.queuePool = append(ws.queuePool, q)
}
func (ws *writeScheduler) getEmptyQueue() *writeQueue {
ln := len(ws.queuePool)
if ln == 0 {
return new(writeQueue)
}
q := ws.queuePool[ln-1]
ws.queuePool = ws.queuePool[:ln-1]
return q
}
func (ws *writeScheduler) empty() bool { return ws.zero.empty() && len(ws.sq) == 0 }
func (ws *writeScheduler) add(wm frameWriteMsg) {
st := wm.stream
if st == nil {
ws.zero.push(wm)
} else {
ws.streamQueue(st.id).push(wm)
}
}
func (ws *writeScheduler) streamQueue(streamID uint32) *writeQueue {
if q, ok := ws.sq[streamID]; ok {
return q
}
if ws.sq == nil {
ws.sq = make(map[uint32]*writeQueue)
}
q := ws.getEmptyQueue()
ws.sq[streamID] = q
return q
}
// take returns the most important frame to write and removes it from the scheduler.
// It is illegal to call this if the scheduler is empty or if there are no connection-level
// flow control bytes available.
func (ws *writeScheduler) take() (wm frameWriteMsg, ok bool) {
if ws.maxFrameSize == 0 {
panic("internal error: ws.maxFrameSize not initialized or invalid")
}
// If there any frames not associated with streams, prefer those first.
// These are usually SETTINGS, etc.
if !ws.zero.empty() {
return ws.zero.shift(), true
}
if len(ws.sq) == 0 {
return
}
// Next, prioritize frames on streams that aren't DATA frames (no cost).
for id, q := range ws.sq {
if q.firstIsNoCost() {
return ws.takeFrom(id, q)
}
}
// Now, all that remains are DATA frames with non-zero bytes to
// send. So pick the best one.
if len(ws.canSend) != 0 {
panic("should be empty")
}
for _, q := range ws.sq {
if n := ws.streamWritableBytes(q); n > 0 {
ws.canSend = append(ws.canSend, q)
}
}
if len(ws.canSend) == 0 {
return
}
defer ws.zeroCanSend()
// TODO: find the best queue
q := ws.canSend[0]
return ws.takeFrom(q.streamID(), q)
}
// zeroCanSend is defered from take.
func (ws *writeScheduler) zeroCanSend() {
for i := range ws.canSend {
ws.canSend[i] = nil
}
ws.canSend = ws.canSend[:0]
}
// streamWritableBytes returns the number of DATA bytes we could write
// from the given queue's stream, if this stream/queue were
// selected. It is an error to call this if q's head isn't a
// *writeData.
func (ws *writeScheduler) streamWritableBytes(q *writeQueue) int32 {
wm := q.head()
ret := wm.stream.flow.available() // max we can write
if ret == 0 {
// StreamID returns the id of the stream this frame will be written to.
// 0 is used for non-stream frames such as PING and SETTINGS.
func (wr FrameWriteRequest) StreamID() uint32 {
if wr.stream == nil {
return 0
}
if int32(ws.maxFrameSize) < ret {
ret = int32(ws.maxFrameSize)
}
if ret == 0 {
panic("internal error: ws.maxFrameSize not initialized or invalid")
}
wd := wm.write.(*writeData)
if len(wd.p) < int(ret) {
ret = int32(len(wd.p))
}
return ret
}
func (ws *writeScheduler) takeFrom(id uint32, q *writeQueue) (wm frameWriteMsg, ok bool) {
wm = q.head()
// If the first item in this queue costs flow control tokens
// and we don't have enough, write as much as we can.
if wd, ok := wm.write.(*writeData); ok && len(wd.p) > 0 {
allowed := wm.stream.flow.available() // max we can write
if allowed == 0 {
// No quota available. Caller can try the next stream.
return frameWriteMsg{}, false
return wr.stream.id
}
// DataSize returns the number of flow control bytes that must be consumed
// to write this entire frame. This is 0 for non-DATA frames.
func (wr FrameWriteRequest) DataSize() int {
if wd, ok := wr.write.(*writeData); ok {
return len(wd.p)
}
return 0
}
// Consume consumes min(n, available) bytes from this frame, where available
// is the number of flow control bytes available on the stream. Consume returns
// 0, 1, or 2 frames, where the integer return value gives the number of frames
// returned.
//
// If flow control prevents consuming any bytes, this returns (_, _, 0). If
// the entire frame was consumed, this returns (wr, _, 1). Otherwise, this
// returns (consumed, rest, 2), where 'consumed' contains the consumed bytes and
// 'rest' contains the remaining bytes. The consumed bytes are deducted from the
// underlying stream's flow control budget.
func (wr FrameWriteRequest) Consume(n int32) (FrameWriteRequest, FrameWriteRequest, int) {
var empty FrameWriteRequest
// Non-DATA frames are always consumed whole.
wd, ok := wr.write.(*writeData)
if !ok || len(wd.p) == 0 {
return wr, empty, 1
}
// Might need to split after applying limits.
allowed := wr.stream.flow.available()
if n < allowed {
allowed = n
}
if wr.stream.sc.maxFrameSize < allowed {
allowed = wr.stream.sc.maxFrameSize
}
if allowed <= 0 {
return empty, empty, 0
}
if len(wd.p) > int(allowed) {
wr.stream.flow.take(allowed)
consumed := FrameWriteRequest{
stream: wr.stream,
write: &writeData{
streamID: wd.streamID,
p: wd.p[:allowed],
// Even if the original had endStream set, there
// are bytes remaining because len(wd.p) > allowed,
// so we know endStream is false.
endStream: false,
},
// Our caller is blocking on the final DATA frame, not
// this intermediate frame, so no need to wait.
done: nil,
}
if int32(ws.maxFrameSize) < allowed {
allowed = int32(ws.maxFrameSize)
}
// TODO: further restrict the allowed size, because even if
// the peer says it's okay to write 16MB data frames, we might
// want to write smaller ones to properly weight competing
// streams' priorities.
if len(wd.p) > int(allowed) {
wm.stream.flow.take(allowed)
chunk := wd.p[:allowed]
wd.p = wd.p[allowed:]
// Make up a new write message of a valid size, rather
// than shifting one off the queue.
return frameWriteMsg{
stream: wm.stream,
write: &writeData{
streamID: wd.streamID,
p: chunk,
// even if the original had endStream set, there
// arebytes remaining because len(wd.p) > allowed,
// so we know endStream is false:
endStream: false,
},
// our caller is blocking on the final DATA frame, not
// these intermediates, so no need to wait:
done: nil,
}, true
rest := FrameWriteRequest{
stream: wr.stream,
write: &writeData{
streamID: wd.streamID,
p: wd.p[allowed:],
endStream: wd.endStream,
},
done: wr.done,
}
wm.stream.flow.take(int32(len(wd.p)))
return consumed, rest, 2
}
q.shift()
if q.empty() {
ws.putEmptyQueue(q)
delete(ws.sq, id)
}
return wm, true
// The frame is consumed whole.
// NB: This cast cannot overflow because allowed is <= math.MaxInt32.
wr.stream.flow.take(int32(len(wd.p)))
return wr, empty, 1
}
func (ws *writeScheduler) forgetStream(id uint32) {
q, ok := ws.sq[id]
if !ok {
return
// String is for debugging only.
func (wr FrameWriteRequest) String() string {
var streamID uint32
if wr.stream != nil {
streamID = wr.stream.id
}
delete(ws.sq, id)
// But keep it for others later.
for i := range q.s {
q.s[i] = frameWriteMsg{}
var des string
if s, ok := wr.write.(fmt.Stringer); ok {
des = s.String()
} else {
des = fmt.Sprintf("%T", wr.write)
}
q.s = q.s[:0]
ws.putEmptyQueue(q)
return fmt.Sprintf("[FrameWriteRequest stream=%d, ch=%v, writer=%v]", streamID, wr.done != nil, des)
}
// writeQueue is used by implementations of WriteScheduler.
type writeQueue struct {
s []frameWriteMsg
s []FrameWriteRequest
}
// streamID returns the stream ID for a non-empty stream-specific queue.
func (q *writeQueue) streamID() uint32 { return q.s[0].stream.id }
func (q *writeQueue) empty() bool { return len(q.s) == 0 }
func (q *writeQueue) push(wm frameWriteMsg) {
q.s = append(q.s, wm)
func (q *writeQueue) push(wr FrameWriteRequest) {
q.s = append(q.s, wr)
}
// head returns the next item that would be removed by shift.
func (q *writeQueue) head() frameWriteMsg {
func (q *writeQueue) shift() FrameWriteRequest {
if len(q.s) == 0 {
panic("invalid use of queue")
}
return q.s[0]
wr := q.s[0]
// TODO: less copy-happy queue.
copy(q.s, q.s[1:])
q.s[len(q.s)-1] = FrameWriteRequest{}
q.s = q.s[:len(q.s)-1]
return wr
}
func (q *writeQueue) shift() frameWriteMsg {
// consume consumes up to n bytes from q.s[0]. If the frame is
// entirely consumed, it is removed from the queue. If the frame
// is partially consumed, the frame is kept with the consumed
// bytes removed. Returns true iff any bytes were consumed.
func (q *writeQueue) consume(n int32) (FrameWriteRequest, bool) {
if len(q.s) == 0 {
panic("invalid use of queue")
return FrameWriteRequest{}, false
}
wm := q.s[0]
// TODO: less copy-happy queue.
copy(q.s, q.s[1:])
q.s[len(q.s)-1] = frameWriteMsg{}
q.s = q.s[:len(q.s)-1]
return wm
consumed, rest, numresult := q.s[0].Consume(n)
switch numresult {
case 0:
return FrameWriteRequest{}, false
case 1:
q.shift()
case 2:
q.s[0] = rest
}
return consumed, true
}
type writeQueuePool []*writeQueue
// put inserts an unused writeQueue into the pool.
func (p *writeQueuePool) put(q *writeQueue) {
for i := range q.s {
q.s[i] = FrameWriteRequest{}
}
q.s = q.s[:0]
*p = append(*p, q)
}
func (q *writeQueue) firstIsNoCost() bool {
if df, ok := q.s[0].write.(*writeData); ok {
return len(df.p) == 0
// get returns an empty writeQueue.
func (p *writeQueuePool) get() *writeQueue {
ln := len(*p)
if ln == 0 {
return new(writeQueue)
}
return true
x := ln - 1
q := (*p)[x]
(*p)[x] = nil
*p = (*p)[:x]
return q
}
// Copyright 2016 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package http2
import (
"fmt"
"math"
"sort"
)
// RFC 7540, Section 5.3.5: the default weight is 16.
const priorityDefaultWeight = 15 // 16 = 15 + 1
// PriorityWriteSchedulerConfig configures a priorityWriteScheduler.
type PriorityWriteSchedulerConfig struct {
// MaxClosedNodesInTree controls the maximum number of closed streams to
// retain in the priority tree. Setting this to zero saves a small amount
// of memory at the cost of performance.
//
// See RFC 7540, Section 5.3.4:
// "It is possible for a stream to become closed while prioritization
// information ... is in transit. ... This potentially creates suboptimal
// prioritization, since the stream could be given a priority that is
// different from what is intended. To avoid these problems, an endpoint
// SHOULD retain stream prioritization state for a period after streams
// become closed. The longer state is retained, the lower the chance that
// streams are assigned incorrect or default priority values."
MaxClosedNodesInTree int
// MaxIdleNodesInTree controls the maximum number of idle streams to
// retain in the priority tree. Setting this to zero saves a small amount
// of memory at the cost of performance.
//
// See RFC 7540, Section 5.3.4:
// Similarly, streams that are in the "idle" state can be assigned
// priority or become a parent of other streams. This allows for the
// creation of a grouping node in the dependency tree, which enables
// more flexible expressions of priority. Idle streams begin with a
// default priority (Section 5.3.5).
MaxIdleNodesInTree int
// ThrottleOutOfOrderWrites enables write throttling to help ensure that
// data is delivered in priority order. This works around a race where
// stream B depends on stream A and both streams are about to call Write
// to queue DATA frames. If B wins the race, a naive scheduler would eagerly
// write as much data from B as possible, but this is suboptimal because A
// is a higher-priority stream. With throttling enabled, we write a small
// amount of data from B to minimize the amount of bandwidth that B can
// steal from A.
ThrottleOutOfOrderWrites bool
}
// NewPriorityWriteScheduler constructs a WriteScheduler that schedules
// frames by following HTTP/2 priorities as described in RFC 7340 Section 5.3.
// If cfg is nil, default options are used.
func NewPriorityWriteScheduler(cfg *PriorityWriteSchedulerConfig) WriteScheduler {
if cfg == nil {
// For justification of these defaults, see:
// https://docs.google.com/document/d/1oLhNg1skaWD4_DtaoCxdSRN5erEXrH-KnLrMwEpOtFY
cfg = &PriorityWriteSchedulerConfig{
MaxClosedNodesInTree: 10,
MaxIdleNodesInTree: 10,
ThrottleOutOfOrderWrites: false,
}
}
ws := &priorityWriteScheduler{
nodes: make(map[uint32]*priorityNode),
maxClosedNodesInTree: cfg.MaxClosedNodesInTree,
maxIdleNodesInTree: cfg.MaxIdleNodesInTree,
enableWriteThrottle: cfg.ThrottleOutOfOrderWrites,
}
ws.nodes[0] = &ws.root
if cfg.ThrottleOutOfOrderWrites {
ws.writeThrottleLimit = 1024
} else {
ws.writeThrottleLimit = math.MaxInt32
}
return ws
}
type priorityNodeState int
const (
priorityNodeOpen priorityNodeState = iota
priorityNodeClosed
priorityNodeIdle
)
// priorityNode is a node in an HTTP/2 priority tree.
// Each node is associated with a single stream ID.
// See RFC 7540, Section 5.3.
type priorityNode struct {
q writeQueue // queue of pending frames to write
id uint32 // id of the stream, or 0 for the root of the tree
weight uint8 // the actual weight is weight+1, so the value is in [1,256]
state priorityNodeState // open | closed | idle
bytes int64 // number of bytes written by this node, or 0 if closed
subtreeBytes int64 // sum(node.bytes) of all nodes in this subtree
// These links form the priority tree.
parent *priorityNode
kids *priorityNode // start of the kids list
prev, next *priorityNode // doubly-linked list of siblings
}
func (n *priorityNode) setParent(parent *priorityNode) {
if n == parent {
panic("setParent to self")
}
if n.parent == parent {
return
}
// Unlink from current parent.
if parent := n.parent; parent != nil {
if n.prev == nil {
parent.kids = n.next
} else {
n.prev.next = n.next
}
if n.next != nil {
n.next.prev = n.prev
}
}
// Link to new parent.
// If parent=nil, remove n from the tree.
// Always insert at the head of parent.kids (this is assumed by walkReadyInOrder).
n.parent = parent
if parent == nil {
n.next = nil
n.prev = nil
} else {
n.next = parent.kids
n.prev = nil
if n.next != nil {
n.next.prev = n
}
parent.kids = n
}
}
func (n *priorityNode) addBytes(b int64) {
n.bytes += b
for ; n != nil; n = n.parent {
n.subtreeBytes += b
}
}
// walkReadyInOrder iterates over the tree in priority order, calling f for each node
// with a non-empty write queue. When f returns true, this funcion returns true and the
// walk halts. tmp is used as scratch space for sorting.
//
// f(n, openParent) takes two arguments: the node to visit, n, and a bool that is true
// if any ancestor p of n is still open (ignoring the root node).
func (n *priorityNode) walkReadyInOrder(openParent bool, tmp *[]*priorityNode, f func(*priorityNode, bool) bool) bool {
if !n.q.empty() && f(n, openParent) {
return true
}
if n.kids == nil {
return false
}
// Don't consider the root "open" when updating openParent since
// we can't send data frames on the root stream (only control frames).
if n.id != 0 {
openParent = openParent || (n.state == priorityNodeOpen)
}
// Common case: only one kid or all kids have the same weight.
// Some clients don't use weights; other clients (like web browsers)
// use mostly-linear priority trees.
w := n.kids.weight
needSort := false
for k := n.kids.next; k != nil; k = k.next {
if k.weight != w {
needSort = true
break
}
}
if !needSort {
for k := n.kids; k != nil; k = k.next {
if k.walkReadyInOrder(openParent, tmp, f) {
return true
}
}
return false
}
// Uncommon case: sort the child nodes. We remove the kids from the parent,
// then re-insert after sorting so we can reuse tmp for future sort calls.
*tmp = (*tmp)[:0]
for n.kids != nil {
*tmp = append(*tmp, n.kids)
n.kids.setParent(nil)
}
sort.Sort(sortPriorityNodeSiblings(*tmp))
for i := len(*tmp) - 1; i >= 0; i-- {
(*tmp)[i].setParent(n) // setParent inserts at the head of n.kids
}
for k := n.kids; k != nil; k = k.next {
if k.walkReadyInOrder(openParent, tmp, f) {
return true
}
}
return false
}
type sortPriorityNodeSiblings []*priorityNode
func (z sortPriorityNodeSiblings) Len() int { return len(z) }
func (z sortPriorityNodeSiblings) Swap(i, k int) { z[i], z[k] = z[k], z[i] }
func (z sortPriorityNodeSiblings) Less(i, k int) bool {
// Prefer the subtree that has sent fewer bytes relative to its weight.
// See sections 5.3.2 and 5.3.4.
wi, bi := float64(z[i].weight+1), float64(z[i].subtreeBytes)
wk, bk := float64(z[k].weight+1), float64(z[k].subtreeBytes)
if bi == 0 && bk == 0 {
return wi >= wk
}
if bk == 0 {
return false
}
return bi/bk <= wi/wk
}
type priorityWriteScheduler struct {
// root is the root of the priority tree, where root.id = 0.
// The root queues control frames that are not associated with any stream.
root priorityNode
// nodes maps stream ids to priority tree nodes.
nodes map[uint32]*priorityNode
// maxID is the maximum stream id in nodes.
maxID uint32
// lists of nodes that have been closed or are idle, but are kept in
// the tree for improved prioritization. When the lengths exceed either
// maxClosedNodesInTree or maxIdleNodesInTree, old nodes are discarded.
closedNodes, idleNodes []*priorityNode
// From the config.
maxClosedNodesInTree int
maxIdleNodesInTree int
writeThrottleLimit int32
enableWriteThrottle bool
// tmp is scratch space for priorityNode.walkReadyInOrder to reduce allocations.
tmp []*priorityNode
// pool of empty queues for reuse.
queuePool writeQueuePool
}
func (ws *priorityWriteScheduler) OpenStream(streamID uint32, options OpenStreamOptions) {
// The stream may be currently idle but cannot be opened or closed.
if curr := ws.nodes[streamID]; curr != nil {
if curr.state != priorityNodeIdle {
panic(fmt.Sprintf("stream %d already opened", streamID))
}
curr.state = priorityNodeOpen
return
}
// RFC 7540, Section 5.3.5:
// "All streams are initially assigned a non-exclusive dependency on stream 0x0.
// Pushed streams initially depend on their associated stream. In both cases,
// streams are assigned a default weight of 16."
parent := ws.nodes[options.PusherID]
if parent == nil {
parent = &ws.root
}
n := &priorityNode{
q: *ws.queuePool.get(),
id: streamID,
weight: priorityDefaultWeight,
state: priorityNodeOpen,
}
n.setParent(parent)
ws.nodes[streamID] = n
if streamID > ws.maxID {
ws.maxID = streamID
}
}
func (ws *priorityWriteScheduler) CloseStream(streamID uint32) {
if streamID == 0 {
panic("violation of WriteScheduler interface: cannot close stream 0")
}
if ws.nodes[streamID] == nil {
panic(fmt.Sprintf("violation of WriteScheduler interface: unknown stream %d", streamID))
}
if ws.nodes[streamID].state != priorityNodeOpen {
panic(fmt.Sprintf("violation of WriteScheduler interface: stream %d already closed", streamID))
}
n := ws.nodes[streamID]
n.state = priorityNodeClosed
n.addBytes(-n.bytes)
q := n.q
ws.queuePool.put(&q)
n.q.s = nil
if ws.maxClosedNodesInTree > 0 {
ws.addClosedOrIdleNode(&ws.closedNodes, ws.maxClosedNodesInTree, n)
} else {
ws.removeNode(n)
}
}
func (ws *priorityWriteScheduler) AdjustStream(streamID uint32, priority PriorityParam) {
if streamID == 0 {
panic("adjustPriority on root")
}
// If streamID does not exist, there are two cases:
// - A closed stream that has been removed (this will have ID <= maxID)
// - An idle stream that is being used for "grouping" (this will have ID > maxID)
n := ws.nodes[streamID]
if n == nil {
if streamID <= ws.maxID || ws.maxIdleNodesInTree == 0 {
return
}
ws.maxID = streamID
n = &priorityNode{
q: *ws.queuePool.get(),
id: streamID,
weight: priorityDefaultWeight,
state: priorityNodeIdle,
}
n.setParent(&ws.root)
ws.nodes[streamID] = n
ws.addClosedOrIdleNode(&ws.idleNodes, ws.maxIdleNodesInTree, n)
}
// Section 5.3.1: A dependency on a stream that is not currently in the tree
// results in that stream being given a default priority (Section 5.3.5).
parent := ws.nodes[priority.StreamDep]
if parent == nil {
n.setParent(&ws.root)
n.weight = priorityDefaultWeight
return
}
// Ignore if the client tries to make a node its own parent.
if n == parent {
return
}
// Section 5.3.3:
// "If a stream is made dependent on one of its own dependencies, the
// formerly dependent stream is first moved to be dependent on the
// reprioritized stream's previous parent. The moved dependency retains
// its weight."
//
// That is: if parent depends on n, move parent to depend on n.parent.
for x := parent.parent; x != nil; x = x.parent {
if x == n {
parent.setParent(n.parent)
break
}
}
// Section 5.3.3: The exclusive flag causes the stream to become the sole
// dependency of its parent stream, causing other dependencies to become
// dependent on the exclusive stream.
if priority.Exclusive {
k := parent.kids
for k != nil {
next := k.next
if k != n {
k.setParent(n)
}
k = next
}
}
n.setParent(parent)
n.weight = priority.Weight
}
func (ws *priorityWriteScheduler) Push(wr FrameWriteRequest) {
var n *priorityNode
if id := wr.StreamID(); id == 0 {
n = &ws.root
} else {
n = ws.nodes[id]
if n == nil {
panic("add on non-open stream")
}
}
n.q.push(wr)
}
func (ws *priorityWriteScheduler) Pop() (wr FrameWriteRequest, ok bool) {
ws.root.walkReadyInOrder(false, &ws.tmp, func(n *priorityNode, openParent bool) bool {
limit := int32(math.MaxInt32)
if openParent {
limit = ws.writeThrottleLimit
}
wr, ok = n.q.consume(limit)
if !ok {
return false
}
n.addBytes(int64(wr.DataSize()))
// If B depends on A and B continuously has data available but A
// does not, gradually increase the throttling limit to allow B to
// steal more and more bandwidth from A.
if openParent {
ws.writeThrottleLimit += 1024
if ws.writeThrottleLimit < 0 {
ws.writeThrottleLimit = math.MaxInt32
}
} else if ws.enableWriteThrottle {
ws.writeThrottleLimit = 1024
}
return true
})
return wr, ok
}
func (ws *priorityWriteScheduler) addClosedOrIdleNode(list *[]*priorityNode, maxSize int, n *priorityNode) {
if maxSize == 0 {
return
}
if len(*list) == maxSize {
// Remove the oldest node, then shift left.
ws.removeNode((*list)[0])
x := (*list)[1:]
copy(*list, x)
*list = (*list)[:len(x)]
}
*list = append(*list, n)
}
func (ws *priorityWriteScheduler) removeNode(n *priorityNode) {
for k := n.kids; k != nil; k = k.next {
k.setParent(n.parent)
}
n.setParent(nil)
delete(ws.nodes, n.id)
}
// Copyright 2016 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package http2
import (
"bytes"
"fmt"
"sort"
"testing"
)
func defaultPriorityWriteScheduler() *priorityWriteScheduler {
return NewPriorityWriteScheduler(nil).(*priorityWriteScheduler)
}
func checkPriorityWellFormed(ws *priorityWriteScheduler) error {
for id, n := range ws.nodes {
if id != n.id {
return fmt.Errorf("bad ws.nodes: ws.nodes[%d] = %d", id, n.id)
}
if n.parent == nil {
if n.next != nil || n.prev != nil {
return fmt.Errorf("bad node %d: nil parent but prev/next not nil", id)
}
continue
}
found := false
for k := n.parent.kids; k != nil; k = k.next {
if k.id == id {
found = true
break
}
}
if !found {
return fmt.Errorf("bad node %d: not found in parent %d kids list", id, n.parent.id)
}
}
return nil
}
func fmtTree(ws *priorityWriteScheduler, fmtNode func(*priorityNode) string) string {
var ids []int
for _, n := range ws.nodes {
ids = append(ids, int(n.id))
}
sort.Ints(ids)
var buf bytes.Buffer
for _, id := range ids {
if buf.Len() != 0 {
buf.WriteString(" ")
}
if id == 0 {
buf.WriteString(fmtNode(&ws.root))
} else {
buf.WriteString(fmtNode(ws.nodes[uint32(id)]))
}
}
return buf.String()
}
func fmtNodeParentSkipRoot(n *priorityNode) string {
switch {
case n.id == 0:
return ""
case n.parent == nil:
return fmt.Sprintf("%d{parent:nil}", n.id)
default:
return fmt.Sprintf("%d{parent:%d}", n.id, n.parent.id)
}
}
func fmtNodeWeightParentSkipRoot(n *priorityNode) string {
switch {
case n.id == 0:
return ""
case n.parent == nil:
return fmt.Sprintf("%d{weight:%d,parent:nil}", n.id, n.weight)
default:
return fmt.Sprintf("%d{weight:%d,parent:%d}", n.id, n.weight, n.parent.id)
}
}
func TestPriorityTwoStreams(t *testing.T) {
ws := defaultPriorityWriteScheduler()
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{})
want := "1{weight:15,parent:0} 2{weight:15,parent:0}"
if got := fmtTree(ws, fmtNodeWeightParentSkipRoot); got != want {
t.Errorf("After open\ngot %q\nwant %q", got, want)
}
// Move 1's parent to 2.
ws.AdjustStream(1, PriorityParam{
StreamDep: 2,
Weight: 32,
Exclusive: false,
})
want = "1{weight:32,parent:2} 2{weight:15,parent:0}"
if got := fmtTree(ws, fmtNodeWeightParentSkipRoot); got != want {
t.Errorf("After adjust\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func TestPriorityAdjustExclusiveZero(t *testing.T) {
// 1, 2, and 3 are all children of the 0 stream.
// Exclusive reprioritization to any of the streams should bring
// the rest of the streams under the reprioritized stream.
ws := defaultPriorityWriteScheduler()
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{})
ws.OpenStream(3, OpenStreamOptions{})
want := "1{weight:15,parent:0} 2{weight:15,parent:0} 3{weight:15,parent:0}"
if got := fmtTree(ws, fmtNodeWeightParentSkipRoot); got != want {
t.Errorf("After open\ngot %q\nwant %q", got, want)
}
ws.AdjustStream(2, PriorityParam{
StreamDep: 0,
Weight: 20,
Exclusive: true,
})
want = "1{weight:15,parent:2} 2{weight:20,parent:0} 3{weight:15,parent:2}"
if got := fmtTree(ws, fmtNodeWeightParentSkipRoot); got != want {
t.Errorf("After adjust\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func TestPriorityAdjustOwnParent(t *testing.T) {
// Assigning a node as its own parent should have no effect.
ws := defaultPriorityWriteScheduler()
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{})
ws.AdjustStream(2, PriorityParam{
StreamDep: 2,
Weight: 20,
Exclusive: true,
})
want := "1{weight:15,parent:0} 2{weight:15,parent:0}"
if got := fmtTree(ws, fmtNodeWeightParentSkipRoot); got != want {
t.Errorf("After adjust\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func TestPriorityClosedStreams(t *testing.T) {
ws := NewPriorityWriteScheduler(&PriorityWriteSchedulerConfig{MaxClosedNodesInTree: 2}).(*priorityWriteScheduler)
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{PusherID: 1})
ws.OpenStream(3, OpenStreamOptions{PusherID: 2})
ws.OpenStream(4, OpenStreamOptions{PusherID: 3})
// Close the first three streams. We lose 1, but keep 2 and 3.
ws.CloseStream(1)
ws.CloseStream(2)
ws.CloseStream(3)
want := "2{weight:15,parent:0} 3{weight:15,parent:2} 4{weight:15,parent:3}"
if got := fmtTree(ws, fmtNodeWeightParentSkipRoot); got != want {
t.Errorf("After close\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
// Adding a stream as an exclusive child of 1 gives it default
// priorities, since 1 is gone.
ws.OpenStream(5, OpenStreamOptions{})
ws.AdjustStream(5, PriorityParam{StreamDep: 1, Weight: 15, Exclusive: true})
// Adding a stream as an exclusive child of 2 should work, since 2 is not gone.
ws.OpenStream(6, OpenStreamOptions{})
ws.AdjustStream(6, PriorityParam{StreamDep: 2, Weight: 15, Exclusive: true})
want = "2{weight:15,parent:0} 3{weight:15,parent:6} 4{weight:15,parent:3} 5{weight:15,parent:0} 6{weight:15,parent:2}"
if got := fmtTree(ws, fmtNodeWeightParentSkipRoot); got != want {
t.Errorf("After add streams\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func TestPriorityClosedStreamsDisabled(t *testing.T) {
ws := NewPriorityWriteScheduler(&PriorityWriteSchedulerConfig{}).(*priorityWriteScheduler)
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{PusherID: 1})
ws.OpenStream(3, OpenStreamOptions{PusherID: 2})
// Close the first two streams. We keep only 3.
ws.CloseStream(1)
ws.CloseStream(2)
want := "3{weight:15,parent:0}"
if got := fmtTree(ws, fmtNodeWeightParentSkipRoot); got != want {
t.Errorf("After close\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func TestPriorityIdleStreams(t *testing.T) {
ws := NewPriorityWriteScheduler(&PriorityWriteSchedulerConfig{MaxIdleNodesInTree: 2}).(*priorityWriteScheduler)
ws.AdjustStream(1, PriorityParam{StreamDep: 0, Weight: 15}) // idle
ws.AdjustStream(2, PriorityParam{StreamDep: 0, Weight: 15}) // idle
ws.AdjustStream(3, PriorityParam{StreamDep: 2, Weight: 20}) // idle
ws.OpenStream(4, OpenStreamOptions{})
ws.OpenStream(5, OpenStreamOptions{})
ws.OpenStream(6, OpenStreamOptions{})
ws.AdjustStream(4, PriorityParam{StreamDep: 1, Weight: 15})
ws.AdjustStream(5, PriorityParam{StreamDep: 2, Weight: 15})
ws.AdjustStream(6, PriorityParam{StreamDep: 3, Weight: 15})
want := "2{weight:15,parent:0} 3{weight:20,parent:2} 4{weight:15,parent:0} 5{weight:15,parent:2} 6{weight:15,parent:3}"
if got := fmtTree(ws, fmtNodeWeightParentSkipRoot); got != want {
t.Errorf("After open\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func TestPriorityIdleStreamsDisabled(t *testing.T) {
ws := NewPriorityWriteScheduler(&PriorityWriteSchedulerConfig{}).(*priorityWriteScheduler)
ws.AdjustStream(1, PriorityParam{StreamDep: 0, Weight: 15}) // idle
ws.AdjustStream(2, PriorityParam{StreamDep: 0, Weight: 15}) // idle
ws.AdjustStream(3, PriorityParam{StreamDep: 2, Weight: 20}) // idle
ws.OpenStream(4, OpenStreamOptions{})
want := "4{weight:15,parent:0}"
if got := fmtTree(ws, fmtNodeWeightParentSkipRoot); got != want {
t.Errorf("After open\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func TestPrioritySection531NonExclusive(t *testing.T) {
// Example from RFC 7540 Section 5.3.1.
// A,B,C,D = 1,2,3,4
ws := defaultPriorityWriteScheduler()
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{PusherID: 1})
ws.OpenStream(3, OpenStreamOptions{PusherID: 1})
ws.OpenStream(4, OpenStreamOptions{})
ws.AdjustStream(4, PriorityParam{
StreamDep: 1,
Weight: 15,
Exclusive: false,
})
want := "1{parent:0} 2{parent:1} 3{parent:1} 4{parent:1}"
if got := fmtTree(ws, fmtNodeParentSkipRoot); got != want {
t.Errorf("After adjust\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func TestPrioritySection531Exclusive(t *testing.T) {
// Example from RFC 7540 Section 5.3.1.
// A,B,C,D = 1,2,3,4
ws := defaultPriorityWriteScheduler()
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{PusherID: 1})
ws.OpenStream(3, OpenStreamOptions{PusherID: 1})
ws.OpenStream(4, OpenStreamOptions{})
ws.AdjustStream(4, PriorityParam{
StreamDep: 1,
Weight: 15,
Exclusive: true,
})
want := "1{parent:0} 2{parent:4} 3{parent:4} 4{parent:1}"
if got := fmtTree(ws, fmtNodeParentSkipRoot); got != want {
t.Errorf("After adjust\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func makeSection533Tree() *priorityWriteScheduler {
// Initial tree from RFC 7540 Section 5.3.3.
// A,B,C,D,E,F = 1,2,3,4,5,6
ws := defaultPriorityWriteScheduler()
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{PusherID: 1})
ws.OpenStream(3, OpenStreamOptions{PusherID: 1})
ws.OpenStream(4, OpenStreamOptions{PusherID: 3})
ws.OpenStream(5, OpenStreamOptions{PusherID: 3})
ws.OpenStream(6, OpenStreamOptions{PusherID: 4})
return ws
}
func TestPrioritySection533NonExclusive(t *testing.T) {
// Example from RFC 7540 Section 5.3.3.
// A,B,C,D,E,F = 1,2,3,4,5,6
ws := defaultPriorityWriteScheduler()
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{PusherID: 1})
ws.OpenStream(3, OpenStreamOptions{PusherID: 1})
ws.OpenStream(4, OpenStreamOptions{PusherID: 3})
ws.OpenStream(5, OpenStreamOptions{PusherID: 3})
ws.OpenStream(6, OpenStreamOptions{PusherID: 4})
ws.AdjustStream(1, PriorityParam{
StreamDep: 4,
Weight: 15,
Exclusive: false,
})
want := "1{parent:4} 2{parent:1} 3{parent:1} 4{parent:0} 5{parent:3} 6{parent:4}"
if got := fmtTree(ws, fmtNodeParentSkipRoot); got != want {
t.Errorf("After adjust\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func TestPrioritySection533Exclusive(t *testing.T) {
// Example from RFC 7540 Section 5.3.3.
// A,B,C,D,E,F = 1,2,3,4,5,6
ws := defaultPriorityWriteScheduler()
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{PusherID: 1})
ws.OpenStream(3, OpenStreamOptions{PusherID: 1})
ws.OpenStream(4, OpenStreamOptions{PusherID: 3})
ws.OpenStream(5, OpenStreamOptions{PusherID: 3})
ws.OpenStream(6, OpenStreamOptions{PusherID: 4})
ws.AdjustStream(1, PriorityParam{
StreamDep: 4,
Weight: 15,
Exclusive: true,
})
want := "1{parent:4} 2{parent:1} 3{parent:1} 4{parent:0} 5{parent:3} 6{parent:1}"
if got := fmtTree(ws, fmtNodeParentSkipRoot); got != want {
t.Errorf("After adjust\ngot %q\nwant %q", got, want)
}
if err := checkPriorityWellFormed(ws); err != nil {
t.Error(err)
}
}
func checkPopAll(ws WriteScheduler, order []uint32) error {
for k, id := range order {
wr, ok := ws.Pop()
if !ok {
return fmt.Errorf("Pop[%d]: got ok=false, want %d (order=%v)", k, id, order)
}
if got := wr.StreamID(); got != id {
return fmt.Errorf("Pop[%d]: got %v, want %d (order=%v)", k, got, id, order)
}
}
wr, ok := ws.Pop()
if ok {
return fmt.Errorf("Pop[%d]: got %v, want ok=false (order=%v)", len(order), wr.StreamID(), order)
}
return nil
}
func TestPriorityPopFrom533Tree(t *testing.T) {
ws := makeSection533Tree()
ws.Push(makeWriteHeadersRequest(3 /*C*/))
ws.Push(makeWriteNonStreamRequest())
ws.Push(makeWriteHeadersRequest(5 /*E*/))
ws.Push(makeWriteHeadersRequest(1 /*A*/))
t.Log("tree:", fmtTree(ws, fmtNodeParentSkipRoot))
if err := checkPopAll(ws, []uint32{0 /*NonStream*/, 1, 3, 5}); err != nil {
t.Error(err)
}
}
func TestPriorityPopFromLinearTree(t *testing.T) {
ws := defaultPriorityWriteScheduler()
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{PusherID: 1})
ws.OpenStream(3, OpenStreamOptions{PusherID: 2})
ws.OpenStream(4, OpenStreamOptions{PusherID: 3})
ws.Push(makeWriteHeadersRequest(3))
ws.Push(makeWriteHeadersRequest(4))
ws.Push(makeWriteHeadersRequest(1))
ws.Push(makeWriteHeadersRequest(2))
ws.Push(makeWriteNonStreamRequest())
ws.Push(makeWriteNonStreamRequest())
t.Log("tree:", fmtTree(ws, fmtNodeParentSkipRoot))
if err := checkPopAll(ws, []uint32{0, 0 /*NonStreams*/, 1, 2, 3, 4}); err != nil {
t.Error(err)
}
}
func TestPriorityFlowControl(t *testing.T) {
ws := NewPriorityWriteScheduler(&PriorityWriteSchedulerConfig{ThrottleOutOfOrderWrites: false})
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{PusherID: 1})
sc := &serverConn{maxFrameSize: 16}
st1 := &stream{id: 1, sc: sc}
st2 := &stream{id: 2, sc: sc}
ws.Push(FrameWriteRequest{&writeData{1, make([]byte, 16), false}, st1, nil})
ws.Push(FrameWriteRequest{&writeData{2, make([]byte, 16), false}, st2, nil})
ws.AdjustStream(2, PriorityParam{StreamDep: 1})
// No flow-control bytes available.
if wr, ok := ws.Pop(); ok {
t.Fatalf("Pop(limited by flow control)=%v,true, want false", wr)
}
// Add enough flow-control bytes to write st2 in two Pop calls.
// Should write data from st2 even though it's lower priority than st1.
for i := 1; i <= 2; i++ {
st2.flow.add(8)
wr, ok := ws.Pop()
if !ok {
t.Fatalf("Pop(%d)=false, want true", i)
}
if got, want := wr.DataSize(), 8; got != want {
t.Fatalf("Pop(%d)=%d bytes, want %d bytes", got, want)
}
}
}
func TestPriorityThrottleOutOfOrderWrites(t *testing.T) {
ws := NewPriorityWriteScheduler(&PriorityWriteSchedulerConfig{ThrottleOutOfOrderWrites: true})
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{PusherID: 1})
sc := &serverConn{maxFrameSize: 4096}
st1 := &stream{id: 1, sc: sc}
st2 := &stream{id: 2, sc: sc}
st1.flow.add(4096)
st2.flow.add(4096)
ws.Push(FrameWriteRequest{&writeData{2, make([]byte, 4096), false}, st2, nil})
ws.AdjustStream(2, PriorityParam{StreamDep: 1})
// We have enough flow-control bytes to write st2 in a single Pop call.
// However, due to out-of-order write throttling, the first call should
// only write 1KB.
wr, ok := ws.Pop()
if !ok {
t.Fatalf("Pop(st2.first)=false, want true")
}
if got, want := wr.StreamID(), uint32(2); got != want {
t.Fatalf("Pop(st2.first)=stream %d, want stream %d", got, want)
}
if got, want := wr.DataSize(), 1024; got != want {
t.Fatalf("Pop(st2.first)=%d bytes, want %d bytes", got, want)
}
// Now add data on st1. This should take precedence.
ws.Push(FrameWriteRequest{&writeData{1, make([]byte, 4096), false}, st1, nil})
wr, ok = ws.Pop()
if !ok {
t.Fatalf("Pop(st1)=false, want true")
}
if got, want := wr.StreamID(), uint32(1); got != want {
t.Fatalf("Pop(st1)=stream %d, want stream %d", got, want)
}
if got, want := wr.DataSize(), 4096; got != want {
t.Fatalf("Pop(st1)=%d bytes, want %d bytes", got, want)
}
// Should go back to writing 1KB from st2.
wr, ok = ws.Pop()
if !ok {
t.Fatalf("Pop(st2.last)=false, want true")
}
if got, want := wr.StreamID(), uint32(2); got != want {
t.Fatalf("Pop(st2.last)=stream %d, want stream %d", got, want)
}
if got, want := wr.DataSize(), 1024; got != want {
t.Fatalf("Pop(st2.last)=%d bytes, want %d bytes", got, want)
}
}
func TestPriorityWeights(t *testing.T) {
ws := defaultPriorityWriteScheduler()
ws.OpenStream(1, OpenStreamOptions{})
ws.OpenStream(2, OpenStreamOptions{})
sc := &serverConn{maxFrameSize: 8}
st1 := &stream{id: 1, sc: sc}
st2 := &stream{id: 2, sc: sc}
st1.flow.add(40)
st2.flow.add(40)
ws.Push(FrameWriteRequest{&writeData{1, make([]byte, 40), false}, st1, nil})
ws.Push(FrameWriteRequest{&writeData{2, make([]byte, 40), false}, st2, nil})
ws.AdjustStream(1, PriorityParam{StreamDep: 0, Weight: 34})
ws.AdjustStream(2, PriorityParam{StreamDep: 0, Weight: 9})
// st1 gets 3.5x the bandwidth of st2 (3.5 = (34+1)/(9+1)).
// The maximum frame size is 8 bytes. The write sequence should be:
// st1, total bytes so far is (st1=8, st=0)
// st2, total bytes so far is (st1=8, st=8)
// st1, total bytes so far is (st1=16, st=8)
// st1, total bytes so far is (st1=24, st=8) // 3x bandwidth
// st1, total bytes so far is (st1=32, st=8) // 4x bandwidth
// st2, total bytes so far is (st1=32, st=16) // 2x bandwidth
// st1, total bytes so far is (st1=40, st=16)
// st2, total bytes so far is (st1=40, st=24)
// st2, total bytes so far is (st1=40, st=32)
// st2, total bytes so far is (st1=40, st=40)
if err := checkPopAll(ws, []uint32{1, 2, 1, 1, 1, 2, 1, 2, 2, 2}); err != nil {
t.Error(err)
}
}
// Copyright 2014 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package http2
import "math"
// NewRandomWriteScheduler constructs a WriteScheduler that ignores HTTP/2
// priorities. Control frames like SETTINGS and PING are written before DATA
// frames, but if no control frames are queued and multiple streams have queued
// HEADERS or DATA frames, Pop selects a ready stream arbitrarily.
func NewRandomWriteScheduler() WriteScheduler {
return &randomWriteScheduler{sq: make(map[uint32]*writeQueue)}
}
type randomWriteScheduler struct {
// zero are frames not associated with a specific stream.
zero writeQueue
// sq contains the stream-specific queues, keyed by stream ID.
// When a stream is idle or closed, it's deleted from the map.
sq map[uint32]*writeQueue
// pool of empty queues for reuse.
queuePool writeQueuePool
}
func (ws *randomWriteScheduler) OpenStream(streamID uint32, options OpenStreamOptions) {
// no-op: idle streams are not tracked
}
func (ws *randomWriteScheduler) CloseStream(streamID uint32) {
q, ok := ws.sq[streamID]
if !ok {
return
}
delete(ws.sq, streamID)
ws.queuePool.put(q)
}
func (ws *randomWriteScheduler) AdjustStream(streamID uint32, priority PriorityParam) {
// no-op: priorities are ignored
}
func (ws *randomWriteScheduler) Push(wr FrameWriteRequest) {
id := wr.StreamID()
if id == 0 {
ws.zero.push(wr)
return
}
q, ok := ws.sq[id]
if !ok {
q = ws.queuePool.get()
ws.sq[id] = q
}
q.push(wr)
}
func (ws *randomWriteScheduler) Pop() (FrameWriteRequest, bool) {
// Control frames first.
if !ws.zero.empty() {
return ws.zero.shift(), true
}
// Iterate over all non-idle streams until finding one that can be consumed.
for _, q := range ws.sq {
if wr, ok := q.consume(math.MaxInt32); ok {
return wr, true
}
}
return FrameWriteRequest{}, false
}
// Copyright 2016 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package http2
import "testing"
func TestRandomScheduler(t *testing.T) {
ws := NewRandomWriteScheduler()
ws.Push(makeWriteHeadersRequest(3))
ws.Push(makeWriteHeadersRequest(4))
ws.Push(makeWriteHeadersRequest(1))
ws.Push(makeWriteHeadersRequest(2))
ws.Push(makeWriteNonStreamRequest())
ws.Push(makeWriteNonStreamRequest())
// Pop all frames. Should get the non-stream requests first,
// followed by the stream requests in any order.
var order []FrameWriteRequest
for {
wr, ok := ws.Pop()
if !ok {
break
}
order = append(order, wr)
}
t.Logf("got frames: %v", order)
if len(order) != 6 {
t.Fatalf("got %d frames, expected 6", len(order))
}
if order[0].StreamID() != 0 || order[1].StreamID() != 0 {
t.Fatalf("expected non-stream frames first", order[0], order[1])
}
got := make(map[uint32]bool)
for _, wr := range order[2:] {
got[wr.StreamID()] = true
}
for id := uint32(1); id <= 4; id++ {
if !got[id] {
t.Errorf("frame not found for stream %d", id)
}
}
}
// Copyright 2016 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package http2
import (
"fmt"
"math"
"reflect"
"testing"
)
func makeWriteNonStreamRequest() FrameWriteRequest {
return FrameWriteRequest{writeSettingsAck{}, nil, nil}
}
func makeWriteHeadersRequest(streamID uint32) FrameWriteRequest {
st := &stream{id: streamID}
return FrameWriteRequest{&writeResHeaders{streamID: streamID, httpResCode: 200}, st, nil}
}
func checkConsume(wr FrameWriteRequest, nbytes int32, want []FrameWriteRequest) error {
consumed, rest, n := wr.Consume(nbytes)
var wantConsumed, wantRest FrameWriteRequest
switch len(want) {
case 0:
case 1:
wantConsumed = want[0]
case 2:
wantConsumed = want[0]
wantRest = want[1]
}
if !reflect.DeepEqual(consumed, wantConsumed) || !reflect.DeepEqual(rest, wantRest) || n != len(want) {
return fmt.Errorf("got %v, %v, %v\nwant %v, %v, %v", consumed, rest, n, wantConsumed, wantRest, len(want))
}
return nil
}
func TestFrameWriteRequestNonData(t *testing.T) {
wr := makeWriteNonStreamRequest()
if got, want := wr.DataSize(), 0; got != want {
t.Errorf("DataSize: got %v, want %v", got, want)
}
// Non-DATA frames are always consumed whole.
if err := checkConsume(wr, 0, []FrameWriteRequest{wr}); err != nil {
t.Errorf("Consume:\n%v", err)
}
}
func TestFrameWriteRequestData(t *testing.T) {
st := &stream{
id: 1,
sc: &serverConn{maxFrameSize: 16},
}
const size = 32
wr := FrameWriteRequest{&writeData{st.id, make([]byte, size), true}, st, make(chan error)}
if got, want := wr.DataSize(), size; got != want {
t.Errorf("DataSize: got %v, want %v", got, want)
}
// No flow-control bytes available: cannot consume anything.
if err := checkConsume(wr, math.MaxInt32, []FrameWriteRequest{}); err != nil {
t.Errorf("Consume(limited by flow control):\n%v", err)
}
// Add enough flow-control bytes to consume the entire frame,
// but we're now restricted by st.sc.maxFrameSize.
st.flow.add(size)
want := []FrameWriteRequest{
{
write: &writeData{st.id, make([]byte, st.sc.maxFrameSize), false},
stream: st,
done: nil,
},
{
write: &writeData{st.id, make([]byte, size-st.sc.maxFrameSize), true},
stream: st,
done: wr.done,
},
}
if err := checkConsume(wr, math.MaxInt32, want); err != nil {
t.Errorf("Consume(limited by maxFrameSize):\n%v", err)
}
rest := want[1]
// Consume 8 bytes from the remaining frame.
want = []FrameWriteRequest{
{
write: &writeData{st.id, make([]byte, 8), false},
stream: st,
done: nil,
},
{
write: &writeData{st.id, make([]byte, size-st.sc.maxFrameSize-8), true},
stream: st,
done: wr.done,
},
}
if err := checkConsume(rest, 8, want); err != nil {
t.Errorf("Consume(8):\n%v", err)
}
rest = want[1]
// Consume all remaining bytes.
want = []FrameWriteRequest{
{
write: &writeData{st.id, make([]byte, size-st.sc.maxFrameSize-8), true},
stream: st,
done: wr.done,
},
}
if err := checkConsume(rest, math.MaxInt32, want); err != nil {
t.Errorf("Consume(remainder):\n%v", err)
}
}
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment