summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/transport
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/tcpip/transport')
-rw-r--r--pkg/tcpip/transport/tcp/accept.go91
-rw-r--r--pkg/tcpip/transport/tcp/connect.go20
-rw-r--r--pkg/tcpip/transport/tcp/dual_stack_test.go14
-rw-r--r--pkg/tcpip/transport/tcp/endpoint.go14
-rw-r--r--pkg/tcpip/transport/tcp/protocol.go77
-rw-r--r--pkg/tcpip/transport/tcp/tcp_sack_test.go14
-rw-r--r--pkg/tcpip/transport/tcp/tcp_test.go85
-rw-r--r--pkg/tcpip/transport/tcp/tcp_timestamp_test.go8
8 files changed, 126 insertions, 197 deletions
diff --git a/pkg/tcpip/transport/tcp/accept.go b/pkg/tcpip/transport/tcp/accept.go
index 025b134e2..7372ebc08 100644
--- a/pkg/tcpip/transport/tcp/accept.go
+++ b/pkg/tcpip/transport/tcp/accept.go
@@ -51,11 +51,6 @@ const (
// timestamp and the current timestamp. If the difference is greater
// than maxTSDiff, the cookie is expired.
maxTSDiff = 2
-
- // SynRcvdCountThreshold is the default global maximum number of
- // connections that are allowed to be in SYN-RCVD state before TCP
- // starts using SYN cookies to accept connections.
- SynRcvdCountThreshold uint64 = 1000
)
var (
@@ -80,9 +75,6 @@ func encodeMSS(mss uint16) uint32 {
type listenContext struct {
stack *stack.Stack
- // synRcvdCount is a reference to the stack level synRcvdCount.
- synRcvdCount *synRcvdCounter
-
// rcvWnd is the receive window that is sent by this listening context
// in the initial SYN-ACK.
rcvWnd seqnum.Size
@@ -138,11 +130,6 @@ func newListenContext(stk *stack.Stack, listenEP *endpoint, rcvWnd seqnum.Size,
listenEP: listenEP,
pendingEndpoints: make(map[stack.TransportEndpointID]*endpoint),
}
- p, ok := stk.TransportProtocolInstance(ProtocolNumber).(*protocol)
- if !ok {
- panic(fmt.Sprintf("unable to get TCP protocol instance from stack: %+v", stk))
- }
- l.synRcvdCount = p.SynRcvdCounter()
rand.Read(l.nonce[0][:])
rand.Read(l.nonce[1][:])
@@ -199,6 +186,14 @@ func (l *listenContext) isCookieValid(id stack.TransportEndpointID, cookie seqnu
return (v - l.cookieHash(id, cookieTS, 1)) & hashMask, true
}
+func (l *listenContext) useSynCookies() bool {
+ var alwaysUseSynCookies tcpip.TCPAlwaysUseSynCookies
+ if err := l.stack.TransportProtocolOption(header.TCPProtocolNumber, &alwaysUseSynCookies); err != nil {
+ panic(fmt.Sprintf("TransportProtocolOption(%d, %T) = %s", header.TCPProtocolNumber, alwaysUseSynCookies, err))
+ }
+ return bool(alwaysUseSynCookies) || (l.listenEP != nil && l.listenEP.synRcvdBacklogFull())
+}
+
// 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) {
@@ -307,6 +302,7 @@ func (l *listenContext) startHandshake(s *segment, opts *header.TCPSynOptions, q
// Initialize and start the handshake.
h := ep.newPassiveHandshake(isn, irs, opts, deferAccept)
+ h.listenEP = l.listenEP
h.start()
return h, nil
}
@@ -485,7 +481,6 @@ func (e *endpoint) handleSynSegment(ctx *listenContext, s *segment, opts *header
}
go func() {
- defer ctx.synRcvdCount.dec()
if err := h.complete(); err != nil {
e.stack.Stats().TCP.FailedConnectionAttempts.Increment()
e.stats.FailedConnectionAttempts.Increment()
@@ -497,24 +492,29 @@ func (e *endpoint) handleSynSegment(ctx *listenContext, s *segment, opts *header
h.ep.startAcceptedLoop()
e.stack.Stats().TCP.PassiveConnectionOpenings.Increment()
e.deliverAccepted(h.ep, false /*withSynCookie*/)
- }() // S/R-SAFE: synRcvdCount is the barrier.
+ }()
return nil
}
-func (e *endpoint) incSynRcvdCount() bool {
+func (e *endpoint) synRcvdBacklogFull() bool {
e.acceptMu.Lock()
- canInc := int(atomic.LoadInt32(&e.synRcvdCount)) < cap(e.acceptedChan)
+ acceptedChanCap := cap(e.acceptedChan)
e.acceptMu.Unlock()
- if canInc {
- atomic.AddInt32(&e.synRcvdCount, 1)
- }
- return canInc
+ // The allocated accepted channel size would always be one greater than the
+ // listen backlog. But, the SYNRCVD connections count is always checked
+ // against the listen backlog value for Linux parity reason.
+ // https://github.com/torvalds/linux/blob/7acac4b3196/include/net/inet_connection_sock.h#L280
+ //
+ // We maintain an equality check here as the synRcvdCount is incremented
+ // and compared only from a single listener context and the capacity of
+ // the accepted channel can only increase by a new listen call.
+ return int(atomic.LoadInt32(&e.synRcvdCount)) == acceptedChanCap-1
}
func (e *endpoint) acceptQueueIsFull() bool {
e.acceptMu.Lock()
- full := len(e.acceptedChan)+int(atomic.LoadInt32(&e.synRcvdCount)) >= cap(e.acceptedChan)
+ full := len(e.acceptedChan) == cap(e.acceptedChan)
e.acceptMu.Unlock()
return full
}
@@ -539,17 +539,13 @@ func (e *endpoint) handleListenSegment(ctx *listenContext, s *segment) tcpip.Err
switch {
case s.flags == header.TCPFlagSyn:
opts := parseSynSegmentOptions(s)
- if ctx.synRcvdCount.inc() {
- // Only handle the syn if the following conditions hold
- // - accept queue is not full.
- // - number of connections in synRcvd state is less than the
- // backlog.
- if !e.acceptQueueIsFull() && e.incSynRcvdCount() {
+ if !ctx.useSynCookies() {
+ if !e.acceptQueueIsFull() {
s.incRef()
+ atomic.AddInt32(&e.synRcvdCount, 1)
_ = e.handleSynSegment(ctx, s, &opts)
return nil
}
- ctx.synRcvdCount.dec()
e.stack.Stats().TCP.ListenOverflowSynDrop.Increment()
e.stats.ReceiveErrors.ListenOverflowSynDrop.Increment()
e.stack.Stats().DroppedPackets.Increment()
@@ -615,25 +611,6 @@ func (e *endpoint) handleListenSegment(ctx *listenContext, s *segment) tcpip.Err
return nil
}
- if !ctx.synRcvdCount.synCookiesInUse() {
- // When not using SYN cookies, as per RFC 793, section 3.9, page 64:
- // Any acknowledgment is bad if it arrives on a connection still in
- // the LISTEN state. An acceptable reset segment should be formed
- // for any arriving ACK-bearing segment. The RST should be
- // formatted as follows:
- //
- // <SEQ=SEG.ACK><CTL=RST>
- //
- // Send a reset as this is an ACK for which there is no
- // half open connections and we are not using cookies
- // yet.
- //
- // The only time we should reach here when a connection
- // was opened and closed really quickly and a delayed
- // ACK was received from the sender.
- return replyWithReset(e.stack, s, e.sendTOS, e.ttl)
- }
-
iss := s.ackNumber - 1
irs := s.sequenceNumber - 1
@@ -651,7 +628,23 @@ func (e *endpoint) handleListenSegment(ctx *listenContext, s *segment) tcpip.Err
if !ok || int(data) >= len(mssTable) {
e.stack.Stats().TCP.ListenOverflowInvalidSynCookieRcvd.Increment()
e.stack.Stats().DroppedPackets.Increment()
- return nil
+
+ // When not using SYN cookies, as per RFC 793, section 3.9, page 64:
+ // Any acknowledgment is bad if it arrives on a connection still in
+ // the LISTEN state. An acceptable reset segment should be formed
+ // for any arriving ACK-bearing segment. The RST should be
+ // formatted as follows:
+ //
+ // <SEQ=SEG.ACK><CTL=RST>
+ //
+ // Send a reset as this is an ACK for which there is no
+ // half open connections and we are not using cookies
+ // yet.
+ //
+ // The only time we should reach here when a connection
+ // was opened and closed really quickly and a delayed
+ // ACK was received from the sender.
+ return replyWithReset(e.stack, s, e.sendTOS, e.ttl)
}
e.stack.Stats().TCP.ListenOverflowSynCookieRcvd.Increment()
// Create newly accepted endpoint and deliver it.
diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go
index a9e978cf6..8f0f0c3e9 100644
--- a/pkg/tcpip/transport/tcp/connect.go
+++ b/pkg/tcpip/transport/tcp/connect.go
@@ -65,11 +65,12 @@ const (
// NOTE: handshake.ep.mu is held during handshake processing. It is released if
// we are going to block and reacquired when we start processing an event.
type handshake struct {
- ep *endpoint
- state handshakeState
- active bool
- flags header.TCPFlags
- ackNum seqnum.Value
+ ep *endpoint
+ listenEP *endpoint
+ state handshakeState
+ active bool
+ flags header.TCPFlags
+ ackNum seqnum.Value
// iss is the initial send sequence number, as defined in RFC 793.
iss seqnum.Value
@@ -394,6 +395,15 @@ func (h *handshake) synRcvdState(s *segment) tcpip.Error {
return nil
}
+ // Drop the ACK if the accept queue is full.
+ // https://github.com/torvalds/linux/blob/7acac4b3196/net/ipv4/tcp_ipv4.c#L1523
+ // We could abort the connection as well with a tunable as in
+ // https://github.com/torvalds/linux/blob/7acac4b3196/net/ipv4/tcp_minisocks.c#L788
+ if listenEP := h.listenEP; listenEP != nil && listenEP.acceptQueueIsFull() {
+ listenEP.stack.Stats().DroppedPackets.Increment()
+ return nil
+ }
+
// Update timestamp if required. See RFC7323, section-4.3.
if h.ep.sendTSOk && s.parsedOptions.TS {
h.ep.updateRecentTimestamp(s.parsedOptions.TSVal, h.ackNum, s.sequenceNumber)
diff --git a/pkg/tcpip/transport/tcp/dual_stack_test.go b/pkg/tcpip/transport/tcp/dual_stack_test.go
index f6a16f96e..d6d68f128 100644
--- a/pkg/tcpip/transport/tcp/dual_stack_test.go
+++ b/pkg/tcpip/transport/tcp/dual_stack_test.go
@@ -565,17 +565,15 @@ func TestV4AcceptOnV4(t *testing.T) {
}
func testV4ListenClose(t *testing.T, c *context.Context) {
- // Set the SynRcvd threshold to zero to force a syn cookie based accept
- // to happen.
- var opt tcpip.TCPSynRcvdCountThresholdOption
+ opt := tcpip.TCPAlwaysUseSynCookies(true)
if err := c.Stack().SetTransportProtocolOption(tcp.ProtocolNumber, &opt); err != nil {
- t.Fatalf("setting TCPSynRcvdCountThresholdOption(%d, &%T(%d)): %s", tcp.ProtocolNumber, opt, opt, err)
+ t.Fatalf("SetTransportProtocolOption(%d, &%T(%t)): %s", tcp.ProtocolNumber, opt, opt, err)
}
- const n = uint16(32)
+ const n = 32
// Start listening.
- if err := c.EP.Listen(int(tcp.SynRcvdCountThreshold + 1)); err != nil {
+ if err := c.EP.Listen(n); err != nil {
t.Fatalf("Listen failed: %v", err)
}
@@ -591,9 +589,9 @@ func testV4ListenClose(t *testing.T, c *context.Context) {
})
}
- // Each of these ACK's will cause a syn-cookie based connection to be
+ // Each of these ACKs will cause a syn-cookie based connection to be
// accepted and delivered to the listening endpoint.
- for i := uint16(0); i < n; i++ {
+ for i := 0; i < n; i++ {
b := c.GetPacket()
tcp := header.TCP(header.IPv4(b).Payload())
iss := seqnum.Value(tcp.SequenceNumber())
diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go
index c5daba232..9438056f9 100644
--- a/pkg/tcpip/transport/tcp/endpoint.go
+++ b/pkg/tcpip/transport/tcp/endpoint.go
@@ -2474,6 +2474,20 @@ func (e *endpoint) shutdownLocked(flags tcpip.ShutdownFlags) tcpip.Error {
// Listen puts the endpoint in "listen" mode, which allows it to accept
// new connections.
func (e *endpoint) Listen(backlog int) tcpip.Error {
+ if uint32(backlog) > MaxListenBacklog {
+ // Linux treats incoming backlog as uint with a limit defined by
+ // sysctl_somaxconn.
+ // https://github.com/torvalds/linux/blob/7acac4b3196/net/socket.c#L1666
+ //
+ // We use the backlog to allocate a channel of that size, hence enforce
+ // a hard limit for the backlog.
+ backlog = MaxListenBacklog
+ } else {
+ // Accept one more than the configured listen backlog to keep in parity with
+ // Linux. Ref, because of missing equality check here:
+ // https://github.com/torvalds/linux/blob/7acac4b3196/include/net/sock.h#L937
+ backlog++
+ }
err := e.listen(backlog)
if err != nil {
if !err.IgnoreStats() {
diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go
index 2a4667906..230fa6ebe 100644
--- a/pkg/tcpip/transport/tcp/protocol.go
+++ b/pkg/tcpip/transport/tcp/protocol.go
@@ -68,6 +68,9 @@ const (
// DefaultSynRetries is the default value for the number of SYN retransmits
// before a connect is aborted.
DefaultSynRetries = 6
+
+ // MaxListenBacklog is the maximum limit of listen backlog supported.
+ MaxListenBacklog = 1024
)
const (
@@ -75,63 +78,6 @@ const (
ccCubic = "cubic"
)
-// syncRcvdCounter tracks the number of endpoints in the SYN-RCVD state. The
-// value is protected by a mutex so that we can increment only when it's
-// guaranteed not to go above a threshold.
-type synRcvdCounter struct {
- sync.Mutex
- value uint64
- pending sync.WaitGroup
- threshold uint64
-}
-
-// inc tries to increment the global number of endpoints in SYN-RCVD state. It
-// succeeds if the increment doesn't make the count go beyond the threshold, and
-// fails otherwise.
-func (s *synRcvdCounter) inc() bool {
- s.Lock()
- defer s.Unlock()
- if s.value >= s.threshold {
- return false
- }
-
- s.pending.Add(1)
- s.value++
-
- return true
-}
-
-// dec atomically decrements the global number of endpoints in SYN-RCVD
-// state. It must only be called if a previous call to inc succeeded.
-func (s *synRcvdCounter) dec() {
- s.Lock()
- defer s.Unlock()
- s.value--
- s.pending.Done()
-}
-
-// synCookiesInUse returns true if the synRcvdCount is greater than
-// SynRcvdCountThreshold.
-func (s *synRcvdCounter) synCookiesInUse() bool {
- s.Lock()
- defer s.Unlock()
- return s.value >= s.threshold
-}
-
-// SetThreshold sets synRcvdCounter.Threshold to ths new threshold.
-func (s *synRcvdCounter) SetThreshold(threshold uint64) {
- s.Lock()
- defer s.Unlock()
- s.threshold = threshold
-}
-
-// Threshold returns the current value of synRcvdCounter.Threhsold.
-func (s *synRcvdCounter) Threshold() uint64 {
- s.Lock()
- defer s.Unlock()
- return s.threshold
-}
-
type protocol struct {
stack *stack.Stack
@@ -139,6 +85,7 @@ type protocol struct {
sackEnabled bool
recovery tcpip.TCPRecovery
delayEnabled bool
+ alwaysUseSynCookies bool
sendBufferSize tcpip.TCPSendBufferSizeRangeOption
recvBufferSize tcpip.TCPReceiveBufferSizeRangeOption
congestionControl string
@@ -150,7 +97,6 @@ type protocol struct {
minRTO time.Duration
maxRTO time.Duration
maxRetries uint32
- synRcvdCount synRcvdCounter
synRetries uint8
dispatcher dispatcher
}
@@ -373,9 +319,9 @@ func (p *protocol) SetOption(option tcpip.SettableTransportProtocolOption) tcpip
p.mu.Unlock()
return nil
- case *tcpip.TCPSynRcvdCountThresholdOption:
+ case *tcpip.TCPAlwaysUseSynCookies:
p.mu.Lock()
- p.synRcvdCount.SetThreshold(uint64(*v))
+ p.alwaysUseSynCookies = bool(*v)
p.mu.Unlock()
return nil
@@ -480,9 +426,9 @@ func (p *protocol) Option(option tcpip.GettableTransportProtocolOption) tcpip.Er
p.mu.RUnlock()
return nil
- case *tcpip.TCPSynRcvdCountThresholdOption:
+ case *tcpip.TCPAlwaysUseSynCookies:
p.mu.RLock()
- *v = tcpip.TCPSynRcvdCountThresholdOption(p.synRcvdCount.Threshold())
+ *v = tcpip.TCPAlwaysUseSynCookies(p.alwaysUseSynCookies)
p.mu.RUnlock()
return nil
@@ -507,12 +453,6 @@ func (p *protocol) Wait() {
p.dispatcher.wait()
}
-// SynRcvdCounter returns a reference to the synRcvdCount for this protocol
-// instance.
-func (p *protocol) SynRcvdCounter() *synRcvdCounter {
- return &p.synRcvdCount
-}
-
// Parse implements stack.TransportProtocol.Parse.
func (*protocol) Parse(pkt *stack.PacketBuffer) bool {
return parse.TCP(pkt)
@@ -537,7 +477,6 @@ func NewProtocol(s *stack.Stack) stack.TransportProtocol {
lingerTimeout: DefaultTCPLingerTimeout,
timeWaitTimeout: DefaultTCPTimeWaitTimeout,
timeWaitReuse: tcpip.TCPTimeWaitReuseLoopbackOnly,
- synRcvdCount: synRcvdCounter{threshold: SynRcvdCountThreshold},
synRetries: DefaultSynRetries,
minRTO: MinRTO,
maxRTO: MaxRTO,
diff --git a/pkg/tcpip/transport/tcp/tcp_sack_test.go b/pkg/tcpip/transport/tcp/tcp_sack_test.go
index 81f800cad..20c9761f2 100644
--- a/pkg/tcpip/transport/tcp/tcp_sack_test.go
+++ b/pkg/tcpip/transport/tcp/tcp_sack_test.go
@@ -160,12 +160,9 @@ func TestSackPermittedAccept(t *testing.T) {
defer c.Cleanup()
if tc.cookieEnabled {
- // Set the SynRcvd threshold to
- // zero to force a syn cookie
- // based accept to happen.
- var opt tcpip.TCPSynRcvdCountThresholdOption
+ opt := tcpip.TCPAlwaysUseSynCookies(true)
if err := c.Stack().SetTransportProtocolOption(tcp.ProtocolNumber, &opt); err != nil {
- t.Fatalf("SetTransportProtocolOption(%d, &%T(%d)): %s", tcp.ProtocolNumber, opt, opt, err)
+ t.Fatalf("SetTransportProtocolOption(%d, &%T(%t)): %s", tcp.ProtocolNumber, opt, opt, err)
}
}
setStackSACKPermitted(t, c, sackEnabled)
@@ -235,12 +232,9 @@ func TestSackDisabledAccept(t *testing.T) {
defer c.Cleanup()
if tc.cookieEnabled {
- // Set the SynRcvd threshold to
- // zero to force a syn cookie
- // based accept to happen.
- var opt tcpip.TCPSynRcvdCountThresholdOption
+ opt := tcpip.TCPAlwaysUseSynCookies(true)
if err := c.Stack().SetTransportProtocolOption(tcp.ProtocolNumber, &opt); err != nil {
- t.Fatalf("SetTransportProtocolOption(%d, &%T(%d)): %s", tcp.ProtocolNumber, opt, opt, err)
+ t.Fatalf("SetTransportProtocolOption(%d, &%T(%t)): %s", tcp.ProtocolNumber, opt, opt, err)
}
}
diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go
index 9c23469f2..5605a4390 100644
--- a/pkg/tcpip/transport/tcp/tcp_test.go
+++ b/pkg/tcpip/transport/tcp/tcp_test.go
@@ -955,11 +955,7 @@ func TestUserSuppliedMSSOnConnect(t *testing.T) {
// when completing the handshake for a new TCP connection from a TCP
// listening socket. It should be present in the sent TCP SYN-ACK segment.
func TestUserSuppliedMSSOnListenAccept(t *testing.T) {
- const (
- nonSynCookieAccepts = 2
- totalAccepts = 4
- mtu = 5000
- )
+ const mtu = 5000
ips := []struct {
name string
@@ -1033,12 +1029,6 @@ func TestUserSuppliedMSSOnListenAccept(t *testing.T) {
ip.createEP(c)
- // Set the SynRcvd threshold to force a syn cookie based accept to happen.
- opt := tcpip.TCPSynRcvdCountThresholdOption(nonSynCookieAccepts)
- if err := c.Stack().SetTransportProtocolOption(tcp.ProtocolNumber, &opt); err != nil {
- t.Fatalf("SetTransportProtocolOption(%d, &%T(%d)): %s", tcp.ProtocolNumber, opt, opt, err)
- }
-
if err := c.EP.SetSockOptInt(tcpip.MaxSegOption, int(test.setMSS)); err != nil {
t.Fatalf("SetSockOptInt(MaxSegOption, %d): %s", test.setMSS, err)
}
@@ -1048,13 +1038,17 @@ func TestUserSuppliedMSSOnListenAccept(t *testing.T) {
t.Fatalf("Bind(%+v): %s:", bindAddr, err)
}
- if err := c.EP.Listen(totalAccepts); err != nil {
- t.Fatalf("Listen(%d): %s:", totalAccepts, err)
+ backlog := 5
+ // Keep the number of client requests twice to the backlog
+ // such that half of the connections do not use syncookies
+ // and the other half does.
+ clientConnects := backlog * 2
+
+ if err := c.EP.Listen(backlog); err != nil {
+ t.Fatalf("Listen(%d): %s:", backlog, err)
}
- // The first nonSynCookieAccepts packets sent will trigger a gorooutine
- // based accept. The rest will trigger a cookie based accept.
- for i := 0; i < totalAccepts; i++ {
+ for i := 0; i < clientConnects; i++ {
// Send a SYN requests.
iss := seqnum.Value(i)
srcPort := context.TestPort + uint16(i)
@@ -3087,11 +3081,9 @@ func TestSynCookiePassiveSendMSSLessThanMTU(t *testing.T) {
c := context.New(t, mtu)
defer c.Cleanup()
- // Set the SynRcvd threshold to zero to force a syn cookie based accept
- // to happen.
- opt := tcpip.TCPSynRcvdCountThresholdOption(0)
+ opt := tcpip.TCPAlwaysUseSynCookies(true)
if err := c.Stack().SetTransportProtocolOption(tcp.ProtocolNumber, &opt); err != nil {
- t.Fatalf("SetTransportProtocolOption(%d, &%T(%d)): %s", tcp.ProtocolNumber, opt, opt, err)
+ t.Fatalf("SetTransportProtocolOption(%d, &%T(%t)): %s", tcp.ProtocolNumber, opt, opt, err)
}
// Create EP and start listening.
@@ -5363,7 +5355,7 @@ func TestListenBacklogFull(t *testing.T) {
}
lastPortOffset := uint16(0)
- for ; int(lastPortOffset) < listenBacklog; lastPortOffset++ {
+ for ; int(lastPortOffset) < listenBacklog+1; lastPortOffset++ {
executeHandshake(t, c, context.TestPort+lastPortOffset, false /*synCookieInUse */)
}
@@ -5671,15 +5663,13 @@ func TestListenSynRcvdQueueFull(t *testing.T) {
}
// Test acceptance.
- // Start listening.
- listenBacklog := 1
- if err := c.EP.Listen(listenBacklog); err != nil {
+ if err := c.EP.Listen(0); err != nil {
t.Fatalf("Listen failed: %s", err)
}
// Send two SYN's the first one should get a SYN-ACK, the
// second one should not get any response and is dropped as
- // the synRcvd count will be equal to backlog.
+ // the accept queue is full.
irs := seqnum.Value(context.TestInitialSequenceNumber)
c.SendPacket(nil, &context.Headers{
SrcPort: context.TestPort,
@@ -5701,23 +5691,7 @@ func TestListenSynRcvdQueueFull(t *testing.T) {
}
checker.IPv4(t, b, checker.TCP(tcpCheckers...))
- // Now execute send one more SYN. The stack should not respond as the backlog
- // is full at this point.
- //
- // NOTE: we did not complete the handshake for the previous one so the
- // accept backlog should be empty and there should be one connection in
- // synRcvd state.
- c.SendPacket(nil, &context.Headers{
- SrcPort: context.TestPort + 1,
- DstPort: context.StackPort,
- Flags: header.TCPFlagSyn,
- SeqNum: seqnum.Value(889),
- RcvWnd: 30000,
- })
- c.CheckNoPacketTimeout("unexpected packet received", 50*time.Millisecond)
-
- // Now complete the previous connection and verify that there is a connection
- // to accept.
+ // Now complete the previous connection.
// Send ACK.
c.SendPacket(nil, &context.Headers{
SrcPort: context.TestPort,
@@ -5728,11 +5702,24 @@ func TestListenSynRcvdQueueFull(t *testing.T) {
RcvWnd: 30000,
})
- // Try to accept the connections in the backlog.
+ // Verify if that is delivered to the accept queue.
we, ch := waiter.NewChannelEntry(nil)
c.WQ.EventRegister(&we, waiter.ReadableEvents)
defer c.WQ.EventUnregister(&we)
+ <-ch
+
+ // Now execute send one more SYN. The stack should not respond as the backlog
+ // is full at this point.
+ c.SendPacket(nil, &context.Headers{
+ SrcPort: context.TestPort + 1,
+ DstPort: context.StackPort,
+ Flags: header.TCPFlagSyn,
+ SeqNum: seqnum.Value(889),
+ RcvWnd: 30000,
+ })
+ c.CheckNoPacketTimeout("unexpected packet received", 50*time.Millisecond)
+ // Try to accept the connections in the backlog.
newEP, _, err := c.EP.Accept(nil)
if _, ok := err.(*tcpip.ErrWouldBlock); ok {
// Wait for connection to be established.
@@ -5764,11 +5751,6 @@ func TestListenBacklogFullSynCookieInUse(t *testing.T) {
c := context.New(t, defaultMTU)
defer c.Cleanup()
- opt := tcpip.TCPSynRcvdCountThresholdOption(1)
- if err := c.Stack().SetTransportProtocolOption(tcp.ProtocolNumber, &opt); err != nil {
- t.Fatalf("SetTransportProtocolOption(%d, &%T(%d)): %s", tcp.ProtocolNumber, opt, opt, err)
- }
-
// Create TCP endpoint.
var err tcpip.Error
c.EP, err = c.Stack().NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &c.WQ)
@@ -5781,9 +5763,8 @@ func TestListenBacklogFullSynCookieInUse(t *testing.T) {
t.Fatalf("Bind failed: %s", err)
}
- // Start listening.
- listenBacklog := 1
- if err := c.EP.Listen(listenBacklog); err != nil {
+ // Test for SynCookies usage after filling up the backlog.
+ if err := c.EP.Listen(0); err != nil {
t.Fatalf("Listen failed: %s", err)
}
@@ -6066,7 +6047,7 @@ func TestPassiveFailedConnectionAttemptIncrement(t *testing.T) {
if err := c.EP.Bind(tcpip.FullAddress{Addr: context.StackAddr, Port: context.StackPort}); err != nil {
t.Fatalf("Bind failed: %s", err)
}
- if err := c.EP.Listen(1); err != nil {
+ if err := c.EP.Listen(0); err != nil {
t.Fatalf("Listen failed: %s", err)
}
diff --git a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go
index 2949588ce..1deb1fe4d 100644
--- a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go
+++ b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go
@@ -139,9 +139,9 @@ func timeStampEnabledAccept(t *testing.T, cookieEnabled bool, wndScale int, wndS
defer c.Cleanup()
if cookieEnabled {
- var opt tcpip.TCPSynRcvdCountThresholdOption
+ opt := tcpip.TCPAlwaysUseSynCookies(true)
if err := c.Stack().SetTransportProtocolOption(tcp.ProtocolNumber, &opt); err != nil {
- t.Fatalf("SetTransportProtocolOption(%d, &%T(%d)): %s", tcp.ProtocolNumber, opt, opt, err)
+ t.Fatalf("SetTransportProtocolOption(%d, &%T(%t)): %s", tcp.ProtocolNumber, opt, opt, err)
}
}
@@ -202,9 +202,9 @@ func timeStampDisabledAccept(t *testing.T, cookieEnabled bool, wndScale int, wnd
defer c.Cleanup()
if cookieEnabled {
- var opt tcpip.TCPSynRcvdCountThresholdOption
+ opt := tcpip.TCPAlwaysUseSynCookies(true)
if err := c.Stack().SetTransportProtocolOption(tcp.ProtocolNumber, &opt); err != nil {
- t.Fatalf("SetTransportProtocolOption(%d, &%T(%d)): %s", tcp.ProtocolNumber, opt, opt, err)
+ t.Fatalf("SetTransportProtocolOption(%d, &%T(%t)): %s", tcp.ProtocolNumber, opt, opt, err)
}
}