summaryrefslogtreecommitdiffhomepage
path: root/pkg
diff options
context:
space:
mode:
Diffstat (limited to 'pkg')
-rw-r--r--pkg/sentry/socket/netstack/netstack.go19
-rw-r--r--pkg/sentry/socket/socket.go1
-rw-r--r--pkg/tcpip/checker/checker.go16
-rw-r--r--pkg/tcpip/tcpip.go100
-rw-r--r--pkg/tcpip/transport/udp/endpoint.go24
-rw-r--r--pkg/tcpip/transport/udp/udp_test.go24
6 files changed, 135 insertions, 49 deletions
diff --git a/pkg/sentry/socket/netstack/netstack.go b/pkg/sentry/socket/netstack/netstack.go
index c0b63a803..e7d2c83d7 100644
--- a/pkg/sentry/socket/netstack/netstack.go
+++ b/pkg/sentry/socket/netstack/netstack.go
@@ -1169,6 +1169,17 @@ func getSockOptSocket(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, fam
return int32(v), nil
+ case linux.SO_NO_CHECK:
+ if outLen < sizeOfInt32 {
+ return nil, syserr.ErrInvalidArgument
+ }
+
+ v, err := ep.GetSockOptBool(tcpip.NoChecksumOption)
+ if err != nil {
+ return nil, syserr.TranslateNetstackError(err)
+ }
+ return boolToInt32(v), nil
+
default:
socket.GetSockOptEmitUnimplementedEvent(t, name)
}
@@ -1720,6 +1731,14 @@ func setSockOptSocket(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, nam
return syserr.TranslateNetstackError(ep.SetSockOpt(tcpip.OutOfBandInlineOption(v)))
+ case linux.SO_NO_CHECK:
+ if len(optVal) < sizeOfInt32 {
+ return syserr.ErrInvalidArgument
+ }
+
+ v := usermem.ByteOrder.Uint32(optVal)
+ return syserr.TranslateNetstackError(ep.SetSockOptBool(tcpip.NoChecksumOption, v != 0))
+
case linux.SO_LINGER:
if len(optVal) < linux.SizeOfLinger {
return syserr.ErrInvalidArgument
diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go
index 6580bd6e9..fcd7f9d7f 100644
--- a/pkg/sentry/socket/socket.go
+++ b/pkg/sentry/socket/socket.go
@@ -407,7 +407,6 @@ func emitUnimplementedEvent(t *kernel.Task, name int) {
linux.SO_MARK,
linux.SO_MAX_PACING_RATE,
linux.SO_NOFCS,
- linux.SO_NO_CHECK,
linux.SO_OOBINLINE,
linux.SO_PASSCRED,
linux.SO_PASSSEC,
diff --git a/pkg/tcpip/checker/checker.go b/pkg/tcpip/checker/checker.go
index c1745ba6a..ee264b726 100644
--- a/pkg/tcpip/checker/checker.go
+++ b/pkg/tcpip/checker/checker.go
@@ -320,6 +320,22 @@ func DstPort(port uint16) TransportChecker {
}
}
+// NoChecksum creates a checker that checks if the checksum is zero.
+func NoChecksum(noChecksum bool) TransportChecker {
+ return func(t *testing.T, h header.Transport) {
+ t.Helper()
+
+ udp, ok := h.(header.UDP)
+ if !ok {
+ return
+ }
+
+ if b := udp.Checksum() == 0; b != noChecksum {
+ t.Errorf("bad checksum state, got %t, want %t", b, noChecksum)
+ }
+ }
+}
+
// SeqNum creates a checker that checks the sequence number.
func SeqNum(seq uint32) TransportChecker {
return func(t *testing.T, h header.Transport) {
diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go
index 4d45dcc42..2be1c107a 100644
--- a/pkg/tcpip/tcpip.go
+++ b/pkg/tcpip/tcpip.go
@@ -585,59 +585,68 @@ type WriteOptions struct {
type SockOptBool int
const (
- // BroadcastOption is used by SetSockOpt/GetSockOpt to specify whether
- // datagram sockets are allowed to send packets to a broadcast address.
+ // 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 SetSockOpt/GetSockOpt to specify if data should be
- // held until segments are full by the TCP transport protocol.
+ // CorkOption is used by SetSockOptBool/GetSockOptBool to specify if
+ // data should be held until segments are full by the TCP transport
+ // protocol.
CorkOption
- // DelayOption is used by SetSockOpt/GetSockOpt to specify if data
- // should be sent out immediately by the transport protocol. For TCP,
- // it determines if the Nagle algorithm is on or off.
+ // DelayOption is used by SetSockOptBool/GetSockOptBool to specify if
+ // data should be sent out immediately by the transport protocol. For
+ // TCP, it determines if the Nagle algorithm is on or off.
DelayOption
- // KeepaliveEnabledOption is used by SetSockOpt/GetSockOpt to specify whether
- // TCP keepalive is enabled for this socket.
+ // KeepaliveEnabledOption is used by SetSockOptBool/GetSockOptBool to
+ // specify whether TCP keepalive is enabled for this socket.
KeepaliveEnabledOption
- // MulticastLoopOption is used by SetSockOpt/GetSockOpt to specify whether
- // multicast packets sent over a non-loopback interface will be looped back.
+ // MulticastLoopOption is used by SetSockOptBool/GetSockOptBool to
+ // specify whether multicast packets sent over a non-loopback interface
+ // will be looped back.
MulticastLoopOption
- // PasscredOption is used by SetSockOpt/GetSockOpt to specify whether
- // SCM_CREDENTIALS socket control messages are enabled.
+ // NoChecksumOption is used by SetSockOptBool/GetSockOptBool to specify
+ // whether UDP checksum is disabled for this socket.
+ NoChecksumOption
+
+ // PasscredOption is used by SetSockOptBool/GetSockOptBool to specify
+ // whether SCM_CREDENTIALS socket control messages are enabled.
//
// Only supported on Unix sockets.
PasscredOption
- // QuickAckOption is stubbed out in SetSockOpt/GetSockOpt.
+ // QuickAckOption is stubbed out in SetSockOptBool/GetSockOptBool.
QuickAckOption
- // ReceiveTClassOption is used by SetSockOpt/GetSockOpt to specify if the
- // IPV6_TCLASS ancillary message is passed with incoming packets.
+ // ReceiveTClassOption is used by SetSockOptBool/GetSockOptBool to
+ // specify if the IPV6_TCLASS ancillary message is passed with incoming
+ // packets.
ReceiveTClassOption
- // ReceiveTOSOption is used by SetSockOpt/GetSockOpt to specify if the TOS
- // ancillary message is passed with incoming packets.
+ // ReceiveTOSOption is used by SetSockOptBool/GetSockOptBool to specify
+ // if the TOS ancillary message is passed with incoming packets.
ReceiveTOSOption
- // ReceiveIPPacketInfoOption is used by {G,S}etSockOptBool to specify
- // if more inforamtion is provided with incoming packets such
- // as interface index and address.
+ // ReceiveIPPacketInfoOption is used by SetSockOptBool/GetSockOptBool to
+ // specify if more inforamtion is provided with incoming packets such as
+ // interface index and address.
ReceiveIPPacketInfoOption
- // ReuseAddressOption is used by SetSockOpt/GetSockOpt to specify whether Bind()
- // should allow reuse of local address.
+ // ReuseAddressOption is used by SetSockOptBool/GetSockOptBool to
+ // specify whether Bind() should allow reuse of local address.
ReuseAddressOption
- // ReusePortOption is used by SetSockOpt/GetSockOpt to permit multiple sockets
- // to be bound to an identical socket address.
+ // ReusePortOption is used by SetSockOptBool/GetSockOptBool to permit
+ // multiple sockets to be bound to an identical socket address.
ReusePortOption
- // V6OnlyOption is used by {G,S}etSockOptBool to specify whether an IPv6
- // socket is to be restricted to sending and receiving IPv6 packets only.
+ // V6OnlyOption is used by SetSockOptBool/GetSockOptBool to specify
+ // whether an IPv6 socket is to be restricted to sending and receiving
+ // IPv6 packets only.
V6OnlyOption
)
@@ -645,25 +654,27 @@ const (
type SockOptInt int
const (
- // KeepaliveCountOption is used by SetSockOpt/GetSockOpt to specify the number
- // of un-ACKed TCP keepalives that will be sent before the connection is
- // closed.
+ // KeepaliveCountOption is used by SetSockOptInt/GetSockOptInt to
+ // specify the number of un-ACKed TCP keepalives that will be sent
+ // before the connection is closed.
KeepaliveCountOption SockOptInt = iota
- // IPv4TOSOption is used by SetSockOpt/GetSockOpt to specify TOS
+ // IPv4TOSOption is used by SetSockOptInt/GetSockOptInt to specify TOS
// for all subsequent outgoing IPv4 packets from the endpoint.
IPv4TOSOption
- // IPv6TrafficClassOption is used by SetSockOpt/GetSockOpt to specify TOS
- // for all subsequent outgoing IPv6 packets from the endpoint.
+ // IPv6TrafficClassOption is used by SetSockOptInt/GetSockOptInt to
+ // specify TOS for all subsequent outgoing IPv6 packets from the
+ // endpoint.
IPv6TrafficClassOption
- // MaxSegOption is used by SetSockOpt/GetSockOpt to set/get the current
- // Maximum Segment Size(MSS) value as specified using the TCP_MAXSEG option.
+ // MaxSegOption is used by SetSockOptInt/GetSockOptInt to set/get the
+ // current Maximum Segment Size(MSS) value as specified using the
+ // TCP_MAXSEG option.
MaxSegOption
- // MulticastTTLOption is used by SetSockOpt/GetSockOpt to control the default
- // TTL value for multicast messages. The default is 1.
+ // MulticastTTLOption is used by SetSockOptInt/GetSockOptInt to control
+ // the default TTL value for multicast messages. The default is 1.
MulticastTTLOption
// ReceiveQueueSizeOption is used in GetSockOptInt to specify that the
@@ -682,21 +693,22 @@ const (
// number of unread bytes in the output buffer should be returned.
SendQueueSizeOption
- // TTLOption is used by SetSockOpt/GetSockOpt to control the default TTL/hop
- // limit value for unicast messages. The default is protocol specific.
+ // TTLOption is used by SetSockOptInt/GetSockOptInt to control the
+ // default TTL/hop limit value for unicast messages. The default is
+ // protocol specific.
//
// A zero value indicates the default.
TTLOption
- // TCPSynCountOption is used by SetSockOpt/GetSockOpt to specify the number of
- // SYN retransmits that TCP should send before aborting the attempt to
- // connect. It cannot exceed 255.
+ // TCPSynCountOption is used by SetSockOptInt/GetSockOptInt to specify
+ // the number of SYN retransmits that TCP should send before aborting
+ // the attempt to connect. It cannot exceed 255.
//
// NOTE: This option is currently only stubbed out and is no-op.
TCPSynCountOption
- // TCPWindowClampOption is used by SetSockOpt/GetSockOpt to bound the size
- // of the advertised window to this value.
+ // TCPWindowClampOption is used by SetSockOptInt/GetSockOptInt to bound
+ // the size of the advertised window to this value.
//
// NOTE: This option is currently only stubed out and is a no-op
TCPWindowClampOption
diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go
index 8bdc1ee1f..cae29fbff 100644
--- a/pkg/tcpip/transport/udp/endpoint.go
+++ b/pkg/tcpip/transport/udp/endpoint.go
@@ -109,6 +109,7 @@ type endpoint struct {
portFlags ports.Flags
bindToDevice tcpip.NICID
broadcast bool
+ noChecksum bool
lastErrorMu sync.Mutex `state:"nosave"`
lastError *tcpip.Error `state:".(string)"`
@@ -529,7 +530,7 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
useDefaultTTL = false
}
- if err := sendUDP(route, buffer.View(v).ToVectorisedView(), e.ID.LocalPort, dstPort, ttl, useDefaultTTL, e.sendTOS, e.owner); err != nil {
+ if err := sendUDP(route, buffer.View(v).ToVectorisedView(), e.ID.LocalPort, dstPort, ttl, useDefaultTTL, e.sendTOS, e.owner, e.noChecksum); err != nil {
return 0, nil, err
}
return int64(len(v)), nil, nil
@@ -553,6 +554,11 @@ func (e *endpoint) SetSockOptBool(opt tcpip.SockOptBool, v bool) *tcpip.Error {
e.multicastLoop = v
e.mu.Unlock()
+ case tcpip.NoChecksumOption:
+ e.mu.Lock()
+ e.noChecksum = v
+ e.mu.Unlock()
+
case tcpip.ReceiveTOSOption:
e.mu.Lock()
e.receiveTOS = v
@@ -825,6 +831,12 @@ func (e *endpoint) GetSockOptBool(opt tcpip.SockOptBool) (bool, *tcpip.Error) {
e.mu.RUnlock()
return v, nil
+ case tcpip.NoChecksumOption:
+ e.mu.RLock()
+ v := e.noChecksum
+ e.mu.RUnlock()
+ return v, nil
+
case tcpip.ReceiveTOSOption:
e.mu.RLock()
v := e.receiveTOS
@@ -959,7 +971,7 @@ func (e *endpoint) GetSockOpt(opt interface{}) *tcpip.Error {
// sendUDP sends a UDP segment via the provided network endpoint and under the
// provided identity.
-func sendUDP(r *stack.Route, data buffer.VectorisedView, localPort, remotePort uint16, ttl uint8, useDefaultTTL bool, tos uint8, owner tcpip.PacketOwner) *tcpip.Error {
+func sendUDP(r *stack.Route, data buffer.VectorisedView, localPort, remotePort uint16, ttl uint8, useDefaultTTL bool, tos uint8, owner tcpip.PacketOwner, noChecksum bool) *tcpip.Error {
// Allocate a buffer for the UDP header.
hdr := buffer.NewPrependable(header.UDPMinimumSize + int(r.MaxHeaderLength()))
@@ -973,8 +985,12 @@ func sendUDP(r *stack.Route, data buffer.VectorisedView, localPort, remotePort u
Length: length,
})
- // Only calculate the checksum if offloading isn't supported.
- if r.Capabilities()&stack.CapabilityTXChecksumOffload == 0 {
+ // Set the checksum field unless TX checksum offload is enabled.
+ // On IPv4, UDP checksum is optional, and a zero value indicates the
+ // transmitter skipped the checksum generation (RFC768).
+ // On IPv6, UDP checksum is not optional (RFC2460 Section 8.1).
+ if r.Capabilities()&stack.CapabilityTXChecksumOffload == 0 &&
+ (!noChecksum || r.NetProto == header.IPv6ProtocolNumber) {
xsum := r.PseudoHeaderChecksum(ProtocolNumber, length)
for _, v := range data.Views() {
xsum = header.Checksum(v, xsum)
diff --git a/pkg/tcpip/transport/udp/udp_test.go b/pkg/tcpip/transport/udp/udp_test.go
index ff9f60cf9..db59eb5a0 100644
--- a/pkg/tcpip/transport/udp/udp_test.go
+++ b/pkg/tcpip/transport/udp/udp_test.go
@@ -1251,6 +1251,30 @@ func TestWriteIncrementsPacketsSent(t *testing.T) {
}
}
+func TestNoChecksum(t *testing.T) {
+ for _, flow := range []testFlow{unicastV4, unicastV6} {
+ t.Run(fmt.Sprintf("flow:%s", flow), func(t *testing.T) {
+ c := newDualTestContext(t, defaultMTU)
+ defer c.cleanup()
+
+ c.createEndpointForFlow(flow)
+
+ // Disable the checksum generation.
+ if err := c.ep.SetSockOptBool(tcpip.NoChecksumOption, true); err != nil {
+ t.Fatalf("SetSockOptBool failed: %s", err)
+ }
+ // This option is effective on IPv4 only.
+ testWrite(c, flow, checker.UDP(checker.NoChecksum(flow.isV4())))
+
+ // Enable the checksum generation.
+ if err := c.ep.SetSockOptBool(tcpip.NoChecksumOption, false); err != nil {
+ t.Fatalf("SetSockOptBool failed: %s", err)
+ }
+ testWrite(c, flow, checker.UDP(checker.NoChecksum(false)))
+ })
+ }
+}
+
func TestTTL(t *testing.T) {
for _, flow := range []testFlow{unicastV4, unicastV4in6, unicastV6, unicastV6Only, multicastV4, multicastV4in6, multicastV6, broadcast, broadcastIn6} {
t.Run(fmt.Sprintf("flow:%s", flow), func(t *testing.T) {