summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/transport
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/tcpip/transport')
-rw-r--r--pkg/tcpip/transport/tcp/BUILD5
-rw-r--r--pkg/tcpip/transport/tcp/protocol.go17
-rw-r--r--pkg/tcpip/transport/tcp/snd.go15
-rw-r--r--pkg/tcpip/transport/tcp/tcp_noracedetector_test.go83
-rw-r--r--pkg/tcpip/transport/tcp/tcp_sack_test.go2
-rw-r--r--pkg/tcpip/transport/tcp/tcp_test.go50
-rw-r--r--pkg/tcpip/transport/tcp/testing/context/context.go17
7 files changed, 130 insertions, 59 deletions
diff --git a/pkg/tcpip/transport/tcp/BUILD b/pkg/tcpip/transport/tcp/BUILD
index 7f94f9646..edb7718a6 100644
--- a/pkg/tcpip/transport/tcp/BUILD
+++ b/pkg/tcpip/transport/tcp/BUILD
@@ -87,7 +87,9 @@ go_test(
"tcp_timestamp_test.go",
],
# FIXME(b/68809571)
- tags = ["flaky"],
+ tags = [
+ "flaky",
+ ],
deps = [
":tcp",
"//pkg/sync",
@@ -104,5 +106,6 @@ go_test(
"//pkg/tcpip/stack",
"//pkg/tcpip/transport/tcp/testing/context",
"//pkg/waiter",
+ "//runsc/testutil",
],
)
diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go
index dce9a1652..91f25c132 100644
--- a/pkg/tcpip/transport/tcp/protocol.go
+++ b/pkg/tcpip/transport/tcp/protocol.go
@@ -105,6 +105,7 @@ type protocol struct {
moderateReceiveBuffer bool
tcpLingerTimeout time.Duration
tcpTimeWaitTimeout time.Duration
+ minRTO time.Duration
dispatcher *dispatcher
}
@@ -272,6 +273,15 @@ func (p *protocol) SetOption(option interface{}) *tcpip.Error {
p.mu.Unlock()
return nil
+ case tcpip.TCPMinRTOOption:
+ if v < 0 {
+ v = tcpip.TCPMinRTOOption(MinRTO)
+ }
+ p.mu.Lock()
+ p.minRTO = time.Duration(v)
+ p.mu.Unlock()
+ return nil
+
default:
return tcpip.ErrUnknownProtocolOption
}
@@ -334,6 +344,12 @@ func (p *protocol) Option(option interface{}) *tcpip.Error {
p.mu.RUnlock()
return nil
+ case *tcpip.TCPMinRTOOption:
+ p.mu.RLock()
+ *v = tcpip.TCPMinRTOOption(p.minRTO)
+ p.mu.RUnlock()
+ return nil
+
default:
return tcpip.ErrUnknownProtocolOption
}
@@ -359,5 +375,6 @@ func NewProtocol() stack.TransportProtocol {
tcpLingerTimeout: DefaultTCPLingerTimeout,
tcpTimeWaitTimeout: DefaultTCPTimeWaitTimeout,
dispatcher: newDispatcher(runtime.GOMAXPROCS(0)),
+ minRTO: MinRTO,
}
}
diff --git a/pkg/tcpip/transport/tcp/snd.go b/pkg/tcpip/transport/tcp/snd.go
index 6b7bac37d..d8cfe3115 100644
--- a/pkg/tcpip/transport/tcp/snd.go
+++ b/pkg/tcpip/transport/tcp/snd.go
@@ -15,6 +15,7 @@
package tcp
import (
+ "fmt"
"math"
"sync/atomic"
"time"
@@ -149,6 +150,9 @@ type sender struct {
rtt rtt
rto time.Duration
+ // minRTO is the minimum permitted value for sender.rto.
+ minRTO time.Duration
+
// maxPayloadSize is the maximum size of the payload of a given segment.
// It is initialized on demand.
maxPayloadSize int
@@ -260,6 +264,13 @@ func newSender(ep *endpoint, iss, irs seqnum.Value, sndWnd seqnum.Size, mss uint
// etc.
s.ep.scoreboard = NewSACKScoreboard(uint16(s.maxPayloadSize), iss)
+ // Get Stack wide minRTO.
+ var v tcpip.TCPMinRTOOption
+ if err := ep.stack.TransportProtocolOption(ProtocolNumber, &v); err != nil {
+ panic(fmt.Sprintf("unable to get minRTO from stack: %s", err))
+ }
+ s.minRTO = time.Duration(v)
+
return s
}
@@ -394,8 +405,8 @@ func (s *sender) updateRTO(rtt time.Duration) {
s.rto = s.rtt.srtt + 4*s.rtt.rttvar
s.rtt.Unlock()
- if s.rto < MinRTO {
- s.rto = MinRTO
+ if s.rto < s.minRTO {
+ s.rto = s.minRTO
}
}
diff --git a/pkg/tcpip/transport/tcp/tcp_noracedetector_test.go b/pkg/tcpip/transport/tcp/tcp_noracedetector_test.go
index 782d7b42c..359a75e73 100644
--- a/pkg/tcpip/transport/tcp/tcp_noracedetector_test.go
+++ b/pkg/tcpip/transport/tcp/tcp_noracedetector_test.go
@@ -31,6 +31,7 @@ import (
"gvisor.dev/gvisor/pkg/tcpip/header"
"gvisor.dev/gvisor/pkg/tcpip/transport/tcp"
"gvisor.dev/gvisor/pkg/tcpip/transport/tcp/testing/context"
+ "gvisor.dev/gvisor/runsc/testutil"
)
func TestFastRecovery(t *testing.T) {
@@ -40,7 +41,7 @@ func TestFastRecovery(t *testing.T) {
c.CreateConnected(789, 30000, -1 /* epRcvBuf */)
- const iterations = 7
+ const iterations = 3
data := buffer.NewView(2 * maxPayload * (tcp.InitialCwnd << (iterations + 1)))
for i := range data {
data[i] = byte(i)
@@ -86,16 +87,23 @@ func TestFastRecovery(t *testing.T) {
// Receive the retransmitted packet.
c.ReceiveAndCheckPacket(data, rtxOffset, maxPayload)
- if got, want := c.Stack().Stats().TCP.FastRetransmit.Value(), uint64(1); got != want {
- t.Errorf("got stats.TCP.FastRetransmit.Value = %v, want = %v", got, want)
- }
+ // Wait before checking metrics.
+ metricPollFn := func() error {
+ if got, want := c.Stack().Stats().TCP.FastRetransmit.Value(), uint64(1); got != want {
+ return fmt.Errorf("got stats.TCP.FastRetransmit.Value = %v, want = %v", got, want)
+ }
+ if got, want := c.Stack().Stats().TCP.Retransmits.Value(), uint64(1); got != want {
+ return fmt.Errorf("got stats.TCP.Retransmit.Value = %v, want = %v", got, want)
+ }
- if got, want := c.Stack().Stats().TCP.Retransmits.Value(), uint64(1); got != want {
- t.Errorf("got stats.TCP.Retransmit.Value = %v, want = %v", got, want)
+ if got, want := c.Stack().Stats().TCP.FastRecovery.Value(), uint64(1); got != want {
+ return fmt.Errorf("got stats.TCP.FastRecovery.Value = %v, want = %v", got, want)
+ }
+ return nil
}
- if got, want := c.Stack().Stats().TCP.FastRecovery.Value(), uint64(1); got != want {
- t.Errorf("got stats.TCP.FastRecovery.Value = %v, want = %v", got, want)
+ if err := testutil.Poll(metricPollFn, 1*time.Second); err != nil {
+ t.Error(err)
}
// Now send 7 mode duplicate acks. Each of these should cause a window
@@ -117,12 +125,18 @@ func TestFastRecovery(t *testing.T) {
// Receive the retransmit due to partial ack.
c.ReceiveAndCheckPacket(data, rtxOffset, maxPayload)
- if got, want := c.Stack().Stats().TCP.FastRetransmit.Value(), uint64(2); got != want {
- t.Errorf("got stats.TCP.FastRetransmit.Value = %v, want = %v", got, want)
+ // Wait before checking metrics.
+ metricPollFn = func() error {
+ if got, want := c.Stack().Stats().TCP.FastRetransmit.Value(), uint64(2); got != want {
+ return fmt.Errorf("got stats.TCP.FastRetransmit.Value = %v, want = %v", got, want)
+ }
+ if got, want := c.Stack().Stats().TCP.Retransmits.Value(), uint64(2); got != want {
+ return fmt.Errorf("got stats.TCP.Retransmit.Value = %v, want = %v", got, want)
+ }
+ return nil
}
-
- if got, want := c.Stack().Stats().TCP.Retransmits.Value(), uint64(2); got != want {
- t.Errorf("got stats.TCP.Retransmit.Value = %v, want = %v", got, want)
+ if err := testutil.Poll(metricPollFn, 1*time.Second); err != nil {
+ t.Error(err)
}
// Receive the 10 extra packets that should have been released due to
@@ -192,7 +206,7 @@ func TestExponentialIncreaseDuringSlowStart(t *testing.T) {
c.CreateConnected(789, 30000, -1 /* epRcvBuf */)
- const iterations = 7
+ const iterations = 3
data := buffer.NewView(maxPayload * (tcp.InitialCwnd << (iterations + 1)))
for i := range data {
data[i] = byte(i)
@@ -234,7 +248,7 @@ func TestCongestionAvoidance(t *testing.T) {
c.CreateConnected(789, 30000, -1 /* epRcvBuf */)
- const iterations = 7
+ const iterations = 3
data := buffer.NewView(2 * maxPayload * (tcp.InitialCwnd << (iterations + 1)))
for i := range data {
data[i] = byte(i)
@@ -338,7 +352,7 @@ func TestCubicCongestionAvoidance(t *testing.T) {
c.CreateConnected(789, 30000, -1 /* epRcvBuf */)
- const iterations = 7
+ const iterations = 3
data := buffer.NewView(2 * maxPayload * (tcp.InitialCwnd << (iterations + 1)))
for i := range data {
@@ -447,7 +461,7 @@ func TestRetransmit(t *testing.T) {
c.CreateConnected(789, 30000, -1 /* epRcvBuf */)
- const iterations = 7
+ const iterations = 3
data := buffer.NewView(maxPayload * (tcp.InitialCwnd << (iterations + 1)))
for i := range data {
data[i] = byte(i)
@@ -492,24 +506,33 @@ func TestRetransmit(t *testing.T) {
rtxOffset := bytesRead - maxPayload*expected
c.ReceiveAndCheckPacket(data, rtxOffset, maxPayload)
- if got, want := c.Stack().Stats().TCP.Timeouts.Value(), uint64(1); got != want {
- t.Errorf("got stats.TCP.Timeouts.Value = %v, want = %v", got, want)
- }
+ metricPollFn := func() error {
+ if got, want := c.Stack().Stats().TCP.Timeouts.Value(), uint64(1); got != want {
+ return fmt.Errorf("got stats.TCP.Timeouts.Value = %v, want = %v", got, want)
+ }
- if got, want := c.Stack().Stats().TCP.Retransmits.Value(), uint64(1); got != want {
- t.Errorf("got stats.TCP.Retransmits.Value = %v, want = %v", got, want)
- }
+ if got, want := c.Stack().Stats().TCP.Retransmits.Value(), uint64(1); got != want {
+ return fmt.Errorf("got stats.TCP.Retransmits.Value = %v, want = %v", got, want)
+ }
- if got, want := c.EP.Stats().(*tcp.Stats).SendErrors.Timeouts.Value(), uint64(1); got != want {
- t.Errorf("got EP SendErrors.Timeouts.Value = %v, want = %v", got, want)
- }
+ if got, want := c.EP.Stats().(*tcp.Stats).SendErrors.Timeouts.Value(), uint64(1); got != want {
+ return fmt.Errorf("got EP SendErrors.Timeouts.Value = %v, want = %v", got, want)
+ }
+
+ if got, want := c.EP.Stats().(*tcp.Stats).SendErrors.Retransmits.Value(), uint64(1); got != want {
+ return fmt.Errorf("got EP stats SendErrors.Retransmits.Value = %v, want = %v", got, want)
+ }
+
+ if got, want := c.Stack().Stats().TCP.SlowStartRetransmits.Value(), uint64(1); got != want {
+ return fmt.Errorf("got stats.TCP.SlowStartRetransmits.Value = %v, want = %v", got, want)
+ }
- if got, want := c.EP.Stats().(*tcp.Stats).SendErrors.Retransmits.Value(), uint64(1); got != want {
- t.Errorf("got EP stats SendErrors.Retransmits.Value = %v, want = %v", got, want)
+ return nil
}
- if got, want := c.Stack().Stats().TCP.SlowStartRetransmits.Value(), uint64(1); got != want {
- t.Errorf("got stats.TCP.SlowStartRetransmits.Value = %v, want = %v", got, want)
+ // Poll when checking metrics.
+ if err := testutil.Poll(metricPollFn, 1*time.Second); err != nil {
+ t.Error(err)
}
// Acknowledge half of the pending data.
diff --git a/pkg/tcpip/transport/tcp/tcp_sack_test.go b/pkg/tcpip/transport/tcp/tcp_sack_test.go
index afea124ec..c439d5281 100644
--- a/pkg/tcpip/transport/tcp/tcp_sack_test.go
+++ b/pkg/tcpip/transport/tcp/tcp_sack_test.go
@@ -387,7 +387,7 @@ func TestSACKRecovery(t *testing.T) {
setStackSACKPermitted(t, c, true)
createConnectedWithSACKAndTS(c)
- const iterations = 7
+ const iterations = 3
data := buffer.NewView(2 * maxPayload * (tcp.InitialCwnd << (iterations + 1)))
for i := range data {
data[i] = byte(i)
diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go
index 29301a45c..41caa9ed4 100644
--- a/pkg/tcpip/transport/tcp/tcp_test.go
+++ b/pkg/tcpip/transport/tcp/tcp_test.go
@@ -590,6 +590,10 @@ func TestClosingWithEnqueuedSegments(t *testing.T) {
),
)
+ // Give the stack a few ms to transition the endpoint out of ESTABLISHED
+ // state.
+ time.Sleep(10 * time.Millisecond)
+
if got, want := tcp.EndpointState(ep.State()), tcp.StateCloseWait; got != want {
t.Errorf("Unexpected endpoint state: want %v, got %v", want, got)
}
@@ -4472,8 +4476,8 @@ func TestKeepalive(t *testing.T) {
c.CreateConnected(789, 30000, -1 /* epRcvBuf */)
- const keepAliveInterval = 10 * time.Millisecond
- c.EP.SetSockOpt(tcpip.KeepaliveIdleOption(10 * time.Millisecond))
+ const keepAliveInterval = 3 * time.Second
+ c.EP.SetSockOpt(tcpip.KeepaliveIdleOption(100 * time.Millisecond))
c.EP.SetSockOpt(tcpip.KeepaliveIntervalOption(keepAliveInterval))
c.EP.SetSockOptInt(tcpip.KeepaliveCountOption, 5)
c.EP.SetSockOptBool(tcpip.KeepaliveEnabledOption, true)
@@ -4567,7 +4571,7 @@ func TestKeepalive(t *testing.T) {
// Sleep for a litte over the KeepAlive interval to make sure
// the timer has time to fire after the last ACK and close the
// close the socket.
- time.Sleep(keepAliveInterval + 5*time.Millisecond)
+ time.Sleep(keepAliveInterval + keepAliveInterval/2)
// The connection should be terminated after 5 unacked keepalives.
// Send an ACK to trigger a RST from the stack as the endpoint should
@@ -6615,14 +6619,17 @@ func TestKeepaliveWithUserTimeout(t *testing.T) {
origEstablishedTimedout := c.Stack().Stats().TCP.EstablishedTimedout.Value()
- const keepAliveInterval = 10 * time.Millisecond
- c.EP.SetSockOpt(tcpip.KeepaliveIdleOption(10 * time.Millisecond))
+ const keepAliveInterval = 3 * time.Second
+ c.EP.SetSockOpt(tcpip.KeepaliveIdleOption(100 * time.Millisecond))
c.EP.SetSockOpt(tcpip.KeepaliveIntervalOption(keepAliveInterval))
c.EP.SetSockOptInt(tcpip.KeepaliveCountOption, 10)
c.EP.SetSockOptBool(tcpip.KeepaliveEnabledOption, true)
- // Set userTimeout to be the duration for 3 keepalive probes.
- userTimeout := 30 * time.Millisecond
+ // Set userTimeout to be the duration to be 1 keepalive
+ // probes. Which means that after the first probe is sent
+ // the second one should cause the connection to be
+ // closed due to userTimeout being hit.
+ userTimeout := 1 * keepAliveInterval
c.EP.SetSockOpt(tcpip.TCPUserTimeoutOption(userTimeout))
// Check that the connection is still alive.
@@ -6630,28 +6637,23 @@ func TestKeepaliveWithUserTimeout(t *testing.T) {
t.Fatalf("got c.EP.Read(nil) = %v, want = %v", err, tcpip.ErrWouldBlock)
}
- // Now receive 2 keepalives, but don't ACK them. The connection should
- // be reset when the 3rd one should be sent due to userTimeout being
- // 30ms and each keepalive probe should be sent 10ms apart as set above after
- // the connection has been idle for 10ms.
- for i := 0; i < 2; i++ {
- b := c.GetPacket()
- checker.IPv4(t, b,
- checker.TCP(
- checker.DstPort(context.TestPort),
- checker.SeqNum(uint32(c.IRS)),
- checker.AckNum(uint32(790)),
- checker.TCPFlags(header.TCPFlagAck),
- ),
- )
- }
+ // Now receive 1 keepalives, but don't ACK it.
+ b := c.GetPacket()
+ checker.IPv4(t, b,
+ checker.TCP(
+ checker.DstPort(context.TestPort),
+ checker.SeqNum(uint32(c.IRS)),
+ checker.AckNum(uint32(790)),
+ checker.TCPFlags(header.TCPFlagAck),
+ ),
+ )
// Sleep for a litte over the KeepAlive interval to make sure
// the timer has time to fire after the last ACK and close the
// close the socket.
- time.Sleep(keepAliveInterval + 5*time.Millisecond)
+ time.Sleep(keepAliveInterval + keepAliveInterval/2)
- // The connection should be terminated after 30ms.
+ // The connection should be closed with a timeout.
// Send an ACK to trigger a RST from the stack as the endpoint should
// be dead.
c.SendPacket(nil, &context.Headers{
diff --git a/pkg/tcpip/transport/tcp/testing/context/context.go b/pkg/tcpip/transport/tcp/testing/context/context.go
index 431ab4e6b..7b1d72cf4 100644
--- a/pkg/tcpip/transport/tcp/testing/context/context.go
+++ b/pkg/tcpip/transport/tcp/testing/context/context.go
@@ -152,6 +152,13 @@ func New(t *testing.T, mtu uint32) *Context {
t.Fatalf("SetTransportProtocolOption failed: %v", err)
}
+ // Increase minimum RTO in tests to avoid test flakes due to early
+ // retransmit in case the test executors are overloaded and cause timers
+ // to fire earlier than expected.
+ if err := s.SetTransportProtocolOption(tcp.ProtocolNumber, tcpip.TCPMinRTOOption(3*time.Second)); err != nil {
+ t.Fatalf("failed to set stack-wide minRTO: %s", err)
+ }
+
// Some of the congestion control tests send up to 640 packets, we so
// set the channel size to 1000.
ep := channel.New(1000, mtu, "")
@@ -236,7 +243,7 @@ func (c *Context) CheckNoPacket(errMsg string) {
func (c *Context) GetPacket() []byte {
c.t.Helper()
- ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
+ ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
p, ok := c.linkEP.ReadContext(ctx)
if !ok {
@@ -417,6 +424,8 @@ func (c *Context) SendAckWithSACK(seq seqnum.Value, bytesReceived int, sackBlock
// verifies that the packet packet payload of packet matches the slice
// of data indicated by offset & size.
func (c *Context) ReceiveAndCheckPacket(data []byte, offset, size int) {
+ c.t.Helper()
+
c.ReceiveAndCheckPacketWithOptions(data, offset, size, 0)
}
@@ -425,6 +434,8 @@ func (c *Context) ReceiveAndCheckPacket(data []byte, offset, size int) {
// data indicated by offset & size and skips optlen bytes in addition to the IP
// TCP headers when comparing the data.
func (c *Context) ReceiveAndCheckPacketWithOptions(data []byte, offset, size, optlen int) {
+ c.t.Helper()
+
b := c.GetPacket()
checker.IPv4(c.t, b,
checker.PayloadLen(size+header.TCPMinimumSize+optlen),
@@ -447,6 +458,8 @@ func (c *Context) ReceiveAndCheckPacketWithOptions(data []byte, offset, size, op
// data indicated by offset & size. It returns true if a packet was received and
// processed.
func (c *Context) ReceiveNonBlockingAndCheckPacket(data []byte, offset, size int) bool {
+ c.t.Helper()
+
b := c.GetPacketNonBlocking()
if b == nil {
return false
@@ -570,6 +583,8 @@ func (c *Context) CreateConnected(iss seqnum.Value, rcvWnd seqnum.Size, epRcvBuf
//
// PreCondition: c.EP must already be created.
func (c *Context) Connect(iss seqnum.Value, rcvWnd seqnum.Size, options []byte) {
+ c.t.Helper()
+
// Start connection attempt.
waitEntry, notifyCh := waiter.NewChannelEntry(nil)
c.WQ.EventRegister(&waitEntry, waiter.EventOut)