summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/tcpip.go
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/tcpip/tcpip.go')
-rw-r--r--pkg/tcpip/tcpip.go436
1 files changed, 207 insertions, 229 deletions
diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go
index 56aac093c..c500a0d1c 100644
--- a/pkg/tcpip/tcpip.go
+++ b/pkg/tcpip/tcpip.go
@@ -29,6 +29,7 @@
package tcpip
import (
+ "bytes"
"errors"
"fmt"
"io"
@@ -46,141 +47,6 @@ import (
// Using header.IPv4AddressSize would cause an import cycle.
const ipv4AddressSize = 4
-// Error represents an error in the netstack error space. Using a special type
-// ensures that errors outside of this space are not accidentally introduced.
-//
-// All errors must have unique msg strings.
-//
-// +stateify savable
-type Error struct {
- msg string
-
- ignoreStats bool
-}
-
-// String implements fmt.Stringer.String.
-func (e *Error) String() string {
- if e == nil {
- return "<nil>"
- }
- return e.msg
-}
-
-// IgnoreStats indicates whether this error type should be included in failure
-// counts in tcpip.Stats structs.
-func (e *Error) IgnoreStats() bool {
- return e.ignoreStats
-}
-
-// Errors that can be returned by the network stack.
-var (
- ErrUnknownProtocol = &Error{msg: "unknown protocol"}
- ErrUnknownNICID = &Error{msg: "unknown nic id"}
- ErrUnknownDevice = &Error{msg: "unknown device"}
- ErrUnknownProtocolOption = &Error{msg: "unknown option for protocol"}
- ErrDuplicateNICID = &Error{msg: "duplicate nic id"}
- ErrDuplicateAddress = &Error{msg: "duplicate address"}
- ErrNoRoute = &Error{msg: "no route"}
- ErrBadLinkEndpoint = &Error{msg: "bad link layer endpoint"}
- ErrAlreadyBound = &Error{msg: "endpoint already bound", ignoreStats: true}
- ErrInvalidEndpointState = &Error{msg: "endpoint is in invalid state"}
- ErrAlreadyConnecting = &Error{msg: "endpoint is already connecting", ignoreStats: true}
- ErrAlreadyConnected = &Error{msg: "endpoint is already connected", ignoreStats: true}
- ErrNoPortAvailable = &Error{msg: "no ports are available"}
- ErrPortInUse = &Error{msg: "port is in use"}
- ErrBadLocalAddress = &Error{msg: "bad local address"}
- ErrClosedForSend = &Error{msg: "endpoint is closed for send"}
- ErrClosedForReceive = &Error{msg: "endpoint is closed for receive"}
- ErrWouldBlock = &Error{msg: "operation would block", ignoreStats: true}
- ErrConnectionRefused = &Error{msg: "connection was refused"}
- ErrTimeout = &Error{msg: "operation timed out"}
- ErrAborted = &Error{msg: "operation aborted"}
- ErrConnectStarted = &Error{msg: "connection attempt started", ignoreStats: true}
- ErrDestinationRequired = &Error{msg: "destination address is required"}
- ErrNotSupported = &Error{msg: "operation not supported"}
- ErrQueueSizeNotSupported = &Error{msg: "queue size querying not supported"}
- ErrNotConnected = &Error{msg: "endpoint not connected"}
- ErrConnectionReset = &Error{msg: "connection reset by peer"}
- ErrConnectionAborted = &Error{msg: "connection aborted"}
- ErrNoSuchFile = &Error{msg: "no such file"}
- ErrInvalidOptionValue = &Error{msg: "invalid option value specified"}
- ErrBadAddress = &Error{msg: "bad address"}
- ErrNetworkUnreachable = &Error{msg: "network is unreachable"}
- ErrMessageTooLong = &Error{msg: "message too long"}
- ErrNoBufferSpace = &Error{msg: "no buffer space available"}
- ErrBroadcastDisabled = &Error{msg: "broadcast socket option disabled"}
- ErrNotPermitted = &Error{msg: "operation not permitted"}
- ErrAddressFamilyNotSupported = &Error{msg: "address family not supported by protocol"}
- ErrMalformedHeader = &Error{msg: "header is malformed"}
- ErrBadBuffer = &Error{msg: "bad buffer"}
-)
-
-var messageToError map[string]*Error
-
-var populate sync.Once
-
-// StringToError converts an error message to the error.
-func StringToError(s string) *Error {
- populate.Do(func() {
- var errors = []*Error{
- ErrUnknownProtocol,
- ErrUnknownNICID,
- ErrUnknownDevice,
- ErrUnknownProtocolOption,
- ErrDuplicateNICID,
- ErrDuplicateAddress,
- ErrNoRoute,
- ErrBadLinkEndpoint,
- ErrAlreadyBound,
- ErrInvalidEndpointState,
- ErrAlreadyConnecting,
- ErrAlreadyConnected,
- ErrNoPortAvailable,
- ErrPortInUse,
- ErrBadLocalAddress,
- ErrClosedForSend,
- ErrClosedForReceive,
- ErrWouldBlock,
- ErrConnectionRefused,
- ErrTimeout,
- ErrAborted,
- ErrConnectStarted,
- ErrDestinationRequired,
- ErrNotSupported,
- ErrQueueSizeNotSupported,
- ErrNotConnected,
- ErrConnectionReset,
- ErrConnectionAborted,
- ErrNoSuchFile,
- ErrInvalidOptionValue,
- ErrBadAddress,
- ErrNetworkUnreachable,
- ErrMessageTooLong,
- ErrNoBufferSpace,
- ErrBroadcastDisabled,
- ErrNotPermitted,
- ErrAddressFamilyNotSupported,
- ErrMalformedHeader,
- ErrBadBuffer,
- }
-
- messageToError = make(map[string]*Error)
- for _, e := range errors {
- if messageToError[e.String()] != nil {
- panic("tcpip errors with duplicated message: " + e.String())
- }
- messageToError[e.String()] = e
- }
- })
-
- e, ok := messageToError[s]
- if !ok {
- panic("unknown error message: " + s)
- }
-
- return e
-}
-
// Errors related to Subnet
var (
errSubnetLengthMismatch = errors.New("subnet length of address and mask differ")
@@ -194,7 +60,7 @@ type ErrSaveRejection struct {
}
// Error returns a sensible description of the save rejection error.
-func (e ErrSaveRejection) Error() string {
+func (e *ErrSaveRejection) Error() string {
return "save rejected due to unsupported networking state: " + e.Err.Error()
}
@@ -471,30 +337,15 @@ type FullAddress struct {
// This interface allows the endpoint to request the amount of data it needs
// based on internal buffers without exposing them.
type Payloader interface {
- // FullPayload returns all available bytes.
- FullPayload() ([]byte, *Error)
+ io.Reader
- // Payload returns a slice containing at most size bytes.
- Payload(size int) ([]byte, *Error)
+ // Len returns the number of bytes of the unread portion of the
+ // Reader.
+ Len() int
}
-// SlicePayload implements Payloader for slices.
-//
-// This is typically used for tests.
-type SlicePayload []byte
-
-// FullPayload implements Payloader.FullPayload.
-func (s SlicePayload) FullPayload() ([]byte, *Error) {
- return s, nil
-}
-
-// Payload implements Payloader.Payload.
-func (s SlicePayload) Payload(size int) ([]byte, *Error) {
- if size > len(s) {
- size = len(s)
- }
- return s[:size], nil
-}
+var _ Payloader = (*bytes.Buffer)(nil)
+var _ Payloader = (*bytes.Reader)(nil)
var _ io.Writer = (*SliceWriter)(nil)
@@ -647,7 +498,7 @@ type Endpoint interface {
// If non-zero number of bytes are successfully read and written to dst, err
// must be nil. Otherwise, if dst failed to write anything, ErrBadBuffer
// should be returned.
- Read(dst io.Writer, opts ReadOptions) (res ReadResult, err *Error)
+ Read(io.Writer, ReadOptions) (ReadResult, Error)
// Write writes data to the endpoint's peer. This method does not block if
// the data cannot be written.
@@ -662,7 +513,7 @@ type Endpoint interface {
// stream (TCP) Endpoints may return partial writes, and even then only
// in the case where writing additional data would block. Other Endpoints
// will either write the entire message or return an error.
- Write(Payloader, WriteOptions) (int64, *Error)
+ Write(Payloader, WriteOptions) (int64, Error)
// Connect connects the endpoint to its peer. Specifying a NIC is
// optional.
@@ -676,21 +527,21 @@ type Endpoint interface {
// connected returns nil. Calling connect again results in ErrAlreadyConnected.
// Anything else -- the attempt to connect failed.
//
- // If address.Addr is empty, this means that Enpoint has to be
+ // If address.Addr is empty, this means that Endpoint has to be
// disconnected if this is supported, otherwise
// ErrAddressFamilyNotSupported must be returned.
- Connect(address FullAddress) *Error
+ Connect(address FullAddress) Error
// Disconnect disconnects the endpoint from its peer.
- Disconnect() *Error
+ Disconnect() Error
// Shutdown closes the read and/or write end of the endpoint connection
// to its peer.
- Shutdown(flags ShutdownFlags) *Error
+ Shutdown(flags ShutdownFlags) Error
// Listen puts the endpoint in "listen" mode, which allows it to accept
// new connections.
- Listen(backlog int) *Error
+ Listen(backlog int) Error
// Accept returns a new endpoint if a peer has established a connection
// to an endpoint previously set to listen mode. This method does not
@@ -700,36 +551,36 @@ type Endpoint interface {
//
// If peerAddr is not nil then it is populated with the peer address of the
// returned endpoint.
- Accept(peerAddr *FullAddress) (Endpoint, *waiter.Queue, *Error)
+ Accept(peerAddr *FullAddress) (Endpoint, *waiter.Queue, Error)
// Bind binds the endpoint to a specific local address and port.
// Specifying a NIC is optional.
- Bind(address FullAddress) *Error
+ Bind(address FullAddress) Error
// GetLocalAddress returns the address to which the endpoint is bound.
- GetLocalAddress() (FullAddress, *Error)
+ GetLocalAddress() (FullAddress, Error)
// GetRemoteAddress returns the address to which the endpoint is
// connected.
- GetRemoteAddress() (FullAddress, *Error)
+ GetRemoteAddress() (FullAddress, Error)
// Readiness returns the current readiness of the endpoint. For example,
// if waiter.EventIn is set, the endpoint is immediately readable.
Readiness(mask waiter.EventMask) waiter.EventMask
// SetSockOpt sets a socket option.
- SetSockOpt(opt SettableSocketOption) *Error
+ SetSockOpt(opt SettableSocketOption) Error
// SetSockOptInt sets a socket option, for simple cases where a value
// has the int type.
- SetSockOptInt(opt SockOptInt, v int) *Error
+ SetSockOptInt(opt SockOptInt, v int) Error
// GetSockOpt gets a socket option.
- GetSockOpt(opt GettableSocketOption) *Error
+ GetSockOpt(opt GettableSocketOption) Error
// GetSockOptInt gets a socket option for simple cases where a return
// value has the int type.
- GetSockOptInt(SockOptInt) (int, *Error)
+ GetSockOptInt(SockOptInt) (int, Error)
// State returns a socket's lifecycle state. The returned value is
// protocol-specific and is primarily used for diagnostics.
@@ -752,7 +603,7 @@ type Endpoint interface {
SetOwner(owner PacketOwner)
// LastError clears and returns the last error reported by the endpoint.
- LastError() *Error
+ LastError() Error
// SocketOptions returns the structure which contains all the socket
// level options.
@@ -840,10 +691,6 @@ const (
// number of unread bytes in the input buffer should be returned.
ReceiveQueueSizeOption
- // SendBufferSizeOption is used by SetSockOptInt/GetSockOptInt to
- // specify the send buffer size option.
- SendBufferSizeOption
-
// ReceiveBufferSizeOption is used by SetSockOptInt/GetSockOptInt to
// specify the receive buffer size option.
ReceiveBufferSizeOption
@@ -1011,12 +858,54 @@ type SettableSocketOption interface {
isSettableSocketOption()
}
+// CongestionControlState indicates the current congestion control state for
+// TCP sender.
+type CongestionControlState int
+
+const (
+ // Open indicates that the sender is receiving acks in order and
+ // no loss or dupACK's etc have been detected.
+ Open CongestionControlState = iota
+ // RTORecovery indicates that an RTO has occurred and the sender
+ // has entered an RTO based recovery phase.
+ RTORecovery
+ // FastRecovery indicates that the sender has entered FastRecovery
+ // based on receiving nDupAck's. This state is entered only when
+ // SACK is not in use.
+ FastRecovery
+ // SACKRecovery indicates that the sender has entered SACK based
+ // recovery.
+ SACKRecovery
+ // Disorder indicates the sender either received some SACK blocks
+ // or dupACK's.
+ Disorder
+)
+
// TCPInfoOption is used by GetSockOpt to expose TCP statistics.
//
// TODO(b/64800844): Add and populate stat fields.
type TCPInfoOption struct {
- RTT time.Duration
+ // RTT is the smoothed round trip time.
+ RTT time.Duration
+
+ // RTTVar is the round trip time variation.
RTTVar time.Duration
+
+ // RTO is the retransmission timeout for the endpoint.
+ RTO time.Duration
+
+ // CcState is the congestion control state.
+ CcState CongestionControlState
+
+ // SndCwnd is the congestion window, in packets.
+ SndCwnd uint32
+
+ // SndSsthresh is the threshold between slow start and congestion
+ // avoidance.
+ SndSsthresh uint32
+
+ // ReorderSeen indicates if reordering is seen in the endpoint.
+ ReorderSeen bool
}
func (*TCPInfoOption) isGettableSocketOption() {}
@@ -1248,6 +1137,31 @@ type IPPacketInfo struct {
DestinationAddr Address
}
+// SendBufferSizeOption is used by stack.(Stack*).Option/SetOption to
+// get/set the default, min and max send buffer sizes.
+type SendBufferSizeOption struct {
+ // Min is the minimum size for send buffer.
+ Min int
+
+ // Default is the default size for send buffer.
+ Default int
+
+ // Max is the maximum size for send buffer.
+ Max int
+}
+
+// GetSendBufferLimits is used to get the send buffer size limits.
+type GetSendBufferLimits func(StackHandler) SendBufferSizeOption
+
+// GetStackSendBufferLimits is used to get default, min and max send buffer size.
+func GetStackSendBufferLimits(so StackHandler) SendBufferSizeOption {
+ var ss SendBufferSizeOption
+ if err := so.Option(&ss); err != nil {
+ panic(fmt.Sprintf("s.Option(%#v) = %s", ss, err))
+ }
+ return ss
+}
+
// Route is a row in the routing table. It specifies through which NIC (and
// gateway) sets of packets should be routed. A row is considered viable if the
// masked target address matches the destination address in the row.
@@ -1317,8 +1231,33 @@ func (s *StatCounter) String() string {
return strconv.FormatUint(s.Value(), 10)
}
+// A MultiCounterStat keeps track of two counters at once.
+type MultiCounterStat struct {
+ a, b *StatCounter
+}
+
+// Init sets both internal counters to point to a and b.
+func (m *MultiCounterStat) Init(a, b *StatCounter) {
+ m.a = a
+ m.b = b
+}
+
+// Increment adds one to the counters.
+func (m *MultiCounterStat) Increment() {
+ m.a.Increment()
+ m.b.Increment()
+}
+
+// IncrementBy increments the counters by v.
+func (m *MultiCounterStat) IncrementBy(v uint64) {
+ m.a.IncrementBy(v)
+ m.b.IncrementBy(v)
+}
+
// ICMPv4PacketStats enumerates counts for all ICMPv4 packet types.
type ICMPv4PacketStats struct {
+ // LINT.IfChange(ICMPv4PacketStats)
+
// Echo is the total number of ICMPv4 echo packets counted.
Echo *StatCounter
@@ -1358,10 +1297,56 @@ type ICMPv4PacketStats struct {
// InfoReply is the total number of ICMPv4 information reply packets
// counted.
InfoReply *StatCounter
+
+ // LINT.ThenChange(network/ipv4/stats.go:multiCounterICMPv4PacketStats)
+}
+
+// ICMPv4SentPacketStats collects outbound ICMPv4-specific stats.
+type ICMPv4SentPacketStats struct {
+ // LINT.IfChange(ICMPv4SentPacketStats)
+
+ ICMPv4PacketStats
+
+ // Dropped is the total number of ICMPv4 packets dropped due to link
+ // layer errors.
+ Dropped *StatCounter
+
+ // RateLimited is the total number of ICMPv4 packets dropped due to
+ // rate limit being exceeded.
+ RateLimited *StatCounter
+
+ // LINT.ThenChange(network/ipv4/stats.go:multiCounterICMPv4SentPacketStats)
+}
+
+// ICMPv4ReceivedPacketStats collects inbound ICMPv4-specific stats.
+type ICMPv4ReceivedPacketStats struct {
+ // LINT.IfChange(ICMPv4ReceivedPacketStats)
+
+ ICMPv4PacketStats
+
+ // Invalid is the total number of invalid ICMPv4 packets received.
+ Invalid *StatCounter
+
+ // LINT.ThenChange(network/ipv4/stats.go:multiCounterICMPv4ReceivedPacketStats)
+}
+
+// ICMPv4Stats collects ICMPv4-specific stats.
+type ICMPv4Stats struct {
+ // LINT.IfChange(ICMPv4Stats)
+
+ // PacketsSent contains statistics about sent packets.
+ PacketsSent ICMPv4SentPacketStats
+
+ // PacketsReceived contains statistics about received packets.
+ PacketsReceived ICMPv4ReceivedPacketStats
+
+ // LINT.ThenChange(network/ipv4/stats.go:multiCounterICMPv4Stats)
}
// ICMPv6PacketStats enumerates counts for all ICMPv6 packet types.
type ICMPv6PacketStats struct {
+ // LINT.IfChange(ICMPv6PacketStats)
+
// EchoRequest is the total number of ICMPv6 echo request packets
// counted.
EchoRequest *StatCounter
@@ -1416,32 +1401,14 @@ type ICMPv6PacketStats struct {
// MulticastListenerDone is the total number of Multicast Listener Done
// messages counted.
MulticastListenerDone *StatCounter
-}
-
-// ICMPv4SentPacketStats collects outbound ICMPv4-specific stats.
-type ICMPv4SentPacketStats struct {
- ICMPv4PacketStats
-
- // Dropped is the total number of ICMPv4 packets dropped due to link
- // layer errors.
- Dropped *StatCounter
- // RateLimited is the total number of ICMPv6 packets dropped due to
- // rate limit being exceeded.
- RateLimited *StatCounter
-}
-
-// ICMPv4ReceivedPacketStats collects inbound ICMPv4-specific stats.
-type ICMPv4ReceivedPacketStats struct {
- ICMPv4PacketStats
-
- // Invalid is the total number of ICMPv4 packets received that the
- // transport layer could not parse.
- Invalid *StatCounter
+ // LINT.ThenChange(network/ipv6/stats.go:multiCounterICMPv6PacketStats)
}
// ICMPv6SentPacketStats collects outbound ICMPv6-specific stats.
type ICMPv6SentPacketStats struct {
+ // LINT.IfChange(ICMPv6SentPacketStats)
+
ICMPv6PacketStats
// Dropped is the total number of ICMPv6 packets dropped due to link
@@ -1451,47 +1418,41 @@ type ICMPv6SentPacketStats struct {
// RateLimited is the total number of ICMPv6 packets dropped due to
// rate limit being exceeded.
RateLimited *StatCounter
+
+ // LINT.ThenChange(network/ipv6/stats.go:multiCounterICMPv6SentPacketStats)
}
// ICMPv6ReceivedPacketStats collects inbound ICMPv6-specific stats.
type ICMPv6ReceivedPacketStats struct {
+ // LINT.IfChange(ICMPv6ReceivedPacketStats)
+
ICMPv6PacketStats
// Unrecognized is the total number of ICMPv6 packets received that the
// transport layer does not know how to parse.
Unrecognized *StatCounter
- // Invalid is the total number of ICMPv6 packets received that the
- // transport layer could not parse.
+ // Invalid is the total number of invalid ICMPv6 packets received.
Invalid *StatCounter
// RouterOnlyPacketsDroppedByHost is the total number of ICMPv6 packets
// dropped due to being router-specific packets.
RouterOnlyPacketsDroppedByHost *StatCounter
-}
-
-// ICMPv4Stats collects ICMPv4-specific stats.
-type ICMPv4Stats struct {
- // ICMPv4SentPacketStats contains counts of sent packets by ICMPv4 packet type
- // and a single count of packets which failed to write to the link
- // layer.
- PacketsSent ICMPv4SentPacketStats
- // ICMPv4ReceivedPacketStats contains counts of received packets by ICMPv4
- // packet type and a single count of invalid packets received.
- PacketsReceived ICMPv4ReceivedPacketStats
+ // LINT.ThenChange(network/ipv6/stats.go:multiCounterICMPv6ReceivedPacketStats)
}
// ICMPv6Stats collects ICMPv6-specific stats.
type ICMPv6Stats struct {
- // ICMPv6SentPacketStats contains counts of sent packets by ICMPv6 packet type
- // and a single count of packets which failed to write to the link
- // layer.
+ // LINT.IfChange(ICMPv6Stats)
+
+ // PacketsSent contains statistics about sent packets.
PacketsSent ICMPv6SentPacketStats
- // ICMPv6ReceivedPacketStats contains counts of received packets by ICMPv6
- // packet type and a single count of invalid packets received.
+ // PacketsReceived contains statistics about received packets.
PacketsReceived ICMPv6ReceivedPacketStats
+
+ // LINT.ThenChange(network/ipv6/stats.go:multiCounterICMPv6Stats)
}
// ICMPStats collects ICMP-specific stats (both v4 and v6).
@@ -1505,6 +1466,8 @@ type ICMPStats struct {
// IGMPPacketStats enumerates counts for all IGMP packet types.
type IGMPPacketStats struct {
+ // LINT.IfChange(IGMPPacketStats)
+
// MembershipQuery is the total number of Membership Query messages counted.
MembershipQuery *StatCounter
@@ -1518,22 +1481,29 @@ type IGMPPacketStats struct {
// LeaveGroup is the total number of Leave Group messages counted.
LeaveGroup *StatCounter
+
+ // LINT.ThenChange(network/ipv4/stats.go:multiCounterIGMPPacketStats)
}
// IGMPSentPacketStats collects outbound IGMP-specific stats.
type IGMPSentPacketStats struct {
+ // LINT.IfChange(IGMPSentPacketStats)
+
IGMPPacketStats
// Dropped is the total number of IGMP packets dropped.
Dropped *StatCounter
+
+ // LINT.ThenChange(network/ipv4/stats.go:multiCounterIGMPSentPacketStats)
}
// IGMPReceivedPacketStats collects inbound IGMP-specific stats.
type IGMPReceivedPacketStats struct {
+ // LINT.IfChange(IGMPReceivedPacketStats)
+
IGMPPacketStats
- // Invalid is the total number of IGMP packets received that IGMP could not
- // parse.
+ // Invalid is the total number of invalid IGMP packets received.
Invalid *StatCounter
// ChecksumErrors is the total number of IGMP packets dropped due to bad
@@ -1543,21 +1513,27 @@ type IGMPReceivedPacketStats struct {
// Unrecognized is the total number of unrecognized messages counted, these
// are silently ignored for forward-compatibilty.
Unrecognized *StatCounter
+
+ // LINT.ThenChange(network/ipv4/stats.go:multiCounterIGMPReceivedPacketStats)
}
-// IGMPStats colelcts IGMP-specific stats.
+// IGMPStats collects IGMP-specific stats.
type IGMPStats struct {
- // IGMPSentPacketStats contains counts of sent packets by IGMP packet type
- // and a single count of invalid packets received.
+ // LINT.IfChange(IGMPStats)
+
+ // PacketsSent contains statistics about sent packets.
PacketsSent IGMPSentPacketStats
- // IGMPReceivedPacketStats contains counts of received packets by IGMP packet
- // type and a single count of invalid packets received.
+ // PacketsReceived contains statistics about received packets.
PacketsReceived IGMPReceivedPacketStats
+
+ // LINT.ThenChange(network/ipv4/stats.go:multiCounterIGMPStats)
}
// IPStats collects IP-specific stats (both v4 and v6).
type IPStats struct {
+ // LINT.IfChange(IPStats)
+
// PacketsReceived is the total number of IP packets received from the
// link layer.
PacketsReceived *StatCounter
@@ -1575,7 +1551,7 @@ type IPStats struct {
InvalidSourceAddressesReceived *StatCounter
// PacketsDelivered is the total number of incoming IP packets that
- // are successfully delivered to the transport layer via HandlePacket.
+ // are successfully delivered to the transport layer.
PacketsDelivered *StatCounter
// PacketsSent is the total number of IP packets sent via WritePacket.
@@ -1613,10 +1589,14 @@ type IPStats struct {
// OptionUnknownReceived is the number of unknown IP options seen.
OptionUnknownReceived *StatCounter
+
+ // LINT.ThenChange(network/ip/stats.go:MultiCounterIPStats)
}
// ARPStats collects ARP-specific stats.
type ARPStats struct {
+ // LINT.IfChange(ARPStats)
+
// PacketsReceived is the number of ARP packets received from the link layer.
PacketsReceived *StatCounter
@@ -1644,10 +1624,6 @@ type ARPStats struct {
// ARP request with a bad local address.
OutgoingRequestBadLocalAddressErrors *StatCounter
- // OutgoingRequestNetworkUnreachableErrors is the number of failures to send
- // an ARP request with a network unreachable error.
- OutgoingRequestNetworkUnreachableErrors *StatCounter
-
// OutgoingRequestsDropped is the number of ARP requests which failed to write
// to a link-layer endpoint.
OutgoingRequestsDropped *StatCounter
@@ -1666,6 +1642,8 @@ type ARPStats struct {
// OutgoingRepliesSent is the number of ARP replies successfully written to a
// link-layer endpoint.
OutgoingRepliesSent *StatCounter
+
+ // LINT.ThenChange(network/arp/stats.go:multiCounterARPStats)
}
// TCPStats collects TCP-specific stats.