summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/tcpip')
-rw-r--r--pkg/tcpip/tcpip.go20
-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/endpoint.go14
-rw-r--r--pkg/tcpip/transport/tcp/protocol.go77
5 files changed, 86 insertions, 136 deletions
diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go
index 87ea09a5e..60de16579 100644
--- a/pkg/tcpip/tcpip.go
+++ b/pkg/tcpip/tcpip.go
@@ -786,6 +786,13 @@ func (*TCPRecovery) isGettableTransportProtocolOption() {}
func (*TCPRecovery) isSettableTransportProtocolOption() {}
+// TCPAlwaysUseSynCookies indicates unconditional usage of syncookies.
+type TCPAlwaysUseSynCookies bool
+
+func (*TCPAlwaysUseSynCookies) isGettableTransportProtocolOption() {}
+
+func (*TCPAlwaysUseSynCookies) isSettableTransportProtocolOption() {}
+
const (
// TCPRACKLossDetection indicates RACK is used for loss detection and
// recovery.
@@ -1020,19 +1027,6 @@ func (*TCPMaxRetriesOption) isGettableTransportProtocolOption() {}
func (*TCPMaxRetriesOption) isSettableTransportProtocolOption() {}
-// TCPSynRcvdCountThresholdOption is used by SetSockOpt/GetSockOpt to specify
-// the number of endpoints that can be in SYN-RCVD state before the stack
-// switches to using SYN cookies.
-type TCPSynRcvdCountThresholdOption uint64
-
-func (*TCPSynRcvdCountThresholdOption) isGettableSocketOption() {}
-
-func (*TCPSynRcvdCountThresholdOption) isSettableSocketOption() {}
-
-func (*TCPSynRcvdCountThresholdOption) isGettableTransportProtocolOption() {}
-
-func (*TCPSynRcvdCountThresholdOption) isSettableTransportProtocolOption() {}
-
// TCPSynRetriesOption is used by SetSockOpt/GetSockOpt to specify stack-wide
// default for number of times SYN is retransmitted before aborting a connect.
type TCPSynRetriesOption uint8
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/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,