summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
authorMithun Iyer <iyerm@google.com>2021-05-20 19:12:27 -0700
committergVisor bot <gvisor-bot@google.com>2021-05-20 19:15:49 -0700
commit9157a91a4eca7e0811edb20952e9f22ea2c3f13e (patch)
tree7b793c5fb55b409da1907321953d975290abde92
parentae96e00bd98f2d8e44e4fe6dc1c9f05454f2cc93 (diff)
Add protocol state to TCPINFO
Add missing protocol state to TCPINFO struct and update packetimpact. This re-arranges the TCP state definitions to align with Linux. Fixes #478 PiperOrigin-RevId: 374996751
-rw-r--r--pkg/sentry/socket/netstack/netstack.go14
-rw-r--r--pkg/tcpip/tcpip.go6
-rw-r--r--pkg/tcpip/transport/tcp/connect.go2
-rw-r--r--pkg/tcpip/transport/tcp/endpoint.go45
-rw-r--r--pkg/tcpip/transport/tcp/endpoint_state.go8
-rw-r--r--pkg/tcpip/transport/udp/endpoint.go13
-rw-r--r--pkg/tcpip/transport/udp/forwarder.go2
-rw-r--r--test/packetimpact/testbench/BUILD2
-rw-r--r--test/packetimpact/testbench/dut.go33
-rw-r--r--test/packetimpact/tests/BUILD7
-rw-r--r--test/packetimpact/tests/tcp_info_test.go18
-rw-r--r--test/packetimpact/tests/tcp_rack_test.go16
-rw-r--r--test/packetimpact/tests/tcp_retransmits_test.go10
-rw-r--r--test/packetimpact/tests/tcp_synsent_reset_test.go19
-rw-r--r--test/syscalls/linux/socket_inet_loopback.cc71
15 files changed, 183 insertions, 83 deletions
diff --git a/pkg/sentry/socket/netstack/netstack.go b/pkg/sentry/socket/netstack/netstack.go
index 335284f50..03a383cf4 100644
--- a/pkg/sentry/socket/netstack/netstack.go
+++ b/pkg/sentry/socket/netstack/netstack.go
@@ -1127,7 +1127,14 @@ func getSockOptTCP(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name,
// TODO(b/64800844): Translate fields once they are added to
// tcpip.TCPInfoOption.
- info := linux.TCPInfo{}
+ info := linux.TCPInfo{
+ State: uint8(v.State),
+ RTO: uint32(v.RTO / time.Microsecond),
+ RTT: uint32(v.RTT / time.Microsecond),
+ RTTVar: uint32(v.RTTVar / time.Microsecond),
+ SndSsthresh: v.SndSsthresh,
+ SndCwnd: v.SndCwnd,
+ }
switch v.CcState {
case tcpip.RTORecovery:
info.CaState = linux.TCP_CA_Loss
@@ -1138,11 +1145,6 @@ func getSockOptTCP(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name,
case tcpip.Open:
info.CaState = linux.TCP_CA_Open
}
- info.RTO = uint32(v.RTO / time.Microsecond)
- info.RTT = uint32(v.RTT / time.Microsecond)
- info.RTTVar = uint32(v.RTTVar / time.Microsecond)
- info.SndSsthresh = v.SndSsthresh
- info.SndCwnd = v.SndCwnd
// In netstack reorderSeen is updated only when RACK is enabled.
// We only track whether the reordering is seen, which is
diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go
index 328470f3e..4dc89e746 100644
--- a/pkg/tcpip/tcpip.go
+++ b/pkg/tcpip/tcpip.go
@@ -861,6 +861,9 @@ type SettableSocketOption interface {
isSettableSocketOption()
}
+// EndpointState represents the state of an endpoint.
+type EndpointState uint8
+
// CongestionControlState indicates the current congestion control state for
// TCP sender.
type CongestionControlState int
@@ -897,6 +900,9 @@ type TCPInfoOption struct {
// RTO is the retransmission timeout for the endpoint.
RTO time.Duration
+ // State is the current endpoint protocol state.
+ State EndpointState
+
// CcState is the congestion control state.
CcState CongestionControlState
diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go
index 5e03e7715..05b41e0f8 100644
--- a/pkg/tcpip/transport/tcp/connect.go
+++ b/pkg/tcpip/transport/tcp/connect.go
@@ -1235,7 +1235,7 @@ func (e *endpoint) handleSegmentLocked(s *segment) (cont bool, err tcpip.Error)
// Now check if the received segment has caused us to transition
// to a CLOSED state, if yes then terminate processing and do
// not invoke the sender.
- state := e.state
+ state := e.EndpointState()
if state == StateClose {
// When we get into StateClose while processing from the queue,
// return immediately and let the protocolMainloop handle it.
diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go
index 50d39cbad..c6bf1fd98 100644
--- a/pkg/tcpip/transport/tcp/endpoint.go
+++ b/pkg/tcpip/transport/tcp/endpoint.go
@@ -38,19 +38,15 @@ import (
)
// EndpointState represents the state of a TCP endpoint.
-type EndpointState uint32
+type EndpointState tcpip.EndpointState
// Endpoint states. Note that are represented in a netstack-specific manner and
// may not be meaningful externally. Specifically, they need to be translated to
// Linux's representation for these states if presented to userspace.
const (
- // Endpoint states internal to netstack. These map to the TCP state CLOSED.
- StateInitial EndpointState = iota
- StateBound
- StateConnecting // Connect() called, but the initial SYN hasn't been sent.
- StateError
-
- // TCP protocol states.
+ _ EndpointState = iota
+ // TCP protocol states in sync with the definitions in
+ // https://github.com/torvalds/linux/blob/7acac4b3196/include/net/tcp_states.h#L13
StateEstablished
StateSynSent
StateSynRecv
@@ -62,6 +58,12 @@ const (
StateLastAck
StateListen
StateClosing
+
+ // Endpoint states internal to netstack.
+ StateInitial
+ StateBound
+ StateConnecting // Connect() called, but the initial SYN hasn't been sent.
+ StateError
)
const (
@@ -97,6 +99,16 @@ func (s EndpointState) connecting() bool {
}
}
+// internal returns true when the state is netstack internal.
+func (s EndpointState) internal() bool {
+ switch s {
+ case StateInitial, StateBound, StateConnecting, StateError:
+ return true
+ default:
+ return false
+ }
+}
+
// handshake returns true when s is one of the states representing an endpoint
// in the middle of a TCP handshake.
func (s EndpointState) handshake() bool {
@@ -422,12 +434,12 @@ type endpoint struct {
// state must be read/set using the EndpointState()/setEndpointState()
// methods.
- state EndpointState `state:".(EndpointState)"`
+ state uint32 `state:".(EndpointState)"`
// origEndpointState is only used during a restore phase to save the
// endpoint state at restore time as the socket is moved to it's correct
// state.
- origEndpointState EndpointState `state:"nosave"`
+ origEndpointState uint32 `state:"nosave"`
isPortReserved bool `state:"manual"`
isRegistered bool `state:"manual"`
@@ -747,7 +759,7 @@ func (e *endpoint) ResumeWork() {
//
// Precondition: e.mu must be held to call this method.
func (e *endpoint) setEndpointState(state EndpointState) {
- oldstate := EndpointState(atomic.LoadUint32((*uint32)(&e.state)))
+ oldstate := EndpointState(atomic.LoadUint32(&e.state))
switch state {
case StateEstablished:
e.stack.Stats().TCP.CurrentEstablished.Increment()
@@ -764,12 +776,12 @@ func (e *endpoint) setEndpointState(state EndpointState) {
e.stack.Stats().TCP.CurrentEstablished.Decrement()
}
}
- atomic.StoreUint32((*uint32)(&e.state), uint32(state))
+ atomic.StoreUint32(&e.state, uint32(state))
}
// EndpointState returns the current state of the endpoint.
func (e *endpoint) EndpointState() EndpointState {
- return EndpointState(atomic.LoadUint32((*uint32)(&e.state)))
+ return EndpointState(atomic.LoadUint32(&e.state))
}
// setRecentTimestamp sets the recentTS field to the provided value.
@@ -810,7 +822,7 @@ func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQue
},
},
waiterQueue: waiterQueue,
- state: StateInitial,
+ state: uint32(StateInitial),
keepalive: keepalive{
// Linux defaults.
idle: 2 * time.Hour,
@@ -1956,6 +1968,11 @@ func (e *endpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, tcpip.Error) {
func (e *endpoint) getTCPInfo() tcpip.TCPInfoOption {
info := tcpip.TCPInfoOption{}
e.LockUser()
+ if state := e.EndpointState(); state.internal() {
+ info.State = tcpip.EndpointState(StateClose)
+ } else {
+ info.State = tcpip.EndpointState(state)
+ }
snd := e.snd
if snd != nil {
// We do not calculate RTT before sending the data packets. If
diff --git a/pkg/tcpip/transport/tcp/endpoint_state.go b/pkg/tcpip/transport/tcp/endpoint_state.go
index 6e9777fe4..a56d34dc5 100644
--- a/pkg/tcpip/transport/tcp/endpoint_state.go
+++ b/pkg/tcpip/transport/tcp/endpoint_state.go
@@ -154,7 +154,7 @@ func (e *endpoint) afterLoad() {
e.origEndpointState = e.state
// Restore the endpoint to InitialState as it will be moved to
// its origEndpointState during Resume.
- e.state = StateInitial
+ e.state = uint32(StateInitial)
// Condition variables and mutexs are not S/R'ed so reinitialize
// acceptCond with e.acceptMu.
e.acceptCond = sync.NewCond(&e.acceptMu)
@@ -167,7 +167,7 @@ func (e *endpoint) Resume(s *stack.Stack) {
e.stack = s
e.ops.InitHandler(e, e.stack, GetTCPSendBufferLimits, GetTCPReceiveBufferLimits)
e.segmentQueue.thaw()
- epState := e.origEndpointState
+ epState := EndpointState(e.origEndpointState)
switch epState {
case StateInitial, StateBound, StateListen, StateConnecting, StateEstablished:
var ss tcpip.TCPSendBufferSizeRangeOption
@@ -281,11 +281,11 @@ func (e *endpoint) Resume(s *stack.Stack) {
}()
case epState == StateClose:
e.isPortReserved = false
- e.state = StateClose
+ e.state = uint32(StateClose)
e.stack.CompleteTransportEndpointCleanup(e)
tcpip.DeleteDanglingEndpoint(e)
case epState == StateError:
- e.state = StateError
+ e.state = uint32(StateError)
e.stack.CompleteTransportEndpointCleanup(e)
tcpip.DeleteDanglingEndpoint(e)
}
diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go
index f7dd50d35..54d583fbd 100644
--- a/pkg/tcpip/transport/udp/endpoint.go
+++ b/pkg/tcpip/transport/udp/endpoint.go
@@ -40,13 +40,14 @@ type udpPacket struct {
}
// EndpointState represents the state of a UDP endpoint.
-type EndpointState uint32
+type EndpointState tcpip.EndpointState
// Endpoint states. Note that are represented in a netstack-specific manner and
// may not be meaningful externally. Specifically, they need to be translated to
// Linux's representation for these states if presented to userspace.
const (
- StateInitial EndpointState = iota
+ _ EndpointState = iota
+ StateInitial
StateBound
StateConnected
StateClosed
@@ -98,7 +99,7 @@ type endpoint struct {
mu sync.RWMutex `state:"nosave"`
// state must be read/set using the EndpointState()/setEndpointState()
// methods.
- state EndpointState
+ state uint32
route *stack.Route `state:"manual"`
dstPort uint16
ttl uint8
@@ -176,7 +177,7 @@ func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQue
// Linux defaults to TTL=1.
multicastTTL: 1,
multicastMemberships: make(map[multicastMembership]struct{}),
- state: StateInitial,
+ state: uint32(StateInitial),
uniqueID: s.UniqueID(),
}
e.ops.InitHandler(e, e.stack, tcpip.GetStackSendBufferLimits, tcpip.GetStackReceiveBufferLimits)
@@ -204,12 +205,12 @@ func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQue
//
// Precondition: e.mu must be held to call this method.
func (e *endpoint) setEndpointState(state EndpointState) {
- atomic.StoreUint32((*uint32)(&e.state), uint32(state))
+ atomic.StoreUint32(&e.state, uint32(state))
}
// EndpointState() returns the current state of the endpoint.
func (e *endpoint) EndpointState() EndpointState {
- return EndpointState(atomic.LoadUint32((*uint32)(&e.state)))
+ return EndpointState(atomic.LoadUint32(&e.state))
}
// UniqueID implements stack.TransportEndpoint.UniqueID.
diff --git a/pkg/tcpip/transport/udp/forwarder.go b/pkg/tcpip/transport/udp/forwarder.go
index 705ad1f64..7c357cb09 100644
--- a/pkg/tcpip/transport/udp/forwarder.go
+++ b/pkg/tcpip/transport/udp/forwarder.go
@@ -90,7 +90,7 @@ func (r *ForwarderRequest) CreateEndpoint(queue *waiter.Queue) (tcpip.Endpoint,
ep.RegisterNICID = r.pkt.NICID
ep.boundPortFlags = ep.portFlags
- ep.state = StateConnected
+ ep.state = uint32(StateConnected)
ep.rcvMu.Lock()
ep.rcvReady = true
diff --git a/test/packetimpact/testbench/BUILD b/test/packetimpact/testbench/BUILD
index 616215dc3..d8059ab98 100644
--- a/test/packetimpact/testbench/BUILD
+++ b/test/packetimpact/testbench/BUILD
@@ -16,6 +16,8 @@ go_library(
],
visibility = ["//test/packetimpact:__subpackages__"],
deps = [
+ "//pkg/abi/linux",
+ "//pkg/binary",
"//pkg/hostarch",
"//pkg/tcpip",
"//pkg/tcpip/buffer",
diff --git a/test/packetimpact/testbench/dut.go b/test/packetimpact/testbench/dut.go
index eabdc8cb3..269e163bb 100644
--- a/test/packetimpact/testbench/dut.go
+++ b/test/packetimpact/testbench/dut.go
@@ -22,11 +22,13 @@ import (
"testing"
"time"
- pb "gvisor.dev/gvisor/test/packetimpact/proto/posix_server_go_proto"
-
"golang.org/x/sys/unix"
"google.golang.org/grpc"
"google.golang.org/grpc/keepalive"
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ bin "gvisor.dev/gvisor/pkg/binary"
+ "gvisor.dev/gvisor/pkg/hostarch"
+ pb "gvisor.dev/gvisor/test/packetimpact/proto/posix_server_go_proto"
)
// DUT communicates with the DUT to force it to make POSIX calls.
@@ -428,6 +430,33 @@ func (dut *DUT) GetSockOptTimevalWithErrno(ctx context.Context, t *testing.T, so
return ret, timeval, errno
}
+// GetSockOptTCPInfo retreives TCPInfo for the given socket descriptor.
+func (dut *DUT) GetSockOptTCPInfo(t *testing.T, sockfd int32) linux.TCPInfo {
+ t.Helper()
+
+ ctx, cancel := context.WithTimeout(context.Background(), RPCTimeout)
+ defer cancel()
+ ret, info, err := dut.GetSockOptTCPInfoWithErrno(ctx, t, sockfd)
+ if ret != 0 || err != unix.Errno(0) {
+ t.Fatalf("failed to GetSockOptTCPInfo: %s", err)
+ }
+ return info
+}
+
+// GetSockOptTCPInfoWithErrno retreives TCPInfo with any errno.
+func (dut *DUT) GetSockOptTCPInfoWithErrno(ctx context.Context, t *testing.T, sockfd int32) (int32, linux.TCPInfo, error) {
+ t.Helper()
+
+ info := linux.TCPInfo{}
+ ret, infoBytes, errno := dut.GetSockOptWithErrno(ctx, t, sockfd, unix.SOL_TCP, unix.TCP_INFO, int32(linux.SizeOfTCPInfo))
+ if got, want := len(infoBytes), linux.SizeOfTCPInfo; got != want {
+ t.Fatalf("expected %T, got %d bytes want %d bytes", info, got, want)
+ }
+ bin.Unmarshal(infoBytes, hostarch.ByteOrder, &info)
+
+ return ret, info, errno
+}
+
// Listen calls listen on the DUT and causes a fatal test failure if it doesn't
// succeed. If more control over the timeout or error handling is needed, use
// ListenWithErrno.
diff --git a/test/packetimpact/tests/BUILD b/test/packetimpact/tests/BUILD
index c4fe293e0..b1d280f98 100644
--- a/test/packetimpact/tests/BUILD
+++ b/test/packetimpact/tests/BUILD
@@ -104,8 +104,6 @@ packetimpact_testbench(
srcs = ["tcp_retransmits_test.go"],
deps = [
"//pkg/abi/linux",
- "//pkg/binary",
- "//pkg/hostarch",
"//pkg/tcpip/header",
"//test/packetimpact/testbench",
"@org_golang_x_sys//unix:go_default_library",
@@ -189,6 +187,7 @@ packetimpact_testbench(
name = "tcp_synsent_reset",
srcs = ["tcp_synsent_reset_test.go"],
deps = [
+ "//pkg/abi/linux",
"//pkg/tcpip/header",
"//test/packetimpact/testbench",
"@org_golang_x_sys//unix:go_default_library",
@@ -353,8 +352,6 @@ packetimpact_testbench(
srcs = ["tcp_rack_test.go"],
deps = [
"//pkg/abi/linux",
- "//pkg/binary",
- "//pkg/hostarch",
"//pkg/tcpip/header",
"//pkg/tcpip/seqnum",
"//test/packetimpact/testbench",
@@ -367,8 +364,6 @@ packetimpact_testbench(
srcs = ["tcp_info_test.go"],
deps = [
"//pkg/abi/linux",
- "//pkg/binary",
- "//pkg/hostarch",
"//pkg/tcpip/header",
"//test/packetimpact/testbench",
"@org_golang_x_sys//unix:go_default_library",
diff --git a/test/packetimpact/tests/tcp_info_test.go b/test/packetimpact/tests/tcp_info_test.go
index 93f58ec49..b7514e846 100644
--- a/test/packetimpact/tests/tcp_info_test.go
+++ b/test/packetimpact/tests/tcp_info_test.go
@@ -21,8 +21,6 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
- "gvisor.dev/gvisor/pkg/binary"
- "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/tcpip/header"
"gvisor.dev/gvisor/test/packetimpact/testbench"
)
@@ -53,13 +51,10 @@ func TestTCPInfo(t *testing.T) {
}
conn.Send(t, testbench.TCP{Flags: testbench.TCPFlags(header.TCPFlagAck)})
- info := linux.TCPInfo{}
- infoBytes := dut.GetSockOpt(t, acceptFD, unix.SOL_TCP, unix.TCP_INFO, int32(linux.SizeOfTCPInfo))
- if got, want := len(infoBytes), linux.SizeOfTCPInfo; got != want {
- t.Fatalf("expected %T, got %d bytes want %d bytes", info, got, want)
+ info := dut.GetSockOptTCPInfo(t, acceptFD)
+ if got, want := uint32(info.State), linux.TCP_ESTABLISHED; got != want {
+ t.Fatalf("got %d want %d", got, want)
}
- binary.Unmarshal(infoBytes, hostarch.ByteOrder, &info)
-
rtt := time.Duration(info.RTT) * time.Microsecond
rttvar := time.Duration(info.RTTVar) * time.Microsecond
rto := time.Duration(info.RTO) * time.Microsecond
@@ -94,12 +89,7 @@ func TestTCPInfo(t *testing.T) {
t.Fatalf("expected a packet with payload %v: %s", samplePayload, err)
}
- info = linux.TCPInfo{}
- infoBytes = dut.GetSockOpt(t, acceptFD, unix.SOL_TCP, unix.TCP_INFO, int32(linux.SizeOfTCPInfo))
- if got, want := len(infoBytes), linux.SizeOfTCPInfo; got != want {
- t.Fatalf("expected %T, got %d bytes want %d bytes", info, got, want)
- }
- binary.Unmarshal(infoBytes, hostarch.ByteOrder, &info)
+ info = dut.GetSockOptTCPInfo(t, acceptFD)
if info.CaState != linux.TCP_CA_Loss {
t.Errorf("expected the connection to be in loss recovery, got: %v want: %v", info.CaState, linux.TCP_CA_Loss)
}
diff --git a/test/packetimpact/tests/tcp_rack_test.go b/test/packetimpact/tests/tcp_rack_test.go
index ff1431bbf..5a60bf712 100644
--- a/test/packetimpact/tests/tcp_rack_test.go
+++ b/test/packetimpact/tests/tcp_rack_test.go
@@ -21,8 +21,6 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
- "gvisor.dev/gvisor/pkg/binary"
- "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/tcpip/header"
"gvisor.dev/gvisor/pkg/tcpip/seqnum"
"gvisor.dev/gvisor/test/packetimpact/testbench"
@@ -69,12 +67,7 @@ func closeSACKConnection(t *testing.T, dut testbench.DUT, conn testbench.TCPIPv4
}
func getRTTAndRTO(t *testing.T, dut testbench.DUT, acceptFd int32) (rtt, rto time.Duration) {
- info := linux.TCPInfo{}
- infoBytes := dut.GetSockOpt(t, acceptFd, unix.SOL_TCP, unix.TCP_INFO, int32(linux.SizeOfTCPInfo))
- if got, want := len(infoBytes), linux.SizeOfTCPInfo; got != want {
- t.Fatalf("expected %T, got %d bytes want %d bytes", info, got, want)
- }
- binary.Unmarshal(infoBytes, hostarch.ByteOrder, &info)
+ info := dut.GetSockOptTCPInfo(t, acceptFd)
return time.Duration(info.RTT) * time.Microsecond, time.Duration(info.RTO) * time.Microsecond
}
@@ -402,12 +395,7 @@ func TestRACKWithLostRetransmission(t *testing.T) {
}
// Check the congestion control state.
- info := linux.TCPInfo{}
- infoBytes := dut.GetSockOpt(t, acceptFd, unix.SOL_TCP, unix.TCP_INFO, int32(linux.SizeOfTCPInfo))
- if got, want := len(infoBytes), linux.SizeOfTCPInfo; got != want {
- t.Fatalf("expected %T, got %d bytes want %d bytes", info, got, want)
- }
- binary.Unmarshal(infoBytes, hostarch.ByteOrder, &info)
+ info := dut.GetSockOptTCPInfo(t, acceptFd)
if info.CaState != linux.TCP_CA_Recovery {
t.Fatalf("expected connection to be in fast recovery, want: %v got: %v", linux.TCP_CA_Recovery, info.CaState)
}
diff --git a/test/packetimpact/tests/tcp_retransmits_test.go b/test/packetimpact/tests/tcp_retransmits_test.go
index 1eafe20c3..d3fb789f4 100644
--- a/test/packetimpact/tests/tcp_retransmits_test.go
+++ b/test/packetimpact/tests/tcp_retransmits_test.go
@@ -21,9 +21,6 @@ import (
"time"
"golang.org/x/sys/unix"
- "gvisor.dev/gvisor/pkg/abi/linux"
- "gvisor.dev/gvisor/pkg/binary"
- "gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/tcpip/header"
"gvisor.dev/gvisor/test/packetimpact/testbench"
)
@@ -33,12 +30,7 @@ func init() {
}
func getRTO(t *testing.T, dut testbench.DUT, acceptFd int32) (rto time.Duration) {
- info := linux.TCPInfo{}
- infoBytes := dut.GetSockOpt(t, acceptFd, unix.SOL_TCP, unix.TCP_INFO, int32(linux.SizeOfTCPInfo))
- if got, want := len(infoBytes), linux.SizeOfTCPInfo; got != want {
- t.Fatalf("unexpected size for TCP_INFO, got %d bytes want %d bytes", got, want)
- }
- binary.Unmarshal(infoBytes, hostarch.ByteOrder, &info)
+ info := dut.GetSockOptTCPInfo(t, acceptFd)
return time.Duration(info.RTO) * time.Microsecond
}
diff --git a/test/packetimpact/tests/tcp_synsent_reset_test.go b/test/packetimpact/tests/tcp_synsent_reset_test.go
index cccb0abc6..fe53e7061 100644
--- a/test/packetimpact/tests/tcp_synsent_reset_test.go
+++ b/test/packetimpact/tests/tcp_synsent_reset_test.go
@@ -20,6 +20,7 @@ import (
"time"
"golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/tcpip/header"
"gvisor.dev/gvisor/test/packetimpact/testbench"
)
@@ -29,7 +30,7 @@ func init() {
}
// dutSynSentState sets up the dut connection in SYN-SENT state.
-func dutSynSentState(t *testing.T) (*testbench.DUT, *testbench.TCPIPv4, uint16, uint16) {
+func dutSynSentState(t *testing.T) (*testbench.DUT, *testbench.TCPIPv4, int32, uint16, uint16) {
t.Helper()
dut := testbench.NewDUT(t)
@@ -46,26 +47,29 @@ func dutSynSentState(t *testing.T) (*testbench.DUT, *testbench.TCPIPv4, uint16,
t.Fatalf("expected SYN\n")
}
- return &dut, &conn, port, clientPort
+ return &dut, &conn, clientFD, port, clientPort
}
// TestTCPSynSentReset tests RFC793, p67: SYN-SENT to CLOSED transition.
func TestTCPSynSentReset(t *testing.T) {
- _, conn, _, _ := dutSynSentState(t)
+ dut, conn, fd, _, _ := dutSynSentState(t)
defer conn.Close(t)
conn.Send(t, testbench.TCP{Flags: testbench.TCPFlags(header.TCPFlagRst | header.TCPFlagAck)})
// Expect the connection to have closed.
- // TODO(gvisor.dev/issue/478): Check for TCP_INFO on the dut side.
conn.Send(t, testbench.TCP{Flags: testbench.TCPFlags(header.TCPFlagAck)})
if _, err := conn.ExpectData(t, &testbench.TCP{Flags: testbench.TCPFlags(header.TCPFlagRst)}, nil, time.Second); err != nil {
t.Fatalf("expected a TCP RST")
}
+ info := dut.GetSockOptTCPInfo(t, fd)
+ if got, want := uint32(info.State), linux.TCP_CLOSE; got != want {
+ t.Fatalf("got %d want %d", got, want)
+ }
}
// TestTCPSynSentRcvdReset tests RFC793, p70, SYN-SENT to SYN-RCVD to CLOSED
// transitions.
func TestTCPSynSentRcvdReset(t *testing.T) {
- dut, c, remotePort, clientPort := dutSynSentState(t)
+ dut, c, fd, remotePort, clientPort := dutSynSentState(t)
defer c.Close(t)
conn := dut.Net.NewTCPIPv4(t, testbench.TCP{SrcPort: &remotePort, DstPort: &clientPort}, testbench.TCP{SrcPort: &clientPort, DstPort: &remotePort})
@@ -79,9 +83,12 @@ func TestTCPSynSentRcvdReset(t *testing.T) {
}
conn.Send(t, testbench.TCP{Flags: testbench.TCPFlags(header.TCPFlagRst)})
// Expect the connection to have transitioned SYN-RCVD to CLOSED.
- // TODO(gvisor.dev/issue/478): Check for TCP_INFO on the dut side.
conn.Send(t, testbench.TCP{Flags: testbench.TCPFlags(header.TCPFlagAck)})
if _, err := conn.ExpectData(t, &testbench.TCP{Flags: testbench.TCPFlags(header.TCPFlagRst)}, nil, time.Second); err != nil {
t.Fatalf("expected a TCP RST")
}
+ info := dut.GetSockOptTCPInfo(t, fd)
+ if got, want := uint32(info.State), linux.TCP_CLOSE; got != want {
+ t.Fatalf("got %d want %d", got, want)
+ }
}
diff --git a/test/syscalls/linux/socket_inet_loopback.cc b/test/syscalls/linux/socket_inet_loopback.cc
index 9a6b089f6..f99d6f1c7 100644
--- a/test/syscalls/linux/socket_inet_loopback.cc
+++ b/test/syscalls/linux/socket_inet_loopback.cc
@@ -472,6 +472,77 @@ TEST_P(SocketInetLoopbackTest, TCPListenClose) {
}
}
+// Test the protocol state information returned by TCPINFO.
+TEST_P(SocketInetLoopbackTest, TCPInfoState) {
+ auto const& param = GetParam();
+ TestAddress const& listener = param.listener;
+ TestAddress const& connector = param.connector;
+
+ // Create the listening socket.
+ FileDescriptor const listen_fd = ASSERT_NO_ERRNO_AND_VALUE(
+ Socket(listener.family(), SOCK_STREAM, IPPROTO_TCP));
+
+ auto state = [](int fd) -> int {
+ struct tcp_info opt = {};
+ socklen_t optLen = sizeof(opt);
+ EXPECT_THAT(getsockopt(fd, SOL_TCP, TCP_INFO, &opt, &optLen),
+ SyscallSucceeds());
+ return opt.tcpi_state;
+ };
+ ASSERT_EQ(state(listen_fd.get()), TCP_CLOSE);
+
+ sockaddr_storage listen_addr = listener.addr;
+ ASSERT_THAT(
+ bind(listen_fd.get(), AsSockAddr(&listen_addr), listener.addr_len),
+ SyscallSucceeds());
+ ASSERT_EQ(state(listen_fd.get()), TCP_CLOSE);
+
+ ASSERT_THAT(listen(listen_fd.get(), SOMAXCONN), SyscallSucceeds());
+ ASSERT_EQ(state(listen_fd.get()), TCP_LISTEN);
+
+ // Get the port bound by the listening socket.
+ socklen_t addrlen = listener.addr_len;
+ ASSERT_THAT(getsockname(listen_fd.get(), AsSockAddr(&listen_addr), &addrlen),
+ SyscallSucceeds());
+ uint16_t const port =
+ ASSERT_NO_ERRNO_AND_VALUE(AddrPort(listener.family(), listen_addr));
+
+ // Connect to the listening socket.
+ FileDescriptor conn_fd = ASSERT_NO_ERRNO_AND_VALUE(
+ Socket(connector.family(), SOCK_STREAM, IPPROTO_TCP));
+ sockaddr_storage conn_addr = connector.addr;
+ ASSERT_NO_ERRNO(SetAddrPort(connector.family(), &conn_addr, port));
+ ASSERT_EQ(state(conn_fd.get()), TCP_CLOSE);
+ ASSERT_THAT(RetryEINTR(connect)(conn_fd.get(), AsSockAddr(&conn_addr),
+ connector.addr_len),
+ SyscallSucceeds());
+ ASSERT_EQ(state(conn_fd.get()), TCP_ESTABLISHED);
+
+ auto accepted =
+ ASSERT_NO_ERRNO_AND_VALUE(Accept(listen_fd.get(), nullptr, nullptr));
+ ASSERT_EQ(state(accepted.get()), TCP_ESTABLISHED);
+
+ ASSERT_THAT(close(accepted.release()), SyscallSucceeds());
+
+ struct pollfd pfd = {
+ .fd = conn_fd.get(),
+ .events = POLLIN | POLLRDHUP,
+ };
+ constexpr int kTimeout = 10000;
+ int n = poll(&pfd, 1, kTimeout);
+ ASSERT_GE(n, 0) << strerror(errno);
+ ASSERT_EQ(n, 1);
+ if (IsRunningOnGvisor()) {
+ // TODO(gvisor.dev/issue/6015): Notify POLLRDHUP on incoming FIN.
+ ASSERT_EQ(pfd.revents, POLLIN);
+ } else {
+ ASSERT_EQ(pfd.revents, POLLIN | POLLRDHUP);
+ }
+
+ ASSERT_THAT(state(conn_fd.get()), TCP_CLOSE_WAIT);
+ ASSERT_THAT(close(conn_fd.release()), SyscallSucceeds());
+}
+
void TestHangupDuringConnect(const TestParam& param,
void (*hangup)(FileDescriptor&)) {
TestAddress const& listener = param.listener;