summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/transport
diff options
context:
space:
mode:
authorNayana Bidari <nybidari@google.com>2021-01-26 08:23:49 -0800
committergVisor bot <gvisor-bot@google.com>2021-01-26 08:25:34 -0800
commitdaf0d3f6ca3aad6f3f9ab4d762546c6dee78fa57 (patch)
tree84ebfd054c94040e516bb48703e507660b742841 /pkg/tcpip/transport
parent3946075403a93907138f13e61bdba075aeabfecf (diff)
Move SO_SNDBUF to socketops.
This CL moves {S,G}etsockopt of SO_SNDBUF from all endpoints to socketops. For unix sockets, we do not support setting of this option. PiperOrigin-RevId: 353871484
Diffstat (limited to 'pkg/tcpip/transport')
-rw-r--r--pkg/tcpip/transport/icmp/endpoint.go18
-rw-r--r--pkg/tcpip/transport/icmp/endpoint_state.go1
-rw-r--r--pkg/tcpip/transport/packet/endpoint.go43
-rw-r--r--pkg/tcpip/transport/packet/endpoint_state.go2
-rw-r--r--pkg/tcpip/transport/raw/endpoint.go42
-rw-r--r--pkg/tcpip/transport/raw/endpoint_state.go1
-rw-r--r--pkg/tcpip/transport/tcp/endpoint.go76
-rw-r--r--pkg/tcpip/transport/tcp/endpoint_state.go6
-rw-r--r--pkg/tcpip/transport/tcp/tcp_test.go12
-rw-r--r--pkg/tcpip/transport/udp/endpoint.go37
-rw-r--r--pkg/tcpip/transport/udp/endpoint_state.go1
11 files changed, 77 insertions, 162 deletions
diff --git a/pkg/tcpip/transport/icmp/endpoint.go b/pkg/tcpip/transport/icmp/endpoint.go
index af00ed548..85b497365 100644
--- a/pkg/tcpip/transport/icmp/endpoint.go
+++ b/pkg/tcpip/transport/icmp/endpoint.go
@@ -69,8 +69,7 @@ type endpoint struct {
rcvClosed bool
// The following fields are protected by the mu mutex.
- mu sync.RWMutex `state:"nosave"`
- sndBufSize int
+ mu sync.RWMutex `state:"nosave"`
// shutdownFlags represent the current shutdown state of the endpoint.
shutdownFlags tcpip.ShutdownFlags
state endpointState
@@ -94,11 +93,17 @@ func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, transProt
},
waiterQueue: waiterQueue,
rcvBufSizeMax: 32 * 1024,
- sndBufSize: 32 * 1024,
state: stateInitial,
uniqueID: s.UniqueID(),
}
- ep.ops.InitHandler(ep)
+ ep.ops.InitHandler(ep, ep.stack)
+ ep.ops.SetSendBufferSize(32*1024, false /* notify */, tcpip.GetStackSendBufferLimits)
+
+ // Override with stack defaults.
+ var ss tcpip.SendBufferSizeOption
+ if err := s.Option(&ss); err == nil {
+ ep.ops.SetSendBufferSize(int64(ss.Default), false /* notify */, tcpip.GetStackSendBufferLimits)
+ }
return ep, nil
}
@@ -363,11 +368,6 @@ func (e *endpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error) {
}
e.rcvMu.Unlock()
return v, nil
- case tcpip.SendBufferSizeOption:
- e.mu.Lock()
- v := e.sndBufSize
- e.mu.Unlock()
- return v, nil
case tcpip.ReceiveBufferSizeOption:
e.rcvMu.Lock()
diff --git a/pkg/tcpip/transport/icmp/endpoint_state.go b/pkg/tcpip/transport/icmp/endpoint_state.go
index 9d263c0ec..9335cbc5a 100644
--- a/pkg/tcpip/transport/icmp/endpoint_state.go
+++ b/pkg/tcpip/transport/icmp/endpoint_state.go
@@ -69,6 +69,7 @@ func (e *endpoint) afterLoad() {
// Resume implements tcpip.ResumableEndpoint.Resume.
func (e *endpoint) Resume(s *stack.Stack) {
e.stack = s
+ e.ops.InitHandler(e, e.stack)
if e.state != stateBound && e.state != stateConnected {
return
diff --git a/pkg/tcpip/transport/packet/endpoint.go b/pkg/tcpip/transport/packet/endpoint.go
index 6fd116a98..9ca9c9780 100644
--- a/pkg/tcpip/transport/packet/endpoint.go
+++ b/pkg/tcpip/transport/packet/endpoint.go
@@ -79,13 +79,11 @@ type endpoint struct {
rcvClosed bool
// The following fields are protected by mu.
- mu sync.RWMutex `state:"nosave"`
- sndBufSize int
- sndBufSizeMax int
- closed bool
- stats tcpip.TransportEndpointStats `state:"nosave"`
- bound bool
- boundNIC tcpip.NICID
+ mu sync.RWMutex `state:"nosave"`
+ closed bool
+ stats tcpip.TransportEndpointStats `state:"nosave"`
+ bound bool
+ boundNIC tcpip.NICID
// lastErrorMu protects lastError.
lastErrorMu sync.Mutex `state:"nosave"`
@@ -106,14 +104,13 @@ func NewEndpoint(s *stack.Stack, cooked bool, netProto tcpip.NetworkProtocolNumb
netProto: netProto,
waiterQueue: waiterQueue,
rcvBufSizeMax: 32 * 1024,
- sndBufSize: 32 * 1024,
}
- ep.ops.InitHandler(ep)
+ ep.ops.InitHandler(ep, ep.stack)
// Override with stack defaults.
- var ss stack.SendBufferSizeOption
+ var ss tcpip.SendBufferSizeOption
if err := s.Option(&ss); err == nil {
- ep.sndBufSizeMax = ss.Default
+ ep.ops.SetSendBufferSize(int64(ss.Default), false /* notify */, tcpip.GetStackSendBufferLimits)
}
var rs stack.ReceiveBufferSizeOption
@@ -320,24 +317,6 @@ func (ep *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error {
// SetSockOptInt implements tcpip.Endpoint.SetSockOptInt.
func (ep *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error {
switch opt {
- case tcpip.SendBufferSizeOption:
- // Make sure the send buffer size is within the min and max
- // allowed.
- var ss stack.SendBufferSizeOption
- if err := ep.stack.Option(&ss); err != nil {
- panic(fmt.Sprintf("s.Option(%#v) = %s", ss, err))
- }
- if v > ss.Max {
- v = ss.Max
- }
- if v < ss.Min {
- v = ss.Min
- }
- ep.mu.Lock()
- ep.sndBufSizeMax = v
- ep.mu.Unlock()
- return nil
-
case tcpip.ReceiveBufferSizeOption:
// Make sure the receive buffer size is within the min and max
// allowed.
@@ -395,12 +374,6 @@ func (ep *endpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error) {
ep.rcvMu.Unlock()
return v, nil
- case tcpip.SendBufferSizeOption:
- ep.mu.Lock()
- v := ep.sndBufSizeMax
- ep.mu.Unlock()
- return v, nil
-
case tcpip.ReceiveBufferSizeOption:
ep.rcvMu.Lock()
v := ep.rcvBufSizeMax
diff --git a/pkg/tcpip/transport/packet/endpoint_state.go b/pkg/tcpip/transport/packet/endpoint_state.go
index e2fa96d17..61221be71 100644
--- a/pkg/tcpip/transport/packet/endpoint_state.go
+++ b/pkg/tcpip/transport/packet/endpoint_state.go
@@ -63,8 +63,8 @@ func (ep *endpoint) loadRcvBufSizeMax(max int) {
// afterLoad is invoked by stateify.
func (ep *endpoint) afterLoad() {
- // StackFromEnv is a stack used specifically for save/restore.
ep.stack = stack.StackFromEnv
+ ep.ops.InitHandler(ep, ep.stack)
// TODO(gvisor.dev/173): Once bind is supported, choose the right NIC.
if err := ep.stack.RegisterPacketEndpoint(0, ep.netProto, ep); err != nil {
diff --git a/pkg/tcpip/transport/raw/endpoint.go b/pkg/tcpip/transport/raw/endpoint.go
index 2dacf5a64..f038097e4 100644
--- a/pkg/tcpip/transport/raw/endpoint.go
+++ b/pkg/tcpip/transport/raw/endpoint.go
@@ -76,12 +76,10 @@ type endpoint struct {
rcvClosed bool
// The following fields are protected by mu.
- mu sync.RWMutex `state:"nosave"`
- sndBufSize int
- sndBufSizeMax int
- closed bool
- connected bool
- bound bool
+ mu sync.RWMutex `state:"nosave"`
+ closed bool
+ connected bool
+ bound bool
// route is the route to a remote network endpoint. It is set via
// Connect(), and is valid only when conneted is true.
route *stack.Route `state:"manual"`
@@ -112,16 +110,16 @@ func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, transProt
},
waiterQueue: waiterQueue,
rcvBufSizeMax: 32 * 1024,
- sndBufSizeMax: 32 * 1024,
associated: associated,
}
- e.ops.InitHandler(e)
+ e.ops.InitHandler(e, e.stack)
e.ops.SetHeaderIncluded(!associated)
+ e.ops.SetSendBufferSize(32*1024, false /* notify */, tcpip.GetStackSendBufferLimits)
// Override with stack defaults.
- var ss stack.SendBufferSizeOption
+ var ss tcpip.SendBufferSizeOption
if err := s.Option(&ss); err == nil {
- e.sndBufSizeMax = ss.Default
+ e.ops.SetSendBufferSize(int64(ss.Default), false /* notify */, tcpip.GetStackSendBufferLimits)
}
var rs stack.ReceiveBufferSizeOption
@@ -511,24 +509,6 @@ func (e *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error {
// SetSockOptInt implements tcpip.Endpoint.SetSockOptInt.
func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error {
switch opt {
- case tcpip.SendBufferSizeOption:
- // Make sure the send buffer size is within the min and max
- // allowed.
- var ss stack.SendBufferSizeOption
- if err := e.stack.Option(&ss); err != nil {
- panic(fmt.Sprintf("s.Option(%#v) = %s", ss, err))
- }
- if v > ss.Max {
- v = ss.Max
- }
- if v < ss.Min {
- v = ss.Min
- }
- e.mu.Lock()
- e.sndBufSizeMax = v
- e.mu.Unlock()
- return nil
-
case tcpip.ReceiveBufferSizeOption:
// Make sure the receive buffer size is within the min and max
// allowed.
@@ -570,12 +550,6 @@ func (e *endpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error) {
e.rcvMu.Unlock()
return v, nil
- case tcpip.SendBufferSizeOption:
- e.mu.Lock()
- v := e.sndBufSizeMax
- e.mu.Unlock()
- return v, nil
-
case tcpip.ReceiveBufferSizeOption:
e.rcvMu.Lock()
v := e.rcvBufSizeMax
diff --git a/pkg/tcpip/transport/raw/endpoint_state.go b/pkg/tcpip/transport/raw/endpoint_state.go
index 4a7e1c039..e11ab38d5 100644
--- a/pkg/tcpip/transport/raw/endpoint_state.go
+++ b/pkg/tcpip/transport/raw/endpoint_state.go
@@ -69,6 +69,7 @@ func (e *endpoint) afterLoad() {
// Resume implements tcpip.ResumableEndpoint.Resume.
func (e *endpoint) Resume(s *stack.Stack) {
e.stack = s
+ e.ops.InitHandler(e, e.stack)
// If the endpoint is connected, re-connect.
if e.connected {
diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go
index 05c431e83..8e43fec81 100644
--- a/pkg/tcpip/transport/tcp/endpoint.go
+++ b/pkg/tcpip/transport/tcp/endpoint.go
@@ -557,7 +557,6 @@ type endpoint struct {
// When the send side is closed, the protocol goroutine is notified via
// sndCloseWaker, and sndClosed is set to true.
sndBufMu sync.Mutex `state:"nosave"`
- sndBufSize int
sndBufUsed int
sndClosed bool
sndBufInQueue seqnum.Size
@@ -869,7 +868,6 @@ func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQue
waiterQueue: waiterQueue,
state: StateInitial,
rcvBufSize: DefaultReceiveBufferSize,
- sndBufSize: DefaultSendBufferSize,
sndMTU: int(math.MaxInt32),
keepalive: keepalive{
// Linux defaults.
@@ -882,13 +880,14 @@ func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQue
windowClamp: DefaultReceiveBufferSize,
maxSynRetries: DefaultSynRetries,
}
- e.ops.InitHandler(e)
+ e.ops.InitHandler(e, e.stack)
e.ops.SetMulticastLoop(true)
e.ops.SetQuickAck(true)
+ e.ops.SetSendBufferSize(DefaultSendBufferSize, false /* notify */, GetTCPSendBufferLimits)
var ss tcpip.TCPSendBufferSizeRangeOption
if err := s.TransportProtocolOption(ProtocolNumber, &ss); err == nil {
- e.sndBufSize = ss.Default
+ e.ops.SetSendBufferSize(int64(ss.Default), false /* notify */, GetTCPSendBufferLimits)
}
var rs tcpip.TCPReceiveBufferSizeRangeOption
@@ -967,7 +966,8 @@ func (e *endpoint) Readiness(mask waiter.EventMask) waiter.EventMask {
// Determine if the endpoint is writable if requested.
if (mask & waiter.EventOut) != 0 {
e.sndBufMu.Lock()
- if e.sndClosed || e.sndBufUsed < e.sndBufSize {
+ sndBufSize := e.getSendBufferSize()
+ if e.sndClosed || e.sndBufUsed < sndBufSize {
result |= waiter.EventOut
}
e.sndBufMu.Unlock()
@@ -1499,7 +1499,8 @@ func (e *endpoint) isEndpointWritableLocked() (int, *tcpip.Error) {
return 0, tcpip.ErrClosedForSend
}
- avail := e.sndBufSize - e.sndBufUsed
+ sndBufSize := e.getSendBufferSize()
+ avail := sndBufSize - e.sndBufUsed
if avail <= 0 {
return 0, tcpip.ErrWouldBlock
}
@@ -1692,6 +1693,14 @@ func (e *endpoint) OnCorkOptionSet(v bool) {
}
}
+func (e *endpoint) getSendBufferSize() int {
+ sndBufSize, err := e.ops.GetSendBufferSize()
+ if err != nil {
+ panic(fmt.Sprintf("e.ops.GetSendBufferSize() = %s", err))
+ }
+ return int(sndBufSize)
+}
+
// SetSockOptInt sets a socket option.
func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error {
// Lower 2 bits represents ECN bits. RFC 3168, section 23.1
@@ -1785,31 +1794,6 @@ func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error {
e.rcvListMu.Unlock()
e.UnlockUser()
- case tcpip.SendBufferSizeOption:
- // Make sure the send buffer size is within the min and max
- // allowed.
- var ss tcpip.TCPSendBufferSizeRangeOption
- if err := e.stack.TransportProtocolOption(ProtocolNumber, &ss); err != nil {
- panic(fmt.Sprintf("e.stack.TransportProtocolOption(%d, %#v) = %s", ProtocolNumber, &ss, err))
- }
-
- if v > ss.Max {
- v = ss.Max
- }
-
- if v < math.MaxInt32/SegOverheadFactor {
- v *= SegOverheadFactor
- if v < ss.Min {
- v = ss.Min
- }
- } else {
- v = math.MaxInt32
- }
-
- e.sndBufMu.Lock()
- e.sndBufSize = v
- e.sndBufMu.Unlock()
-
case tcpip.TTLOption:
e.LockUser()
e.ttl = uint8(v)
@@ -1995,12 +1979,6 @@ func (e *endpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error) {
case tcpip.ReceiveQueueSizeOption:
return e.readyReceiveSize()
- case tcpip.SendBufferSizeOption:
- e.sndBufMu.Lock()
- v := e.sndBufSize
- e.sndBufMu.Unlock()
- return v, nil
-
case tcpip.ReceiveBufferSizeOption:
e.rcvListMu.Lock()
v := e.rcvBufSize
@@ -2749,13 +2727,14 @@ func (e *endpoint) HandleControlPacket(typ stack.ControlType, extra uint32, pkt
// updateSndBufferUsage is called by the protocol goroutine when room opens up
// in the send buffer. The number of newly available bytes is v.
func (e *endpoint) updateSndBufferUsage(v int) {
+ sendBufferSize := e.getSendBufferSize()
e.sndBufMu.Lock()
- notify := e.sndBufUsed >= e.sndBufSize>>1
+ notify := e.sndBufUsed >= sendBufferSize>>1
e.sndBufUsed -= v
- // We only notify when there is half the sndBufSize available after
+ // We only notify when there is half the sendBufferSize available after
// a full buffer event occurs. This ensures that we don't wake up
// writers to queue just 1-2 segments and go back to sleep.
- notify = notify && e.sndBufUsed < e.sndBufSize>>1
+ notify = notify && e.sndBufUsed < int(sendBufferSize)>>1
e.sndBufMu.Unlock()
if notify {
@@ -2967,8 +2946,9 @@ func (e *endpoint) completeState() stack.TCPEndpointState {
s.SACK.ReceivedBlocks, s.SACK.MaxSACKED = e.scoreboard.Copy()
// Copy endpoint send state.
+ sndBufSize := e.getSendBufferSize()
e.sndBufMu.Lock()
- s.SndBufSize = e.sndBufSize
+ s.SndBufSize = sndBufSize
s.SndBufUsed = e.sndBufUsed
s.SndClosed = e.sndClosed
s.SndBufInQueue = e.sndBufInQueue
@@ -3113,3 +3093,17 @@ func (e *endpoint) Wait() {
func (e *endpoint) SocketOptions() *tcpip.SocketOptions {
return &e.ops
}
+
+// GetTCPSendBufferLimits is used to get send buffer size limits for TCP.
+func GetTCPSendBufferLimits(s tcpip.StackHandler) tcpip.SendBufferSizeOption {
+ var ss tcpip.TCPSendBufferSizeRangeOption
+ if err := s.TransportProtocolOption(header.TCPProtocolNumber, &ss); err != nil {
+ panic(fmt.Sprintf("s.TransportProtocolOption(%d, %#v) = %s", header.TCPProtocolNumber, ss, err))
+ }
+
+ return tcpip.SendBufferSizeOption{
+ Min: ss.Min,
+ Default: ss.Default,
+ Max: ss.Max,
+ }
+}
diff --git a/pkg/tcpip/transport/tcp/endpoint_state.go b/pkg/tcpip/transport/tcp/endpoint_state.go
index ba67176b5..19c1dc67a 100644
--- a/pkg/tcpip/transport/tcp/endpoint_state.go
+++ b/pkg/tcpip/transport/tcp/endpoint_state.go
@@ -179,14 +179,16 @@ func (e *endpoint) afterLoad() {
// Resume implements tcpip.ResumableEndpoint.Resume.
func (e *endpoint) Resume(s *stack.Stack) {
e.stack = s
+ e.ops.InitHandler(e, e.stack)
e.segmentQueue.thaw()
epState := e.origEndpointState
switch epState {
case StateInitial, StateBound, StateListen, StateConnecting, StateEstablished:
var ss tcpip.TCPSendBufferSizeRangeOption
if err := e.stack.TransportProtocolOption(ProtocolNumber, &ss); err == nil {
- if e.sndBufSize < ss.Min || e.sndBufSize > ss.Max {
- panic(fmt.Sprintf("endpoint.sndBufSize %d is outside the min and max allowed [%d, %d]", e.sndBufSize, ss.Min, ss.Max))
+ sendBufferSize := e.getSendBufferSize()
+ if sendBufferSize < ss.Min || sendBufferSize > ss.Max {
+ panic(fmt.Sprintf("endpoint sendBufferSize %d is outside the min and max allowed [%d, %d]", sendBufferSize, ss.Min, ss.Max))
}
}
diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go
index db9dc4e25..754f6b92a 100644
--- a/pkg/tcpip/transport/tcp/tcp_test.go
+++ b/pkg/tcpip/transport/tcp/tcp_test.go
@@ -4365,9 +4365,9 @@ func checkRecvBufferSize(t *testing.T, ep tcpip.Endpoint, v int) {
func checkSendBufferSize(t *testing.T, ep tcpip.Endpoint, v int) {
t.Helper()
- s, err := ep.GetSockOptInt(tcpip.SendBufferSizeOption)
+ s, err := ep.SocketOptions().GetSendBufferSize()
if err != nil {
- t.Fatalf("GetSockOpt failed: %s", err)
+ t.Fatalf("GetSendBufferSize failed: %s", err)
}
if int(s) != v {
@@ -4473,9 +4473,7 @@ func TestMinMaxBufferSizes(t *testing.T) {
checkRecvBufferSize(t, ep, 200)
- if err := ep.SetSockOptInt(tcpip.SendBufferSizeOption, 149); err != nil {
- t.Fatalf("SetSockOptInt(SendBufferSizeOption, 299) failed: %s", err)
- }
+ ep.SocketOptions().SetSendBufferSize(149, true, tcp.GetTCPSendBufferLimits)
checkSendBufferSize(t, ep, 300)
@@ -4487,9 +4485,7 @@ func TestMinMaxBufferSizes(t *testing.T) {
// Values above max are capped at max and then doubled.
checkRecvBufferSize(t, ep, tcp.DefaultReceiveBufferSize*20*2)
- if err := ep.SetSockOptInt(tcpip.SendBufferSizeOption, 1+tcp.DefaultSendBufferSize*30); err != nil {
- t.Fatalf("SetSockOptInt(SendBufferSizeOption) failed: %s", err)
- }
+ ep.SocketOptions().SetSendBufferSize(1+tcp.DefaultSendBufferSize*30, true, tcp.GetTCPSendBufferLimits)
// Values above max are capped at max and then doubled.
checkSendBufferSize(t, ep, tcp.DefaultSendBufferSize*30*2)
diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go
index 8544fcb08..ee7e43b97 100644
--- a/pkg/tcpip/transport/udp/endpoint.go
+++ b/pkg/tcpip/transport/udp/endpoint.go
@@ -97,9 +97,7 @@ type endpoint struct {
rcvClosed bool
// The following fields are protected by the mu mutex.
- mu sync.RWMutex `state:"nosave"`
- sndBufSize int
- sndBufSizeMax int
+ mu sync.RWMutex `state:"nosave"`
// state must be read/set using the EndpointState()/setEndpointState()
// methods.
state EndpointState
@@ -176,18 +174,18 @@ func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQue
// Linux defaults to TTL=1.
multicastTTL: 1,
rcvBufSizeMax: 32 * 1024,
- sndBufSizeMax: 32 * 1024,
multicastMemberships: make(map[multicastMembership]struct{}),
state: StateInitial,
uniqueID: s.UniqueID(),
}
- e.ops.InitHandler(e)
+ e.ops.InitHandler(e, e.stack)
e.ops.SetMulticastLoop(true)
+ e.ops.SetSendBufferSize(32*1024, false /* notify */, tcpip.GetStackSendBufferLimits)
// Override with stack defaults.
- var ss stack.SendBufferSizeOption
+ var ss tcpip.SendBufferSizeOption
if err := s.Option(&ss); err == nil {
- e.sndBufSizeMax = ss.Default
+ e.ops.SetSendBufferSize(int64(ss.Default), false /* notify */, tcpip.GetStackSendBufferLimits)
}
var rs stack.ReceiveBufferSizeOption
@@ -632,25 +630,6 @@ func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error {
e.rcvBufSizeMax = v
e.mu.Unlock()
return nil
- case tcpip.SendBufferSizeOption:
- // Make sure the send buffer size is within the min and max
- // allowed.
- var ss stack.SendBufferSizeOption
- if err := e.stack.Option(&ss); err != nil {
- panic(fmt.Sprintf("e.stack.Option(%#v) = %s", ss, err))
- }
-
- if v < ss.Min {
- v = ss.Min
- }
- if v > ss.Max {
- v = ss.Max
- }
-
- e.mu.Lock()
- e.sndBufSizeMax = v
- e.mu.Unlock()
- return nil
}
return nil
@@ -811,12 +790,6 @@ func (e *endpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error) {
e.rcvMu.Unlock()
return v, nil
- case tcpip.SendBufferSizeOption:
- e.mu.Lock()
- v := e.sndBufSizeMax
- e.mu.Unlock()
- return v, nil
-
case tcpip.ReceiveBufferSizeOption:
e.rcvMu.Lock()
v := e.rcvBufSizeMax
diff --git a/pkg/tcpip/transport/udp/endpoint_state.go b/pkg/tcpip/transport/udp/endpoint_state.go
index 13b72dc88..903397f1c 100644
--- a/pkg/tcpip/transport/udp/endpoint_state.go
+++ b/pkg/tcpip/transport/udp/endpoint_state.go
@@ -91,6 +91,7 @@ func (e *endpoint) Resume(s *stack.Stack) {
defer e.mu.Unlock()
e.stack = s
+ e.ops.InitHandler(e, e.stack)
for m := range e.multicastMemberships {
if err := e.stack.JoinGroup(e.NetProto, m.nicID, m.multicastAddr); err != nil {