summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
-rw-r--r--pkg/sentry/socket/netstack/netstack.go45
-rw-r--r--pkg/sentry/socket/unix/transport/unix.go10
-rw-r--r--pkg/tcpip/tcpip.go13
-rw-r--r--pkg/tcpip/transport/tcp/endpoint.go33
-rw-r--r--test/packetimpact/testbench/dut.go16
-rw-r--r--test/packetimpact/tests/BUILD10
-rw-r--r--test/packetimpact/tests/tcp_linger_test.go253
-rw-r--r--test/syscalls/linux/socket_inet_loopback.cc23
-rw-r--r--test/syscalls/linux/socket_ip_tcp_generic.cc119
-rw-r--r--test/syscalls/linux/socket_ip_udp_generic.cc30
10 files changed, 538 insertions, 14 deletions
diff --git a/pkg/sentry/socket/netstack/netstack.go b/pkg/sentry/socket/netstack/netstack.go
index 2e568bc3d..b462924af 100644
--- a/pkg/sentry/socket/netstack/netstack.go
+++ b/pkg/sentry/socket/netstack/netstack.go
@@ -482,8 +482,35 @@ func (s *socketOpsCommon) fetchReadView() *syserr.Error {
}
// Release implements fs.FileOperations.Release.
-func (s *socketOpsCommon) Release(context.Context) {
+func (s *socketOpsCommon) Release(ctx context.Context) {
+ e, ch := waiter.NewChannelEntry(nil)
+ s.EventRegister(&e, waiter.EventHUp|waiter.EventErr)
+ defer s.EventUnregister(&e)
+
s.Endpoint.Close()
+
+ // SO_LINGER option is valid only for TCP. For other socket types
+ // return after endpoint close.
+ if family, skType, _ := s.Type(); skType != linux.SOCK_STREAM || (family != linux.AF_INET && family != linux.AF_INET6) {
+ return
+ }
+
+ var v tcpip.LingerOption
+ if err := s.Endpoint.GetSockOpt(&v); err != nil {
+ return
+ }
+
+ // The case for zero timeout is handled in tcp endpoint close function.
+ // Close is blocked until either:
+ // 1. The endpoint state is not in any of the states: FIN-WAIT1,
+ // CLOSING and LAST_ACK.
+ // 2. Timeout is reached.
+ if v.Enabled && v.Timeout != 0 {
+ t := kernel.TaskFromContext(ctx)
+ start := t.Kernel().MonotonicClock().Now()
+ deadline := start.Add(v.Timeout)
+ t.BlockWithDeadline(ch, true, deadline)
+ }
}
// Read implements fs.FileOperations.Read.
@@ -1155,7 +1182,16 @@ func getSockOptSocket(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, fam
return nil, syserr.ErrInvalidArgument
}
- linger := linux.Linger{}
+ var v tcpip.LingerOption
+ var linger linux.Linger
+ if err := ep.GetSockOpt(&v); err != nil {
+ return &linger, nil
+ }
+
+ if v.Enabled {
+ linger.OnOff = 1
+ }
+ linger.Linger = int32(v.Timeout.Seconds())
return &linger, nil
case linux.SO_SNDTIMEO:
@@ -1884,7 +1920,10 @@ func setSockOptSocket(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, nam
socket.SetSockOptEmitUnimplementedEvent(t, name)
}
- return nil
+ return syserr.TranslateNetstackError(
+ ep.SetSockOpt(&tcpip.LingerOption{
+ Enabled: v.OnOff != 0,
+ Timeout: time.Second * time.Duration(v.Linger)}))
case linux.SO_DETACH_FILTER:
// optval is ignored.
diff --git a/pkg/sentry/socket/unix/transport/unix.go b/pkg/sentry/socket/unix/transport/unix.go
index cbbdd000f..08504560c 100644
--- a/pkg/sentry/socket/unix/transport/unix.go
+++ b/pkg/sentry/socket/unix/transport/unix.go
@@ -945,8 +945,14 @@ func (e *baseEndpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error) {
// GetSockOpt implements tcpip.Endpoint.GetSockOpt.
func (e *baseEndpoint) GetSockOpt(opt tcpip.GettableSocketOption) *tcpip.Error {
- log.Warningf("Unsupported socket option: %T", opt)
- return tcpip.ErrUnknownProtocolOption
+ switch opt.(type) {
+ case *tcpip.LingerOption:
+ return nil
+
+ default:
+ log.Warningf("Unsupported socket option: %T", opt)
+ return tcpip.ErrUnknownProtocolOption
+ }
}
// LastError implements Endpoint.LastError.
diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go
index 5e34e27ba..b2ddb24ec 100644
--- a/pkg/tcpip/tcpip.go
+++ b/pkg/tcpip/tcpip.go
@@ -1194,6 +1194,19 @@ const (
TCPTimeWaitReuseLoopbackOnly
)
+// LingerOption is used by SetSockOpt/GetSockOpt to set/get the
+// duration for which a socket lingers before returning from Close.
+//
+// +stateify savable
+type LingerOption struct {
+ Enabled bool
+ Timeout time.Duration
+}
+
+func (*LingerOption) isGettableSocketOption() {}
+
+func (*LingerOption) isSettableSocketOption() {}
+
// IPPacketInfo is the message structure for IP_PKTINFO.
//
// +stateify savable
diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go
index 6d5046a3d..faea7f2bb 100644
--- a/pkg/tcpip/transport/tcp/endpoint.go
+++ b/pkg/tcpip/transport/tcp/endpoint.go
@@ -654,6 +654,9 @@ type endpoint struct {
// owner is used to get uid and gid of the packet.
owner tcpip.PacketOwner
+
+ // linger is used for SO_LINGER socket option.
+ linger tcpip.LingerOption
}
// UniqueID implements stack.TransportEndpoint.UniqueID.
@@ -1007,6 +1010,26 @@ func (e *endpoint) Close() {
return
}
+ if e.linger.Enabled && e.linger.Timeout == 0 {
+ s := e.EndpointState()
+ isResetState := s == StateEstablished || s == StateCloseWait || s == StateFinWait1 || s == StateFinWait2 || s == StateSynRecv
+ if isResetState {
+ // Close the endpoint without doing full shutdown and
+ // send a RST.
+ e.resetConnectionLocked(tcpip.ErrConnectionAborted)
+ e.closeNoShutdownLocked()
+
+ // Wake up worker to close the endpoint.
+ switch s {
+ case StateSynRecv:
+ e.notifyProtocolGoroutine(notifyClose)
+ default:
+ e.notifyProtocolGoroutine(notifyTickleWorker)
+ }
+ return
+ }
+ }
+
// Issue a shutdown so that the peer knows we won't send any more data
// if we're connected, or stop accepting if we're listening.
e.shutdownLocked(tcpip.ShutdownWrite | tcpip.ShutdownRead)
@@ -1807,6 +1830,11 @@ func (e *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error {
case *tcpip.SocketDetachFilterOption:
return nil
+ case *tcpip.LingerOption:
+ e.LockUser()
+ e.linger = *v
+ e.UnlockUser()
+
default:
return nil
}
@@ -2031,6 +2059,11 @@ func (e *endpoint) GetSockOpt(opt tcpip.GettableSocketOption) *tcpip.Error {
Port: port,
}
+ case *tcpip.LingerOption:
+ e.LockUser()
+ *o = e.linger
+ e.UnlockUser()
+
default:
return tcpip.ErrUnknownProtocolOption
}
diff --git a/test/packetimpact/testbench/dut.go b/test/packetimpact/testbench/dut.go
index ff269d949..6165ab293 100644
--- a/test/packetimpact/testbench/dut.go
+++ b/test/packetimpact/testbench/dut.go
@@ -16,11 +16,13 @@ package testbench
import (
"context"
+ "encoding/binary"
"flag"
"net"
"strconv"
"syscall"
"testing"
+ "time"
pb "gvisor.dev/gvisor/test/packetimpact/proto/posix_server_go_proto"
@@ -701,6 +703,20 @@ func (dut *DUT) RecvWithErrno(ctx context.Context, t *testing.T, sockfd, len, fl
return resp.GetRet(), resp.GetBuf(), syscall.Errno(resp.GetErrno_())
}
+// SetSockLingerOption sets SO_LINGER socket option on the DUT.
+func (dut *DUT) SetSockLingerOption(t *testing.T, sockfd int32, timeout time.Duration, enable bool) {
+ var linger unix.Linger
+ if enable {
+ linger.Onoff = 1
+ }
+ linger.Linger = int32(timeout / time.Second)
+
+ buf := make([]byte, 8)
+ binary.LittleEndian.PutUint32(buf, uint32(linger.Onoff))
+ binary.LittleEndian.PutUint32(buf[4:], uint32(linger.Linger))
+ dut.SetSockOpt(t, sockfd, unix.SOL_SOCKET, unix.SO_LINGER, buf)
+}
+
// Shutdown calls shutdown 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
// ShutdownWithErrno.
diff --git a/test/packetimpact/tests/BUILD b/test/packetimpact/tests/BUILD
index 6dda05102..f850dfcd8 100644
--- a/test/packetimpact/tests/BUILD
+++ b/test/packetimpact/tests/BUILD
@@ -320,3 +320,13 @@ packetimpact_go_test(
"@org_golang_x_sys//unix:go_default_library",
],
)
+
+packetimpact_go_test(
+ name = "tcp_linger",
+ srcs = ["tcp_linger_test.go"],
+ deps = [
+ "//pkg/tcpip/header",
+ "//test/packetimpact/testbench",
+ "@org_golang_x_sys//unix:go_default_library",
+ ],
+)
diff --git a/test/packetimpact/tests/tcp_linger_test.go b/test/packetimpact/tests/tcp_linger_test.go
new file mode 100644
index 000000000..913e49e06
--- /dev/null
+++ b/test/packetimpact/tests/tcp_linger_test.go
@@ -0,0 +1,253 @@
+// 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_linger_test
+
+import (
+ "context"
+ "flag"
+ "syscall"
+ "testing"
+ "time"
+
+ "golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/tcpip/header"
+ "gvisor.dev/gvisor/test/packetimpact/testbench"
+)
+
+func init() {
+ testbench.RegisterFlags(flag.CommandLine)
+}
+
+func createSocket(t *testing.T, dut testbench.DUT) (int32, int32, testbench.TCPIPv4) {
+ listenFD, remotePort := dut.CreateListener(t, unix.SOCK_STREAM, unix.IPPROTO_TCP, 1)
+ conn := testbench.NewTCPIPv4(t, testbench.TCP{DstPort: &remotePort}, testbench.TCP{SrcPort: &remotePort})
+ conn.Connect(t)
+ acceptFD, _ := dut.Accept(t, listenFD)
+ return acceptFD, listenFD, conn
+}
+
+func closeAll(t *testing.T, dut testbench.DUT, listenFD int32, conn testbench.TCPIPv4) {
+ conn.Close(t)
+ dut.Close(t, listenFD)
+ dut.TearDown()
+}
+
+// lingerDuration is the timeout value used with SO_LINGER socket option.
+const lingerDuration = 3 * time.Second
+
+// TestTCPLingerZeroTimeout tests when SO_LINGER is set with zero timeout. DUT
+// should send RST-ACK when socket is closed.
+func TestTCPLingerZeroTimeout(t *testing.T) {
+ // Create a socket, listen, TCP connect, and accept.
+ dut := testbench.NewDUT(t)
+ acceptFD, listenFD, conn := createSocket(t, dut)
+ defer closeAll(t, dut, listenFD, conn)
+
+ dut.SetSockLingerOption(t, acceptFD, 0, true)
+ dut.Close(t, acceptFD)
+
+ // If the linger timeout is set to zero, the DUT should send a RST.
+ if _, err := conn.Expect(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagRst | header.TCPFlagAck)}, time.Second); err != nil {
+ t.Errorf("expected RST-ACK packet within a second but got none: %s", err)
+ }
+ conn.Send(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagAck)})
+}
+
+// TestTCPLingerOff tests when SO_LINGER is not set. DUT should send FIN-ACK
+// when socket is closed.
+func TestTCPLingerOff(t *testing.T) {
+ // Create a socket, listen, TCP connect, and accept.
+ dut := testbench.NewDUT(t)
+ acceptFD, listenFD, conn := createSocket(t, dut)
+ defer closeAll(t, dut, listenFD, conn)
+
+ dut.Close(t, acceptFD)
+
+ // If SO_LINGER is not set, DUT should send a FIN-ACK.
+ if _, err := conn.Expect(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagFin | header.TCPFlagAck)}, time.Second); err != nil {
+ t.Errorf("expected FIN-ACK packet within a second but got none: %s", err)
+ }
+ conn.Send(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagAck)})
+}
+
+// TestTCPLingerNonZeroTimeout tests when SO_LINGER is set with non-zero timeout.
+// DUT should close the socket after timeout.
+func TestTCPLingerNonZeroTimeout(t *testing.T) {
+ for _, tt := range []struct {
+ description string
+ lingerOn bool
+ }{
+ {"WithNonZeroLinger", true},
+ {"WithoutLinger", false},
+ } {
+ t.Run(tt.description, func(t *testing.T) {
+ // Create a socket, listen, TCP connect, and accept.
+ dut := testbench.NewDUT(t)
+ acceptFD, listenFD, conn := createSocket(t, dut)
+ defer closeAll(t, dut, listenFD, conn)
+
+ dut.SetSockLingerOption(t, acceptFD, lingerDuration, tt.lingerOn)
+
+ // Increase timeout as Close will take longer time to
+ // return when SO_LINGER is set with non-zero timeout.
+ timeout := lingerDuration + 1*time.Second
+ ctx, cancel := context.WithTimeout(context.Background(), timeout)
+ defer cancel()
+ start := time.Now()
+ dut.CloseWithErrno(ctx, t, acceptFD)
+ end := time.Now()
+ diff := end.Sub(start)
+
+ if tt.lingerOn && diff < lingerDuration {
+ t.Errorf("expected close to return after %v seconds, but returned sooner", lingerDuration)
+ } else if !tt.lingerOn && diff > 1*time.Second {
+ t.Errorf("expected close to return within a second, but returned later")
+ }
+
+ if _, err := conn.Expect(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagFin | header.TCPFlagAck)}, time.Second); err != nil {
+ t.Errorf("expected FIN-ACK packet within a second but got none: %s", err)
+ }
+ conn.Send(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagAck)})
+ })
+ }
+}
+
+// TestTCPLingerSendNonZeroTimeout tests when SO_LINGER is set with non-zero
+// timeout and send a packet. DUT should close the socket after timeout.
+func TestTCPLingerSendNonZeroTimeout(t *testing.T) {
+ for _, tt := range []struct {
+ description string
+ lingerOn bool
+ }{
+ {"WithSendNonZeroLinger", true},
+ {"WithoutLinger", false},
+ } {
+ t.Run(tt.description, func(t *testing.T) {
+ // Create a socket, listen, TCP connect, and accept.
+ dut := testbench.NewDUT(t)
+ acceptFD, listenFD, conn := createSocket(t, dut)
+ defer closeAll(t, dut, listenFD, conn)
+
+ dut.SetSockLingerOption(t, acceptFD, lingerDuration, tt.lingerOn)
+
+ // Send data.
+ sampleData := []byte("Sample Data")
+ dut.Send(t, acceptFD, sampleData, 0)
+
+ // Increase timeout as Close will take longer time to
+ // return when SO_LINGER is set with non-zero timeout.
+ timeout := lingerDuration + 1*time.Second
+ ctx, cancel := context.WithTimeout(context.Background(), timeout)
+ defer cancel()
+ start := time.Now()
+ dut.CloseWithErrno(ctx, t, acceptFD)
+ end := time.Now()
+ diff := end.Sub(start)
+
+ if tt.lingerOn && diff < lingerDuration {
+ t.Errorf("expected close to return after %v seconds, but returned sooner", lingerDuration)
+ } else if !tt.lingerOn && diff > 1*time.Second {
+ t.Errorf("expected close to return within a second, but returned later")
+ }
+
+ samplePayload := &testbench.Payload{Bytes: sampleData}
+ if _, err := conn.ExpectData(t, &testbench.TCP{}, samplePayload, time.Second); err != nil {
+ t.Fatalf("expected a packet with payload %v: %s", samplePayload, err)
+ }
+
+ if _, err := conn.Expect(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagFin | header.TCPFlagAck)}, time.Second); err != nil {
+ t.Errorf("expected FIN-ACK packet within a second but got none: %s", err)
+ }
+ conn.Send(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagAck)})
+ })
+ }
+}
+
+// TestTCPLingerShutdownZeroTimeout tests SO_LINGER with shutdown() and zero
+// timeout. DUT should send RST-ACK when socket is closed.
+func TestTCPLingerShutdownZeroTimeout(t *testing.T) {
+ // Create a socket, listen, TCP connect, and accept.
+ dut := testbench.NewDUT(t)
+ acceptFD, listenFD, conn := createSocket(t, dut)
+ defer closeAll(t, dut, listenFD, conn)
+
+ dut.SetSockLingerOption(t, acceptFD, 0, true)
+ dut.Shutdown(t, acceptFD, syscall.SHUT_RDWR)
+ dut.Close(t, acceptFD)
+
+ // Shutdown will send FIN-ACK with read/write option.
+ if _, err := conn.Expect(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagFin | header.TCPFlagAck)}, time.Second); err != nil {
+ t.Errorf("expected FIN-ACK packet within a second but got none: %s", err)
+ }
+
+ // If the linger timeout is set to zero, the DUT should send a RST.
+ if _, err := conn.Expect(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagRst | header.TCPFlagAck)}, time.Second); err != nil {
+ t.Errorf("expected RST-ACK packet within a second but got none: %s", err)
+ }
+ conn.Send(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagAck)})
+}
+
+// TestTCPLingerShutdownSendNonZeroTimeout tests SO_LINGER with shutdown() and
+// non-zero timeout. DUT should close the socket after timeout.
+func TestTCPLingerShutdownSendNonZeroTimeout(t *testing.T) {
+ for _, tt := range []struct {
+ description string
+ lingerOn bool
+ }{
+ {"shutdownRDWR", true},
+ {"shutdownRDWR", false},
+ } {
+ t.Run(tt.description, func(t *testing.T) {
+ // Create a socket, listen, TCP connect, and accept.
+ dut := testbench.NewDUT(t)
+ acceptFD, listenFD, conn := createSocket(t, dut)
+ defer closeAll(t, dut, listenFD, conn)
+
+ dut.SetSockLingerOption(t, acceptFD, lingerDuration, tt.lingerOn)
+
+ // Send data.
+ sampleData := []byte("Sample Data")
+ dut.Send(t, acceptFD, sampleData, 0)
+
+ dut.Shutdown(t, acceptFD, syscall.SHUT_RDWR)
+
+ // Increase timeout as Close will take longer time to
+ // return when SO_LINGER is set with non-zero timeout.
+ timeout := lingerDuration + 1*time.Second
+ ctx, cancel := context.WithTimeout(context.Background(), timeout)
+ defer cancel()
+ start := time.Now()
+ dut.CloseWithErrno(ctx, t, acceptFD)
+ end := time.Now()
+ diff := end.Sub(start)
+
+ if tt.lingerOn && diff < lingerDuration {
+ t.Errorf("expected close to return after %v seconds, but returned sooner", lingerDuration)
+ } else if !tt.lingerOn && diff > 1*time.Second {
+ t.Errorf("expected close to return within a second, but returned later")
+ }
+
+ samplePayload := &testbench.Payload{Bytes: sampleData}
+ if _, err := conn.ExpectData(t, &testbench.TCP{}, samplePayload, time.Second); err != nil {
+ t.Fatalf("expected a packet with payload %v: %s", samplePayload, err)
+ }
+
+ if _, err := conn.Expect(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagFin | header.TCPFlagAck)}, time.Second); err != nil {
+ t.Errorf("expected FIN-ACK packet within a second but got none: %s", err)
+ }
+ conn.Send(t, testbench.TCP{Flags: testbench.Uint8(header.TCPFlagAck)})
+ })
+ }
+}
diff --git a/test/syscalls/linux/socket_inet_loopback.cc b/test/syscalls/linux/socket_inet_loopback.cc
index 54fee2e82..11fcec443 100644
--- a/test/syscalls/linux/socket_inet_loopback.cc
+++ b/test/syscalls/linux/socket_inet_loopback.cc
@@ -1116,9 +1116,6 @@ TEST_P(SocketInetLoopbackTest, TCPAcceptAfterReset) {
TestAddress const& listener = param.listener;
TestAddress const& connector = param.connector;
- // TODO(gvisor.dev/issue/1400): Remove this after SO_LINGER is fixed.
- SKIP_IF(IsRunningOnGvisor());
-
// Create the listening socket.
const FileDescriptor listen_fd = ASSERT_NO_ERRNO_AND_VALUE(
Socket(listener.family(), SOCK_STREAM, IPPROTO_TCP));
@@ -1178,12 +1175,20 @@ TEST_P(SocketInetLoopbackTest, TCPAcceptAfterReset) {
listen_fd.get(), reinterpret_cast<sockaddr*>(&accept_addr), &addrlen));
ASSERT_EQ(addrlen, listener.addr_len);
- int err;
- socklen_t optlen = sizeof(err);
- ASSERT_THAT(getsockopt(accept_fd.get(), SOL_SOCKET, SO_ERROR, &err, &optlen),
- SyscallSucceeds());
- ASSERT_EQ(err, ECONNRESET);
- ASSERT_EQ(optlen, sizeof(err));
+ // TODO(gvisor.dev/issue/3812): Remove after SO_ERROR is fixed.
+ if (IsRunningOnGvisor()) {
+ char buf[10];
+ ASSERT_THAT(ReadFd(accept_fd.get(), buf, sizeof(buf)),
+ SyscallFailsWithErrno(ECONNRESET));
+ } else {
+ int err;
+ socklen_t optlen = sizeof(err);
+ ASSERT_THAT(
+ getsockopt(accept_fd.get(), SOL_SOCKET, SO_ERROR, &err, &optlen),
+ SyscallSucceeds());
+ ASSERT_EQ(err, ECONNRESET);
+ ASSERT_EQ(optlen, sizeof(err));
+ }
}
// TODO(gvisor.dev/issue/1688): Partially completed passive endpoints are not
diff --git a/test/syscalls/linux/socket_ip_tcp_generic.cc b/test/syscalls/linux/socket_ip_tcp_generic.cc
index 04356b780..f4b69c46c 100644
--- a/test/syscalls/linux/socket_ip_tcp_generic.cc
+++ b/test/syscalls/linux/socket_ip_tcp_generic.cc
@@ -1080,5 +1080,124 @@ TEST_P(TCPSocketPairTest, TCPResetDuringClose_NoRandomSave) {
}
}
+// Test setsockopt and getsockopt for a socket with SO_LINGER option.
+TEST_P(TCPSocketPairTest, SetAndGetLingerOption) {
+ auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair());
+
+ // Check getsockopt before SO_LINGER option is set.
+ struct linger got_linger = {-1, -1};
+ socklen_t got_len = sizeof(got_linger);
+
+ ASSERT_THAT(getsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER,
+ &got_linger, &got_len),
+ SyscallSucceeds());
+ ASSERT_THAT(got_len, sizeof(got_linger));
+ struct linger want_linger = {};
+ EXPECT_EQ(0, memcmp(&want_linger, &got_linger, got_len));
+
+ // Set and get SO_LINGER with negative values.
+ struct linger sl;
+ sl.l_onoff = 1;
+ sl.l_linger = -3;
+ ASSERT_THAT(
+ setsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER, &sl, sizeof(sl)),
+ SyscallSucceeds());
+ ASSERT_THAT(getsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER,
+ &got_linger, &got_len),
+ SyscallSucceeds());
+ ASSERT_EQ(got_len, sizeof(got_linger));
+ EXPECT_EQ(sl.l_onoff, got_linger.l_onoff);
+ // Linux returns a different value as it uses HZ to convert the seconds to
+ // jiffies which overflows for negative values. We want to be compatible with
+ // linux for getsockopt return value.
+ if (IsRunningOnGvisor()) {
+ EXPECT_EQ(sl.l_linger, got_linger.l_linger);
+ }
+
+ // Set and get SO_LINGER option with positive values.
+ sl.l_onoff = 1;
+ sl.l_linger = 5;
+ ASSERT_THAT(
+ setsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER, &sl, sizeof(sl)),
+ SyscallSucceeds());
+ ASSERT_THAT(getsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER,
+ &got_linger, &got_len),
+ SyscallSucceeds());
+ ASSERT_EQ(got_len, sizeof(got_linger));
+ EXPECT_EQ(0, memcmp(&sl, &got_linger, got_len));
+}
+
+// Test socket to disable SO_LINGER option.
+TEST_P(TCPSocketPairTest, SetOffLingerOption) {
+ auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair());
+
+ // Set the SO_LINGER option.
+ struct linger sl;
+ sl.l_onoff = 1;
+ sl.l_linger = 5;
+ ASSERT_THAT(
+ setsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER, &sl, sizeof(sl)),
+ SyscallSucceeds());
+
+ // Check getsockopt after SO_LINGER option is set.
+ struct linger got_linger = {-1, -1};
+ socklen_t got_len = sizeof(got_linger);
+ ASSERT_THAT(getsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER,
+ &got_linger, &got_len),
+ SyscallSucceeds());
+ ASSERT_EQ(got_len, sizeof(got_linger));
+ EXPECT_EQ(0, memcmp(&sl, &got_linger, got_len));
+
+ sl.l_onoff = 0;
+ sl.l_linger = 5;
+ ASSERT_THAT(
+ setsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER, &sl, sizeof(sl)),
+ SyscallSucceeds());
+
+ // Check getsockopt after SO_LINGER option is set to zero.
+ ASSERT_THAT(getsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER,
+ &got_linger, &got_len),
+ SyscallSucceeds());
+ ASSERT_EQ(got_len, sizeof(got_linger));
+ EXPECT_EQ(0, memcmp(&sl, &got_linger, got_len));
+}
+
+// Test close on dup'd socket with SO_LINGER option set.
+TEST_P(TCPSocketPairTest, CloseWithLingerOption) {
+ auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair());
+
+ // Set the SO_LINGER option.
+ struct linger sl;
+ sl.l_onoff = 1;
+ sl.l_linger = 5;
+ ASSERT_THAT(
+ setsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER, &sl, sizeof(sl)),
+ SyscallSucceeds());
+
+ // Check getsockopt after SO_LINGER option is set.
+ struct linger got_linger = {-1, -1};
+ socklen_t got_len = sizeof(got_linger);
+ ASSERT_THAT(getsockopt(sockets->first_fd(), SOL_SOCKET, SO_LINGER,
+ &got_linger, &got_len),
+ SyscallSucceeds());
+ ASSERT_EQ(got_len, sizeof(got_linger));
+ EXPECT_EQ(0, memcmp(&sl, &got_linger, got_len));
+
+ FileDescriptor dupFd = FileDescriptor(dup(sockets->first_fd()));
+ ASSERT_THAT(close(sockets->release_first_fd()), SyscallSucceeds());
+ char buf[10] = {};
+ // Write on dupFd should succeed as socket will not be closed until
+ // all references are removed.
+ ASSERT_THAT(RetryEINTR(write)(dupFd.get(), buf, sizeof(buf)),
+ SyscallSucceedsWithValue(sizeof(buf)));
+ ASSERT_THAT(RetryEINTR(write)(sockets->first_fd(), buf, sizeof(buf)),
+ SyscallFailsWithErrno(EBADF));
+
+ // Close the socket.
+ dupFd.reset();
+ // Write on dupFd should fail as all references for socket are removed.
+ ASSERT_THAT(RetryEINTR(write)(dupFd.get(), buf, sizeof(buf)),
+ SyscallFailsWithErrno(EBADF));
+}
} // namespace testing
} // namespace gvisor
diff --git a/test/syscalls/linux/socket_ip_udp_generic.cc b/test/syscalls/linux/socket_ip_udp_generic.cc
index bbe356116..6e4ecd680 100644
--- a/test/syscalls/linux/socket_ip_udp_generic.cc
+++ b/test/syscalls/linux/socket_ip_udp_generic.cc
@@ -450,5 +450,35 @@ TEST_P(UDPSocketPairTest, TClassRecvMismatch) {
SyscallFailsWithErrno(EOPNOTSUPP));
}
+// Test the SO_LINGER option can be set/get on udp socket.
+TEST_P(UDPSocketPairTest, SoLingerFail) {
+ auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair());
+ int level = SOL_SOCKET;
+ int type = SO_LINGER;
+
+ struct linger sl;
+ sl.l_onoff = 1;
+ sl.l_linger = 5;
+ ASSERT_THAT(setsockopt(sockets->first_fd(), level, type, &sl, sizeof(sl)),
+ SyscallSucceedsWithValue(0));
+
+ struct linger got_linger = {};
+ socklen_t length = sizeof(sl);
+ ASSERT_THAT(
+ getsockopt(sockets->first_fd(), level, type, &got_linger, &length),
+ SyscallSucceedsWithValue(0));
+
+ ASSERT_EQ(length, sizeof(got_linger));
+ // Linux returns the values which are set in the SetSockOpt for SO_LINGER.
+ // In gVisor, we do not store the linger values for UDP as SO_LINGER for UDP
+ // is a no-op.
+ if (IsRunningOnGvisor()) {
+ struct linger want_linger = {};
+ EXPECT_EQ(0, memcmp(&want_linger, &got_linger, length));
+ } else {
+ EXPECT_EQ(0, memcmp(&sl, &got_linger, length));
+ }
+}
+
} // namespace testing
} // namespace gvisor