summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/transport/tcp
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/tcpip/transport/tcp')
-rw-r--r--pkg/tcpip/transport/tcp/accept.go14
-rw-r--r--pkg/tcpip/transport/tcp/connect.go82
-rw-r--r--pkg/tcpip/transport/tcp/dual_stack_test.go16
-rw-r--r--pkg/tcpip/transport/tcp/endpoint.go178
-rw-r--r--pkg/tcpip/transport/tcp/endpoint_state.go44
-rw-r--r--pkg/tcpip/transport/tcp/forwarder.go4
-rw-r--r--pkg/tcpip/transport/tcp/protocol.go26
-rw-r--r--pkg/tcpip/transport/tcp/rack.go2
-rw-r--r--pkg/tcpip/transport/tcp/rcv.go8
-rw-r--r--pkg/tcpip/transport/tcp/snd.go4
-rw-r--r--pkg/tcpip/transport/tcp/tcp_test.go247
-rw-r--r--pkg/tcpip/transport/tcp/testing/context/context.go13
12 files changed, 324 insertions, 314 deletions
diff --git a/pkg/tcpip/transport/tcp/accept.go b/pkg/tcpip/transport/tcp/accept.go
index e475c36f3..842c1622b 100644
--- a/pkg/tcpip/transport/tcp/accept.go
+++ b/pkg/tcpip/transport/tcp/accept.go
@@ -199,7 +199,7 @@ func (l *listenContext) isCookieValid(id stack.TransportEndpointID, cookie seqnu
// createConnectingEndpoint creates a new endpoint in a connecting state, with
// the connection parameters given by the arguments.
-func (l *listenContext) createConnectingEndpoint(s *segment, iss seqnum.Value, irs seqnum.Value, rcvdSynOpts *header.TCPSynOptions, queue *waiter.Queue) (*endpoint, *tcpip.Error) {
+func (l *listenContext) createConnectingEndpoint(s *segment, iss seqnum.Value, irs seqnum.Value, rcvdSynOpts *header.TCPSynOptions, queue *waiter.Queue) (*endpoint, tcpip.Error) {
// Create a new endpoint.
netProto := l.netProto
if netProto == 0 {
@@ -242,7 +242,7 @@ func (l *listenContext) createConnectingEndpoint(s *segment, iss seqnum.Value, i
// On success, a handshake h is returned with h.ep.mu held.
//
// Precondition: if l.listenEP != nil, l.listenEP.mu must be locked.
-func (l *listenContext) startHandshake(s *segment, opts *header.TCPSynOptions, queue *waiter.Queue, owner tcpip.PacketOwner) (*handshake, *tcpip.Error) {
+func (l *listenContext) startHandshake(s *segment, opts *header.TCPSynOptions, queue *waiter.Queue, owner tcpip.PacketOwner) (*handshake, tcpip.Error) {
// Create new endpoint.
irs := s.sequenceNumber
isn := generateSecureISN(s.id, l.stack.Seed())
@@ -267,7 +267,7 @@ func (l *listenContext) startHandshake(s *segment, opts *header.TCPSynOptions, q
ep.mu.Unlock()
ep.Close()
- return nil, tcpip.ErrConnectionAborted
+ return nil, &tcpip.ErrConnectionAborted{}
}
l.addPendingEndpoint(ep)
@@ -281,7 +281,7 @@ func (l *listenContext) startHandshake(s *segment, opts *header.TCPSynOptions, q
l.removePendingEndpoint(ep)
- return nil, tcpip.ErrConnectionAborted
+ return nil, &tcpip.ErrConnectionAborted{}
}
deferAccept = l.listenEP.deferAccept
@@ -313,7 +313,7 @@ func (l *listenContext) startHandshake(s *segment, opts *header.TCPSynOptions, q
// established endpoint is returned with e.mu held.
//
// Precondition: if l.listenEP != nil, l.listenEP.mu must be locked.
-func (l *listenContext) performHandshake(s *segment, opts *header.TCPSynOptions, queue *waiter.Queue, owner tcpip.PacketOwner) (*endpoint, *tcpip.Error) {
+func (l *listenContext) performHandshake(s *segment, opts *header.TCPSynOptions, queue *waiter.Queue, owner tcpip.PacketOwner) (*endpoint, tcpip.Error) {
h, err := l.startHandshake(s, opts, queue, owner)
if err != nil {
return nil, err
@@ -467,7 +467,7 @@ func (e *endpoint) notifyAborted() {
// cookies to accept connections.
//
// Precondition: if ctx.listenEP != nil, ctx.listenEP.mu must be locked.
-func (e *endpoint) handleSynSegment(ctx *listenContext, s *segment, opts *header.TCPSynOptions) *tcpip.Error {
+func (e *endpoint) handleSynSegment(ctx *listenContext, s *segment, opts *header.TCPSynOptions) tcpip.Error {
defer s.decRef()
h, err := ctx.startHandshake(s, opts, &waiter.Queue{}, e.owner)
@@ -522,7 +522,7 @@ func (e *endpoint) acceptQueueIsFull() bool {
// and needs to handle it.
//
// Precondition: if ctx.listenEP != nil, ctx.listenEP.mu must be locked.
-func (e *endpoint) handleListenSegment(ctx *listenContext, s *segment) *tcpip.Error {
+func (e *endpoint) handleListenSegment(ctx *listenContext, s *segment) tcpip.Error {
e.rcvListMu.Lock()
rcvClosed := e.rcvClosed
e.rcvListMu.Unlock()
diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go
index 6df4e6525..4695b66d6 100644
--- a/pkg/tcpip/transport/tcp/connect.go
+++ b/pkg/tcpip/transport/tcp/connect.go
@@ -226,7 +226,7 @@ func (h *handshake) checkAck(s *segment) bool {
// synSentState handles a segment received when the TCP 3-way handshake is in
// the SYN-SENT state.
-func (h *handshake) synSentState(s *segment) *tcpip.Error {
+func (h *handshake) synSentState(s *segment) tcpip.Error {
// RFC 793, page 37, states that in the SYN-SENT state, a reset is
// acceptable if the ack field acknowledges the SYN.
if s.flagIsSet(header.TCPFlagRst) {
@@ -237,7 +237,7 @@ func (h *handshake) synSentState(s *segment) *tcpip.Error {
h.ep.workerCleanup = true
// Although the RFC above calls out ECONNRESET, Linux actually returns
// ECONNREFUSED here so we do as well.
- return tcpip.ErrConnectionRefused
+ return &tcpip.ErrConnectionRefused{}
}
return nil
}
@@ -314,12 +314,12 @@ func (h *handshake) synSentState(s *segment) *tcpip.Error {
// synRcvdState handles a segment received when the TCP 3-way handshake is in
// the SYN-RCVD state.
-func (h *handshake) synRcvdState(s *segment) *tcpip.Error {
+func (h *handshake) synRcvdState(s *segment) tcpip.Error {
if s.flagIsSet(header.TCPFlagRst) {
// RFC 793, page 37, states that in the SYN-RCVD state, a reset
// is acceptable if the sequence number is in the window.
if s.sequenceNumber.InWindow(h.ackNum, h.rcvWnd) {
- return tcpip.ErrConnectionRefused
+ return &tcpip.ErrConnectionRefused{}
}
return nil
}
@@ -349,7 +349,7 @@ func (h *handshake) synRcvdState(s *segment) *tcpip.Error {
h.ep.sendRaw(buffer.VectorisedView{}, header.TCPFlagRst|header.TCPFlagAck, seq, ack, 0)
if !h.active {
- return tcpip.ErrInvalidEndpointState
+ return &tcpip.ErrInvalidEndpointState{}
}
h.resetState()
@@ -412,7 +412,7 @@ func (h *handshake) synRcvdState(s *segment) *tcpip.Error {
return nil
}
-func (h *handshake) handleSegment(s *segment) *tcpip.Error {
+func (h *handshake) handleSegment(s *segment) tcpip.Error {
h.sndWnd = s.window
if !s.flagIsSet(header.TCPFlagSyn) && h.sndWndScale > 0 {
h.sndWnd <<= uint8(h.sndWndScale)
@@ -429,7 +429,7 @@ func (h *handshake) handleSegment(s *segment) *tcpip.Error {
// processSegments goes through the segment queue and processes up to
// maxSegmentsPerWake (if they're available).
-func (h *handshake) processSegments() *tcpip.Error {
+func (h *handshake) processSegments() tcpip.Error {
for i := 0; i < maxSegmentsPerWake; i++ {
s := h.ep.segmentQueue.dequeue()
if s == nil {
@@ -505,7 +505,7 @@ func (h *handshake) start() {
}
// complete completes the TCP 3-way handshake initiated by h.start().
-func (h *handshake) complete() *tcpip.Error {
+func (h *handshake) complete() tcpip.Error {
// Set up the wakers.
var s sleep.Sleeper
resendWaker := sleep.Waker{}
@@ -555,7 +555,7 @@ func (h *handshake) complete() *tcpip.Error {
case wakerForNotification:
n := h.ep.fetchNotifications()
if (n&notifyClose)|(n&notifyAbort) != 0 {
- return tcpip.ErrAborted
+ return &tcpip.ErrAborted{}
}
if n&notifyDrain != 0 {
for !h.ep.segmentQueue.empty() {
@@ -593,19 +593,19 @@ type backoffTimer struct {
t *time.Timer
}
-func newBackoffTimer(timeout, maxTimeout time.Duration, f func()) (*backoffTimer, *tcpip.Error) {
+func newBackoffTimer(timeout, maxTimeout time.Duration, f func()) (*backoffTimer, tcpip.Error) {
if timeout > maxTimeout {
- return nil, tcpip.ErrTimeout
+ return nil, &tcpip.ErrTimeout{}
}
bt := &backoffTimer{timeout: timeout, maxTimeout: maxTimeout}
bt.t = time.AfterFunc(timeout, f)
return bt, nil
}
-func (bt *backoffTimer) reset() *tcpip.Error {
+func (bt *backoffTimer) reset() tcpip.Error {
bt.timeout *= 2
if bt.timeout > MaxRTO {
- return tcpip.ErrTimeout
+ return &tcpip.ErrTimeout{}
}
bt.t.Reset(bt.timeout)
return nil
@@ -706,7 +706,7 @@ type tcpFields struct {
txHash uint32
}
-func (e *endpoint) sendSynTCP(r *stack.Route, tf tcpFields, opts header.TCPSynOptions) *tcpip.Error {
+func (e *endpoint) sendSynTCP(r *stack.Route, tf tcpFields, opts header.TCPSynOptions) tcpip.Error {
tf.opts = makeSynOptions(opts)
// We ignore SYN send errors and let the callers re-attempt send.
if err := e.sendTCP(r, tf, buffer.VectorisedView{}, nil); err != nil {
@@ -716,7 +716,7 @@ func (e *endpoint) sendSynTCP(r *stack.Route, tf tcpFields, opts header.TCPSynOp
return nil
}
-func (e *endpoint) sendTCP(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso *stack.GSO) *tcpip.Error {
+func (e *endpoint) sendTCP(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso *stack.GSO) tcpip.Error {
tf.txHash = e.txHash
if err := sendTCP(r, tf, data, gso, e.owner); err != nil {
e.stats.SendErrors.SegmentSendToNetworkFailed.Increment()
@@ -755,7 +755,7 @@ func buildTCPHdr(r *stack.Route, tf tcpFields, pkt *stack.PacketBuffer, gso *sta
}
}
-func sendTCPBatch(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso *stack.GSO, owner tcpip.PacketOwner) *tcpip.Error {
+func sendTCPBatch(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso *stack.GSO, owner tcpip.PacketOwner) tcpip.Error {
// We need to shallow clone the VectorisedView here as ReadToView will
// split the VectorisedView and Trim underlying views as it splits. Not
// doing the clone here will cause the underlying views of data itself
@@ -803,7 +803,7 @@ func sendTCPBatch(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso
// sendTCP sends a TCP segment with the provided options via the provided
// network endpoint and under the provided identity.
-func sendTCP(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso *stack.GSO, owner tcpip.PacketOwner) *tcpip.Error {
+func sendTCP(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso *stack.GSO, owner tcpip.PacketOwner) tcpip.Error {
optLen := len(tf.opts)
if tf.rcvWnd > math.MaxUint16 {
tf.rcvWnd = math.MaxUint16
@@ -875,7 +875,7 @@ func (e *endpoint) makeOptions(sackBlocks []header.SACKBlock) []byte {
}
// sendRaw sends a TCP segment to the endpoint's peer.
-func (e *endpoint) sendRaw(data buffer.VectorisedView, flags byte, seq, ack seqnum.Value, rcvWnd seqnum.Size) *tcpip.Error {
+func (e *endpoint) sendRaw(data buffer.VectorisedView, flags byte, seq, ack seqnum.Value, rcvWnd seqnum.Size) tcpip.Error {
var sackBlocks []header.SACKBlock
if e.EndpointState() == StateEstablished && e.rcv.pendingRcvdSegments.Len() > 0 && (flags&header.TCPFlagAck != 0) {
sackBlocks = e.sack.Blocks[:e.sack.NumBlocks]
@@ -941,12 +941,14 @@ func (e *endpoint) handleClose() {
// error code and sends a RST if and only if the error is not ErrConnectionReset
// indicating that the connection is being reset due to receiving a RST. This
// method must only be called from the protocol goroutine.
-func (e *endpoint) resetConnectionLocked(err *tcpip.Error) {
+func (e *endpoint) resetConnectionLocked(err tcpip.Error) {
// Only send a reset if the connection is being aborted for a reason
// other than receiving a reset.
e.setEndpointState(StateError)
e.hardError = err
- if err != tcpip.ErrConnectionReset && err != tcpip.ErrTimeout {
+ switch err.(type) {
+ case *tcpip.ErrConnectionReset, *tcpip.ErrTimeout:
+ default:
// The exact sequence number to be used for the RST is the same as the
// one used by Linux. We need to handle the case of window being shrunk
// which can cause sndNxt to be outside the acceptable window on the
@@ -1056,7 +1058,7 @@ func (e *endpoint) drainClosingSegmentQueue() {
}
}
-func (e *endpoint) handleReset(s *segment) (ok bool, err *tcpip.Error) {
+func (e *endpoint) handleReset(s *segment) (ok bool, err tcpip.Error) {
if e.rcv.acceptable(s.sequenceNumber, 0) {
// RFC 793, page 37 states that "in all states
// except SYN-SENT, all reset (RST) segments are
@@ -1084,7 +1086,7 @@ func (e *endpoint) handleReset(s *segment) (ok bool, err *tcpip.Error) {
// delete the TCB, and return.
case StateCloseWait:
e.transitionToStateCloseLocked()
- e.hardError = tcpip.ErrAborted
+ e.hardError = &tcpip.ErrAborted{}
e.notifyProtocolGoroutine(notifyTickleWorker)
return false, nil
default:
@@ -1097,14 +1099,14 @@ func (e *endpoint) handleReset(s *segment) (ok bool, err *tcpip.Error) {
// handleSegment is invoked from the processor goroutine
// rather than the worker goroutine.
e.notifyProtocolGoroutine(notifyResetByPeer)
- return false, tcpip.ErrConnectionReset
+ return false, &tcpip.ErrConnectionReset{}
}
}
return true, nil
}
// handleSegments processes all inbound segments.
-func (e *endpoint) handleSegments(fastPath bool) *tcpip.Error {
+func (e *endpoint) handleSegments(fastPath bool) tcpip.Error {
checkRequeue := true
for i := 0; i < maxSegmentsPerWake; i++ {
if e.EndpointState().closed() {
@@ -1151,7 +1153,7 @@ func (e *endpoint) probeSegment() {
// handleSegment handles a given segment and notifies the worker goroutine if
// if the connection should be terminated.
-func (e *endpoint) handleSegment(s *segment) (cont bool, err *tcpip.Error) {
+func (e *endpoint) handleSegment(s *segment) (cont bool, err tcpip.Error) {
// Invoke the tcp probe if installed. The tcp probe function will update
// the TCPEndpointState after the segment is processed.
defer e.probeSegment()
@@ -1225,7 +1227,7 @@ func (e *endpoint) handleSegment(s *segment) (cont bool, err *tcpip.Error) {
// keepaliveTimerExpired is called when the keepaliveTimer fires. We send TCP
// keepalive packets periodically when the connection is idle. If we don't hear
// from the other side after a number of tries, we terminate the connection.
-func (e *endpoint) keepaliveTimerExpired() *tcpip.Error {
+func (e *endpoint) keepaliveTimerExpired() tcpip.Error {
userTimeout := e.userTimeout
e.keepalive.Lock()
@@ -1239,13 +1241,13 @@ func (e *endpoint) keepaliveTimerExpired() *tcpip.Error {
if userTimeout != 0 && time.Since(e.rcv.lastRcvdAckTime) >= userTimeout && e.keepalive.unacked > 0 {
e.keepalive.Unlock()
e.stack.Stats().TCP.EstablishedTimedout.Increment()
- return tcpip.ErrTimeout
+ return &tcpip.ErrTimeout{}
}
if e.keepalive.unacked >= e.keepalive.count {
e.keepalive.Unlock()
e.stack.Stats().TCP.EstablishedTimedout.Increment()
- return tcpip.ErrTimeout
+ return &tcpip.ErrTimeout{}
}
// RFC1122 4.2.3.6: TCP keepalive is a dataless ACK with
@@ -1289,7 +1291,7 @@ func (e *endpoint) disableKeepaliveTimer() {
// protocolMainLoop is the main loop of the TCP protocol. It runs in its own
// goroutine and is responsible for sending segments and handling received
// segments.
-func (e *endpoint) protocolMainLoop(handshake bool, wakerInitDone chan<- struct{}) *tcpip.Error {
+func (e *endpoint) protocolMainLoop(handshake bool, wakerInitDone chan<- struct{}) tcpip.Error {
e.mu.Lock()
var closeTimer *time.Timer
var closeWaker sleep.Waker
@@ -1355,25 +1357,25 @@ func (e *endpoint) protocolMainLoop(handshake bool, wakerInitDone chan<- struct{
// wakes up.
funcs := []struct {
w *sleep.Waker
- f func() *tcpip.Error
+ f func() tcpip.Error
}{
{
w: &e.sndWaker,
- f: func() *tcpip.Error {
+ f: func() tcpip.Error {
e.handleWrite()
return nil
},
},
{
w: &e.sndCloseWaker,
- f: func() *tcpip.Error {
+ f: func() tcpip.Error {
e.handleClose()
return nil
},
},
{
w: &closeWaker,
- f: func() *tcpip.Error {
+ f: func() tcpip.Error {
// This means the socket is being closed due
// to the TCP-FIN-WAIT2 timeout was hit. Just
// mark the socket as closed.
@@ -1384,10 +1386,10 @@ func (e *endpoint) protocolMainLoop(handshake bool, wakerInitDone chan<- struct{
},
{
w: &e.snd.resendWaker,
- f: func() *tcpip.Error {
+ f: func() tcpip.Error {
if !e.snd.retransmitTimerExpired() {
e.stack.Stats().TCP.EstablishedTimedout.Increment()
- return tcpip.ErrTimeout
+ return &tcpip.ErrTimeout{}
}
return nil
},
@@ -1398,7 +1400,7 @@ func (e *endpoint) protocolMainLoop(handshake bool, wakerInitDone chan<- struct{
},
{
w: &e.newSegmentWaker,
- f: func() *tcpip.Error {
+ f: func() tcpip.Error {
return e.handleSegments(false /* fastPath */)
},
},
@@ -1408,7 +1410,7 @@ func (e *endpoint) protocolMainLoop(handshake bool, wakerInitDone chan<- struct{
},
{
w: &e.notificationWaker,
- f: func() *tcpip.Error {
+ f: func() tcpip.Error {
n := e.fetchNotifications()
if n&notifyNonZeroReceiveWindow != 0 {
e.rcv.nonZeroWindow()
@@ -1425,11 +1427,11 @@ func (e *endpoint) protocolMainLoop(handshake bool, wakerInitDone chan<- struct{
}
if n&notifyReset != 0 || n&notifyAbort != 0 {
- return tcpip.ErrConnectionAborted
+ return &tcpip.ErrConnectionAborted{}
}
if n&notifyResetByPeer != 0 {
- return tcpip.ErrConnectionReset
+ return &tcpip.ErrConnectionReset{}
}
if n&notifyClose != 0 && closeTimer == nil {
@@ -1508,7 +1510,7 @@ func (e *endpoint) protocolMainLoop(handshake bool, wakerInitDone chan<- struct{
// Main loop. Handle segments until both send and receive ends of the
// connection have completed.
- cleanupOnError := func(err *tcpip.Error) {
+ cleanupOnError := func(err tcpip.Error) {
e.stack.Stats().TCP.CurrentConnected.Decrement()
e.workerCleanup = true
if err != nil {
diff --git a/pkg/tcpip/transport/tcp/dual_stack_test.go b/pkg/tcpip/transport/tcp/dual_stack_test.go
index 809c88732..2d90246e4 100644
--- a/pkg/tcpip/transport/tcp/dual_stack_test.go
+++ b/pkg/tcpip/transport/tcp/dual_stack_test.go
@@ -37,7 +37,7 @@ func TestV4MappedConnectOnV6Only(t *testing.T) {
// Start connection attempt, it must fail.
err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestV4MappedAddr, Port: context.TestPort})
- if err != tcpip.ErrNoRoute {
+ if _, ok := err.(*tcpip.ErrNoRoute); !ok {
t.Fatalf("Unexpected return value from Connect: %v", err)
}
}
@@ -49,7 +49,7 @@ func testV4Connect(t *testing.T, c *context.Context, checkers ...checker.Network
defer c.WQ.EventUnregister(&we)
err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestV4MappedAddr, Port: context.TestPort})
- if err != tcpip.ErrConnectStarted {
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
t.Fatalf("Unexpected return value from Connect: %v", err)
}
@@ -156,7 +156,7 @@ func testV6Connect(t *testing.T, c *context.Context, checkers ...checker.Network
defer c.WQ.EventUnregister(&we)
err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestV6Addr, Port: context.TestPort})
- if err != tcpip.ErrConnectStarted {
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
t.Fatalf("Unexpected return value from Connect: %v", err)
}
@@ -391,7 +391,7 @@ func testV4Accept(t *testing.T, c *context.Context) {
defer c.WQ.EventUnregister(&we)
nep, _, err := c.EP.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -525,7 +525,7 @@ func TestV6AcceptOnV6(t *testing.T) {
defer c.WQ.EventUnregister(&we)
var addr tcpip.FullAddress
_, _, err := c.EP.Accept(&addr)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -549,7 +549,7 @@ func TestV4AcceptOnV4(t *testing.T) {
defer c.Cleanup()
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
t.Fatalf("NewEndpoint failed: %v", err)
@@ -613,7 +613,7 @@ func testV4ListenClose(t *testing.T, c *context.Context) {
c.WQ.EventRegister(&we, waiter.EventIn)
defer c.WQ.EventUnregister(&we)
nep, _, err := c.EP.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -635,7 +635,7 @@ func TestV4ListenCloseOnV4(t *testing.T) {
defer c.Cleanup()
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
t.Fatalf("NewEndpoint failed: %v", err)
diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go
index bfa5b01fb..9a53f0d56 100644
--- a/pkg/tcpip/transport/tcp/endpoint.go
+++ b/pkg/tcpip/transport/tcp/endpoint.go
@@ -386,12 +386,12 @@ type endpoint struct {
// hardError is meaningful only when state is stateError. It stores the
// error to be returned when read/write syscalls are called and the
// endpoint is in this state. hardError is protected by endpoint mu.
- hardError *tcpip.Error `state:".(string)"`
+ hardError tcpip.Error
// lastError represents the last error that the endpoint reported;
// access to it is protected by the following mutex.
- lastErrorMu sync.Mutex `state:"nosave"`
- lastError *tcpip.Error `state:".(string)"`
+ lastErrorMu sync.Mutex `state:"nosave"`
+ lastError tcpip.Error
// rcvReadMu synchronizes calls to Read.
//
@@ -1059,7 +1059,7 @@ func (e *endpoint) Close() {
if isResetState {
// Close the endpoint without doing full shutdown and
// send a RST.
- e.resetConnectionLocked(tcpip.ErrConnectionAborted)
+ e.resetConnectionLocked(&tcpip.ErrConnectionAborted{})
e.closeNoShutdownLocked()
// Wake up worker to close the endpoint.
@@ -1293,14 +1293,14 @@ func (e *endpoint) SetOwner(owner tcpip.PacketOwner) {
}
// Preconditions: e.mu must be held to call this function.
-func (e *endpoint) hardErrorLocked() *tcpip.Error {
+func (e *endpoint) hardErrorLocked() tcpip.Error {
err := e.hardError
e.hardError = nil
return err
}
// Preconditions: e.mu must be held to call this function.
-func (e *endpoint) lastErrorLocked() *tcpip.Error {
+func (e *endpoint) lastErrorLocked() tcpip.Error {
e.lastErrorMu.Lock()
defer e.lastErrorMu.Unlock()
err := e.lastError
@@ -1309,7 +1309,7 @@ func (e *endpoint) lastErrorLocked() *tcpip.Error {
}
// LastError implements tcpip.Endpoint.LastError.
-func (e *endpoint) LastError() *tcpip.Error {
+func (e *endpoint) LastError() tcpip.Error {
e.LockUser()
defer e.UnlockUser()
if err := e.hardErrorLocked(); err != nil {
@@ -1319,7 +1319,7 @@ func (e *endpoint) LastError() *tcpip.Error {
}
// UpdateLastError implements tcpip.SocketOptionsHandler.UpdateLastError.
-func (e *endpoint) UpdateLastError(err *tcpip.Error) {
+func (e *endpoint) UpdateLastError(err tcpip.Error) {
e.LockUser()
e.lastErrorMu.Lock()
e.lastError = err
@@ -1328,7 +1328,7 @@ func (e *endpoint) UpdateLastError(err *tcpip.Error) {
}
// Read implements tcpip.Endpoint.Read.
-func (e *endpoint) Read(dst io.Writer, opts tcpip.ReadOptions) (tcpip.ReadResult, *tcpip.Error) {
+func (e *endpoint) Read(dst io.Writer, opts tcpip.ReadOptions) (tcpip.ReadResult, tcpip.Error) {
e.rcvReadMu.Lock()
defer e.rcvReadMu.Unlock()
@@ -1337,7 +1337,7 @@ func (e *endpoint) Read(dst io.Writer, opts tcpip.ReadOptions) (tcpip.ReadResult
// can remove segments from the list through commitRead().
first, last, serr := e.startRead()
if serr != nil {
- if serr == tcpip.ErrClosedForReceive {
+ if _, ok := serr.(*tcpip.ErrClosedForReceive); ok {
e.stats.ReadErrors.ReadClosed.Increment()
}
return tcpip.ReadResult{}, serr
@@ -1377,7 +1377,7 @@ func (e *endpoint) Read(dst io.Writer, opts tcpip.ReadOptions) (tcpip.ReadResult
// If something is read, we must report it. Report error when nothing is read.
if done == 0 && err != nil {
- return tcpip.ReadResult{}, tcpip.ErrBadBuffer
+ return tcpip.ReadResult{}, &tcpip.ErrBadBuffer{}
}
return tcpip.ReadResult{
Count: done,
@@ -1389,7 +1389,7 @@ func (e *endpoint) Read(dst io.Writer, opts tcpip.ReadOptions) (tcpip.ReadResult
// inclusive range of segments that can be read.
//
// Precondition: e.rcvReadMu must be held.
-func (e *endpoint) startRead() (first, last *segment, err *tcpip.Error) {
+func (e *endpoint) startRead() (first, last *segment, err tcpip.Error) {
e.LockUser()
defer e.UnlockUser()
@@ -1398,7 +1398,7 @@ func (e *endpoint) startRead() (first, last *segment, err *tcpip.Error) {
// on a receive. It can expect to read any data after the handshake
// is complete. RFC793, section 3.9, p58.
if e.EndpointState() == StateSynSent {
- return nil, nil, tcpip.ErrWouldBlock
+ return nil, nil, &tcpip.ErrWouldBlock{}
}
// The endpoint can be read if it's connected, or if it's already closed
@@ -1414,17 +1414,17 @@ func (e *endpoint) startRead() (first, last *segment, err *tcpip.Error) {
if err := e.hardErrorLocked(); err != nil {
return nil, nil, err
}
- return nil, nil, tcpip.ErrClosedForReceive
+ return nil, nil, &tcpip.ErrClosedForReceive{}
}
e.stats.ReadErrors.NotConnected.Increment()
- return nil, nil, tcpip.ErrNotConnected
+ return nil, nil, &tcpip.ErrNotConnected{}
}
if e.rcvBufUsed == 0 {
if e.rcvClosed || !e.EndpointState().connected() {
- return nil, nil, tcpip.ErrClosedForReceive
+ return nil, nil, &tcpip.ErrClosedForReceive{}
}
- return nil, nil, tcpip.ErrWouldBlock
+ return nil, nil, &tcpip.ErrWouldBlock{}
}
return e.rcvList.Front(), e.rcvList.Back(), nil
@@ -1476,39 +1476,39 @@ func (e *endpoint) commitRead(done int) *segment {
// moment. If the endpoint is not writable then it returns an error
// indicating the reason why it's not writable.
// Caller must hold e.mu and e.sndBufMu
-func (e *endpoint) isEndpointWritableLocked() (int, *tcpip.Error) {
+func (e *endpoint) isEndpointWritableLocked() (int, tcpip.Error) {
// The endpoint cannot be written to if it's not connected.
switch s := e.EndpointState(); {
case s == StateError:
if err := e.hardErrorLocked(); err != nil {
return 0, err
}
- return 0, tcpip.ErrClosedForSend
+ return 0, &tcpip.ErrClosedForSend{}
case !s.connecting() && !s.connected():
- return 0, tcpip.ErrClosedForSend
+ return 0, &tcpip.ErrClosedForSend{}
case s.connecting():
// As per RFC793, page 56, a send request arriving when in connecting
// state, can be queued to be completed after the state becomes
// connected. Return an error code for the caller of endpoint Write to
// try again, until the connection handshake is complete.
- return 0, tcpip.ErrWouldBlock
+ return 0, &tcpip.ErrWouldBlock{}
}
// Check if the connection has already been closed for sends.
if e.sndClosed {
- return 0, tcpip.ErrClosedForSend
+ return 0, &tcpip.ErrClosedForSend{}
}
sndBufSize := e.getSendBufferSize()
avail := sndBufSize - e.sndBufUsed
if avail <= 0 {
- return 0, tcpip.ErrWouldBlock
+ return 0, &tcpip.ErrWouldBlock{}
}
return avail, nil
}
// Write writes data to the endpoint's peer.
-func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tcpip.Error) {
+func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, tcpip.Error) {
// Linux completely ignores any address passed to sendto(2) for TCP sockets
// (without the MSG_FASTOPEN flag). Corking is unimplemented, so opts.More
// and opts.EndOfRecord are also ignored.
@@ -1516,7 +1516,7 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tc
e.LockUser()
defer e.UnlockUser()
- nextSeg, n, err := func() (*segment, int, *tcpip.Error) {
+ nextSeg, n, err := func() (*segment, int, tcpip.Error) {
e.sndBufMu.Lock()
defer e.sndBufMu.Unlock()
@@ -1526,7 +1526,7 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tc
return nil, 0, err
}
- v, err := func() ([]byte, *tcpip.Error) {
+ v, err := func() ([]byte, tcpip.Error) {
// We can release locks while copying data.
//
// This is not possible if atomic is set, because we can't allow the
@@ -1549,7 +1549,7 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tc
}
v := make([]byte, avail)
if _, err := io.ReadFull(p, v); err != nil {
- return nil, tcpip.ErrBadBuffer
+ return nil, &tcpip.ErrBadBuffer{}
}
return v, nil
}()
@@ -1702,7 +1702,7 @@ func (e *endpoint) getSendBufferSize() int {
}
// SetSockOptInt sets a socket option.
-func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error {
+func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) tcpip.Error {
// Lower 2 bits represents ECN bits. RFC 3168, section 23.1
const inetECNMask = 3
@@ -1730,7 +1730,7 @@ func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error {
case tcpip.MaxSegOption:
userMSS := v
if userMSS < header.TCPMinimumMSS || userMSS > header.TCPMaximumMSS {
- return tcpip.ErrInvalidOptionValue
+ return &tcpip.ErrInvalidOptionValue{}
}
e.LockUser()
e.userMSS = uint16(userMSS)
@@ -1741,7 +1741,7 @@ func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error {
// Return not supported if attempting to set this option to
// anything other than path MTU discovery disabled.
if v != tcpip.PMTUDiscoveryDont {
- return tcpip.ErrNotSupported
+ return &tcpip.ErrNotSupported{}
}
case tcpip.ReceiveBufferSizeOption:
@@ -1801,7 +1801,7 @@ func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error {
case tcpip.TCPSynCountOption:
if v < 1 || v > 255 {
- return tcpip.ErrInvalidOptionValue
+ return &tcpip.ErrInvalidOptionValue{}
}
e.LockUser()
e.maxSynRetries = uint8(v)
@@ -1817,7 +1817,7 @@ func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error {
return nil
default:
e.UnlockUser()
- return tcpip.ErrInvalidOptionValue
+ return &tcpip.ErrInvalidOptionValue{}
}
}
var rs tcpip.TCPReceiveBufferSizeRangeOption
@@ -1838,7 +1838,7 @@ func (e *endpoint) HasNIC(id int32) bool {
}
// SetSockOpt sets a socket option.
-func (e *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error {
+func (e *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) tcpip.Error {
switch v := opt.(type) {
case *tcpip.KeepaliveIdleOption:
e.keepalive.Lock()
@@ -1884,7 +1884,7 @@ func (e *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error {
// Linux returns ENOENT when an invalid congestion
// control algorithm is specified.
- return tcpip.ErrNoSuchFile
+ return &tcpip.ErrNoSuchFile{}
case *tcpip.TCPLingerTimeoutOption:
e.LockUser()
@@ -1927,13 +1927,13 @@ func (e *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error {
}
// readyReceiveSize returns the number of bytes ready to be received.
-func (e *endpoint) readyReceiveSize() (int, *tcpip.Error) {
+func (e *endpoint) readyReceiveSize() (int, tcpip.Error) {
e.LockUser()
defer e.UnlockUser()
// The endpoint cannot be in listen state.
if e.EndpointState() == StateListen {
- return 0, tcpip.ErrInvalidEndpointState
+ return 0, &tcpip.ErrInvalidEndpointState{}
}
e.rcvListMu.Lock()
@@ -1943,7 +1943,7 @@ func (e *endpoint) readyReceiveSize() (int, *tcpip.Error) {
}
// GetSockOptInt implements tcpip.Endpoint.GetSockOptInt.
-func (e *endpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error) {
+func (e *endpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, tcpip.Error) {
switch opt {
case tcpip.KeepaliveCountOption:
e.keepalive.Lock()
@@ -2007,7 +2007,7 @@ func (e *endpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error) {
return 1, nil
default:
- return -1, tcpip.ErrUnknownProtocolOption
+ return -1, &tcpip.ErrUnknownProtocolOption{}
}
}
@@ -2035,7 +2035,7 @@ func (e *endpoint) getTCPInfo() tcpip.TCPInfoOption {
}
// GetSockOpt implements tcpip.Endpoint.GetSockOpt.
-func (e *endpoint) GetSockOpt(opt tcpip.GettableSocketOption) *tcpip.Error {
+func (e *endpoint) GetSockOpt(opt tcpip.GettableSocketOption) tcpip.Error {
switch o := opt.(type) {
case *tcpip.TCPInfoOption:
*o = e.getTCPInfo()
@@ -2084,14 +2084,14 @@ func (e *endpoint) GetSockOpt(opt tcpip.GettableSocketOption) *tcpip.Error {
}
default:
- return tcpip.ErrUnknownProtocolOption
+ return &tcpip.ErrUnknownProtocolOption{}
}
return nil
}
// checkV4MappedLocked determines the effective network protocol and converts
// addr to its canonical form.
-func (e *endpoint) checkV4MappedLocked(addr tcpip.FullAddress) (tcpip.FullAddress, tcpip.NetworkProtocolNumber, *tcpip.Error) {
+func (e *endpoint) checkV4MappedLocked(addr tcpip.FullAddress) (tcpip.FullAddress, tcpip.NetworkProtocolNumber, tcpip.Error) {
unwrapped, netProto, err := e.TransportEndpointInfo.AddrNetProtoLocked(addr, e.ops.GetV6Only())
if err != nil {
return tcpip.FullAddress{}, 0, err
@@ -2100,18 +2100,20 @@ func (e *endpoint) checkV4MappedLocked(addr tcpip.FullAddress) (tcpip.FullAddres
}
// Disconnect implements tcpip.Endpoint.Disconnect.
-func (*endpoint) Disconnect() *tcpip.Error {
- return tcpip.ErrNotSupported
+func (*endpoint) Disconnect() tcpip.Error {
+ return &tcpip.ErrNotSupported{}
}
// Connect connects the endpoint to its peer.
-func (e *endpoint) Connect(addr tcpip.FullAddress) *tcpip.Error {
+func (e *endpoint) Connect(addr tcpip.FullAddress) tcpip.Error {
err := e.connect(addr, true, true)
- if err != nil && !err.IgnoreStats() {
- // Connect failed. Let's wake up any waiters.
- e.waiterQueue.Notify(waiter.EventHUp | waiter.EventErr | waiter.EventIn | waiter.EventOut)
- e.stack.Stats().TCP.FailedConnectionAttempts.Increment()
- e.stats.FailedConnectionAttempts.Increment()
+ if err != nil {
+ if !err.IgnoreStats() {
+ // Connect failed. Let's wake up any waiters.
+ e.waiterQueue.Notify(waiter.EventHUp | waiter.EventErr | waiter.EventIn | waiter.EventOut)
+ e.stack.Stats().TCP.FailedConnectionAttempts.Increment()
+ e.stats.FailedConnectionAttempts.Increment()
+ }
}
return err
}
@@ -2122,7 +2124,7 @@ func (e *endpoint) Connect(addr tcpip.FullAddress) *tcpip.Error {
// created (so no new handshaking is done); for stack-accepted connections not
// yet accepted by the app, they are restored without running the main goroutine
// here.
-func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) *tcpip.Error {
+func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) tcpip.Error {
e.LockUser()
defer e.UnlockUser()
@@ -2141,7 +2143,7 @@ func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) *tc
return nil
}
// Otherwise return that it's already connected.
- return tcpip.ErrAlreadyConnected
+ return &tcpip.ErrAlreadyConnected{}
}
nicID := addr.NIC
@@ -2154,7 +2156,7 @@ func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) *tc
}
if nicID != 0 && nicID != e.boundNICID {
- return tcpip.ErrNoRoute
+ return &tcpip.ErrNoRoute{}
}
nicID = e.boundNICID
@@ -2166,16 +2168,16 @@ func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) *tc
case StateConnecting, StateSynSent, StateSynRecv:
// A connection request has already been issued but hasn't completed
// yet.
- return tcpip.ErrAlreadyConnecting
+ return &tcpip.ErrAlreadyConnecting{}
case StateError:
if err := e.hardErrorLocked(); err != nil {
return err
}
- return tcpip.ErrConnectionAborted
+ return &tcpip.ErrConnectionAborted{}
default:
- return tcpip.ErrInvalidEndpointState
+ return &tcpip.ErrInvalidEndpointState{}
}
// Find a route to the desired destination.
@@ -2231,12 +2233,12 @@ func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) *tc
}
bindToDevice := tcpip.NICID(e.ops.GetBindToDevice())
- if _, err := e.stack.PickEphemeralPortStable(portOffset, func(p uint16) (bool, *tcpip.Error) {
+ if _, err := e.stack.PickEphemeralPortStable(portOffset, func(p uint16) (bool, tcpip.Error) {
if sameAddr && p == e.ID.RemotePort {
return false, nil
}
if _, err := e.stack.ReservePort(netProtos, ProtocolNumber, e.ID.LocalAddress, p, e.portFlags, bindToDevice, addr, nil /* testPort */); err != nil {
- if err != tcpip.ErrPortInUse || !reuse {
+ if _, ok := err.(*tcpip.ErrPortInUse); !ok || !reuse {
return false, nil
}
transEPID := e.ID
@@ -2282,7 +2284,7 @@ func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) *tc
id.LocalPort = p
if err := e.stack.RegisterTransportEndpoint(netProtos, ProtocolNumber, id, e, e.portFlags, bindToDevice); err != nil {
e.stack.ReleasePort(netProtos, ProtocolNumber, e.ID.LocalAddress, p, e.portFlags, bindToDevice, addr)
- if err == tcpip.ErrPortInUse {
+ if _, ok := err.(*tcpip.ErrPortInUse); ok {
return false, nil
}
return false, err
@@ -2337,23 +2339,23 @@ func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) *tc
go e.protocolMainLoop(handshake, nil) // S/R-SAFE: will be drained before save.
}
- return tcpip.ErrConnectStarted
+ return &tcpip.ErrConnectStarted{}
}
// ConnectEndpoint is not supported.
-func (*endpoint) ConnectEndpoint(tcpip.Endpoint) *tcpip.Error {
- return tcpip.ErrInvalidEndpointState
+func (*endpoint) ConnectEndpoint(tcpip.Endpoint) tcpip.Error {
+ return &tcpip.ErrInvalidEndpointState{}
}
// Shutdown closes the read and/or write end of the endpoint connection to its
// peer.
-func (e *endpoint) Shutdown(flags tcpip.ShutdownFlags) *tcpip.Error {
+func (e *endpoint) Shutdown(flags tcpip.ShutdownFlags) tcpip.Error {
e.LockUser()
defer e.UnlockUser()
return e.shutdownLocked(flags)
}
-func (e *endpoint) shutdownLocked(flags tcpip.ShutdownFlags) *tcpip.Error {
+func (e *endpoint) shutdownLocked(flags tcpip.ShutdownFlags) tcpip.Error {
e.shutdownFlags |= flags
switch {
case e.EndpointState().connected():
@@ -2368,7 +2370,7 @@ func (e *endpoint) shutdownLocked(flags tcpip.ShutdownFlags) *tcpip.Error {
// If we're fully closed and we have unread data we need to abort
// the connection with a RST.
if e.shutdownFlags&tcpip.ShutdownWrite != 0 && rcvBufUsed > 0 {
- e.resetConnectionLocked(tcpip.ErrConnectionAborted)
+ e.resetConnectionLocked(&tcpip.ErrConnectionAborted{})
// Wake up worker to terminate loop.
e.notifyProtocolGoroutine(notifyTickleWorker)
return nil
@@ -2382,7 +2384,7 @@ func (e *endpoint) shutdownLocked(flags tcpip.ShutdownFlags) *tcpip.Error {
// Already closed.
e.sndBufMu.Unlock()
if e.EndpointState() == StateTimeWait {
- return tcpip.ErrNotConnected
+ return &tcpip.ErrNotConnected{}
}
return nil
}
@@ -2415,22 +2417,24 @@ func (e *endpoint) shutdownLocked(flags tcpip.ShutdownFlags) *tcpip.Error {
}
return nil
default:
- return tcpip.ErrNotConnected
+ return &tcpip.ErrNotConnected{}
}
}
// Listen puts the endpoint in "listen" mode, which allows it to accept
// new connections.
-func (e *endpoint) Listen(backlog int) *tcpip.Error {
+func (e *endpoint) Listen(backlog int) tcpip.Error {
err := e.listen(backlog)
- if err != nil && !err.IgnoreStats() {
- e.stack.Stats().TCP.FailedConnectionAttempts.Increment()
- e.stats.FailedConnectionAttempts.Increment()
+ if err != nil {
+ if !err.IgnoreStats() {
+ e.stack.Stats().TCP.FailedConnectionAttempts.Increment()
+ e.stats.FailedConnectionAttempts.Increment()
+ }
}
return err
}
-func (e *endpoint) listen(backlog int) *tcpip.Error {
+func (e *endpoint) listen(backlog int) tcpip.Error {
e.LockUser()
defer e.UnlockUser()
@@ -2448,7 +2452,7 @@ func (e *endpoint) listen(backlog int) *tcpip.Error {
// Adjust the size of the channel iff we can fix
// existing pending connections into the new one.
if len(e.acceptedChan) > backlog {
- return tcpip.ErrInvalidEndpointState
+ return &tcpip.ErrInvalidEndpointState{}
}
if cap(e.acceptedChan) == backlog {
return nil
@@ -2480,7 +2484,7 @@ func (e *endpoint) listen(backlog int) *tcpip.Error {
// Endpoint must be bound before it can transition to listen mode.
if e.EndpointState() != StateBound {
e.stats.ReadErrors.InvalidEndpointState.Increment()
- return tcpip.ErrInvalidEndpointState
+ return &tcpip.ErrInvalidEndpointState{}
}
// Register the endpoint.
@@ -2520,7 +2524,7 @@ func (e *endpoint) startAcceptedLoop() {
// to an endpoint previously set to listen mode.
//
// addr if not-nil will contain the peer address of the returned endpoint.
-func (e *endpoint) Accept(peerAddr *tcpip.FullAddress) (tcpip.Endpoint, *waiter.Queue, *tcpip.Error) {
+func (e *endpoint) Accept(peerAddr *tcpip.FullAddress) (tcpip.Endpoint, *waiter.Queue, tcpip.Error) {
e.LockUser()
defer e.UnlockUser()
@@ -2529,7 +2533,7 @@ func (e *endpoint) Accept(peerAddr *tcpip.FullAddress) (tcpip.Endpoint, *waiter.
e.rcvListMu.Unlock()
// Endpoint must be in listen state before it can accept connections.
if rcvClosed || e.EndpointState() != StateListen {
- return nil, nil, tcpip.ErrInvalidEndpointState
+ return nil, nil, &tcpip.ErrInvalidEndpointState{}
}
// Get the new accepted endpoint.
@@ -2540,7 +2544,7 @@ func (e *endpoint) Accept(peerAddr *tcpip.FullAddress) (tcpip.Endpoint, *waiter.
case n = <-e.acceptedChan:
e.acceptCond.Signal()
default:
- return nil, nil, tcpip.ErrWouldBlock
+ return nil, nil, &tcpip.ErrWouldBlock{}
}
if peerAddr != nil {
*peerAddr = n.getRemoteAddress()
@@ -2549,19 +2553,19 @@ func (e *endpoint) Accept(peerAddr *tcpip.FullAddress) (tcpip.Endpoint, *waiter.
}
// Bind binds the endpoint to a specific local port and optionally address.
-func (e *endpoint) Bind(addr tcpip.FullAddress) (err *tcpip.Error) {
+func (e *endpoint) Bind(addr tcpip.FullAddress) (err tcpip.Error) {
e.LockUser()
defer e.UnlockUser()
return e.bindLocked(addr)
}
-func (e *endpoint) bindLocked(addr tcpip.FullAddress) (err *tcpip.Error) {
+func (e *endpoint) bindLocked(addr tcpip.FullAddress) (err tcpip.Error) {
// Don't allow binding once endpoint is not in the initial state
// anymore. This is because once the endpoint goes into a connected or
// listen state, it is already bound.
if e.EndpointState() != StateInitial {
- return tcpip.ErrAlreadyBound
+ return &tcpip.ErrAlreadyBound{}
}
e.BindAddr = addr.Addr
@@ -2589,7 +2593,7 @@ func (e *endpoint) bindLocked(addr tcpip.FullAddress) (err *tcpip.Error) {
if len(addr.Addr) != 0 {
nic = e.stack.CheckLocalAddress(addr.NIC, netProto, addr.Addr)
if nic == 0 {
- return tcpip.ErrBadLocalAddress
+ return &tcpip.ErrBadLocalAddress{}
}
e.ID.LocalAddress = addr.Addr
}
@@ -2630,7 +2634,7 @@ func (e *endpoint) bindLocked(addr tcpip.FullAddress) (err *tcpip.Error) {
}
// GetLocalAddress returns the address to which the endpoint is bound.
-func (e *endpoint) GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) {
+func (e *endpoint) GetLocalAddress() (tcpip.FullAddress, tcpip.Error) {
e.LockUser()
defer e.UnlockUser()
@@ -2642,12 +2646,12 @@ func (e *endpoint) GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) {
}
// GetRemoteAddress returns the address to which the endpoint is connected.
-func (e *endpoint) GetRemoteAddress() (tcpip.FullAddress, *tcpip.Error) {
+func (e *endpoint) GetRemoteAddress() (tcpip.FullAddress, tcpip.Error) {
e.LockUser()
defer e.UnlockUser()
if !e.EndpointState().connected() {
- return tcpip.FullAddress{}, tcpip.ErrNotConnected
+ return tcpip.FullAddress{}, &tcpip.ErrNotConnected{}
}
return e.getRemoteAddress(), nil
@@ -2679,7 +2683,7 @@ func (e *endpoint) enqueueSegment(s *segment) bool {
return true
}
-func (e *endpoint) onICMPError(err *tcpip.Error, errType byte, errCode byte, extra uint32, pkt *stack.PacketBuffer) {
+func (e *endpoint) onICMPError(err tcpip.Error, errType byte, errCode byte, extra uint32, pkt *stack.PacketBuffer) {
// Update last error first.
e.lastErrorMu.Lock()
e.lastError = err
@@ -2728,13 +2732,13 @@ func (e *endpoint) HandleControlPacket(typ stack.ControlType, extra uint32, pkt
e.notifyProtocolGoroutine(notifyMTUChanged)
case stack.ControlNoRoute:
- e.onICMPError(tcpip.ErrNoRoute, byte(header.ICMPv4DstUnreachable), byte(header.ICMPv4HostUnreachable), extra, pkt)
+ e.onICMPError(&tcpip.ErrNoRoute{}, byte(header.ICMPv4DstUnreachable), byte(header.ICMPv4HostUnreachable), extra, pkt)
case stack.ControlAddressUnreachable:
- e.onICMPError(tcpip.ErrNoRoute, byte(header.ICMPv6DstUnreachable), byte(header.ICMPv6AddressUnreachable), extra, pkt)
+ e.onICMPError(&tcpip.ErrNoRoute{}, byte(header.ICMPv6DstUnreachable), byte(header.ICMPv6AddressUnreachable), extra, pkt)
case stack.ControlNetworkUnreachable:
- e.onICMPError(tcpip.ErrNetworkUnreachable, byte(header.ICMPv6DstUnreachable), byte(header.ICMPv6NetworkUnreachable), extra, pkt)
+ e.onICMPError(&tcpip.ErrNetworkUnreachable{}, byte(header.ICMPv6DstUnreachable), byte(header.ICMPv6NetworkUnreachable), extra, pkt)
}
}
diff --git a/pkg/tcpip/transport/tcp/endpoint_state.go b/pkg/tcpip/transport/tcp/endpoint_state.go
index 4a01c81b4..c21dbc682 100644
--- a/pkg/tcpip/transport/tcp/endpoint_state.go
+++ b/pkg/tcpip/transport/tcp/endpoint_state.go
@@ -59,7 +59,7 @@ func (e *endpoint) beforeSave() {
Err: fmt.Errorf("endpoint cannot be saved in connected state: local %s:%d, remote %s:%d", e.ID.LocalAddress, e.ID.LocalPort, e.ID.RemoteAddress, e.ID.RemotePort),
})
}
- e.resetConnectionLocked(tcpip.ErrConnectionAborted)
+ e.resetConnectionLocked(&tcpip.ErrConnectionAborted{})
e.mu.Unlock()
e.Close()
e.mu.Lock()
@@ -232,7 +232,8 @@ func (e *endpoint) Resume(s *stack.Stack) {
// Reset the scoreboard to reinitialize the sack information as
// we do not restore SACK information.
e.scoreboard.Reset()
- if err := e.connect(tcpip.FullAddress{NIC: e.boundNICID, Addr: e.connectingAddress, Port: e.ID.RemotePort}, false, e.workerRunning); err != tcpip.ErrConnectStarted {
+ err := e.connect(tcpip.FullAddress{NIC: e.boundNICID, Addr: e.connectingAddress, Port: e.ID.RemotePort}, false, e.workerRunning)
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
panic("endpoint connecting failed: " + err.String())
}
e.mu.Lock()
@@ -269,7 +270,8 @@ func (e *endpoint) Resume(s *stack.Stack) {
connectedLoading.Wait()
listenLoading.Wait()
bind()
- if err := e.Connect(tcpip.FullAddress{NIC: e.boundNICID, Addr: e.connectingAddress, Port: e.ID.RemotePort}); err != tcpip.ErrConnectStarted {
+ err := e.Connect(tcpip.FullAddress{NIC: e.boundNICID, Addr: e.connectingAddress, Port: e.ID.RemotePort})
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
panic("endpoint connecting failed: " + err.String())
}
connectingLoading.Done()
@@ -296,24 +298,6 @@ func (e *endpoint) Resume(s *stack.Stack) {
}
}
-// saveLastError is invoked by stateify.
-func (e *endpoint) saveLastError() string {
- if e.lastError == nil {
- return ""
- }
-
- return e.lastError.String()
-}
-
-// loadLastError is invoked by stateify.
-func (e *endpoint) loadLastError(s string) {
- if s == "" {
- return
- }
-
- e.lastError = tcpip.StringToError(s)
-}
-
// saveRecentTSTime is invoked by stateify.
func (e *endpoint) saveRecentTSTime() unixTime {
return unixTime{e.recentTSTime.Unix(), e.recentTSTime.UnixNano()}
@@ -324,24 +308,6 @@ func (e *endpoint) loadRecentTSTime(unix unixTime) {
e.recentTSTime = time.Unix(unix.second, unix.nano)
}
-// saveHardError is invoked by stateify.
-func (e *endpoint) saveHardError() string {
- if e.hardError == nil {
- return ""
- }
-
- return e.hardError.String()
-}
-
-// loadHardError is invoked by stateify.
-func (e *endpoint) loadHardError(s string) {
- if s == "" {
- return
- }
-
- e.hardError = tcpip.StringToError(s)
-}
-
// saveMeasureTime is invoked by stateify.
func (r *rcvBufAutoTuneParams) saveMeasureTime() unixTime {
return unixTime{r.measureTime.Unix(), r.measureTime.UnixNano()}
diff --git a/pkg/tcpip/transport/tcp/forwarder.go b/pkg/tcpip/transport/tcp/forwarder.go
index 596178625..2f9fe7ee0 100644
--- a/pkg/tcpip/transport/tcp/forwarder.go
+++ b/pkg/tcpip/transport/tcp/forwarder.go
@@ -143,12 +143,12 @@ func (r *ForwarderRequest) Complete(sendReset bool) {
// CreateEndpoint creates a TCP endpoint for the connection request, performing
// the 3-way handshake in the process.
-func (r *ForwarderRequest) CreateEndpoint(queue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) {
+func (r *ForwarderRequest) CreateEndpoint(queue *waiter.Queue) (tcpip.Endpoint, tcpip.Error) {
r.mu.Lock()
defer r.mu.Unlock()
if r.segment == nil {
- return nil, tcpip.ErrInvalidEndpointState
+ return nil, &tcpip.ErrInvalidEndpointState{}
}
f := r.forwarder
diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go
index 1720370c9..04012cd40 100644
--- a/pkg/tcpip/transport/tcp/protocol.go
+++ b/pkg/tcpip/transport/tcp/protocol.go
@@ -161,13 +161,13 @@ func (*protocol) Number() tcpip.TransportProtocolNumber {
}
// NewEndpoint creates a new tcp endpoint.
-func (p *protocol) NewEndpoint(netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) {
+func (p *protocol) NewEndpoint(netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, tcpip.Error) {
return newEndpoint(p.stack, netProto, waiterQueue), nil
}
// NewRawEndpoint creates a new raw TCP endpoint. Raw TCP sockets are currently
// unsupported. It implements stack.TransportProtocol.NewRawEndpoint.
-func (p *protocol) NewRawEndpoint(netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) {
+func (p *protocol) NewRawEndpoint(netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, tcpip.Error) {
return raw.NewEndpoint(p.stack, netProto, header.TCPProtocolNumber, waiterQueue)
}
@@ -178,7 +178,7 @@ func (*protocol) MinimumPacketSize() int {
// ParsePorts returns the source and destination ports stored in the given tcp
// packet.
-func (*protocol) ParsePorts(v buffer.View) (src, dst uint16, err *tcpip.Error) {
+func (*protocol) ParsePorts(v buffer.View) (src, dst uint16, err tcpip.Error) {
h := header.TCP(v)
return h.SourcePort(), h.DestinationPort(), nil
}
@@ -216,7 +216,7 @@ func (p *protocol) HandleUnknownDestinationPacket(id stack.TransportEndpointID,
// replyWithReset replies to the given segment with a reset segment.
//
// If the passed TTL is 0, then the route's default TTL will be used.
-func replyWithReset(stack *stack.Stack, s *segment, tos, ttl uint8) *tcpip.Error {
+func replyWithReset(stack *stack.Stack, s *segment, tos, ttl uint8) tcpip.Error {
route, err := stack.FindRoute(s.nicID, s.dstAddr, s.srcAddr, s.netProto, false /* multicastLoop */)
if err != nil {
return err
@@ -261,7 +261,7 @@ func replyWithReset(stack *stack.Stack, s *segment, tos, ttl uint8) *tcpip.Error
}
// SetOption implements stack.TransportProtocol.SetOption.
-func (p *protocol) SetOption(option tcpip.SettableTransportProtocolOption) *tcpip.Error {
+func (p *protocol) SetOption(option tcpip.SettableTransportProtocolOption) tcpip.Error {
switch v := option.(type) {
case *tcpip.TCPSACKEnabled:
p.mu.Lock()
@@ -283,7 +283,7 @@ func (p *protocol) SetOption(option tcpip.SettableTransportProtocolOption) *tcpi
case *tcpip.TCPSendBufferSizeRangeOption:
if v.Min <= 0 || v.Default < v.Min || v.Default > v.Max {
- return tcpip.ErrInvalidOptionValue
+ return &tcpip.ErrInvalidOptionValue{}
}
p.mu.Lock()
p.sendBufferSize = *v
@@ -292,7 +292,7 @@ func (p *protocol) SetOption(option tcpip.SettableTransportProtocolOption) *tcpi
case *tcpip.TCPReceiveBufferSizeRangeOption:
if v.Min <= 0 || v.Default < v.Min || v.Default > v.Max {
- return tcpip.ErrInvalidOptionValue
+ return &tcpip.ErrInvalidOptionValue{}
}
p.mu.Lock()
p.recvBufferSize = *v
@@ -310,7 +310,7 @@ func (p *protocol) SetOption(option tcpip.SettableTransportProtocolOption) *tcpi
}
// linux returns ENOENT when an invalid congestion control
// is specified.
- return tcpip.ErrNoSuchFile
+ return &tcpip.ErrNoSuchFile{}
case *tcpip.TCPModerateReceiveBufferOption:
p.mu.Lock()
@@ -340,7 +340,7 @@ func (p *protocol) SetOption(option tcpip.SettableTransportProtocolOption) *tcpi
case *tcpip.TCPTimeWaitReuseOption:
if *v < tcpip.TCPTimeWaitReuseDisabled || *v > tcpip.TCPTimeWaitReuseLoopbackOnly {
- return tcpip.ErrInvalidOptionValue
+ return &tcpip.ErrInvalidOptionValue{}
}
p.mu.Lock()
p.timeWaitReuse = *v
@@ -381,7 +381,7 @@ func (p *protocol) SetOption(option tcpip.SettableTransportProtocolOption) *tcpi
case *tcpip.TCPSynRetriesOption:
if *v < 1 || *v > 255 {
- return tcpip.ErrInvalidOptionValue
+ return &tcpip.ErrInvalidOptionValue{}
}
p.mu.Lock()
p.synRetries = uint8(*v)
@@ -389,12 +389,12 @@ func (p *protocol) SetOption(option tcpip.SettableTransportProtocolOption) *tcpi
return nil
default:
- return tcpip.ErrUnknownProtocolOption
+ return &tcpip.ErrUnknownProtocolOption{}
}
}
// Option implements stack.TransportProtocol.Option.
-func (p *protocol) Option(option tcpip.GettableTransportProtocolOption) *tcpip.Error {
+func (p *protocol) Option(option tcpip.GettableTransportProtocolOption) tcpip.Error {
switch v := option.(type) {
case *tcpip.TCPSACKEnabled:
p.mu.RLock()
@@ -493,7 +493,7 @@ func (p *protocol) Option(option tcpip.GettableTransportProtocolOption) *tcpip.E
return nil
default:
- return tcpip.ErrUnknownProtocolOption
+ return &tcpip.ErrUnknownProtocolOption{}
}
}
diff --git a/pkg/tcpip/transport/tcp/rack.go b/pkg/tcpip/transport/tcp/rack.go
index fdb7e3dc6..c0578f2d6 100644
--- a/pkg/tcpip/transport/tcp/rack.go
+++ b/pkg/tcpip/transport/tcp/rack.go
@@ -193,7 +193,7 @@ func (s *sender) schedulePTO() {
// probeTimerExpired is the same as TLP_send_probe() as defined in
// https://tools.ietf.org/html/draft-ietf-tcpm-rack-08#section-7.5.2.
-func (s *sender) probeTimerExpired() *tcpip.Error {
+func (s *sender) probeTimerExpired() tcpip.Error {
if !s.rc.probeTimer.checkExpiration() {
return nil
}
diff --git a/pkg/tcpip/transport/tcp/rcv.go b/pkg/tcpip/transport/tcp/rcv.go
index 405a6dce7..7a7c402c4 100644
--- a/pkg/tcpip/transport/tcp/rcv.go
+++ b/pkg/tcpip/transport/tcp/rcv.go
@@ -347,7 +347,7 @@ func (r *receiver) updateRTT() {
r.ep.rcvListMu.Unlock()
}
-func (r *receiver) handleRcvdSegmentClosing(s *segment, state EndpointState, closed bool) (drop bool, err *tcpip.Error) {
+func (r *receiver) handleRcvdSegmentClosing(s *segment, state EndpointState, closed bool) (drop bool, err tcpip.Error) {
r.ep.rcvListMu.Lock()
rcvClosed := r.ep.rcvClosed || r.closed
r.ep.rcvListMu.Unlock()
@@ -395,7 +395,7 @@ func (r *receiver) handleRcvdSegmentClosing(s *segment, state EndpointState, clo
// trigger a RST.
endDataSeq := s.sequenceNumber.Add(seqnum.Size(s.data.Size()))
if state != StateCloseWait && rcvClosed && r.rcvNxt.LessThan(endDataSeq) {
- return true, tcpip.ErrConnectionAborted
+ return true, &tcpip.ErrConnectionAborted{}
}
if state == StateFinWait1 {
break
@@ -424,7 +424,7 @@ func (r *receiver) handleRcvdSegmentClosing(s *segment, state EndpointState, clo
// the last actual data octet in a segment in
// which it occurs.
if closed && (!s.flagIsSet(header.TCPFlagFin) || s.sequenceNumber.Add(s.logicalLen()) != r.rcvNxt+1) {
- return true, tcpip.ErrConnectionAborted
+ return true, &tcpip.ErrConnectionAborted{}
}
}
@@ -443,7 +443,7 @@ func (r *receiver) handleRcvdSegmentClosing(s *segment, state EndpointState, clo
// handleRcvdSegment handles TCP segments directed at the connection managed by
// r as they arrive. It is called by the protocol main loop.
-func (r *receiver) handleRcvdSegment(s *segment) (drop bool, err *tcpip.Error) {
+func (r *receiver) handleRcvdSegment(s *segment) (drop bool, err tcpip.Error) {
state := r.ep.EndpointState()
closed := r.ep.closed
diff --git a/pkg/tcpip/transport/tcp/snd.go b/pkg/tcpip/transport/tcp/snd.go
index 027c2a4a8..2967c9f97 100644
--- a/pkg/tcpip/transport/tcp/snd.go
+++ b/pkg/tcpip/transport/tcp/snd.go
@@ -1497,7 +1497,7 @@ func (s *sender) handleRcvdSegment(rcvdSeg *segment) {
}
// sendSegment sends the specified segment.
-func (s *sender) sendSegment(seg *segment) *tcpip.Error {
+func (s *sender) sendSegment(seg *segment) tcpip.Error {
if seg.xmitCount > 0 {
s.ep.stack.Stats().TCP.Retransmits.Increment()
s.ep.stats.SendErrors.Retransmits.Increment()
@@ -1528,7 +1528,7 @@ func (s *sender) sendSegment(seg *segment) *tcpip.Error {
// sendSegmentFromView sends a new segment containing the given payload, flags
// and sequence number.
-func (s *sender) sendSegmentFromView(data buffer.VectorisedView, flags byte, seq seqnum.Value) *tcpip.Error {
+func (s *sender) sendSegmentFromView(data buffer.VectorisedView, flags byte, seq seqnum.Value) tcpip.Error {
s.lastSendTime = time.Now()
if seq == s.rttMeasureSeqNum {
s.rttMeasureTime = s.lastSendTime
diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go
index 87ff2b909..da2730e27 100644
--- a/pkg/tcpip/transport/tcp/tcp_test.go
+++ b/pkg/tcpip/transport/tcp/tcp_test.go
@@ -48,7 +48,7 @@ type endpointTester struct {
}
// CheckReadError issues a read to the endpoint and checking for an error.
-func (e *endpointTester) CheckReadError(t *testing.T, want *tcpip.Error) {
+func (e *endpointTester) CheckReadError(t *testing.T, want tcpip.Error) {
t.Helper()
res, got := e.ep.Read(ioutil.Discard, tcpip.ReadOptions{})
if got != want {
@@ -87,7 +87,7 @@ func (e *endpointTester) CheckReadFull(t *testing.T, count int, notifyRead <-cha
}
for w.N != 0 {
_, err := e.ep.Read(&w, tcpip.ReadOptions{})
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for receive to be notified.
select {
case <-notifyRead:
@@ -128,8 +128,11 @@ func TestGiveUpConnect(t *testing.T) {
wq.EventRegister(&waitEntry, waiter.EventHUp)
defer wq.EventUnregister(&waitEntry)
- if err := ep.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort}); err != tcpip.ErrConnectStarted {
- t.Fatalf("got ep.Connect(...) = %s, want = %s", err, tcpip.ErrConnectStarted)
+ {
+ err := ep.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort})
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
+ t.Fatalf("got ep.Connect(...) = %v, want = %s", err, &tcpip.ErrConnectStarted{})
+ }
}
// Close the connection, wait for completion.
@@ -140,8 +143,11 @@ func TestGiveUpConnect(t *testing.T) {
// Call Connect again to retreive the handshake failure status
// and stats updates.
- if err := ep.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort}); err != tcpip.ErrAborted {
- t.Fatalf("got ep.Connect(...) = %s, want = %s", err, tcpip.ErrAborted)
+ {
+ err := ep.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort})
+ if _, ok := err.(*tcpip.ErrAborted); !ok {
+ t.Fatalf("got ep.Connect(...) = %v, want = %s", err, &tcpip.ErrAborted{})
+ }
}
if got := c.Stack().Stats().TCP.FailedConnectionAttempts.Value(); got != 1 {
@@ -194,8 +200,11 @@ func TestActiveFailedConnectionAttemptIncrement(t *testing.T) {
c.EP = ep
want := stats.TCP.FailedConnectionAttempts.Value() + 1
- if err := c.EP.Connect(tcpip.FullAddress{NIC: 2, Addr: context.TestAddr, Port: context.TestPort}); err != tcpip.ErrNoRoute {
- t.Errorf("got c.EP.Connect(...) = %s, want = %s", err, tcpip.ErrNoRoute)
+ {
+ err := c.EP.Connect(tcpip.FullAddress{NIC: 2, Addr: context.TestAddr, Port: context.TestPort})
+ if _, ok := err.(*tcpip.ErrNoRoute); !ok {
+ t.Errorf("got c.EP.Connect(...) = %v, want = %s", err, &tcpip.ErrNoRoute{})
+ }
}
if got := stats.TCP.FailedConnectionAttempts.Value(); got != want {
@@ -211,7 +220,7 @@ func TestCloseWithoutConnect(t *testing.T) {
defer c.Cleanup()
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
t.Fatalf("NewEndpoint failed: %s", err)
@@ -384,7 +393,7 @@ func TestTCPResetSentForACKWhenNotUsingSynCookies(t *testing.T) {
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -925,8 +934,11 @@ func TestUserSuppliedMSSOnConnect(t *testing.T) {
ws := tcp.FindWndScale(seqnum.Size(rcvBufSize))
connectAddr := tcpip.FullAddress{Addr: ip.connectAddr, Port: context.TestPort}
- if err := c.EP.Connect(connectAddr); err != tcpip.ErrConnectStarted {
- t.Fatalf("Connect(%+v): %s", connectAddr, err)
+ {
+ err := c.EP.Connect(connectAddr)
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
+ t.Fatalf("Connect(%+v): %s", connectAddr, err)
+ }
}
// Receive SYN packet with our user supplied MSS.
@@ -1442,7 +1454,8 @@ func TestConnectBindToDevice(t *testing.T) {
c.WQ.EventRegister(&waitEntry, waiter.EventOut)
defer c.WQ.EventUnregister(&waitEntry)
- if err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort}); err != tcpip.ErrConnectStarted {
+ err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort})
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
t.Fatalf("unexpected return value from Connect: %s", err)
}
@@ -1502,8 +1515,9 @@ func TestSynSent(t *testing.T) {
defer c.WQ.EventUnregister(&waitEntry)
addr := tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort}
- if err := c.EP.Connect(addr); err != tcpip.ErrConnectStarted {
- t.Fatalf("got Connect(%+v) = %s, want %s", addr, err, tcpip.ErrConnectStarted)
+ err := c.EP.Connect(addr)
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
+ t.Fatalf("got Connect(%+v) = %v, want %s", addr, err, &tcpip.ErrConnectStarted{})
}
// Receive SYN packet.
@@ -1548,9 +1562,9 @@ func TestSynSent(t *testing.T) {
ept := endpointTester{c.EP}
if test.reset {
- ept.CheckReadError(t, tcpip.ErrConnectionRefused)
+ ept.CheckReadError(t, &tcpip.ErrConnectionRefused{})
} else {
- ept.CheckReadError(t, tcpip.ErrAborted)
+ ept.CheckReadError(t, &tcpip.ErrAborted{})
}
if got := c.Stack().Stats().TCP.CurrentConnected.Value(); got != 0 {
@@ -1576,7 +1590,7 @@ func TestOutOfOrderReceive(t *testing.T) {
defer c.WQ.EventUnregister(&we)
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
// Send second half of data first, with seqnum 3 ahead of expected.
data := []byte{1, 2, 3, 4, 5, 6}
@@ -1601,7 +1615,7 @@ func TestOutOfOrderReceive(t *testing.T) {
// Wait 200ms and check that no data has been received.
time.Sleep(200 * time.Millisecond)
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
// Send the first 3 bytes now.
c.SendPacket(data[:3], &context.Headers{
@@ -1640,7 +1654,7 @@ func TestOutOfOrderFlood(t *testing.T) {
c.CreateConnected(789, 30000, rcvBufSz)
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
// Send 100 packets before the actual one that is expected.
data := []byte{1, 2, 3, 4, 5, 6}
@@ -1716,7 +1730,7 @@ func TestRstOnCloseWithUnreadData(t *testing.T) {
defer c.WQ.EventUnregister(&we)
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
data := []byte{1, 2, 3}
c.SendPacket(data, &context.Headers{
@@ -1784,7 +1798,7 @@ func TestRstOnCloseWithUnreadDataFinConvertRst(t *testing.T) {
defer c.WQ.EventUnregister(&we)
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
data := []byte{1, 2, 3}
c.SendPacket(data, &context.Headers{
@@ -1866,13 +1880,13 @@ func TestShutdownRead(t *testing.T) {
c.CreateConnected(789, 30000, -1 /* epRcvBuf */)
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
if err := c.EP.Shutdown(tcpip.ShutdownRead); err != nil {
t.Fatalf("Shutdown failed: %s", err)
}
- ept.CheckReadError(t, tcpip.ErrClosedForReceive)
+ ept.CheckReadError(t, &tcpip.ErrClosedForReceive{})
var want uint64 = 1
if got := c.EP.Stats().(*tcp.Stats).ReadErrors.ReadClosed.Value(); got != want {
t.Fatalf("got EP stats Stats.ReadErrors.ReadClosed got %d want %d", got, want)
@@ -1891,7 +1905,7 @@ func TestFullWindowReceive(t *testing.T) {
defer c.WQ.EventUnregister(&we)
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
// Fill up the window w/ tcp.SegOverheadFactor*rcvBufSz as netstack multiplies
// the provided buffer value by tcp.SegOverheadFactor to calculate the actual
@@ -2052,7 +2066,7 @@ func TestNoWindowShrinking(t *testing.T) {
defer c.WQ.EventUnregister(&we)
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
// Send a 1 byte payload so that we can record the current receive window.
// Send a payload of half the size of rcvBufSize.
@@ -2370,7 +2384,7 @@ func TestScaledWindowAccept(t *testing.T) {
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -2443,7 +2457,7 @@ func TestNonScaledWindowAccept(t *testing.T) {
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -2958,7 +2972,7 @@ func TestSetTTL(t *testing.T) {
c := context.New(t, 65535)
defer c.Cleanup()
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &waiter.Queue{})
if err != nil {
t.Fatalf("NewEndpoint failed: %s", err)
@@ -2968,8 +2982,11 @@ func TestSetTTL(t *testing.T) {
t.Fatalf("SetSockOptInt(TTLOption, %d) failed: %s", wantTTL, err)
}
- if err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort}); err != tcpip.ErrConnectStarted {
- t.Fatalf("unexpected return value from Connect: %s", err)
+ {
+ err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort})
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
+ t.Fatalf("unexpected return value from Connect: %s", err)
+ }
}
// Receive SYN packet.
@@ -3029,7 +3046,7 @@ func TestPassiveSendMSSLessThanMTU(t *testing.T) {
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -3085,7 +3102,7 @@ func TestSynCookiePassiveSendMSSLessThanMTU(t *testing.T) {
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -3110,9 +3127,9 @@ func TestForwarderSendMSSLessThanMTU(t *testing.T) {
defer c.Cleanup()
s := c.Stack()
- ch := make(chan *tcpip.Error, 1)
+ ch := make(chan tcpip.Error, 1)
f := tcp.NewForwarder(s, 65536, 10, func(r *tcp.ForwarderRequest) {
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = r.CreateEndpoint(&c.WQ)
ch <- err
})
@@ -3141,7 +3158,7 @@ func TestSynOptionsOnActiveConnect(t *testing.T) {
defer c.Cleanup()
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
t.Fatalf("NewEndpoint failed: %s", err)
@@ -3160,8 +3177,11 @@ func TestSynOptionsOnActiveConnect(t *testing.T) {
c.WQ.EventRegister(&we, waiter.EventOut)
defer c.WQ.EventUnregister(&we)
- if err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort}); err != tcpip.ErrConnectStarted {
- t.Fatalf("got c.EP.Connect(...) = %s, want = %s", err, tcpip.ErrConnectStarted)
+ {
+ err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort})
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
+ t.Fatalf("got c.EP.Connect(...) = %v, want = %s", err, &tcpip.ErrConnectStarted{})
+ }
}
// Receive SYN packet.
@@ -3271,22 +3291,23 @@ func TestReceiveOnResetConnection(t *testing.T) {
loop:
for {
- switch _, err := c.EP.Read(ioutil.Discard, tcpip.ReadOptions{}); err {
- case tcpip.ErrWouldBlock:
+ switch _, err := c.EP.Read(ioutil.Discard, tcpip.ReadOptions{}); err.(type) {
+ case *tcpip.ErrWouldBlock:
select {
case <-ch:
// Expect the state to be StateError and subsequent Reads to fail with HardError.
- if _, err := c.EP.Read(ioutil.Discard, tcpip.ReadOptions{}); err != tcpip.ErrConnectionReset {
- t.Fatalf("got c.EP.Read() = %s, want = %s", err, tcpip.ErrConnectionReset)
+ _, err := c.EP.Read(ioutil.Discard, tcpip.ReadOptions{})
+ if _, ok := err.(*tcpip.ErrConnectionReset); !ok {
+ t.Fatalf("got c.EP.Read() = %v, want = %s", err, &tcpip.ErrConnectionReset{})
}
break loop
case <-time.After(1 * time.Second):
t.Fatalf("Timed out waiting for reset to arrive")
}
- case tcpip.ErrConnectionReset:
+ case *tcpip.ErrConnectionReset:
break loop
default:
- t.Fatalf("got c.EP.Read(nil) = %s, want = %s", err, tcpip.ErrConnectionReset)
+ t.Fatalf("got c.EP.Read(nil) = %v, want = %s", err, &tcpip.ErrConnectionReset{})
}
}
@@ -3325,8 +3346,9 @@ func TestSendOnResetConnection(t *testing.T) {
// Try to write.
var r bytes.Reader
r.Reset(make([]byte, 10))
- if _, err := c.EP.Write(&r, tcpip.WriteOptions{}); err != tcpip.ErrConnectionReset {
- t.Fatalf("got c.EP.Write(...) = %s, want = %s", err, tcpip.ErrConnectionReset)
+ _, err := c.EP.Write(&r, tcpip.WriteOptions{})
+ if _, ok := err.(*tcpip.ErrConnectionReset); !ok {
+ t.Fatalf("got c.EP.Write(...) = %v, want = %s", err, &tcpip.ErrConnectionReset{})
}
}
@@ -4184,7 +4206,7 @@ func TestReadAfterClosedState(t *testing.T) {
defer c.WQ.EventUnregister(&we)
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
// Shutdown immediately for write, check that we get a FIN.
if err := c.EP.Shutdown(tcpip.ShutdownWrite); err != nil {
@@ -4263,10 +4285,13 @@ func TestReadAfterClosedState(t *testing.T) {
// Now that we drained the queue, check that functions fail with the
// right error code.
- ept.CheckReadError(t, tcpip.ErrClosedForReceive)
+ ept.CheckReadError(t, &tcpip.ErrClosedForReceive{})
var buf bytes.Buffer
- if _, err := c.EP.Read(&buf, tcpip.ReadOptions{Peek: true}); err != tcpip.ErrClosedForReceive {
- t.Fatalf("c.EP.Read(_, {Peek: true}) = %v, %s; want _, %s", res, err, tcpip.ErrClosedForReceive)
+ {
+ _, err := c.EP.Read(&buf, tcpip.ReadOptions{Peek: true})
+ if _, ok := err.(*tcpip.ErrClosedForReceive); !ok {
+ t.Fatalf("c.EP.Read(_, {Peek: true}) = %v, %s; want _, %s", res, err, &tcpip.ErrClosedForReceive{})
+ }
}
}
@@ -4277,7 +4302,7 @@ func TestReusePort(t *testing.T) {
defer c.Cleanup()
// First case, just an endpoint that was bound.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &waiter.Queue{})
if err != nil {
t.Fatalf("NewEndpoint failed; %s", err)
@@ -4307,8 +4332,11 @@ func TestReusePort(t *testing.T) {
if err := c.EP.Bind(tcpip.FullAddress{Port: context.StackPort}); err != nil {
t.Fatalf("Bind failed: %s", err)
}
- if err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort}); err != tcpip.ErrConnectStarted {
- t.Fatalf("got c.EP.Connect(...) = %s, want = %s", err, tcpip.ErrConnectStarted)
+ {
+ err := c.EP.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort})
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
+ t.Fatalf("got c.EP.Connect(...) = %v, want = %s", err, &tcpip.ErrConnectStarted{})
+ }
}
c.EP.Close()
@@ -4515,11 +4543,11 @@ func TestBindToDeviceOption(t *testing.T) {
testActions := []struct {
name string
setBindToDevice *tcpip.NICID
- setBindToDeviceError *tcpip.Error
+ setBindToDeviceError tcpip.Error
getBindToDevice int32
}{
{"GetDefaultValue", nil, nil, 0},
- {"BindToNonExistent", nicIDPtr(999), tcpip.ErrUnknownDevice, 0},
+ {"BindToNonExistent", nicIDPtr(999), &tcpip.ErrUnknownDevice{}, 0},
{"BindToExistent", nicIDPtr(321), nil, 321},
{"UnbindToDevice", nicIDPtr(0), nil, 0},
}
@@ -4539,7 +4567,7 @@ func TestBindToDeviceOption(t *testing.T) {
}
}
-func makeStack() (*stack.Stack, *tcpip.Error) {
+func makeStack() (*stack.Stack, tcpip.Error) {
s := stack.New(stack.Options{
NetworkProtocols: []stack.NetworkProtocolFactory{
ipv4.NewProtocol,
@@ -4609,8 +4637,11 @@ func TestSelfConnect(t *testing.T) {
wq.EventRegister(&waitEntry, waiter.EventOut)
defer wq.EventUnregister(&waitEntry)
- if err := ep.Connect(tcpip.FullAddress{Addr: context.StackAddr, Port: context.StackPort}); err != tcpip.ErrConnectStarted {
- t.Fatalf("got ep.Connect(...) = %s, want = %s", err, tcpip.ErrConnectStarted)
+ {
+ err := ep.Connect(tcpip.FullAddress{Addr: context.StackAddr, Port: context.StackPort})
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
+ t.Fatalf("got ep.Connect(...) = %v, want = %s", err, &tcpip.ErrConnectStarted{})
+ }
}
<-notifyCh
@@ -4762,9 +4793,9 @@ func TestConnectAvoidsBoundPorts(t *testing.T) {
t.Fatalf("Bind(%d) failed: %s", i, err)
}
}
- want := tcpip.ErrConnectStarted
+ var want tcpip.Error = &tcpip.ErrConnectStarted{}
if collides {
- want = tcpip.ErrNoPortAvailable
+ want = &tcpip.ErrNoPortAvailable{}
}
if err := makeEP(candidateNetwork).Connect(tcpip.FullAddress{Addr: address(t, candidateAddressType, false), Port: 31337}); err != want {
t.Fatalf("got ep.Connect(..) = %s, want = %s", err, want)
@@ -4889,11 +4920,11 @@ func TestTCPEndpointProbe(t *testing.T) {
func TestStackSetCongestionControl(t *testing.T) {
testCases := []struct {
cc tcpip.CongestionControlOption
- err *tcpip.Error
+ err tcpip.Error
}{
{"reno", nil},
{"cubic", nil},
- {"blahblah", tcpip.ErrNoSuchFile},
+ {"blahblah", &tcpip.ErrNoSuchFile{}},
}
for _, tc := range testCases {
@@ -4975,11 +5006,11 @@ func TestStackSetAvailableCongestionControl(t *testing.T) {
func TestEndpointSetCongestionControl(t *testing.T) {
testCases := []struct {
cc tcpip.CongestionControlOption
- err *tcpip.Error
+ err tcpip.Error
}{
{"reno", nil},
{"cubic", nil},
- {"blahblah", tcpip.ErrNoSuchFile},
+ {"blahblah", &tcpip.ErrNoSuchFile{}},
}
for _, connected := range []bool{false, true} {
@@ -4989,7 +5020,7 @@ func TestEndpointSetCongestionControl(t *testing.T) {
defer c.Cleanup()
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
t.Fatalf("NewEndpoint failed: %s", err)
@@ -5085,7 +5116,7 @@ func TestKeepalive(t *testing.T) {
// Check that the connection is still alive.
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
// Send some data and wait before ACKing it. Keepalives should be disabled
// during this period.
@@ -5176,7 +5207,7 @@ func TestKeepalive(t *testing.T) {
t.Errorf("got c.Stack().Stats().TCP.EstablishedTimedout.Value() = %d, want = 1", got)
}
- ept.CheckReadError(t, tcpip.ErrTimeout)
+ ept.CheckReadError(t, &tcpip.ErrTimeout{})
if got := c.Stack().Stats().TCP.CurrentEstablished.Value(); got != 0 {
t.Errorf("got stats.TCP.CurrentEstablished.Value() = %d, want = 0", got)
@@ -5283,7 +5314,7 @@ func TestListenBacklogFull(t *testing.T) {
defer c.Cleanup()
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
t.Fatalf("NewEndpoint failed: %s", err)
@@ -5326,7 +5357,7 @@ func TestListenBacklogFull(t *testing.T) {
for i := 0; i < listenBacklog; i++ {
_, _, err = c.EP.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -5343,7 +5374,7 @@ func TestListenBacklogFull(t *testing.T) {
// Now verify that there are no more connections that can be accepted.
_, _, err = c.EP.Accept(nil)
- if err != tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); !ok {
select {
case <-ch:
t.Fatalf("unexpected endpoint delivered on Accept: %+v", c.EP)
@@ -5355,7 +5386,7 @@ func TestListenBacklogFull(t *testing.T) {
executeHandshake(t, c, context.TestPort+lastPortOffset, false /*synCookieInUse */)
newEP, _, err := c.EP.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -5598,7 +5629,7 @@ func TestListenSynRcvdQueueFull(t *testing.T) {
defer c.Cleanup()
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
t.Fatalf("NewEndpoint failed: %s", err)
@@ -5673,7 +5704,7 @@ func TestListenSynRcvdQueueFull(t *testing.T) {
defer c.WQ.EventUnregister(&we)
newEP, _, err := c.EP.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -5709,7 +5740,7 @@ func TestListenBacklogFullSynCookieInUse(t *testing.T) {
}
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
t.Fatalf("NewEndpoint failed: %s", err)
@@ -5750,7 +5781,7 @@ func TestListenBacklogFullSynCookieInUse(t *testing.T) {
defer c.WQ.EventUnregister(&we)
_, _, err = c.EP.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -5766,7 +5797,7 @@ func TestListenBacklogFullSynCookieInUse(t *testing.T) {
// Now verify that there are no more connections that can be accepted.
_, _, err = c.EP.Accept(nil)
- if err != tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); !ok {
select {
case <-ch:
t.Fatalf("unexpected endpoint delivered on Accept: %+v", c.EP)
@@ -5780,7 +5811,7 @@ func TestSYNRetransmit(t *testing.T) {
defer c.Cleanup()
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
t.Fatalf("NewEndpoint failed: %s", err)
@@ -5824,7 +5855,7 @@ func TestSynRcvdBadSeqNumber(t *testing.T) {
defer c.Cleanup()
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
t.Fatalf("NewEndpoint failed: %s", err)
@@ -5899,12 +5930,13 @@ func TestSynRcvdBadSeqNumber(t *testing.T) {
})
newEP, _, err := c.EP.Accept(nil)
-
- if err != nil && err != tcpip.ErrWouldBlock {
+ switch err.(type) {
+ case nil, *tcpip.ErrWouldBlock:
+ default:
t.Fatalf("Accept failed: %s", err)
}
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Try to accept the connections in the backlog.
we, ch := waiter.NewChannelEntry(nil)
c.WQ.EventRegister(&we, waiter.EventIn)
@@ -5972,7 +6004,7 @@ func TestPassiveConnectionAttemptIncrement(t *testing.T) {
// Verify that there is only one acceptable connection at this point.
_, _, err = c.EP.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -6042,7 +6074,7 @@ func TestPassiveFailedConnectionAttemptIncrement(t *testing.T) {
// Now check that there is one acceptable connections.
_, _, err = c.EP.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -6074,7 +6106,7 @@ func TestEndpointBindListenAcceptState(t *testing.T) {
}
ept := endpointTester{ep}
- ept.CheckReadError(t, tcpip.ErrNotConnected)
+ ept.CheckReadError(t, &tcpip.ErrNotConnected{})
if got := ep.Stats().(*tcp.Stats).ReadErrors.NotConnected.Value(); got != 1 {
t.Errorf("got EP stats Stats.ReadErrors.NotConnected got %d want %d", got, 1)
}
@@ -6094,7 +6126,7 @@ func TestEndpointBindListenAcceptState(t *testing.T) {
defer wq.EventUnregister(&we)
aep, _, err := ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -6110,8 +6142,11 @@ func TestEndpointBindListenAcceptState(t *testing.T) {
if got, want := tcp.EndpointState(aep.State()), tcp.StateEstablished; got != want {
t.Errorf("unexpected endpoint state: want %s, got %s", want, got)
}
- if err := aep.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort}); err != tcpip.ErrAlreadyConnected {
- t.Errorf("unexpected error attempting to call connect on an established endpoint, got: %s, want: %s", err, tcpip.ErrAlreadyConnected)
+ {
+ err := aep.Connect(tcpip.FullAddress{Addr: context.TestAddr, Port: context.TestPort})
+ if _, ok := err.(*tcpip.ErrAlreadyConnected); !ok {
+ t.Errorf("unexpected error attempting to call connect on an established endpoint, got: %v, want: %s", err, &tcpip.ErrAlreadyConnected{})
+ }
}
// Listening endpoint remains in listen state.
if got, want := tcp.EndpointState(ep.State()), tcp.StateListen; got != want {
@@ -6230,7 +6265,7 @@ func TestReceiveBufferAutoTuningApplicationLimited(t *testing.T) {
// window increases to the full available buffer size.
for {
_, err := c.EP.Read(ioutil.Discard, tcpip.ReadOptions{})
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
break
}
}
@@ -6354,7 +6389,7 @@ func TestReceiveBufferAutoTuning(t *testing.T) {
totalCopied := 0
for {
res, err := c.EP.Read(ioutil.Discard, tcpip.ReadOptions{})
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
break
}
totalCopied += res.Count
@@ -6546,7 +6581,7 @@ func TestTCPTimeWaitRSTIgnored(t *testing.T) {
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -6665,7 +6700,7 @@ func TestTCPTimeWaitOutOfOrder(t *testing.T) {
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -6772,7 +6807,7 @@ func TestTCPTimeWaitNewSyn(t *testing.T) {
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -6862,7 +6897,7 @@ func TestTCPTimeWaitNewSyn(t *testing.T) {
// Try to accept the connection.
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -6936,7 +6971,7 @@ func TestTCPTimeWaitDuplicateFINExtendsTimeWait(t *testing.T) {
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -7086,7 +7121,7 @@ func TestTCPCloseWithData(t *testing.T) {
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch:
@@ -7277,7 +7312,7 @@ func TestTCPUserTimeout(t *testing.T) {
)
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrTimeout)
+ ept.CheckReadError(t, &tcpip.ErrTimeout{})
if got, want := c.Stack().Stats().TCP.EstablishedTimedout.Value(), origEstablishedTimedout+1; got != want {
t.Errorf("got c.Stack().Stats().TCP.EstablishedTimedout = %d, want = %d", got, want)
@@ -7321,7 +7356,7 @@ func TestKeepaliveWithUserTimeout(t *testing.T) {
// Check that the connection is still alive.
ept := endpointTester{c.EP}
- ept.CheckReadError(t, tcpip.ErrWouldBlock)
+ ept.CheckReadError(t, &tcpip.ErrWouldBlock{})
// Now receive 1 keepalives, but don't ACK it.
b := c.GetPacket()
@@ -7360,7 +7395,7 @@ func TestKeepaliveWithUserTimeout(t *testing.T) {
),
)
- ept.CheckReadError(t, tcpip.ErrTimeout)
+ ept.CheckReadError(t, &tcpip.ErrTimeout{})
if got, want := c.Stack().Stats().TCP.EstablishedTimedout.Value(), origEstablishedTimedout+1; got != want {
t.Errorf("got c.Stack().Stats().TCP.EstablishedTimedout = %d, want = %d", got, want)
}
@@ -7515,8 +7550,9 @@ func TestTCPDeferAccept(t *testing.T) {
irs, iss := executeHandshake(t, c, context.TestPort, false /* synCookiesInUse */)
- if _, _, err := c.EP.Accept(nil); err != tcpip.ErrWouldBlock {
- t.Fatalf("got c.EP.Accept(nil) = %s, want: %s", err, tcpip.ErrWouldBlock)
+ _, _, err := c.EP.Accept(nil)
+ if _, ok := err.(*tcpip.ErrWouldBlock); !ok {
+ t.Fatalf("got c.EP.Accept(nil) = %v, want: %s", err, &tcpip.ErrWouldBlock{})
}
// Send data. This should result in an acceptable endpoint.
@@ -7573,8 +7609,9 @@ func TestTCPDeferAcceptTimeout(t *testing.T) {
irs, iss := executeHandshake(t, c, context.TestPort, false /* synCookiesInUse */)
- if _, _, err := c.EP.Accept(nil); err != tcpip.ErrWouldBlock {
- t.Fatalf("got c.EP.Accept(nil) = %s, want: %s", err, tcpip.ErrWouldBlock)
+ _, _, err := c.EP.Accept(nil)
+ if _, ok := err.(*tcpip.ErrWouldBlock); !ok {
+ t.Fatalf("got c.EP.Accept(nil) = %v, want: %s", err, &tcpip.ErrWouldBlock{})
}
// Sleep for a little of the tcpDeferAccept timeout.
@@ -7696,13 +7733,13 @@ func TestSetStackTimeWaitReuse(t *testing.T) {
s := c.Stack()
testCases := []struct {
v int
- err *tcpip.Error
+ err tcpip.Error
}{
{int(tcpip.TCPTimeWaitReuseDisabled), nil},
{int(tcpip.TCPTimeWaitReuseGlobal), nil},
{int(tcpip.TCPTimeWaitReuseLoopbackOnly), nil},
- {int(tcpip.TCPTimeWaitReuseLoopbackOnly) + 1, tcpip.ErrInvalidOptionValue},
- {int(tcpip.TCPTimeWaitReuseDisabled) - 1, tcpip.ErrInvalidOptionValue},
+ {int(tcpip.TCPTimeWaitReuseLoopbackOnly) + 1, &tcpip.ErrInvalidOptionValue{}},
+ {int(tcpip.TCPTimeWaitReuseDisabled) - 1, &tcpip.ErrInvalidOptionValue{}},
}
for _, tc := range testCases {
diff --git a/pkg/tcpip/transport/tcp/testing/context/context.go b/pkg/tcpip/transport/tcp/testing/context/context.go
index ee55f030c..b1cb9a324 100644
--- a/pkg/tcpip/transport/tcp/testing/context/context.go
+++ b/pkg/tcpip/transport/tcp/testing/context/context.go
@@ -586,7 +586,7 @@ func (c *Context) ReceiveNonBlockingAndCheckPacket(data []byte, offset, size int
// is true then it sets the IP_V6ONLY option on the socket to make it a IPv6
// only endpoint instead of a default dual stack socket.
func (c *Context) CreateV6Endpoint(v6only bool) {
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.s.NewEndpoint(tcp.ProtocolNumber, ipv6.ProtocolNumber, &c.WQ)
if err != nil {
c.t.Fatalf("NewEndpoint failed: %v", err)
@@ -689,7 +689,8 @@ func (c *Context) Connect(iss seqnum.Value, rcvWnd seqnum.Size, options []byte)
c.WQ.EventRegister(&waitEntry, waiter.EventOut)
defer c.WQ.EventUnregister(&waitEntry)
- if err := c.EP.Connect(tcpip.FullAddress{Addr: TestAddr, Port: TestPort}); err != tcpip.ErrConnectStarted {
+ err := c.EP.Connect(tcpip.FullAddress{Addr: TestAddr, Port: TestPort})
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
c.t.Fatalf("Unexpected return value from Connect: %v", err)
}
@@ -749,7 +750,7 @@ func (c *Context) Connect(iss seqnum.Value, rcvWnd seqnum.Size, options []byte)
// Create creates a TCP endpoint.
func (c *Context) Create(epRcvBuf int) {
// Create TCP endpoint.
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.s.NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
c.t.Fatalf("NewEndpoint failed: %v", err)
@@ -887,7 +888,7 @@ func (r *RawEndpoint) VerifyACKHasSACK(sackBlocks []header.SACKBlock) {
// It also verifies where required(eg.Timestamp) that the ACK to the SYN-ACK
// does not carry an option that was not requested.
func (c *Context) CreateConnectedWithOptions(wantOptions header.TCPSynOptions) *RawEndpoint {
- var err *tcpip.Error
+ var err tcpip.Error
c.EP, err = c.s.NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
if err != nil {
c.t.Fatalf("c.s.NewEndpoint(tcp, ipv4...) = %v", err)
@@ -903,7 +904,7 @@ func (c *Context) CreateConnectedWithOptions(wantOptions header.TCPSynOptions) *
testFullAddr := tcpip.FullAddress{Addr: TestAddr, Port: TestPort}
err = c.EP.Connect(testFullAddr)
- if err != tcpip.ErrConnectStarted {
+ if _, ok := err.(*tcpip.ErrConnectStarted); !ok {
c.t.Fatalf("c.ep.Connect(%v) = %v", testFullAddr, err)
}
// Receive SYN packet.
@@ -1054,7 +1055,7 @@ func (c *Context) AcceptWithOptions(wndScale int, synOptions header.TCPSynOption
defer wq.EventUnregister(&we)
c.EP, _, err = ep.Accept(nil)
- if err == tcpip.ErrWouldBlock {
+ if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
select {
case <-ch: