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/dut/posix_server.cc9
-rw-r--r--test/packetimpact/proto/posix_server.proto11
-rw-r--r--test/packetimpact/testbench/dut.go42
-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.cc3
-rw-r--r--test/syscalls/linux/socket_ip_tcp_generic.cc119
-rw-r--r--test/syscalls/linux/socket_ip_udp_generic.cc30
12 files changed, 8 insertions, 570 deletions
diff --git a/pkg/sentry/socket/netstack/netstack.go b/pkg/sentry/socket/netstack/netstack.go
index 0bf21f7d8..36c17d1ba 100644
--- a/pkg/sentry/socket/netstack/netstack.go
+++ b/pkg/sentry/socket/netstack/netstack.go
@@ -482,35 +482,8 @@ func (s *socketOpsCommon) fetchReadView() *syserr.Error {
}
// Release implements fs.FileOperations.Release.
-func (s *socketOpsCommon) Release(ctx context.Context) {
- e, ch := waiter.NewChannelEntry(nil)
- s.EventRegister(&e, waiter.EventHUp|waiter.EventErr)
- defer s.EventUnregister(&e)
-
+func (s *socketOpsCommon) Release(context.Context) {
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.
@@ -1184,16 +1157,7 @@ func getSockOptSocket(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, fam
return nil, syserr.ErrInvalidArgument
}
- 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())
+ linger := linux.Linger{}
return &linger, nil
case linux.SO_SNDTIMEO:
@@ -1922,10 +1886,7 @@ func setSockOptSocket(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, nam
socket.SetSockOptEmitUnimplementedEvent(t, name)
}
- return syserr.TranslateNetstackError(
- ep.SetSockOpt(&tcpip.LingerOption{
- Enabled: v.OnOff != 0,
- Timeout: time.Second * time.Duration(v.Linger)}))
+ return nil
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 cc9d650fb..1200cf9bb 100644
--- a/pkg/sentry/socket/unix/transport/unix.go
+++ b/pkg/sentry/socket/unix/transport/unix.go
@@ -942,14 +942,8 @@ func (e *baseEndpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error) {
// GetSockOpt implements tcpip.Endpoint.GetSockOpt.
func (e *baseEndpoint) GetSockOpt(opt tcpip.GettableSocketOption) *tcpip.Error {
- switch opt.(type) {
- case *tcpip.LingerOption:
- return nil
-
- default:
- log.Warningf("Unsupported socket option: %T", opt)
- return tcpip.ErrUnknownProtocolOption
- }
+ 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 47a8d7c86..b113d8613 100644
--- a/pkg/tcpip/tcpip.go
+++ b/pkg/tcpip/tcpip.go
@@ -1074,19 +1074,6 @@ 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 c5d9eba5d..3f18efeef 100644
--- a/pkg/tcpip/transport/tcp/endpoint.go
+++ b/pkg/tcpip/transport/tcp/endpoint.go
@@ -654,9 +654,6 @@ 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.
@@ -1010,26 +1007,6 @@ 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)
@@ -1830,11 +1807,6 @@ 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
}
@@ -2057,11 +2029,6 @@ 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/dut/posix_server.cc b/test/packetimpact/dut/posix_server.cc
index de5b4be93..2476998f8 100644
--- a/test/packetimpact/dut/posix_server.cc
+++ b/test/packetimpact/dut/posix_server.cc
@@ -336,15 +336,6 @@ class PosixImpl final : public posix_server::Posix::Service {
return ::grpc::Status::OK;
}
- ::grpc::Status Shutdown(grpc_impl::ServerContext *context,
- const ::posix_server::ShutdownRequest *request,
- ::posix_server::ShutdownResponse *response) override {
- if (shutdown(request->fd(), request->how()) < 0) {
- response->set_errno_(errno);
- }
- return ::grpc::Status::OK;
- }
-
::grpc::Status Recv(::grpc::ServerContext *context,
const ::posix_server::RecvRequest *request,
::posix_server::RecvResponse *response) override {
diff --git a/test/packetimpact/proto/posix_server.proto b/test/packetimpact/proto/posix_server.proto
index f32ed54ef..ccd20b10d 100644
--- a/test/packetimpact/proto/posix_server.proto
+++ b/test/packetimpact/proto/posix_server.proto
@@ -188,15 +188,6 @@ message SocketResponse {
int32 errno_ = 2; // "errno" may fail to compile in c++.
}
-message ShutdownRequest {
- int32 fd = 1;
- int32 how = 2;
-}
-
-message ShutdownResponse {
- int32 errno_ = 1; // "errno" may fail to compile in c++.
-}
-
message RecvRequest {
int32 sockfd = 1;
int32 len = 2;
@@ -234,8 +225,6 @@ service Posix {
rpc SetSockOpt(SetSockOptRequest) returns (SetSockOptResponse);
// Call socket() on the DUT.
rpc Socket(SocketRequest) returns (SocketResponse);
- // Call shutdown() on the DUT.
- rpc Shutdown(ShutdownRequest) returns (ShutdownResponse);
// Call recv() on the DUT.
rpc Recv(RecvRequest) returns (RecvResponse);
}
diff --git a/test/packetimpact/testbench/dut.go b/test/packetimpact/testbench/dut.go
index 6165ab293..73c532e75 100644
--- a/test/packetimpact/testbench/dut.go
+++ b/test/packetimpact/testbench/dut.go
@@ -16,13 +16,11 @@ package testbench
import (
"context"
- "encoding/binary"
"flag"
"net"
"strconv"
"syscall"
"testing"
- "time"
pb "gvisor.dev/gvisor/test/packetimpact/proto/posix_server_go_proto"
@@ -702,43 +700,3 @@ 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.
-func (dut *DUT) Shutdown(t *testing.T, fd, how int32) error {
- t.Helper()
-
- ctx, cancel := context.WithTimeout(context.Background(), RPCTimeout)
- defer cancel()
- return dut.ShutdownWithErrno(ctx, t, fd, how)
-}
-
-// ShutdownWithErrno calls shutdown on the DUT.
-func (dut *DUT) ShutdownWithErrno(ctx context.Context, t *testing.T, fd, how int32) error {
- t.Helper()
-
- req := pb.ShutdownRequest{
- Fd: fd,
- How: how,
- }
- resp, err := dut.posixServer.Shutdown(ctx, &req)
- if err != nil {
- t.Fatalf("failed to call Shutdown: %s", err)
- }
- return syscall.Errno(resp.GetErrno_())
-}
diff --git a/test/packetimpact/tests/BUILD b/test/packetimpact/tests/BUILD
index 7a7152fa5..74658fea0 100644
--- a/test/packetimpact/tests/BUILD
+++ b/test/packetimpact/tests/BUILD
@@ -308,13 +308,3 @@ 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
deleted file mode 100644
index 913e49e06..000000000
--- a/test/packetimpact/tests/tcp_linger_test.go
+++ /dev/null
@@ -1,253 +0,0 @@
-// 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 425084228..ffcd90475 100644
--- a/test/syscalls/linux/socket_inet_loopback.cc
+++ b/test/syscalls/linux/socket_inet_loopback.cc
@@ -1116,6 +1116,9 @@ 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));
diff --git a/test/syscalls/linux/socket_ip_tcp_generic.cc b/test/syscalls/linux/socket_ip_tcp_generic.cc
index f4b69c46c..04356b780 100644
--- a/test/syscalls/linux/socket_ip_tcp_generic.cc
+++ b/test/syscalls/linux/socket_ip_tcp_generic.cc
@@ -1080,124 +1080,5 @@ 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 6e4ecd680..bbe356116 100644
--- a/test/syscalls/linux/socket_ip_udp_generic.cc
+++ b/test/syscalls/linux/socket_ip_udp_generic.cc
@@ -450,35 +450,5 @@ 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