summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/transport/icmp/endpoint.go
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/tcpip/transport/icmp/endpoint.go')
-rw-r--r--pkg/tcpip/transport/icmp/endpoint.go221
1 files changed, 143 insertions, 78 deletions
diff --git a/pkg/tcpip/transport/icmp/endpoint.go b/pkg/tcpip/transport/icmp/endpoint.go
index 451d3880e..3187b336b 100644
--- a/pkg/tcpip/transport/icmp/endpoint.go
+++ b/pkg/tcpip/transport/icmp/endpoint.go
@@ -15,7 +15,6 @@
package icmp
import (
- "encoding/binary"
"sync"
"gvisor.dev/gvisor/pkg/tcpip"
@@ -53,11 +52,11 @@ const (
//
// +stateify savable
type endpoint struct {
+ stack.TransportEndpointInfo
+
// The following fields are initialized at creation time and are
// immutable.
stack *stack.Stack `state:"manual"`
- netProto tcpip.NetworkProtocolNumber
- transProto tcpip.TransportProtocolNumber
waiterQueue *waiter.Queue
// The following fields are used to manage the receive queue, and are
@@ -74,27 +73,23 @@ type endpoint struct {
sndBufSize int
// shutdownFlags represent the current shutdown state of the endpoint.
shutdownFlags tcpip.ShutdownFlags
- id stack.TransportEndpointID
state endpointState
- // bindNICID and bindAddr are set via calls to Bind(). They are used to
- // reject attempts to send data or connect via a different NIC or
- // address
- bindNICID tcpip.NICID
- bindAddr tcpip.Address
- // regNICID is the default NIC to be used when callers don't specify a
- // NIC.
- regNICID tcpip.NICID
- route stack.Route `state:"manual"`
-}
-
-func newEndpoint(stack *stack.Stack, netProto tcpip.NetworkProtocolNumber, transProto tcpip.TransportProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) {
+ route stack.Route `state:"manual"`
+ ttl uint8
+ stats tcpip.TransportEndpointStats `state:"nosave"`
+}
+
+func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, transProto tcpip.TransportProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) {
return &endpoint{
- stack: stack,
- netProto: netProto,
- transProto: transProto,
+ stack: s,
+ TransportEndpointInfo: stack.TransportEndpointInfo{
+ NetProto: netProto,
+ TransProto: transProto,
+ },
waiterQueue: waiterQueue,
rcvBufSizeMax: 32 * 1024,
sndBufSize: 32 * 1024,
+ state: stateInitial,
}, nil
}
@@ -105,7 +100,7 @@ func (e *endpoint) Close() {
e.shutdownFlags = tcpip.ShutdownRead | tcpip.ShutdownWrite
switch e.state {
case stateBound, stateConnected:
- e.stack.UnregisterTransportEndpoint(e.regNICID, []tcpip.NetworkProtocolNumber{e.netProto}, e.transProto, e.id, e)
+ e.stack.UnregisterTransportEndpoint(e.RegisterNICID, []tcpip.NetworkProtocolNumber{e.NetProto}, e.TransProto, e.ID, e, 0 /* bindToDevice */)
}
// Close the receive list and drain it.
@@ -144,6 +139,7 @@ func (e *endpoint) Read(addr *tcpip.FullAddress) (buffer.View, tcpip.ControlMess
if e.rcvList.Empty() {
err := tcpip.ErrWouldBlock
if e.rcvClosed {
+ e.stats.ReadErrors.ReadClosed.Increment()
err = tcpip.ErrClosedForReceive
}
e.rcvMu.Unlock()
@@ -205,7 +201,30 @@ func (e *endpoint) prepareForWrite(to *tcpip.FullAddress) (retry bool, err *tcpi
// Write writes data to the endpoint's peer. This method does not block
// if the data cannot be written.
-func (e *endpoint) Write(p tcpip.Payload, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
+func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
+ n, ch, err := e.write(p, opts)
+ switch err {
+ case nil:
+ e.stats.PacketsSent.Increment()
+ case tcpip.ErrMessageTooLong, tcpip.ErrInvalidOptionValue:
+ e.stats.WriteErrors.InvalidArgs.Increment()
+ case tcpip.ErrClosedForSend:
+ e.stats.WriteErrors.WriteClosed.Increment()
+ case tcpip.ErrInvalidEndpointState:
+ e.stats.WriteErrors.InvalidEndpointState.Increment()
+ case tcpip.ErrNoLinkAddress:
+ e.stats.SendErrors.NoLinkAddr.Increment()
+ case tcpip.ErrNoRoute, tcpip.ErrBroadcastDisabled, tcpip.ErrNetworkUnreachable:
+ // Errors indicating any problem with IP routing of the packet.
+ e.stats.SendErrors.NoRoute.Increment()
+ default:
+ // For all other errors when writing to the network layer.
+ e.stats.SendErrors.SendToNetworkFailed.Increment()
+ }
+ return n, ch, err
+}
+
+func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
// MSG_MORE is unimplemented. (This also means that MSG_EOR is a no-op.)
if opts.More {
return 0, nil, tcpip.ErrInvalidOptionValue
@@ -256,12 +275,12 @@ func (e *endpoint) Write(p tcpip.Payload, opts tcpip.WriteOptions) (int64, <-cha
// Reject destination address if it goes through a different
// NIC than the endpoint was bound to.
nicid := to.NIC
- if e.bindNICID != 0 {
- if nicid != 0 && nicid != e.bindNICID {
+ if e.BindNICID != 0 {
+ if nicid != 0 && nicid != e.BindNICID {
return 0, nil, tcpip.ErrNoRoute
}
- nicid = e.bindNICID
+ nicid = e.BindNICID
}
toCopy := *to
@@ -272,7 +291,7 @@ func (e *endpoint) Write(p tcpip.Payload, opts tcpip.WriteOptions) (int64, <-cha
}
// Find the enpoint.
- r, err := e.stack.FindRoute(nicid, e.bindAddr, to.Addr, netProto, false /* multicastLoop */)
+ r, err := e.stack.FindRoute(nicid, e.BindAddr, to.Addr, netProto, false /* multicastLoop */)
if err != nil {
return 0, nil, err
}
@@ -290,17 +309,17 @@ func (e *endpoint) Write(p tcpip.Payload, opts tcpip.WriteOptions) (int64, <-cha
}
}
- v, err := p.Get(p.Size())
+ v, err := p.FullPayload()
if err != nil {
return 0, nil, err
}
- switch e.netProto {
+ switch e.NetProto {
case header.IPv4ProtocolNumber:
- err = send4(route, e.id.LocalPort, v)
+ err = send4(route, e.ID.LocalPort, v, e.ttl)
case header.IPv6ProtocolNumber:
- err = send6(route, e.id.LocalPort, v)
+ err = send6(route, e.ID.LocalPort, v, e.ttl)
}
if err != nil {
@@ -315,8 +334,20 @@ func (e *endpoint) Peek([][]byte) (int64, tcpip.ControlMessages, *tcpip.Error) {
return 0, tcpip.ControlMessages{}, nil
}
-// SetSockOpt sets a socket option. Currently not supported.
+// SetSockOpt sets a socket option.
func (e *endpoint) SetSockOpt(opt interface{}) *tcpip.Error {
+ switch o := opt.(type) {
+ case tcpip.TTLOption:
+ e.mu.Lock()
+ e.ttl = uint8(o)
+ e.mu.Unlock()
+ }
+
+ return nil
+}
+
+// SetSockOptInt sets a socket option. Currently not supported.
+func (e *endpoint) SetSockOptInt(opt tcpip.SockOpt, v int) *tcpip.Error {
return nil
}
@@ -332,6 +363,18 @@ func (e *endpoint) GetSockOptInt(opt tcpip.SockOpt) (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()
+ v := e.rcvBufSizeMax
+ e.rcvMu.Unlock()
+ return v, nil
+
}
return -1, tcpip.ErrUnknownProtocolOption
}
@@ -342,40 +385,33 @@ func (e *endpoint) GetSockOpt(opt interface{}) *tcpip.Error {
case tcpip.ErrorOption:
return nil
- case *tcpip.SendBufferSizeOption:
- e.mu.Lock()
- *o = tcpip.SendBufferSizeOption(e.sndBufSize)
- e.mu.Unlock()
+ case *tcpip.KeepaliveEnabledOption:
+ *o = 0
return nil
- case *tcpip.ReceiveBufferSizeOption:
+ case *tcpip.TTLOption:
e.rcvMu.Lock()
- *o = tcpip.ReceiveBufferSizeOption(e.rcvBufSizeMax)
+ *o = tcpip.TTLOption(e.ttl)
e.rcvMu.Unlock()
return nil
- case *tcpip.KeepaliveEnabledOption:
- *o = 0
- return nil
-
default:
return tcpip.ErrUnknownProtocolOption
}
}
-func send4(r *stack.Route, ident uint16, data buffer.View) *tcpip.Error {
+func send4(r *stack.Route, ident uint16, data buffer.View, ttl uint8) *tcpip.Error {
if len(data) < header.ICMPv4MinimumSize {
return tcpip.ErrInvalidEndpointState
}
- // Set the ident to the user-specified port. Sequence number should
- // already be set by the user.
- binary.BigEndian.PutUint16(data[header.ICMPv4PayloadOffset:], ident)
-
hdr := buffer.NewPrependable(header.ICMPv4MinimumSize + int(r.MaxHeaderLength()))
icmpv4 := header.ICMPv4(hdr.Prepend(header.ICMPv4MinimumSize))
copy(icmpv4, data)
+ // Set the ident to the user-specified port. Sequence number should
+ // already be set by the user.
+ icmpv4.SetIdent(ident)
data = data[header.ICMPv4MinimumSize:]
// Linux performs these basic checks.
@@ -386,22 +422,24 @@ func send4(r *stack.Route, ident uint16, data buffer.View) *tcpip.Error {
icmpv4.SetChecksum(0)
icmpv4.SetChecksum(^header.Checksum(icmpv4, header.Checksum(data, 0)))
- return r.WritePacket(nil /* gso */, hdr, data.ToVectorisedView(), header.ICMPv4ProtocolNumber, r.DefaultTTL())
+ if ttl == 0 {
+ ttl = r.DefaultTTL()
+ }
+ return r.WritePacket(nil /* gso */, hdr, data.ToVectorisedView(), stack.NetworkHeaderParams{Protocol: header.ICMPv4ProtocolNumber, TTL: ttl, TOS: stack.DefaultTOS})
}
-func send6(r *stack.Route, ident uint16, data buffer.View) *tcpip.Error {
+func send6(r *stack.Route, ident uint16, data buffer.View, ttl uint8) *tcpip.Error {
if len(data) < header.ICMPv6EchoMinimumSize {
return tcpip.ErrInvalidEndpointState
}
- // Set the ident. Sequence number is provided by the user.
- binary.BigEndian.PutUint16(data[header.ICMPv6MinimumSize:], ident)
-
- hdr := buffer.NewPrependable(header.ICMPv6EchoMinimumSize + int(r.MaxHeaderLength()))
+ hdr := buffer.NewPrependable(header.ICMPv6MinimumSize + int(r.MaxHeaderLength()))
- icmpv6 := header.ICMPv6(hdr.Prepend(header.ICMPv6EchoMinimumSize))
+ icmpv6 := header.ICMPv6(hdr.Prepend(header.ICMPv6MinimumSize))
copy(icmpv6, data)
- data = data[header.ICMPv6EchoMinimumSize:]
+ // Set the ident. Sequence number is provided by the user.
+ icmpv6.SetIdent(ident)
+ data = data[header.ICMPv6MinimumSize:]
if icmpv6.Type() != header.ICMPv6EchoRequest || icmpv6.Code() != 0 {
return tcpip.ErrInvalidEndpointState
@@ -410,18 +448,21 @@ func send6(r *stack.Route, ident uint16, data buffer.View) *tcpip.Error {
icmpv6.SetChecksum(0)
icmpv6.SetChecksum(^header.Checksum(icmpv6, header.Checksum(data, 0)))
- return r.WritePacket(nil /* gso */, hdr, data.ToVectorisedView(), header.ICMPv6ProtocolNumber, r.DefaultTTL())
+ if ttl == 0 {
+ ttl = r.DefaultTTL()
+ }
+ return r.WritePacket(nil /* gso */, hdr, data.ToVectorisedView(), stack.NetworkHeaderParams{Protocol: header.ICMPv6ProtocolNumber, TTL: ttl, TOS: stack.DefaultTOS})
}
func (e *endpoint) checkV4Mapped(addr *tcpip.FullAddress, allowMismatch bool) (tcpip.NetworkProtocolNumber, *tcpip.Error) {
- netProto := e.netProto
+ netProto := e.NetProto
if header.IsV4MappedAddress(addr.Addr) {
return 0, tcpip.ErrNoRoute
}
// Fail if we're bound to an address length different from the one we're
// checking.
- if l := len(e.id.LocalAddress); !allowMismatch && l != 0 && l != len(addr.Addr) {
+ if l := len(e.ID.LocalAddress); !allowMismatch && l != 0 && l != len(addr.Addr) {
return 0, tcpip.ErrInvalidEndpointState
}
@@ -442,16 +483,16 @@ func (e *endpoint) Connect(addr tcpip.FullAddress) *tcpip.Error {
localPort := uint16(0)
switch e.state {
case stateBound, stateConnected:
- localPort = e.id.LocalPort
- if e.bindNICID == 0 {
+ localPort = e.ID.LocalPort
+ if e.BindNICID == 0 {
break
}
- if nicid != 0 && nicid != e.bindNICID {
+ if nicid != 0 && nicid != e.BindNICID {
return tcpip.ErrInvalidEndpointState
}
- nicid = e.bindNICID
+ nicid = e.BindNICID
default:
return tcpip.ErrInvalidEndpointState
}
@@ -462,7 +503,7 @@ func (e *endpoint) Connect(addr tcpip.FullAddress) *tcpip.Error {
}
// Find a route to the desired destination.
- r, err := e.stack.FindRoute(nicid, e.bindAddr, addr.Addr, netProto, false /* multicastLoop */)
+ r, err := e.stack.FindRoute(nicid, e.BindAddr, addr.Addr, netProto, false /* multicastLoop */)
if err != nil {
return err
}
@@ -484,9 +525,9 @@ func (e *endpoint) Connect(addr tcpip.FullAddress) *tcpip.Error {
return err
}
- e.id = id
+ e.ID = id
e.route = r.Clone()
- e.regNICID = nicid
+ e.RegisterNICID = nicid
e.state = stateConnected
@@ -541,14 +582,14 @@ func (e *endpoint) registerWithStack(nicid tcpip.NICID, netProtos []tcpip.Networ
if id.LocalPort != 0 {
// The endpoint already has a local port, just attempt to
// register it.
- err := e.stack.RegisterTransportEndpoint(nicid, netProtos, e.transProto, id, e, false)
+ err := e.stack.RegisterTransportEndpoint(nicid, netProtos, e.TransProto, id, e, false /* reuse */, 0 /* bindToDevice */)
return id, err
}
// We need to find a port for the endpoint.
_, err := e.stack.PickEphemeralPort(func(p uint16) (bool, *tcpip.Error) {
id.LocalPort = p
- err := e.stack.RegisterTransportEndpoint(nicid, netProtos, e.transProto, id, e, false)
+ err := e.stack.RegisterTransportEndpoint(nicid, netProtos, e.TransProto, id, e, false /* reuse */, 0 /* bindtodevice */)
switch err {
case nil:
return true, nil
@@ -595,8 +636,8 @@ func (e *endpoint) bindLocked(addr tcpip.FullAddress) *tcpip.Error {
return err
}
- e.id = id
- e.regNICID = addr.NIC
+ e.ID = id
+ e.RegisterNICID = addr.NIC
// Mark endpoint as bound.
e.state = stateBound
@@ -619,8 +660,8 @@ func (e *endpoint) Bind(addr tcpip.FullAddress) *tcpip.Error {
return err
}
- e.bindNICID = addr.NIC
- e.bindAddr = addr.Addr
+ e.BindNICID = addr.NIC
+ e.BindAddr = addr.Addr
return nil
}
@@ -631,9 +672,9 @@ func (e *endpoint) GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) {
defer e.mu.RUnlock()
return tcpip.FullAddress{
- NIC: e.regNICID,
- Addr: e.id.LocalAddress,
- Port: e.id.LocalPort,
+ NIC: e.RegisterNICID,
+ Addr: e.ID.LocalAddress,
+ Port: e.ID.LocalPort,
}, nil
}
@@ -647,9 +688,9 @@ func (e *endpoint) GetRemoteAddress() (tcpip.FullAddress, *tcpip.Error) {
}
return tcpip.FullAddress{
- NIC: e.regNICID,
- Addr: e.id.RemoteAddress,
- Port: e.id.RemotePort,
+ NIC: e.RegisterNICID,
+ Addr: e.ID.RemoteAddress,
+ Port: e.ID.RemotePort,
}, nil
}
@@ -675,17 +716,19 @@ func (e *endpoint) Readiness(mask waiter.EventMask) waiter.EventMask {
// endpoint.
func (e *endpoint) HandlePacket(r *stack.Route, id stack.TransportEndpointID, vv buffer.VectorisedView) {
// Only accept echo replies.
- switch e.netProto {
+ switch e.NetProto {
case header.IPv4ProtocolNumber:
h := header.ICMPv4(vv.First())
if h.Type() != header.ICMPv4EchoReply {
e.stack.Stats().DroppedPackets.Increment()
+ e.stats.ReceiveErrors.MalformedPacketsReceived.Increment()
return
}
case header.IPv6ProtocolNumber:
h := header.ICMPv6(vv.First())
if h.Type() != header.ICMPv6EchoReply {
e.stack.Stats().DroppedPackets.Increment()
+ e.stats.ReceiveErrors.MalformedPacketsReceived.Increment()
return
}
}
@@ -693,9 +736,17 @@ func (e *endpoint) HandlePacket(r *stack.Route, id stack.TransportEndpointID, vv
e.rcvMu.Lock()
// Drop the packet if our buffer is currently full.
- if !e.rcvReady || e.rcvClosed || e.rcvBufSize >= e.rcvBufSizeMax {
+ if !e.rcvReady || e.rcvClosed {
+ e.rcvMu.Unlock()
e.stack.Stats().DroppedPackets.Increment()
+ e.stats.ReceiveErrors.ClosedReceiver.Increment()
+ return
+ }
+
+ if e.rcvBufSize >= e.rcvBufSizeMax {
e.rcvMu.Unlock()
+ e.stack.Stats().DroppedPackets.Increment()
+ e.stats.ReceiveErrors.ReceiveBufferOverflow.Increment()
return
}
@@ -717,7 +768,7 @@ func (e *endpoint) HandlePacket(r *stack.Route, id stack.TransportEndpointID, vv
pkt.timestamp = e.stack.NowNanoseconds()
e.rcvMu.Unlock()
-
+ e.stats.PacketsReceived.Increment()
// Notify any waiters that there's data to be read now.
if wasEmpty {
e.waiterQueue.Notify(waiter.EventIn)
@@ -733,3 +784,17 @@ func (e *endpoint) HandleControlPacket(id stack.TransportEndpointID, typ stack.C
func (e *endpoint) State() uint32 {
return 0
}
+
+// Info returns a copy of the endpoint info.
+func (e *endpoint) Info() tcpip.EndpointInfo {
+ e.mu.RLock()
+ // Make a copy of the endpoint info.
+ ret := e.TransportEndpointInfo
+ e.mu.RUnlock()
+ return &ret
+}
+
+// Stats returns a pointer to the endpoint stats.
+func (e *endpoint) Stats() tcpip.EndpointStats {
+ return &e.stats
+}