diff --git a/const.go b/const.go index e4b2bc2..08199af 100644 --- a/const.go +++ b/const.go @@ -3,6 +3,7 @@ package yamux import ( "encoding/binary" "fmt" + "time" ) type Error struct { @@ -22,6 +23,64 @@ func (ye *Error) Temporary() bool { return ye.temporary } +type GoAwayError struct { + ErrorCode uint32 + Remote bool +} + +func (e *GoAwayError) Error() string { + if e.Remote { + return fmt.Sprintf("remote sent go away, code: %d", e.ErrorCode) + } + return fmt.Sprintf("sent go away, code: %d", e.ErrorCode) +} + +func (e *GoAwayError) Timeout() bool { + return false +} + +func (e *GoAwayError) Temporary() bool { + return false +} + +func (e *GoAwayError) Is(target error) bool { + // to maintain compatibility with errors returned by previous versions + if e.Remote && target == ErrRemoteGoAway { + return true + } else if !e.Remote && target == ErrSessionShutdown { + return true + } else if target == ErrStreamReset { + // A GoAway on a connection also resets all the streams. + return true + } + + if err, ok := target.(*GoAwayError); ok { + return *e == *err + } + return false +} + +// A StreamError is used for errors returned from Read and Write calls after the stream is Reset +type StreamError struct { + ErrorCode uint32 + Remote bool +} + +func (s *StreamError) Error() string { + if s.Remote { + return fmt.Sprintf("stream reset by remote, error code: %d", s.ErrorCode) + } + return fmt.Sprintf("stream reset, error code: %d", s.ErrorCode) +} + +func (s *StreamError) Is(target error) bool { + if target == ErrStreamReset { + return true + } + e, ok := target.(*StreamError) + return ok && *e == *s +} + var ( // ErrInvalidVersion means we received a frame with an // invalid version @@ -33,7 +92,7 @@ var ( // ErrSessionShutdown is used if there is a shutdown during // an operation - ErrSessionShutdown = &Error{msg: "session shutdown"} + ErrSessionShutdown = &GoAwayError{ErrorCode: goAwayNormal, Remote: false} // ErrStreamsExhausted is returned if we have no more // stream ids to issue @@ -55,8 +114,9 @@ var ( // ErrUnexpectedFlag is set when we get an unexpected flag ErrUnexpectedFlag = &Error{msg: "unexpected flag"} - // ErrRemoteGoAway is used when we get a go away from the other side - ErrRemoteGoAway = &Error{msg: "remote end is not accepting connections"} + // ErrRemoteGoAway is used when we get a go away from the other side with error code + // goAwayNormal(0). + ErrRemoteGoAway = &GoAwayError{Remote: true, ErrorCode: goAwayNormal} // ErrStreamReset is sent if a stream is reset. This can happen // if the backlog is exceeded, or if there was a remote GoAway. @@ -117,6 +177,7 @@ const ( // It's not an implementation choice, the value defined in the specification. initialStreamWindow = 256 * 1024 maxStreamWindow = 16 * 1024 * 1024 + goAwayWaitTime = 100 * time.Millisecond ) const ( diff --git a/session.go b/session.go index c4cd1bd..e229730 100644 --- a/session.go +++ b/session.go @@ -46,10 +46,6 @@ var nullMemoryManager = &nullMemoryManagerImpl{} type Session struct { rtt int64 // to be accessed atomically, in nanoseconds - // remoteGoAway indicates the remote side does - // not want futher connections. Must be first for alignment. - remoteGoAway int32 - // localGoAway indicates that we should stop // accepting futher connections. Must be first for alignment. localGoAway int32 @@ -102,6 +98,8 @@ type Session struct { // recvDoneCh is closed when recv() exits to avoid a race // between stream registration and stream shutdown recvDoneCh chan struct{} + // recvErr is the error the receive loop ended with + recvErr error // sendDoneCh is closed when send() exits to avoid a race // between returning from a Stream.Write and exiting from the send loop @@ -203,9 +201,6 @@ func (s *Session) OpenStream(ctx context.Context) (*Stream, error) { if s.IsClosed() { return nil, s.shutdownErr } - if atomic.LoadInt32(&s.remoteGoAway) == 1 { - return nil, ErrRemoteGoAway - } // Block if we have too many inflight SYNs select { @@ -283,9 +278,23 @@ func (s *Session) AcceptStream() (*Stream, error) { } } -// Close is used to close the session and all streams. -// Attempts to send a GoAway before closing the connection. +// Close is used to close the session and all streams. It doesn't send a GoAway before +// closing the connection. func (s *Session) Close() error { + return s.close(ErrSessionShutdown, false, goAwayNormal) +} + +// CloseWithError is used to close the session and all streams after sending a GoAway message with errCode. +// Blocks for ConnectionWriteTimeout to write the GoAway message. +// +// The GoAway may not actually be sent depending on the semantics of the underlying net.Conn. +// For TCP connections, it may be dropped depending on LINGER value or if there's unread data in the kernel +// receive buffer. +func (s *Session) CloseWithError(errCode uint32) error { + return s.close(&GoAwayError{Remote: false, ErrorCode: errCode}, true, errCode) +} + +func (s *Session) close(shutdownErr error, sendGoAway bool, errCode uint32) error { s.shutdownLock.Lock() defer s.shutdownLock.Unlock() @@ -294,35 +303,42 @@ func (s *Session) Close() error { } s.shutdown = true if s.shutdownErr == nil { - s.shutdownErr = ErrSessionShutdown + s.shutdownErr = shutdownErr } close(s.shutdownCh) - s.conn.Close() s.stopKeepalive() - <-s.recvDoneCh + + // Only send GoAway if we have an error code. + if sendGoAway && errCode != goAwayNormal { + // wait for write loop to exit + // We need to write the current frame completely before sending a goaway. + // This will wait for at most s.config.ConnectionWriteTimeout + <-s.sendDoneCh + ga := s.goAway(errCode) + if err := s.conn.SetWriteDeadline(time.Now().Add(goAwayWaitTime)); err == nil { + _, _ = s.conn.Write(ga[:]) // there's nothing we can do on error here + } + s.conn.SetWriteDeadline(time.Time{}) + } + + s.conn.Close() <-s.sendDoneCh + <-s.recvDoneCh + resetErr := shutdownErr + if _, ok := resetErr.(*GoAwayError); !ok { + resetErr = fmt.Errorf("%w: connection closed: %w", ErrStreamReset, shutdownErr) + } s.streamLock.Lock() defer s.streamLock.Unlock() for id, stream := range s.streams { - stream.forceClose() + stream.forceClose(resetErr) delete(s.streams, id) stream.memorySpan.Done() } return nil } -// exitErr is used to handle an error that is causing the -// session to terminate. -func (s *Session) exitErr(err error) { - s.shutdownLock.Lock() - if s.shutdownErr == nil { - s.shutdownErr = err - } - s.shutdownLock.Unlock() - s.Close() -} - // GoAway can be used to prevent accepting further // connections. It does not close the underlying conn. func (s *Session) GoAway() error { @@ -451,7 +467,7 @@ func (s *Session) startKeepalive() { if err != nil { s.logger.Printf("[ERR] yamux: keepalive failed: %v", err) - s.exitErr(ErrKeepAliveTimeout) + s.close(ErrKeepAliveTimeout, false, 0) } }) } @@ -516,7 +532,25 @@ func (s *Session) sendMsg(hdr header, body []byte, deadline <-chan struct{}) err // send is a long running goroutine that sends data func (s *Session) send() { if err := s.sendLoop(); err != nil { - s.exitErr(err) + // If we are shutting down because remote closed the connection, prefer the recvLoop error + // over the sendLoop error. The receive loop might have error code received in a GoAway frame, + // which was received just before the TCP RST that closed the sendLoop. + // + // If we are closing because of an write error, we use the error from the sendLoop and not the recvLoop. + // We hold the shutdownLock, close the connection, and wait for the receive loop to finish and + // use the sendLoop error. Holding the shutdownLock ensures that the recvLoop doesn't trigger connection close + // but the sendLoop does. + s.shutdownLock.Lock() + if s.shutdownErr == nil { + s.conn.Close() + <-s.recvDoneCh + if _, ok := s.recvErr.(*GoAwayError); ok { + err = s.recvErr + } + s.shutdownErr = err + } + s.shutdownLock.Unlock() + s.close(err, false, 0) } } @@ -644,7 +678,7 @@ func (s *Session) sendLoop() (err error) { // recv is a long running goroutine that accepts new data func (s *Session) recv() { if err := s.recvLoop(); err != nil { - s.exitErr(err) + s.close(err, false, 0) } } @@ -666,7 +700,10 @@ func (s *Session) recvLoop() (err error) { err = fmt.Errorf("panic in yamux receive loop: %s", rerr) } }() - defer close(s.recvDoneCh) + defer func() { + s.recvErr = err + close(s.recvDoneCh) + }() var hdr header for { // fmt.Printf("ReadFull from %#v\n", s.reader) @@ -781,18 +818,15 @@ func (s *Session) handleGoAway(hdr header) error { code := hdr.Length() switch code { case goAwayNormal: - atomic.SwapInt32(&s.remoteGoAway, 1) + return ErrRemoteGoAway case goAwayProtoErr: s.logger.Printf("[ERR] yamux: received protocol error go away") - return fmt.Errorf("yamux protocol error") case goAwayInternalErr: s.logger.Printf("[ERR] yamux: received internal error go away") - return fmt.Errorf("remote yamux internal error") default: - s.logger.Printf("[ERR] yamux: received unexpected go away") - return fmt.Errorf("unexpected go away received") + s.logger.Printf("[ERR] yamux: received go away with error code: %d", code) } - return nil + return &GoAwayError{Remote: true, ErrorCode: code} } // incomingStream is used to create a new incoming stream diff --git a/session_test.go b/session_test.go index 974b6d5..6d3bce0 100644 --- a/session_test.go +++ b/session_test.go @@ -3,6 +3,7 @@ package yamux import ( "bytes" "context" + "errors" "fmt" "io" "math/rand" @@ -15,6 +16,7 @@ import ( "testing" "time" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -39,6 +41,8 @@ type pipeConn struct { writeDeadline pipeDeadline writeBlocker chan struct{} closeCh chan struct{} + closeOnce sync.Once + closeErr error } func (p *pipeConn) SetDeadline(t time.Time) error { @@ -65,10 +69,12 @@ func (p *pipeConn) Write(b []byte) (int, error) { } func (p *pipeConn) Close() error { - p.writeDeadline.set(time.Time{}) - err := p.Conn.Close() - close(p.closeCh) - return err + p.closeOnce.Do(func() { + p.writeDeadline.set(time.Time{}) + p.closeErr = p.Conn.Close() + close(p.closeCh) + }) + return p.closeErr } func (p *pipeConn) BlockWrites() { @@ -642,14 +648,44 @@ func TestGoAway(t *testing.T) { for i := 0; i < 100; i++ { s, err := client.Open(context.Background()) - switch err { - case nil: + if err == nil { s.Close() - case ErrRemoteGoAway: + time.Sleep(50 * time.Millisecond) + continue + } + if err != ErrRemoteGoAway { + t.Fatalf("expected %s, got %s", ErrRemoteGoAway, err) + } else { return - default: + } + } + t.Fatalf("expected GoAway error") +} + +func TestCloseWithError(t *testing.T) { + // This test is noisy. + conf := testConf() + conf.LogOutput = io.Discard + + client, server := testClientServerConfig(conf) + defer client.Close() + defer server.Close() + + if err := server.CloseWithError(42); err != nil { + t.Fatalf("err: %v", err) + } + + for i := 0; i < 100; i++ { + s, err := client.Open(context.Background()) + if err == nil { + s.Close() + time.Sleep(50 * time.Millisecond) + continue + } + if !errors.Is(err, &GoAwayError{ErrorCode: 42, Remote: true}) { t.Fatalf("err: %v", err) } + return } t.Fatalf("expected GoAway error") } @@ -1048,6 +1084,7 @@ func TestKeepAlive_Timeout(t *testing.T) { // Prevent the client from responding clientConn := client.conn.(*pipeConn) clientConn.BlockWrites() + defer clientConn.UnblockWrites() select { case err := <-errCh: @@ -1536,6 +1573,51 @@ func TestStreamResetRead(t *testing.T) { wc.Wait() } +func TestStreamResetWithError(t *testing.T) { + client, server := testClientServer() + defer client.Close() + defer server.Close() + + wc := new(sync.WaitGroup) + wc.Add(1) + go func() { + defer wc.Done() + stream, err := server.AcceptStream() + if err != nil { + t.Error(err) + } + + se := &StreamError{} + _, err = io.ReadAll(stream) + if !errors.As(err, &se) { + t.Errorf("expected StreamError, got type:%T, err: %s", err, err) + return + } + expected := &StreamError{Remote: true, ErrorCode: 42} + assert.Equal(t, se, expected) + }() + + stream, err := client.OpenStream(context.Background()) + if err != nil { + t.Error(err) + } + + time.Sleep(1 * time.Second) + err = stream.ResetWithError(42) + if err != nil { + t.Fatal(err) + } + se := &StreamError{} + _, err = io.ReadAll(stream) + if !errors.As(err, &se) { + t.Errorf("expected StreamError, got type:%T, err: %s", err, err) + return + } + expected := &StreamError{Remote: false, ErrorCode: 42} + assert.Equal(t, se, expected) + wc.Wait() +} + func TestLotsOfWritesWithStreamDeadline(t *testing.T) { config := testConf() config.EnableKeepAlive = false @@ -1774,7 +1856,7 @@ func TestMaxIncomingStreams(t *testing.T) { require.NoError(t, err) str.SetDeadline(time.Now().Add(time.Second)) _, err = str.Read([]byte{0}) - require.EqualError(t, err, "stream reset") + require.ErrorIs(t, err, ErrStreamReset) // Now close one of the streams. // This should then allow the client to open a new stream. diff --git a/stream.go b/stream.go index e1e5602..15a8b56 100644 --- a/stream.go +++ b/stream.go @@ -41,6 +41,7 @@ type Stream struct { state streamState writeState, readState halfStreamState + writeErr, readErr error stateLock sync.Mutex recvBuf segmentedBuffer @@ -89,6 +90,7 @@ func (s *Stream) Read(b []byte) (n int, err error) { START: s.stateLock.Lock() state := s.readState + resetErr := s.readErr s.stateLock.Unlock() switch state { @@ -101,7 +103,7 @@ START: } // Closed, but we have data pending -> read. case halfReset: - return 0, ErrStreamReset + return 0, resetErr default: panic("unknown state") } @@ -147,6 +149,7 @@ func (s *Stream) write(b []byte) (n int, err error) { START: s.stateLock.Lock() state := s.writeState + resetErr := s.writeErr s.stateLock.Unlock() switch state { @@ -155,7 +158,7 @@ START: case halfClosed: return 0, ErrStreamClosed case halfReset: - return 0, ErrStreamReset + return 0, resetErr default: panic("unknown state") } @@ -250,13 +253,17 @@ func (s *Stream) sendClose() error { } // sendReset is used to send a RST -func (s *Stream) sendReset() error { - hdr := encode(typeWindowUpdate, flagRST, s.id, 0) +func (s *Stream) sendReset(errCode uint32) error { + hdr := encode(typeWindowUpdate, flagRST, s.id, errCode) return s.session.sendMsg(hdr, nil, nil) } // Reset resets the stream (forcibly closes the stream) func (s *Stream) Reset() error { + return s.ResetWithError(0) +} + +func (s *Stream) ResetWithError(errCode uint32) error { sendReset := false s.stateLock.Lock() switch s.state { @@ -276,15 +283,17 @@ func (s *Stream) Reset() error { // If we've already sent/received an EOF, no need to reset that side. if s.writeState == halfOpen { s.writeState = halfReset + s.writeErr = &StreamError{Remote: false, ErrorCode: errCode} } if s.readState == halfOpen { s.readState = halfReset + s.readErr = &StreamError{Remote: false, ErrorCode: errCode} } s.state = streamFinished s.notifyWaiting() s.stateLock.Unlock() if sendReset { - _ = s.sendReset() + _ = s.sendReset(errCode) } s.cleanup() return nil @@ -301,7 +310,7 @@ func (s *Stream) CloseWrite() error { return nil case halfReset: s.stateLock.Unlock() - return ErrStreamReset + return s.writeErr default: panic("invalid state") } @@ -322,7 +331,8 @@ func (s *Stream) CloseWrite() error { return err } -// CloseRead is used to close the stream for writing. +// CloseRead is used to close the stream for reading. +// Note: Remote is not notified. func (s *Stream) CloseRead() error { cleanup := false s.stateLock.Lock() @@ -336,6 +346,7 @@ func (s *Stream) CloseRead() error { panic("invalid state") } s.readState = halfReset + s.readErr = ErrStreamReset cleanup = s.writeState != halfOpen if cleanup { s.state = streamFinished @@ -357,13 +368,15 @@ func (s *Stream) Close() error { } // forceClose is used for when the session is exiting -func (s *Stream) forceClose() { +func (s *Stream) forceClose(err error) { s.stateLock.Lock() if s.readState == halfOpen { s.readState = halfReset + s.readErr = err } if s.writeState == halfOpen { s.writeState = halfReset + s.writeErr = err } s.state = streamFinished s.notifyWaiting() @@ -382,7 +395,7 @@ func (s *Stream) cleanup() { // processFlags is used to update the state of the stream // based on set flags, if any. Lock must be held -func (s *Stream) processFlags(flags uint16) { +func (s *Stream) processFlags(hdr header, flags uint16) { // Close the stream without holding the state lock var closeStream bool defer func() { @@ -418,11 +431,18 @@ func (s *Stream) processFlags(flags uint16) { } if flags&flagRST == flagRST { s.stateLock.Lock() + var resetErr error = ErrStreamReset + // Length in a window update frame with RST flag encodes an error code. + if hdr.MsgType() == typeWindowUpdate { + resetErr = &StreamError{Remote: true, ErrorCode: hdr.Length()} + } if s.readState == halfOpen { s.readState = halfReset + s.readErr = resetErr } if s.writeState == halfOpen { s.writeState = halfReset + s.writeErr = resetErr } s.state = streamFinished s.stateLock.Unlock() @@ -439,7 +459,7 @@ func (s *Stream) notifyWaiting() { // incrSendWindow updates the size of our send window func (s *Stream) incrSendWindow(hdr header, flags uint16) { - s.processFlags(flags) + s.processFlags(hdr, flags) // Increase window, unblock a sender atomic.AddUint32(&s.sendWindow, hdr.Length()) asyncNotify(s.sendNotifyCh) @@ -447,7 +467,7 @@ func (s *Stream) incrSendWindow(hdr header, flags uint16) { // readData is used to handle a data frame func (s *Stream) readData(hdr header, flags uint16, conn io.Reader) error { - s.processFlags(flags) + s.processFlags(hdr, flags) // Check that our recv window is not exceeded length := hdr.Length()