diff options
author | gVisor bot <gvisor-bot@google.com> | 2021-04-06 04:58:10 +0000 |
---|---|---|
committer | gVisor bot <gvisor-bot@google.com> | 2021-04-06 04:58:10 +0000 |
commit | 50251104a2c8b0752d43707a759e7f984999419f (patch) | |
tree | 221de41c6fe69fc07c7e2837479fc8d8d4709ad9 /pkg/tcpip | |
parent | 3de9e8a7a5097f030dfedd2ea8e2f877a962d1e2 (diff) | |
parent | 56c69fb0e7f96e5bd5da9e0d29a78e05dd3e2bba (diff) |
Merge release-20210322.0-51-g56c69fb0e (automated)
Diffstat (limited to 'pkg/tcpip')
-rw-r--r-- | pkg/tcpip/tcpip.go | 20 | ||||
-rw-r--r-- | pkg/tcpip/transport/tcp/accept.go | 91 | ||||
-rw-r--r-- | pkg/tcpip/transport/tcp/connect.go | 20 | ||||
-rw-r--r-- | pkg/tcpip/transport/tcp/endpoint.go | 14 | ||||
-rw-r--r-- | pkg/tcpip/transport/tcp/protocol.go | 77 |
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, |