Refactor SOL_SOCKET options

Store all the socket level options in a struct and call {Get/Set}SockOpt on
this struct. This will avoid implementing socket level options on all
endpoints. This CL contains implementing one socket level option for tcp and
udp endpoints.

PiperOrigin-RevId: 342203981
This commit is contained in:
Nayana Bidari 2020-11-12 22:55:09 -08:00 committed by gVisor bot
parent bf392dcc7d
commit 5bb64ce1b8
20 changed files with 147 additions and 258 deletions

View File

@ -262,6 +262,9 @@ type commonEndpoint interface {
// LastError implements tcpip.Endpoint.LastError.
LastError() *tcpip.Error
// SocketOptions implements tcpip.Endpoint.SocketOptions.
SocketOptions() *tcpip.SocketOptions
}
// LINT.IfChange
@ -1163,13 +1166,8 @@ func getSockOptSocket(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, fam
return nil, syserr.ErrInvalidArgument
}
v, err := ep.GetSockOptBool(tcpip.BroadcastOption)
if err != nil {
return nil, syserr.TranslateNetstackError(err)
}
vP := primitive.Int32(boolToInt32(v))
return &vP, nil
v := primitive.Int32(boolToInt32(ep.SocketOptions().GetBroadcast()))
return &v, nil
case linux.SO_KEEPALIVE:
if outLen < sizeOfInt32 {
@ -1916,7 +1914,8 @@ func setSockOptSocket(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, nam
}
v := usermem.ByteOrder.Uint32(optVal)
return syserr.TranslateNetstackError(ep.SetSockOptBool(tcpip.BroadcastOption, v != 0))
ep.SocketOptions().SetBroadcast(v != 0)
return nil
case linux.SO_PASSCRED:
if len(optVal) < sizeOfInt32 {

View File

@ -205,6 +205,9 @@ type Endpoint interface {
// LastError implements tcpip.Endpoint.LastError.
LastError() *tcpip.Error
// SocketOptions implements tcpip.Endpoint.SocketOptions.
SocketOptions() *tcpip.SocketOptions
}
// A Credentialer is a socket or endpoint that supports the SO_PASSCRED socket
@ -757,6 +760,8 @@ type baseEndpoint struct {
// linger is used for SO_LINGER socket option.
linger tcpip.LingerOption
ops tcpip.SocketOptions
}
// EventRegister implements waiter.Waitable.EventRegister.
@ -865,7 +870,6 @@ func (e *baseEndpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error {
func (e *baseEndpoint) SetSockOptBool(opt tcpip.SockOptBool, v bool) *tcpip.Error {
switch opt {
case tcpip.BroadcastOption:
case tcpip.PasscredOption:
e.setPasscred(v)
case tcpip.ReuseAddressOption:
@ -980,6 +984,11 @@ func (*baseEndpoint) LastError() *tcpip.Error {
return nil
}
// SocketOptions implements Endpoint.SocketOptions.
func (e *baseEndpoint) SocketOptions() *tcpip.SocketOptions {
return &e.ops
}
// Shutdown closes the read and/or write end of the endpoint connection to its
// peer.
func (e *baseEndpoint) Shutdown(flags tcpip.ShutdownFlags) *syserr.Error {

View File

@ -115,6 +115,9 @@ type socketOpsCommon struct {
// bound, they cannot be modified.
abstractName string
abstractNamespace *kernel.AbstractSocketNamespace
// ops is used to get socket level options.
ops tcpip.SocketOptions
}
func (s *socketOpsCommon) isPacket() bool {

View File

@ -5,6 +5,7 @@ package(licenses = ["notice"])
go_library(
name = "tcpip",
srcs = [
"socketops.go",
"tcpip.go",
"time_unsafe.go",
"timer.go",

45
pkg/tcpip/socketops.go Normal file
View File

@ -0,0 +1,45 @@
// 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 tcpip
import (
"gvisor.dev/gvisor/pkg/sync"
)
// SocketOptions contains all the variables which store values for socket
// level options.
//
// +stateify savable
type SocketOptions struct {
// mu protects fields below.
mu sync.Mutex `state:"nosave"`
broadcastEnabled bool
}
// GetBroadcast gets value for SO_BROADCAST option.
func (so *SocketOptions) GetBroadcast() bool {
so.mu.Lock()
defer so.mu.Unlock()
return so.broadcastEnabled
}
// SetBroadcast sets value for SO_BROADCAST option.
func (so *SocketOptions) SetBroadcast(v bool) {
so.mu.Lock()
defer so.mu.Unlock()
so.broadcastEnabled = v
}

View File

@ -46,6 +46,9 @@ type fakeTransportEndpoint struct {
// acceptQueue is non-nil iff bound.
acceptQueue []fakeTransportEndpoint
// ops is used to set and get socket options.
ops tcpip.SocketOptions
}
func (f *fakeTransportEndpoint) Info() tcpip.EndpointInfo {
@ -58,6 +61,9 @@ func (*fakeTransportEndpoint) Stats() tcpip.EndpointStats {
func (*fakeTransportEndpoint) SetOwner(owner tcpip.PacketOwner) {}
func (f *fakeTransportEndpoint) SocketOptions() *tcpip.SocketOptions {
return &f.ops
}
func newFakeTransportEndpoint(proto *fakeTransportProtocol, netProto tcpip.NetworkProtocolNumber, uniqueID uint64) tcpip.Endpoint {
return &fakeTransportEndpoint{TransportEndpointInfo: stack.TransportEndpointInfo{NetProto: netProto}, proto: proto, uniqueID: uniqueID}
}
@ -183,9 +189,9 @@ func (f *fakeTransportEndpoint) Accept(*tcpip.FullAddress) (tcpip.Endpoint, *wai
if len(f.acceptQueue) == 0 {
return nil, nil, nil
}
a := f.acceptQueue[0]
a := &f.acceptQueue[0]
f.acceptQueue = f.acceptQueue[1:]
return &a, nil, nil
return a, nil, nil
}
func (f *fakeTransportEndpoint) Bind(a tcpip.FullAddress) *tcpip.Error {

View File

@ -634,6 +634,10 @@ type Endpoint interface {
// LastError clears and returns the last error reported by the endpoint.
LastError() *Error
// SocketOptions returns the structure which contains all the socket
// level options.
SocketOptions() *SocketOptions
}
// LinkPacketInfo holds Link layer information for a received packet.
@ -694,15 +698,10 @@ type WriteOptions struct {
type SockOptBool int
const (
// BroadcastOption is used by SetSockOptBool/GetSockOptBool to specify
// whether datagram sockets are allowed to send packets to a broadcast
// address.
BroadcastOption SockOptBool = iota
// CorkOption is used by SetSockOptBool/GetSockOptBool to specify if
// data should be held until segments are full by the TCP transport
// protocol.
CorkOption
CorkOption SockOptBool = iota
// DelayOption is used by SetSockOptBool/GetSockOptBool to specify if
// data should be sent out immediately by the transport protocol. For

View File

@ -514,9 +514,7 @@ func TestReuseAddrAndBroadcast(t *testing.T) {
t.Fatalf("eps[%d].SetSockOptBool(tcpip.ReuseAddressOption, true): %s", len(eps), err)
}
if err := ep.SetSockOptBool(tcpip.BroadcastOption, true); err != nil {
t.Fatalf("eps[%d].SetSockOptBool(tcpip.BroadcastOption, true): %s", len(eps), err)
}
ep.SocketOptions().SetBroadcast(true)
bindAddr := tcpip.FullAddress{Port: localPort}
if bindWildcard {

View File

@ -79,6 +79,9 @@ type endpoint struct {
// owner is used to get uid and gid of the packet.
owner tcpip.PacketOwner
// ops is used to get socket level options.
ops tcpip.SocketOptions
}
func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, transProto tcpip.TransportProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) {
@ -853,3 +856,7 @@ func (*endpoint) Wait() {}
func (*endpoint) LastError() *tcpip.Error {
return nil
}
func (e *endpoint) SocketOptions() *tcpip.SocketOptions {
return &e.ops
}

View File

@ -89,6 +89,9 @@ type endpoint struct {
// lastErrorMu protects lastError.
lastErrorMu sync.Mutex `state:"nosave"`
lastError *tcpip.Error `state:".(string)"`
// ops is used to get socket level options.
ops tcpip.SocketOptions
}
// NewEndpoint returns a new packet endpoint.
@ -549,3 +552,7 @@ func (ep *endpoint) Stats() tcpip.EndpointStats {
}
func (ep *endpoint) SetOwner(owner tcpip.PacketOwner) {}
func (ep *endpoint) SocketOptions() *tcpip.SocketOptions {
return &ep.ops
}

View File

@ -89,6 +89,9 @@ type endpoint struct {
// owner is used to get uid and gid of the packet.
owner tcpip.PacketOwner
// ops is used to get socket level options.
ops tcpip.SocketOptions
}
// NewEndpoint returns a raw endpoint for the given protocols.
@ -756,3 +759,7 @@ func (*endpoint) Wait() {}
func (*endpoint) LastError() *tcpip.Error {
return nil
}
func (e *endpoint) SocketOptions() *tcpip.SocketOptions {
return &e.ops
}

View File

@ -440,9 +440,6 @@ type endpoint struct {
ttl uint8
v6only bool
isConnectNotified bool
// TCP should never broadcast but Linux nevertheless supports enabling/
// disabling SO_BROADCAST, albeit as a NOOP.
broadcast bool
// portFlags stores the current values of port related flags.
portFlags ports.Flags
@ -685,6 +682,9 @@ type endpoint struct {
// linger is used for SO_LINGER socket option.
linger tcpip.LingerOption
// ops is used to get socket level options.
ops tcpip.SocketOptions
}
// UniqueID implements stack.TransportEndpoint.UniqueID.
@ -1599,11 +1599,6 @@ func (e *endpoint) windowCrossedACKThresholdLocked(deltaBefore int) (crossed boo
func (e *endpoint) SetSockOptBool(opt tcpip.SockOptBool, v bool) *tcpip.Error {
switch opt {
case tcpip.BroadcastOption:
e.LockUser()
e.broadcast = v
e.UnlockUser()
case tcpip.CorkOption:
e.LockUser()
if !v {
@ -1950,11 +1945,6 @@ func (e *endpoint) readyReceiveSize() (int, *tcpip.Error) {
// GetSockOptBool implements tcpip.Endpoint.GetSockOptBool.
func (e *endpoint) GetSockOptBool(opt tcpip.SockOptBool) (bool, *tcpip.Error) {
switch opt {
case tcpip.BroadcastOption:
e.LockUser()
v := e.broadcast
e.UnlockUser()
return v, nil
case tcpip.CorkOption:
return atomic.LoadUint32(&e.cork) != 0, nil
@ -3130,3 +3120,7 @@ func (e *endpoint) Wait() {
<-notifyCh
}
}
func (e *endpoint) SocketOptions() *tcpip.SocketOptions {
return &e.ops
}

View File

@ -108,7 +108,6 @@ type endpoint struct {
multicastLoop bool
portFlags ports.Flags
bindToDevice tcpip.NICID
broadcast bool
noChecksum bool
lastErrorMu sync.Mutex `state:"nosave"`
@ -157,6 +156,9 @@ type endpoint struct {
// linger is used for SO_LINGER socket option.
linger tcpip.LingerOption
// ops is used to get socket level options.
ops tcpip.SocketOptions
}
// +stateify savable
@ -508,7 +510,7 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
resolve = route.Resolve
}
if !e.broadcast && route.IsOutboundBroadcast() {
if !e.ops.GetBroadcast() && route.IsOutboundBroadcast() {
return 0, nil, tcpip.ErrBroadcastDisabled
}
@ -553,11 +555,6 @@ func (e *endpoint) Peek([][]byte) (int64, tcpip.ControlMessages, *tcpip.Error) {
// SetSockOptBool implements tcpip.Endpoint.SetSockOptBool.
func (e *endpoint) SetSockOptBool(opt tcpip.SockOptBool, v bool) *tcpip.Error {
switch opt {
case tcpip.BroadcastOption:
e.mu.Lock()
e.broadcast = v
e.mu.Unlock()
case tcpip.MulticastLoopOption:
e.mu.Lock()
e.multicastLoop = v
@ -614,7 +611,6 @@ func (e *endpoint) SetSockOptBool(opt tcpip.SockOptBool, v bool) *tcpip.Error {
e.v6only = v
}
return nil
}
@ -830,12 +826,6 @@ func (e *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error {
// GetSockOptBool implements tcpip.Endpoint.GetSockOptBool.
func (e *endpoint) GetSockOptBool(opt tcpip.SockOptBool) (bool, *tcpip.Error) {
switch opt {
case tcpip.BroadcastOption:
e.mu.RLock()
v := e.broadcast
e.mu.RUnlock()
return v, nil
case tcpip.KeepaliveEnabledOption:
return false, nil
@ -1525,3 +1515,7 @@ func isBroadcastOrMulticast(a tcpip.Address) bool {
func (e *endpoint) SetOwner(owner tcpip.PacketOwner) {
e.owner = owner
}
func (e *endpoint) SocketOptions() *tcpip.SocketOptions {
return &e.ops
}

View File

@ -364,9 +364,7 @@ func (c *testContext) createEndpointForFlow(flow testFlow) {
c.t.Fatalf("SetSockOptBool failed: %s", err)
}
} else if flow.isBroadcast() {
if err := c.ep.SetSockOptBool(tcpip.BroadcastOption, true); err != nil {
c.t.Fatalf("SetSockOptBool failed: %s", err)
}
c.ep.SocketOptions().SetBroadcast(true)
}
}
@ -2397,17 +2395,13 @@ func TestOutgoingSubnetBroadcast(t *testing.T) {
t.Fatalf("got ep.Write(_, _) = (%d, _, %v), want = (_, _, %v)", n, err, expectedErrWithoutBcastOpt)
}
if err := ep.SetSockOptBool(tcpip.BroadcastOption, true); err != nil {
t.Fatalf("got SetSockOptBool(BroadcastOption, true): %s", err)
}
ep.SocketOptions().SetBroadcast(true)
if n, _, err := ep.Write(data, opts); err != nil {
t.Fatalf("got ep.Write(_, _) = (%d, _, %s), want = (_, _, nil)", n, err)
}
if err := ep.SetSockOptBool(tcpip.BroadcastOption, false); err != nil {
t.Fatalf("got SetSockOptBool(BroadcastOption, false): %s", err)
}
ep.SocketOptions().SetBroadcast(false)
if n, _, err := ep.Write(data, opts); err != expectedErrWithoutBcastOpt {
t.Fatalf("got ep.Write(_, _) = (%d, _, %v), want = (_, _, %v)", n, err, expectedErrWithoutBcastOpt)

View File

@ -19,7 +19,6 @@ exports_files(
"socket_ip_udp_loopback_blocking.cc",
"socket_ip_udp_loopback_nonblock.cc",
"socket_ip_unbound.cc",
"socket_ipv4_tcp_unbound_external_networking_test.cc",
"socket_ipv4_udp_unbound_external_networking_test.cc",
"socket_ipv4_udp_unbound_loopback.cc",
"socket_ipv4_udp_unbound_loopback_nogotsan.cc",
@ -2505,24 +2504,6 @@ cc_library(
alwayslink = 1,
)
cc_library(
name = "socket_ipv4_tcp_unbound_external_networking_test_cases",
testonly = 1,
srcs = [
"socket_ipv4_tcp_unbound_external_networking.cc",
],
hdrs = [
"socket_ipv4_tcp_unbound_external_networking.h",
],
deps = [
":ip_socket_test_util",
":socket_test_util",
gtest,
"//test/util:test_util",
],
alwayslink = 1,
)
cc_binary(
name = "socket_abstract_test",
testonly = 1,
@ -2717,22 +2698,6 @@ cc_binary(
],
)
cc_binary(
name = "socket_ipv4_tcp_unbound_external_networking_test",
testonly = 1,
srcs = [
"socket_ipv4_tcp_unbound_external_networking_test.cc",
],
linkstatic = 1,
deps = [
":ip_socket_test_util",
":socket_ipv4_tcp_unbound_external_networking_test_cases",
":socket_test_util",
"//test/util:test_main",
"//test/util:test_util",
],
)
cc_binary(
name = "socket_bind_to_device_test",
testonly = 1,

View File

@ -818,5 +818,33 @@ TEST_P(AllSocketPairTest, GetSockoptProtocol) {
}
}
TEST_P(AllSocketPairTest, GetSockoptBroadcast) {
auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair());
int opt = -1;
socklen_t optlen = sizeof(opt);
EXPECT_THAT(
getsockopt(sockets->first_fd(), SOL_SOCKET, SO_BROADCAST, &opt, &optlen),
SyscallSucceeds());
ASSERT_EQ(optlen, sizeof(opt));
EXPECT_EQ(opt, 0);
}
TEST_P(AllSocketPairTest, SetAndGetSocketBroadcastOption) {
auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair());
int kSockOptOn = 1;
ASSERT_THAT(setsockopt(sockets->first_fd(), SOL_SOCKET, SO_BROADCAST,
&kSockOptOn, sizeof(kSockOptOn)),
SyscallSucceedsWithValue(0));
int got = -1;
socklen_t length = sizeof(got);
ASSERT_THAT(
getsockopt(sockets->first_fd(), SOL_SOCKET, SO_BROADCAST, &got, &length),
SyscallSucceedsWithValue(0));
ASSERT_EQ(length, sizeof(got));
EXPECT_EQ(got, kSockOptOn);
}
} // namespace testing
} // namespace gvisor

View File

@ -1,66 +0,0 @@
// Copyright 2019 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.
#include "test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h"
#include <netinet/in.h>
#include <sys/socket.h>
#include <sys/types.h>
#include <sys/un.h>
#include <cstdio>
#include <cstring>
#include "gmock/gmock.h"
#include "gtest/gtest.h"
#include "test/syscalls/linux/socket_test_util.h"
#include "test/util/test_util.h"
namespace gvisor {
namespace testing {
// Verifies that a newly instantiated TCP socket does not have the
// broadcast socket option enabled.
TEST_P(IPv4TCPUnboundExternalNetworkingSocketTest, TCPBroadcastDefault) {
auto socket = ASSERT_NO_ERRNO_AND_VALUE(NewSocket());
int get = -1;
socklen_t get_sz = sizeof(get);
EXPECT_THAT(
getsockopt(socket->get(), SOL_SOCKET, SO_BROADCAST, &get, &get_sz),
SyscallSucceedsWithValue(0));
EXPECT_EQ(get, kSockOptOff);
EXPECT_EQ(get_sz, sizeof(get));
}
// Verifies that a newly instantiated TCP socket returns true after enabling
// the broadcast socket option.
TEST_P(IPv4TCPUnboundExternalNetworkingSocketTest, SetTCPBroadcast) {
auto socket = ASSERT_NO_ERRNO_AND_VALUE(NewSocket());
EXPECT_THAT(setsockopt(socket->get(), SOL_SOCKET, SO_BROADCAST, &kSockOptOn,
sizeof(kSockOptOn)),
SyscallSucceedsWithValue(0));
int get = -1;
socklen_t get_sz = sizeof(get);
EXPECT_THAT(
getsockopt(socket->get(), SOL_SOCKET, SO_BROADCAST, &get, &get_sz),
SyscallSucceedsWithValue(0));
EXPECT_EQ(get, kSockOptOn);
EXPECT_EQ(get_sz, sizeof(get));
}
} // namespace testing
} // namespace gvisor

View File

@ -1,30 +0,0 @@
// Copyright 2019 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.
#ifndef GVISOR_TEST_SYSCALLS_LINUX_SOCKET_IPV4_TCP_UNBOUND_EXTERNAL_NETWORKING_H_
#define GVISOR_TEST_SYSCALLS_LINUX_SOCKET_IPV4_TCP_UNBOUND_EXTERNAL_NETWORKING_H_
#include "test/syscalls/linux/socket_test_util.h"
namespace gvisor {
namespace testing {
// Test fixture for tests that apply to unbound IPv4 TCP sockets in a sandbox
// with external networking support.
using IPv4TCPUnboundExternalNetworkingSocketTest = SimpleSocketTest;
} // namespace testing
} // namespace gvisor
#endif // GVISOR_TEST_SYSCALLS_LINUX_SOCKET_IPV4_TCP_UNBOUND_EXTERNAL_NETWORKING_H_

View File

@ -1,39 +0,0 @@
// Copyright 2019 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.
#include "test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h"
#include <vector>
#include "test/syscalls/linux/ip_socket_test_util.h"
#include "test/syscalls/linux/socket_test_util.h"
#include "test/util/test_util.h"
namespace gvisor {
namespace testing {
namespace {
std::vector<SocketKind> GetSockets() {
return ApplyVec<SocketKind>(
IPv4TCPUnboundSocket,
AllBitwiseCombinations(List<int>{0, SOCK_NONBLOCK}));
}
INSTANTIATE_TEST_SUITE_P(IPv4TCPUnboundSockets,
IPv4TCPUnboundExternalNetworkingSocketTest,
::testing::ValuesIn(GetSockets()));
} // namespace
} // namespace testing
} // namespace gvisor

View File

@ -76,38 +76,6 @@ void IPv4UDPUnboundExternalNetworkingSocketTest::SetUp() {
found_net_interfaces_ = true;
}
// Verifies that a newly instantiated UDP socket does not have the
// broadcast socket option enabled.
TEST_P(IPv4UDPUnboundExternalNetworkingSocketTest, UDPBroadcastDefault) {
auto socket = ASSERT_NO_ERRNO_AND_VALUE(NewSocket());
int get = -1;
socklen_t get_sz = sizeof(get);
EXPECT_THAT(
getsockopt(socket->get(), SOL_SOCKET, SO_BROADCAST, &get, &get_sz),
SyscallSucceedsWithValue(0));
EXPECT_EQ(get, kSockOptOff);
EXPECT_EQ(get_sz, sizeof(get));
}
// Verifies that a newly instantiated UDP socket returns true after enabling
// the broadcast socket option.
TEST_P(IPv4UDPUnboundExternalNetworkingSocketTest, SetUDPBroadcast) {
auto socket = ASSERT_NO_ERRNO_AND_VALUE(NewSocket());
EXPECT_THAT(setsockopt(socket->get(), SOL_SOCKET, SO_BROADCAST, &kSockOptOn,
sizeof(kSockOptOn)),
SyscallSucceedsWithValue(0));
int get = -1;
socklen_t get_sz = sizeof(get);
EXPECT_THAT(
getsockopt(socket->get(), SOL_SOCKET, SO_BROADCAST, &get, &get_sz),
SyscallSucceedsWithValue(0));
EXPECT_EQ(get, kSockOptOn);
EXPECT_EQ(get_sz, sizeof(get));
}
// Verifies that a broadcast UDP packet will arrive at all UDP sockets with
// the destination port number.
TEST_P(IPv4UDPUnboundExternalNetworkingSocketTest,