summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
-rw-r--r--pkg/tcpip/transport/tcp/snd.go94
-rw-r--r--pkg/tcpip/transport/tcp/tcp_test.go17
-rw-r--r--test/packetimpact/tests/BUILD32
-rw-r--r--test/packetimpact/tests/tcp_zero_window_probe_retransmit_test.go99
-rw-r--r--test/packetimpact/tests/tcp_zero_window_probe_test.go107
-rw-r--r--test/packetimpact/tests/tcp_zero_window_probe_usertimeout_test.go93
6 files changed, 433 insertions, 9 deletions
diff --git a/pkg/tcpip/transport/tcp/snd.go b/pkg/tcpip/transport/tcp/snd.go
index d8cfe3115..a3018914b 100644
--- a/pkg/tcpip/transport/tcp/snd.go
+++ b/pkg/tcpip/transport/tcp/snd.go
@@ -41,6 +41,10 @@ const (
// nDupAckThreshold is the number of duplicate ACK's required
// before fast-retransmit is entered.
nDupAckThreshold = 3
+
+ // MaxRetries is the maximum number of probe retries sender does
+ // before timing out the connection, Linux default TCP_RETR2.
+ MaxRetries = 15
)
// ccState indicates the current congestion control state for this sender.
@@ -138,6 +142,14 @@ type sender struct {
// the first segment that was retransmitted due to RTO expiration.
firstRetransmittedSegXmitTime time.Time `state:".(unixTime)"`
+ // zeroWindowProbing is set if the sender is currently probing
+ // for zero receive window.
+ zeroWindowProbing bool `state:"nosave"`
+
+ // unackZeroWindowProbes is the number of unacknowledged zero
+ // window probes.
+ unackZeroWindowProbes uint32 `state:"nosave"`
+
closed bool
writeNext *segment
writeList segmentList
@@ -479,10 +491,24 @@ func (s *sender) retransmitTimerExpired() bool {
remaining = uto - elapsed
}
- if remaining <= 0 || s.rto >= MaxRTO {
+ // Always honor the user-timeout irrespective of whether the zero
+ // window probes were acknowledged.
+ // net/ipv4/tcp_timer.c::tcp_probe_timer()
+ if remaining <= 0 || s.unackZeroWindowProbes >= MaxRetries {
return false
}
+ if s.rto >= MaxRTO {
+ // RFC 1122 section: 4.2.2.17
+ // A TCP MAY keep its offered receive window closed
+ // indefinitely. As long as the receiving TCP continues to
+ // send acknowledgments in response to the probe segments, the
+ // sending TCP MUST allow the connection to stay open.
+ if !(s.zeroWindowProbing && s.unackZeroWindowProbes == 0) {
+ return false
+ }
+ }
+
// Set new timeout. The timer will be restarted by the call to sendData
// below.
s.rto *= 2
@@ -533,6 +559,15 @@ func (s *sender) retransmitTimerExpired() bool {
// information is usable after an RTO.
s.ep.scoreboard.Reset()
s.writeNext = s.writeList.Front()
+
+ // RFC 1122 4.2.2.17: Start sending zero window probes when we still see a
+ // zero receive window after retransmission interval and we have data to
+ // send.
+ if s.zeroWindowProbing {
+ s.sendZeroWindowProbe()
+ return true
+ }
+
s.sendData()
return true
@@ -827,6 +862,34 @@ func (s *sender) handleSACKRecovery(limit int, end seqnum.Value) (dataSent bool)
return dataSent
}
+func (s *sender) sendZeroWindowProbe() {
+ ack, win := s.ep.rcv.getSendParams()
+ s.unackZeroWindowProbes++
+ // Send a zero window probe with sequence number pointing to
+ // the last acknowledged byte.
+ s.ep.sendRaw(buffer.VectorisedView{}, header.TCPFlagAck, s.sndUna-1, ack, win)
+ // Rearm the timer to continue probing.
+ s.resendTimer.enable(s.rto)
+}
+
+func (s *sender) enableZeroWindowProbing() {
+ s.zeroWindowProbing = true
+ // We piggyback the probing on the retransmit timer with the
+ // current retranmission interval, as we may start probing while
+ // segment retransmissions.
+ if s.firstRetransmittedSegXmitTime.IsZero() {
+ s.firstRetransmittedSegXmitTime = time.Now()
+ }
+ s.resendTimer.enable(s.rto)
+}
+
+func (s *sender) disableZeroWindowProbing() {
+ s.zeroWindowProbing = false
+ s.unackZeroWindowProbes = 0
+ s.firstRetransmittedSegXmitTime = time.Time{}
+ s.resendTimer.disable()
+}
+
// sendData sends new data segments. It is called when data becomes available or
// when the send window opens up.
func (s *sender) sendData() {
@@ -875,6 +938,13 @@ func (s *sender) sendData() {
s.ep.disableKeepaliveTimer()
}
+ // If the sender has advertized zero receive window and we have
+ // data to be sent out, start zero window probing to query the
+ // the remote for it's receive window size.
+ if s.writeNext != nil && s.sndWnd == 0 {
+ s.enableZeroWindowProbing()
+ }
+
// Enable the timer if we have pending data and it's not enabled yet.
if !s.resendTimer.enabled() && s.sndUna != s.sndNxt {
s.resendTimer.enable(s.rto)
@@ -1122,8 +1192,26 @@ func (s *sender) handleRcvdSegment(seg *segment) {
// Stash away the current window size.
s.sndWnd = seg.window
- // Ignore ack if it doesn't acknowledge any new data.
ack := seg.ackNumber
+
+ // Disable zero window probing if remote advertizes a non-zero receive
+ // window. This can be with an ACK to the zero window probe (where the
+ // acknumber refers to the already acknowledged byte) OR to any previously
+ // unacknowledged segment.
+ if s.zeroWindowProbing && seg.window > 0 &&
+ (ack == s.sndUna || (ack-1).InRange(s.sndUna, s.sndNxt)) {
+ s.disableZeroWindowProbing()
+ }
+
+ // On receiving the ACK for the zero window probe, account for it and
+ // skip trying to send any segment as we are still probing for
+ // receive window to become non-zero.
+ if s.zeroWindowProbing && s.unackZeroWindowProbes > 0 && ack == s.sndUna {
+ s.unackZeroWindowProbes--
+ return
+ }
+
+ // Ignore ack if it doesn't acknowledge any new data.
if (ack - 1).InRange(s.sndUna, s.sndNxt) {
s.dupAckCount = 0
@@ -1143,7 +1231,7 @@ func (s *sender) handleRcvdSegment(seg *segment) {
}
// When an ack is received we must rearm the timer.
- // RFC 6298 5.2
+ // RFC 6298 5.3
s.resendTimer.enable(s.rto)
// Remove all acknowledged data from the write list.
diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go
index 33e2b9a09..49e4ba214 100644
--- a/pkg/tcpip/transport/tcp/tcp_test.go
+++ b/pkg/tcpip/transport/tcp/tcp_test.go
@@ -1900,7 +1900,7 @@ func TestZeroWindowSend(t *testing.T) {
c := context.New(t, defaultMTU)
defer c.Cleanup()
- c.CreateConnected(789, 0, -1 /* epRcvBuf */)
+ c.CreateConnected(789 /* iss */, 0 /* rcvWnd */, -1 /* epRcvBuf */)
data := []byte{1, 2, 3}
view := buffer.NewView(len(data))
@@ -1911,8 +1911,17 @@ func TestZeroWindowSend(t *testing.T) {
t.Fatalf("Write failed: %v", err)
}
- // Since the window is currently zero, check that no packet is received.
- c.CheckNoPacket("Packet received when window is zero")
+ // Check if we got a zero-window probe.
+ b := c.GetPacket()
+ checker.IPv4(t, b,
+ checker.PayloadLen(header.TCPMinimumSize),
+ checker.TCP(
+ checker.DstPort(context.TestPort),
+ checker.SeqNum(uint32(c.IRS)),
+ checker.AckNum(790),
+ checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)),
+ ),
+ )
// Open up the window. Data should be received now.
c.SendPacket(nil, &context.Headers{
@@ -1925,7 +1934,7 @@ func TestZeroWindowSend(t *testing.T) {
})
// Check that data is received.
- b := c.GetPacket()
+ b = c.GetPacket()
checker.IPv4(t, b,
checker.PayloadLen(len(data)+header.TCPMinimumSize),
checker.TCP(
diff --git a/test/packetimpact/tests/BUILD b/test/packetimpact/tests/BUILD
index 6beccbfd0..e4ced444b 100644
--- a/test/packetimpact/tests/BUILD
+++ b/test/packetimpact/tests/BUILD
@@ -44,8 +44,36 @@ packetimpact_go_test(
packetimpact_go_test(
name = "tcp_window_shrink",
srcs = ["tcp_window_shrink_test.go"],
- # TODO(b/153202472): Fix netstack then remove the line below.
- netstack = False,
+ deps = [
+ "//pkg/tcpip/header",
+ "//test/packetimpact/testbench",
+ "@org_golang_x_sys//unix:go_default_library",
+ ],
+)
+
+packetimpact_go_test(
+ name = "tcp_zero_window_probe",
+ srcs = ["tcp_zero_window_probe_test.go"],
+ deps = [
+ "//pkg/tcpip/header",
+ "//test/packetimpact/testbench",
+ "@org_golang_x_sys//unix:go_default_library",
+ ],
+)
+
+packetimpact_go_test(
+ name = "tcp_zero_window_probe_retransmit",
+ srcs = ["tcp_zero_window_probe_retransmit_test.go"],
+ deps = [
+ "//pkg/tcpip/header",
+ "//test/packetimpact/testbench",
+ "@org_golang_x_sys//unix:go_default_library",
+ ],
+)
+
+packetimpact_go_test(
+ name = "tcp_zero_window_probe_usertimeout",
+ srcs = ["tcp_zero_window_probe_usertimeout_test.go"],
deps = [
"//pkg/tcpip/header",
"//test/packetimpact/testbench",
diff --git a/test/packetimpact/tests/tcp_zero_window_probe_retransmit_test.go b/test/packetimpact/tests/tcp_zero_window_probe_retransmit_test.go
new file mode 100644
index 000000000..864e5a634
--- /dev/null
+++ b/test/packetimpact/tests/tcp_zero_window_probe_retransmit_test.go
@@ -0,0 +1,99 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package tcp_zero_window_probe_retransmit_test
+
+import (
+ "testing"
+ "time"
+
+ "golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/tcpip/header"
+ tb "gvisor.dev/gvisor/test/packetimpact/testbench"
+)
+
+// TestZeroWindowProbeRetransmit tests retransmits of zero window probes
+// to be sent at exponentially inreasing time intervals.
+func TestZeroWindowProbeRetransmit(t *testing.T) {
+ dut := tb.NewDUT(t)
+ defer dut.TearDown()
+ listenFd, remotePort := dut.CreateListener(unix.SOCK_STREAM, unix.IPPROTO_TCP, 1)
+ defer dut.Close(listenFd)
+ conn := tb.NewTCPIPv4(t, tb.TCP{DstPort: &remotePort}, tb.TCP{SrcPort: &remotePort})
+ defer conn.Close()
+
+ conn.Handshake()
+ acceptFd, _ := dut.Accept(listenFd)
+ defer dut.Close(acceptFd)
+
+ dut.SetSockOptInt(acceptFd, unix.IPPROTO_TCP, unix.TCP_NODELAY, 1)
+
+ sampleData := []byte("Sample Data")
+ samplePayload := &tb.Payload{Bytes: sampleData}
+
+ // Send and receive sample data to the dut.
+ dut.Send(acceptFd, sampleData, 0)
+ if _, err := conn.ExpectData(&tb.TCP{}, samplePayload, time.Second); err != nil {
+ t.Fatalf("expected a packet with payload %v: %s", samplePayload, err)
+ }
+ conn.Send(tb.TCP{Flags: tb.Uint8(header.TCPFlagAck | header.TCPFlagPsh)}, samplePayload)
+ if _, err := conn.ExpectData(&tb.TCP{Flags: tb.Uint8(header.TCPFlagAck)}, nil, time.Second); err != nil {
+ t.Fatalf("expected a packet with sequence number %s", err)
+ }
+
+ // Check for the dut to keep the connection alive as long as the zero window
+ // probes are acknowledged. Check if the zero window probes are sent at
+ // exponentially increasing intervals. The timeout intervals are function
+ // of the recorded first zero probe transmission duration.
+ //
+ // Advertize zero receive window again.
+ conn.Send(tb.TCP{Flags: tb.Uint8(header.TCPFlagAck), WindowSize: tb.Uint16(0)})
+ probeSeq := tb.Uint32(uint32(*conn.RemoteSeqNum() - 1))
+ ackProbe := tb.Uint32(uint32(*conn.RemoteSeqNum()))
+
+ startProbeDuration := time.Second
+ current := startProbeDuration
+ first := time.Now()
+ // Ask the dut to send out data.
+ dut.Send(acceptFd, sampleData, 0)
+ // Expect the dut to keep the connection alive as long as the remote is
+ // acknowledging the zero-window probes.
+ for i := 0; i < 5; i++ {
+ start := time.Now()
+ // Expect zero-window probe with a timeout which is a function of the typical
+ // first retransmission time. The retransmission times is supposed to
+ // exponentially increase.
+ if _, err := conn.ExpectData(&tb.TCP{SeqNum: probeSeq}, nil, 2*current); err != nil {
+ t.Fatalf("expected a probe with sequence number %v: loop %d", probeSeq, i)
+ }
+ if i == 0 {
+ startProbeDuration = time.Now().Sub(first)
+ current = 2 * startProbeDuration
+ continue
+ }
+ // Check if the probes came at exponentially increasing intervals.
+ if p := time.Since(start); p < current-startProbeDuration {
+ t.Fatalf("zero probe came sooner interval %d probe %d\n", p, i)
+ }
+ // Acknowledge the zero-window probes from the dut.
+ conn.Send(tb.TCP{AckNum: ackProbe, Flags: tb.Uint8(header.TCPFlagAck), WindowSize: tb.Uint16(0)})
+ current *= 2
+ }
+ // Advertize non-zero window.
+ conn.Send(tb.TCP{AckNum: ackProbe, Flags: tb.Uint8(header.TCPFlagAck)})
+ // Expect the dut to recover and transmit data.
+ if _, err := conn.ExpectData(&tb.TCP{SeqNum: ackProbe}, samplePayload, time.Second); err != nil {
+ t.Fatalf("expected a packet with payload %v: %s", samplePayload, err)
+ }
+}
diff --git a/test/packetimpact/tests/tcp_zero_window_probe_test.go b/test/packetimpact/tests/tcp_zero_window_probe_test.go
new file mode 100644
index 000000000..4fa3d0cd4
--- /dev/null
+++ b/test/packetimpact/tests/tcp_zero_window_probe_test.go
@@ -0,0 +1,107 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package tcp_zero_window_probe_test
+
+import (
+ "testing"
+ "time"
+
+ "golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/tcpip/header"
+ tb "gvisor.dev/gvisor/test/packetimpact/testbench"
+)
+
+// TestZeroWindowProbe tests few cases of zero window probing over the
+// same connection.
+func TestZeroWindowProbe(t *testing.T) {
+ dut := tb.NewDUT(t)
+ defer dut.TearDown()
+ listenFd, remotePort := dut.CreateListener(unix.SOCK_STREAM, unix.IPPROTO_TCP, 1)
+ defer dut.Close(listenFd)
+ conn := tb.NewTCPIPv4(t, tb.TCP{DstPort: &remotePort}, tb.TCP{SrcPort: &remotePort})
+ defer conn.Close()
+
+ conn.Handshake()
+ acceptFd, _ := dut.Accept(listenFd)
+ defer dut.Close(acceptFd)
+
+ dut.SetSockOptInt(acceptFd, unix.IPPROTO_TCP, unix.TCP_NODELAY, 1)
+
+ sampleData := []byte("Sample Data")
+ samplePayload := &tb.Payload{Bytes: sampleData}
+
+ start := time.Now()
+ // Send and receive sample data to the dut.
+ dut.Send(acceptFd, sampleData, 0)
+ if _, err := conn.ExpectData(&tb.TCP{}, samplePayload, time.Second); err != nil {
+ t.Fatalf("expected a packet with payload %v: %s", samplePayload, err)
+ }
+ sendTime := time.Now().Sub(start)
+ conn.Send(tb.TCP{Flags: tb.Uint8(header.TCPFlagAck | header.TCPFlagPsh)}, samplePayload)
+ if _, err := conn.ExpectData(&tb.TCP{Flags: tb.Uint8(header.TCPFlagAck)}, nil, time.Second); err != nil {
+ t.Fatalf("expected a packet with sequence number %s", err)
+ }
+
+ // Test 1: Check for receive of a zero window probe, record the duration for
+ // probe to be sent.
+ //
+ // Advertize zero window to the dut.
+ conn.Send(tb.TCP{Flags: tb.Uint8(header.TCPFlagAck), WindowSize: tb.Uint16(0)})
+
+ // Expected sequence number of the zero window probe.
+ probeSeq := tb.Uint32(uint32(*conn.RemoteSeqNum() - 1))
+ // Expected ack number of the ACK for the probe.
+ ackProbe := tb.Uint32(uint32(*conn.RemoteSeqNum()))
+
+ // Expect there are no zero-window probes sent until there is data to be sent out
+ // from the dut.
+ if _, err := conn.ExpectData(&tb.TCP{SeqNum: probeSeq}, nil, 2*time.Second); err == nil {
+ t.Fatalf("unexpected a packet with sequence number %v: %s", probeSeq, err)
+ }
+
+ start = time.Now()
+ // Ask the dut to send out data.
+ dut.Send(acceptFd, sampleData, 0)
+ // Expect zero-window probe from the dut.
+ if _, err := conn.ExpectData(&tb.TCP{SeqNum: probeSeq}, nil, time.Second); err != nil {
+ t.Fatalf("expected a packet with sequence number %v: %s", probeSeq, err)
+ }
+ // Expect the probe to be sent after some time. Compare against the previous
+ // time recorded when the dut immediately sends out data on receiving the
+ // send command.
+ if startProbeDuration := time.Now().Sub(start); startProbeDuration <= sendTime {
+ t.Fatalf("expected the first probe to be sent out after retransmission interval, got %v want > %v\n", startProbeDuration, sendTime)
+ }
+
+ // Test 2: Check if the dut recovers on advertizing non-zero receive window.
+ // and sends out the sample payload after the send window opens.
+ //
+ // Advertize non-zero window to the dut and ack the zero window probe.
+ conn.Send(tb.TCP{AckNum: ackProbe, Flags: tb.Uint8(header.TCPFlagAck)})
+ // Expect the dut to recover and transmit data.
+ if _, err := conn.ExpectData(&tb.TCP{SeqNum: ackProbe}, samplePayload, time.Second); err != nil {
+ t.Fatalf("expected a packet with payload %v: %s", samplePayload, err)
+ }
+
+ // Test 3: Sanity check for dut's processing of a similar probe it sent.
+ // Check if the dut responds as we do for a similar probe sent to it.
+ // Basically with sequence number to one byte behind the unacknowledged
+ // sequence number.
+ p := tb.Uint32(uint32(*conn.LocalSeqNum()))
+ conn.Send(tb.TCP{Flags: tb.Uint8(header.TCPFlagAck), SeqNum: tb.Uint32(uint32(*conn.LocalSeqNum() - 1))})
+ if _, err := conn.ExpectData(&tb.TCP{Flags: tb.Uint8(header.TCPFlagAck), AckNum: p}, nil, time.Second); err != nil {
+ t.Fatalf("expected a packet with ack number: %d: %s", p, err)
+ }
+}
diff --git a/test/packetimpact/tests/tcp_zero_window_probe_usertimeout_test.go b/test/packetimpact/tests/tcp_zero_window_probe_usertimeout_test.go
new file mode 100644
index 000000000..7d81c276c
--- /dev/null
+++ b/test/packetimpact/tests/tcp_zero_window_probe_usertimeout_test.go
@@ -0,0 +1,93 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package tcp_zero_window_probe_usertimeout_test
+
+import (
+ "testing"
+ "time"
+
+ "golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/tcpip/header"
+ tb "gvisor.dev/gvisor/test/packetimpact/testbench"
+)
+
+// TestZeroWindowProbeUserTimeout sanity tests user timeout when we are
+// retransmitting zero window probes.
+func TestZeroWindowProbeUserTimeout(t *testing.T) {
+ dut := tb.NewDUT(t)
+ defer dut.TearDown()
+ listenFd, remotePort := dut.CreateListener(unix.SOCK_STREAM, unix.IPPROTO_TCP, 1)
+ defer dut.Close(listenFd)
+ conn := tb.NewTCPIPv4(t, tb.TCP{DstPort: &remotePort}, tb.TCP{SrcPort: &remotePort})
+ defer conn.Close()
+
+ conn.Handshake()
+ acceptFd, _ := dut.Accept(listenFd)
+ defer dut.Close(acceptFd)
+
+ dut.SetSockOptInt(acceptFd, unix.IPPROTO_TCP, unix.TCP_NODELAY, 1)
+
+ sampleData := []byte("Sample Data")
+ samplePayload := &tb.Payload{Bytes: sampleData}
+
+ // Send and receive sample data to the dut.
+ dut.Send(acceptFd, sampleData, 0)
+ if _, err := conn.ExpectData(&tb.TCP{}, samplePayload, time.Second); err != nil {
+ t.Fatalf("expected a packet with payload %v: %s", samplePayload, err)
+ }
+ conn.Send(tb.TCP{Flags: tb.Uint8(header.TCPFlagAck | header.TCPFlagPsh)}, samplePayload)
+ if _, err := conn.ExpectData(&tb.TCP{Flags: tb.Uint8(header.TCPFlagAck)}, nil, time.Second); err != nil {
+ t.Fatalf("expected a packet with sequence number %s", err)
+ }
+
+ // Test 1: Check for receive of a zero window probe, record the duration for
+ // probe to be sent.
+ //
+ // Advertize zero window to the dut.
+ conn.Send(tb.TCP{Flags: tb.Uint8(header.TCPFlagAck), WindowSize: tb.Uint16(0)})
+
+ // Expected sequence number of the zero window probe.
+ probeSeq := tb.Uint32(uint32(*conn.RemoteSeqNum() - 1))
+ start := time.Now()
+ // Ask the dut to send out data.
+ dut.Send(acceptFd, sampleData, 0)
+ // Expect zero-window probe from the dut.
+ if _, err := conn.ExpectData(&tb.TCP{SeqNum: probeSeq}, nil, time.Second); err != nil {
+ t.Fatalf("expected a packet with sequence number %v: %s", probeSeq, err)
+ }
+ // Record the duration for first probe, the dut sends the zero window probe after
+ // a retransmission time interval.
+ startProbeDuration := time.Now().Sub(start)
+
+ // Test 2: Check if the dut times out the connection by honoring usertimeout
+ // when the dut is sending zero-window probes.
+ //
+ // Reduce the retransmit timeout.
+ dut.SetSockOptInt(acceptFd, unix.IPPROTO_TCP, unix.TCP_USER_TIMEOUT, int32(startProbeDuration.Milliseconds()))
+ // Advertize zero window again.
+ conn.Send(tb.TCP{Flags: tb.Uint8(header.TCPFlagAck), WindowSize: tb.Uint16(0)})
+ // Ask the dut to send out data that would trigger zero window probe retransmissions.
+ dut.Send(acceptFd, sampleData, 0)
+
+ // Wait for the connection to timeout after multiple zero-window probe retransmissions.
+ time.Sleep(8 * startProbeDuration)
+
+ // Expect the connection to have timed out and closed which would cause the dut
+ // to reply with a RST to the ACK we send.
+ conn.Send(tb.TCP{Flags: tb.Uint8(header.TCPFlagAck)})
+ if _, err := conn.ExpectData(&tb.TCP{Flags: tb.Uint8(header.TCPFlagRst)}, nil, time.Second); err != nil {
+ t.Fatalf("expected a TCP RST")
+ }
+}