summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/stack/nic.go
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/tcpip/stack/nic.go')
-rw-r--r--pkg/tcpip/stack/nic.go258
1 files changed, 120 insertions, 138 deletions
diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go
index 41a489047..f66db16a7 100644
--- a/pkg/tcpip/stack/nic.go
+++ b/pkg/tcpip/stack/nic.go
@@ -24,40 +24,26 @@ import (
"gvisor.dev/gvisor/pkg/tcpip/header"
)
-type neighborTable interface {
- neighbors() ([]NeighborEntry, tcpip.Error)
- addStaticEntry(tcpip.Address, tcpip.LinkAddress)
- get(addr, localAddr tcpip.Address, onResolve func(LinkResolutionResult)) (tcpip.LinkAddress, <-chan struct{}, tcpip.Error)
- remove(tcpip.Address) tcpip.Error
- removeAll() tcpip.Error
-
- handleProbe(tcpip.Address, tcpip.LinkAddress)
- handleConfirmation(tcpip.Address, tcpip.LinkAddress, ReachabilityConfirmationFlags)
- handleUpperLevelConfirmation(tcpip.Address)
-
- nudConfig() (NUDConfigurations, tcpip.Error)
- setNUDConfig(NUDConfigurations) tcpip.Error
-}
-
-var _ NetworkInterface = (*NIC)(nil)
-
type linkResolver struct {
resolver LinkAddressResolver
- neighborTable neighborTable
+ neigh neighborCache
}
func (l *linkResolver) getNeighborLinkAddress(addr, localAddr tcpip.Address, onResolve func(LinkResolutionResult)) (tcpip.LinkAddress, <-chan struct{}, tcpip.Error) {
- return l.neighborTable.get(addr, localAddr, onResolve)
+ entry, ch, err := l.neigh.entry(addr, localAddr, onResolve)
+ return entry.LinkAddr, ch, err
}
func (l *linkResolver) confirmReachable(addr tcpip.Address) {
- l.neighborTable.handleUpperLevelConfirmation(addr)
+ l.neigh.handleUpperLevelConfirmation(addr)
}
-// NIC represents a "network interface card" to which the networking stack is
+var _ NetworkInterface = (*nic)(nil)
+
+// nic represents a "network interface card" to which the networking stack is
// attached.
-type NIC struct {
+type nic struct {
LinkEndpoint
stack *Stack
@@ -69,8 +55,9 @@ type NIC struct {
// The network endpoints themselves may be modified by calling the interface's
// methods, but the map reference and entries must be constant.
- networkEndpoints map[tcpip.NetworkProtocolNumber]NetworkEndpoint
- linkAddrResolvers map[tcpip.NetworkProtocolNumber]linkResolver
+ networkEndpoints map[tcpip.NetworkProtocolNumber]NetworkEndpoint
+ linkAddrResolvers map[tcpip.NetworkProtocolNumber]*linkResolver
+ duplicateAddressDetectors map[tcpip.NetworkProtocolNumber]DuplicateAddressDetector
// enabled is set to 1 when the NIC is enabled and 0 when it is disabled.
//
@@ -147,7 +134,7 @@ func (p *packetEndpointList) forEach(fn func(PacketEndpoint)) {
}
// newNIC returns a new NIC using the default NDP configurations from stack.
-func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICContext) *NIC {
+func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICContext) *nic {
// TODO(b/141011931): Validate a LinkEndpoint (ep) is valid. For
// example, make sure that the link address it provides is a valid
// unicast ethernet address.
@@ -156,16 +143,17 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICC
// observe an MTU of at least 1280 bytes. Ensure that this requirement
// of IPv6 is supported on this endpoint's LinkEndpoint.
- nic := &NIC{
+ nic := &nic{
LinkEndpoint: ep,
- stack: stack,
- id: id,
- name: name,
- context: ctx,
- stats: makeNICStats(),
- networkEndpoints: make(map[tcpip.NetworkProtocolNumber]NetworkEndpoint),
- linkAddrResolvers: make(map[tcpip.NetworkProtocolNumber]linkResolver),
+ stack: stack,
+ id: id,
+ name: name,
+ context: ctx,
+ stats: makeNICStats(),
+ networkEndpoints: make(map[tcpip.NetworkProtocolNumber]NetworkEndpoint),
+ linkAddrResolvers: make(map[tcpip.NetworkProtocolNumber]*linkResolver),
+ duplicateAddressDetectors: make(map[tcpip.NetworkProtocolNumber]DuplicateAddressDetector),
}
nic.linkResQueue.init(nic)
nic.mu.packetEPs = make(map[tcpip.NetworkProtocolNumber]*packetEndpointList)
@@ -185,26 +173,15 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICC
if resolutionRequired {
if r, ok := netEP.(LinkAddressResolver); ok {
- l := linkResolver{
- resolver: r,
- }
-
- if stack.useNeighborCache {
- l.neighborTable = &neighborCache{
- nic: nic,
- state: NewNUDState(stack.nudConfigs, stack.randomGenerator),
- linkRes: r,
-
- cache: make(map[tcpip.Address]*neighborEntry, neighborCacheSize),
- }
- } else {
- cache := new(linkAddrCache)
- cache.init(nic, ageLimit, resolutionTimeout, resolutionAttempts, r)
- l.neighborTable = cache
- }
+ l := &linkResolver{resolver: r}
+ l.neigh.init(nic, r)
nic.linkAddrResolvers[r.LinkAddressProtocol()] = l
}
}
+
+ if d, ok := netEP.(DuplicateAddressDetector); ok {
+ nic.duplicateAddressDetectors[d.DuplicateAddressProtocol()] = d
+ }
}
nic.LinkEndpoint.Attach(nic)
@@ -212,19 +189,19 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICC
return nic
}
-func (n *NIC) getNetworkEndpoint(proto tcpip.NetworkProtocolNumber) NetworkEndpoint {
+func (n *nic) getNetworkEndpoint(proto tcpip.NetworkProtocolNumber) NetworkEndpoint {
return n.networkEndpoints[proto]
}
// Enabled implements NetworkInterface.
-func (n *NIC) Enabled() bool {
+func (n *nic) Enabled() bool {
return atomic.LoadUint32(&n.enabled) == 1
}
// setEnabled sets the enabled status for the NIC.
//
// Returns true if the enabled status was updated.
-func (n *NIC) setEnabled(v bool) bool {
+func (n *nic) setEnabled(v bool) bool {
if v {
return atomic.SwapUint32(&n.enabled, 1) == 0
}
@@ -234,7 +211,7 @@ func (n *NIC) setEnabled(v bool) bool {
// disable disables n.
//
// It undoes the work done by enable.
-func (n *NIC) disable() {
+func (n *nic) disable() {
n.mu.Lock()
n.disableLocked()
n.mu.Unlock()
@@ -245,7 +222,7 @@ func (n *NIC) disable() {
// It undoes the work done by enable.
//
// n MUST be locked.
-func (n *NIC) disableLocked() {
+func (n *nic) disableLocked() {
if !n.Enabled() {
return
}
@@ -283,7 +260,7 @@ func (n *NIC) disableLocked() {
// address (ff02::1), start DAD for permanent addresses, and start soliciting
// routers if the stack is not operating as a router. If the stack is also
// configured to auto-generate a link-local address, one will be generated.
-func (n *NIC) enable() tcpip.Error {
+func (n *nic) enable() tcpip.Error {
n.mu.Lock()
defer n.mu.Unlock()
@@ -303,7 +280,7 @@ func (n *NIC) enable() tcpip.Error {
// remove detaches NIC from the link endpoint and releases network endpoint
// resources. This guarantees no packets between this NIC and the network
// stack.
-func (n *NIC) remove() tcpip.Error {
+func (n *nic) remove() tcpip.Error {
n.mu.Lock()
defer n.mu.Unlock()
@@ -319,14 +296,14 @@ func (n *NIC) remove() tcpip.Error {
}
// setPromiscuousMode enables or disables promiscuous mode.
-func (n *NIC) setPromiscuousMode(enable bool) {
+func (n *nic) setPromiscuousMode(enable bool) {
n.mu.Lock()
n.mu.promiscuous = enable
n.mu.Unlock()
}
// Promiscuous implements NetworkInterface.
-func (n *NIC) Promiscuous() bool {
+func (n *nic) Promiscuous() bool {
n.mu.RLock()
rv := n.mu.promiscuous
n.mu.RUnlock()
@@ -334,17 +311,17 @@ func (n *NIC) Promiscuous() bool {
}
// IsLoopback implements NetworkInterface.
-func (n *NIC) IsLoopback() bool {
+func (n *nic) IsLoopback() bool {
return n.LinkEndpoint.Capabilities()&CapabilityLoopback != 0
}
// WritePacket implements NetworkLinkEndpoint.
-func (n *NIC) WritePacket(r *Route, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) tcpip.Error {
+func (n *nic) WritePacket(r *Route, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) tcpip.Error {
_, err := n.enqueuePacketBuffer(r, gso, protocol, pkt)
return err
}
-func (n *NIC) writePacketBuffer(r RouteInfo, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkt pendingPacketBuffer) (int, tcpip.Error) {
+func (n *nic) writePacketBuffer(r RouteInfo, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkt pendingPacketBuffer) (int, tcpip.Error) {
switch pkt := pkt.(type) {
case *PacketBuffer:
if err := n.writePacket(r, gso, protocol, pkt); err != nil {
@@ -358,7 +335,7 @@ func (n *NIC) writePacketBuffer(r RouteInfo, gso *GSO, protocol tcpip.NetworkPro
}
}
-func (n *NIC) enqueuePacketBuffer(r *Route, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkt pendingPacketBuffer) (int, tcpip.Error) {
+func (n *nic) enqueuePacketBuffer(r *Route, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkt pendingPacketBuffer) (int, tcpip.Error) {
routeInfo, _, err := r.resolvedFields(nil)
switch err.(type) {
case nil:
@@ -388,14 +365,14 @@ func (n *NIC) enqueuePacketBuffer(r *Route, gso *GSO, protocol tcpip.NetworkProt
}
// WritePacketToRemote implements NetworkInterface.
-func (n *NIC) WritePacketToRemote(remoteLinkAddr tcpip.LinkAddress, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) tcpip.Error {
+func (n *nic) WritePacketToRemote(remoteLinkAddr tcpip.LinkAddress, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) tcpip.Error {
var r RouteInfo
r.NetProto = protocol
r.RemoteLinkAddress = remoteLinkAddr
return n.writePacket(r, gso, protocol, pkt)
}
-func (n *NIC) writePacket(r RouteInfo, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) tcpip.Error {
+func (n *nic) writePacket(r RouteInfo, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) tcpip.Error {
// WritePacket takes ownership of pkt, calculate numBytes first.
numBytes := pkt.Size()
@@ -412,11 +389,11 @@ func (n *NIC) writePacket(r RouteInfo, gso *GSO, protocol tcpip.NetworkProtocolN
}
// WritePackets implements NetworkLinkEndpoint.
-func (n *NIC) WritePackets(r *Route, gso *GSO, pkts PacketBufferList, protocol tcpip.NetworkProtocolNumber) (int, tcpip.Error) {
+func (n *nic) WritePackets(r *Route, gso *GSO, pkts PacketBufferList, protocol tcpip.NetworkProtocolNumber) (int, tcpip.Error) {
return n.enqueuePacketBuffer(r, gso, protocol, &pkts)
}
-func (n *NIC) writePackets(r RouteInfo, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkts PacketBufferList) (int, tcpip.Error) {
+func (n *nic) writePackets(r RouteInfo, gso *GSO, protocol tcpip.NetworkProtocolNumber, pkts PacketBufferList) (int, tcpip.Error) {
for pkt := pkts.Front(); pkt != nil; pkt = pkt.Next() {
pkt.EgressRoute = r
pkt.GSOOptions = gso
@@ -435,15 +412,22 @@ func (n *NIC) writePackets(r RouteInfo, gso *GSO, protocol tcpip.NetworkProtocol
}
// setSpoofing enables or disables address spoofing.
-func (n *NIC) setSpoofing(enable bool) {
+func (n *nic) setSpoofing(enable bool) {
n.mu.Lock()
n.mu.spoofing = enable
n.mu.Unlock()
}
+// Spoofing implements NetworkInterface.
+func (n *nic) Spoofing() bool {
+ n.mu.RLock()
+ defer n.mu.RUnlock()
+ return n.mu.spoofing
+}
+
// primaryAddress returns an address that can be used to communicate with
// remoteAddr.
-func (n *NIC) primaryEndpoint(protocol tcpip.NetworkProtocolNumber, remoteAddr tcpip.Address) AssignableAddressEndpoint {
+func (n *nic) primaryEndpoint(protocol tcpip.NetworkProtocolNumber, remoteAddr tcpip.Address) AssignableAddressEndpoint {
ep, ok := n.networkEndpoints[protocol]
if !ok {
return nil
@@ -473,11 +457,11 @@ const (
promiscuous
)
-func (n *NIC) getAddress(protocol tcpip.NetworkProtocolNumber, dst tcpip.Address) AssignableAddressEndpoint {
+func (n *nic) getAddress(protocol tcpip.NetworkProtocolNumber, dst tcpip.Address) AssignableAddressEndpoint {
return n.getAddressOrCreateTemp(protocol, dst, CanBePrimaryEndpoint, promiscuous)
}
-func (n *NIC) hasAddress(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) bool {
+func (n *nic) hasAddress(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) bool {
ep := n.getAddressOrCreateTempInner(protocol, addr, false, NeverPrimaryEndpoint)
if ep != nil {
ep.DecRef()
@@ -488,7 +472,7 @@ func (n *NIC) hasAddress(protocol tcpip.NetworkProtocolNumber, addr tcpip.Addres
}
// findEndpoint finds the endpoint, if any, with the given address.
-func (n *NIC) findEndpoint(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, peb PrimaryEndpointBehavior) AssignableAddressEndpoint {
+func (n *nic) findEndpoint(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, peb PrimaryEndpointBehavior) AssignableAddressEndpoint {
return n.getAddressOrCreateTemp(protocol, address, peb, spoofing)
}
@@ -501,7 +485,7 @@ func (n *NIC) findEndpoint(protocol tcpip.NetworkProtocolNumber, address tcpip.A
//
// If the address is the IPv4 broadcast address for an endpoint's network, that
// endpoint will be returned.
-func (n *NIC) getAddressOrCreateTemp(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, peb PrimaryEndpointBehavior, tempRef getAddressBehaviour) AssignableAddressEndpoint {
+func (n *nic) getAddressOrCreateTemp(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, peb PrimaryEndpointBehavior, tempRef getAddressBehaviour) AssignableAddressEndpoint {
n.mu.RLock()
var spoofingOrPromiscuous bool
switch tempRef {
@@ -516,7 +500,7 @@ func (n *NIC) getAddressOrCreateTemp(protocol tcpip.NetworkProtocolNumber, addre
// getAddressOrCreateTempInner is like getAddressEpOrCreateTemp except a boolean
// is passed to indicate whether or not we should generate temporary endpoints.
-func (n *NIC) getAddressOrCreateTempInner(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, createTemp bool, peb PrimaryEndpointBehavior) AssignableAddressEndpoint {
+func (n *nic) getAddressOrCreateTempInner(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, createTemp bool, peb PrimaryEndpointBehavior) AssignableAddressEndpoint {
ep, ok := n.networkEndpoints[protocol]
if !ok {
return nil
@@ -532,7 +516,7 @@ func (n *NIC) getAddressOrCreateTempInner(protocol tcpip.NetworkProtocolNumber,
// addAddress adds a new address to n, so that it starts accepting packets
// targeted at the given address (and network protocol).
-func (n *NIC) addAddress(protocolAddress tcpip.ProtocolAddress, peb PrimaryEndpointBehavior) tcpip.Error {
+func (n *nic) addAddress(protocolAddress tcpip.ProtocolAddress, peb PrimaryEndpointBehavior) tcpip.Error {
ep, ok := n.networkEndpoints[protocolAddress.Protocol]
if !ok {
return &tcpip.ErrUnknownProtocol{}
@@ -553,7 +537,7 @@ func (n *NIC) addAddress(protocolAddress tcpip.ProtocolAddress, peb PrimaryEndpo
// allPermanentAddresses returns all permanent addresses associated with
// this NIC.
-func (n *NIC) allPermanentAddresses() []tcpip.ProtocolAddress {
+func (n *nic) allPermanentAddresses() []tcpip.ProtocolAddress {
var addrs []tcpip.ProtocolAddress
for p, ep := range n.networkEndpoints {
addressableEndpoint, ok := ep.(AddressableEndpoint)
@@ -569,7 +553,7 @@ func (n *NIC) allPermanentAddresses() []tcpip.ProtocolAddress {
}
// primaryAddresses returns the primary addresses associated with this NIC.
-func (n *NIC) primaryAddresses() []tcpip.ProtocolAddress {
+func (n *nic) primaryAddresses() []tcpip.ProtocolAddress {
var addrs []tcpip.ProtocolAddress
for p, ep := range n.networkEndpoints {
addressableEndpoint, ok := ep.(AddressableEndpoint)
@@ -589,7 +573,7 @@ func (n *NIC) primaryAddresses() []tcpip.ProtocolAddress {
// primaryAddress will return the first non-deprecated address if such an
// address exists. If no non-deprecated address exists, the first deprecated
// address will be returned.
-func (n *NIC) primaryAddress(proto tcpip.NetworkProtocolNumber) tcpip.AddressWithPrefix {
+func (n *nic) primaryAddress(proto tcpip.NetworkProtocolNumber) tcpip.AddressWithPrefix {
ep, ok := n.networkEndpoints[proto]
if !ok {
return tcpip.AddressWithPrefix{}
@@ -604,7 +588,7 @@ func (n *NIC) primaryAddress(proto tcpip.NetworkProtocolNumber) tcpip.AddressWit
}
// removeAddress removes an address from n.
-func (n *NIC) removeAddress(addr tcpip.Address) tcpip.Error {
+func (n *nic) removeAddress(addr tcpip.Address) tcpip.Error {
for _, ep := range n.networkEndpoints {
addressableEndpoint, ok := ep.(AddressableEndpoint)
if !ok {
@@ -622,7 +606,7 @@ func (n *NIC) removeAddress(addr tcpip.Address) tcpip.Error {
return &tcpip.ErrBadLocalAddress{}
}
-func (n *NIC) getLinkAddress(addr, localAddr tcpip.Address, protocol tcpip.NetworkProtocolNumber, onResolve func(LinkResolutionResult)) tcpip.Error {
+func (n *nic) getLinkAddress(addr, localAddr tcpip.Address, protocol tcpip.NetworkProtocolNumber, onResolve func(LinkResolutionResult)) tcpip.Error {
linkRes, ok := n.linkAddrResolvers[protocol]
if !ok {
return &tcpip.ErrNotSupported{}
@@ -637,34 +621,38 @@ func (n *NIC) getLinkAddress(addr, localAddr tcpip.Address, protocol tcpip.Netwo
return err
}
-func (n *NIC) neighbors(protocol tcpip.NetworkProtocolNumber) ([]NeighborEntry, tcpip.Error) {
+func (n *nic) neighbors(protocol tcpip.NetworkProtocolNumber) ([]NeighborEntry, tcpip.Error) {
if linkRes, ok := n.linkAddrResolvers[protocol]; ok {
- return linkRes.neighborTable.neighbors()
+ return linkRes.neigh.entries(), nil
}
return nil, &tcpip.ErrNotSupported{}
}
-func (n *NIC) addStaticNeighbor(addr tcpip.Address, protocol tcpip.NetworkProtocolNumber, linkAddress tcpip.LinkAddress) tcpip.Error {
+func (n *nic) addStaticNeighbor(addr tcpip.Address, protocol tcpip.NetworkProtocolNumber, linkAddress tcpip.LinkAddress) tcpip.Error {
if linkRes, ok := n.linkAddrResolvers[protocol]; ok {
- linkRes.neighborTable.addStaticEntry(addr, linkAddress)
+ linkRes.neigh.addStaticEntry(addr, linkAddress)
return nil
}
return &tcpip.ErrNotSupported{}
}
-func (n *NIC) removeNeighbor(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) tcpip.Error {
+func (n *nic) removeNeighbor(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) tcpip.Error {
if linkRes, ok := n.linkAddrResolvers[protocol]; ok {
- return linkRes.neighborTable.remove(addr)
+ if !linkRes.neigh.removeEntry(addr) {
+ return &tcpip.ErrBadAddress{}
+ }
+ return nil
}
return &tcpip.ErrNotSupported{}
}
-func (n *NIC) clearNeighbors(protocol tcpip.NetworkProtocolNumber) tcpip.Error {
+func (n *nic) clearNeighbors(protocol tcpip.NetworkProtocolNumber) tcpip.Error {
if linkRes, ok := n.linkAddrResolvers[protocol]; ok {
- return linkRes.neighborTable.removeAll()
+ linkRes.neigh.clear()
+ return nil
}
return &tcpip.ErrNotSupported{}
@@ -672,7 +660,7 @@ func (n *NIC) clearNeighbors(protocol tcpip.NetworkProtocolNumber) tcpip.Error {
// joinGroup adds a new endpoint for the given multicast address, if none
// exists yet. Otherwise it just increments its count.
-func (n *NIC) joinGroup(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) tcpip.Error {
+func (n *nic) joinGroup(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) tcpip.Error {
// TODO(b/143102137): When implementing MLD, make sure MLD packets are
// not sent unless a valid link-local address is available for use on n
// as an MLD packet's source address must be a link-local address as
@@ -693,7 +681,7 @@ func (n *NIC) joinGroup(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address
// leaveGroup decrements the count for the given multicast address, and when it
// reaches zero removes the endpoint for this address.
-func (n *NIC) leaveGroup(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) tcpip.Error {
+func (n *nic) leaveGroup(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) tcpip.Error {
ep, ok := n.networkEndpoints[protocol]
if !ok {
return &tcpip.ErrNotSupported{}
@@ -708,7 +696,7 @@ func (n *NIC) leaveGroup(protocol tcpip.NetworkProtocolNumber, addr tcpip.Addres
}
// isInGroup returns true if n has joined the multicast group addr.
-func (n *NIC) isInGroup(addr tcpip.Address) bool {
+func (n *nic) isInGroup(addr tcpip.Address) bool {
for _, ep := range n.networkEndpoints {
gep, ok := ep.(GroupAddressableEndpoint)
if !ok {
@@ -729,7 +717,7 @@ func (n *NIC) isInGroup(addr tcpip.Address) bool {
// Note that the ownership of the slice backing vv is retained by the caller.
// This rule applies only to the slice itself, not to the items of the slice;
// the ownership of the items is not retained by the caller.
-func (n *NIC) DeliverNetworkPacket(remote, local tcpip.LinkAddress, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) {
+func (n *nic) DeliverNetworkPacket(remote, local tcpip.LinkAddress, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) {
n.mu.RLock()
enabled := n.Enabled()
// If the NIC is not yet enabled, don't receive any packets.
@@ -777,41 +765,11 @@ func (n *NIC) DeliverNetworkPacket(remote, local tcpip.LinkAddress, protocol tcp
anyEPs.forEach(deliverPacketEPs)
}
- // Parse headers.
- netProto := n.stack.NetworkProtocolInstance(protocol)
- transProtoNum, hasTransportHdr, ok := netProto.Parse(pkt)
- if !ok {
- // The packet is too small to contain a network header.
- n.stack.stats.MalformedRcvdPackets.Increment()
- return
- }
- if hasTransportHdr {
- pkt.TransportProtocolNumber = transProtoNum
- // Parse the transport header if present.
- if state, ok := n.stack.transportProtocols[transProtoNum]; ok {
- state.proto.Parse(pkt)
- }
- }
-
- if n.stack.handleLocal && !n.IsLoopback() {
- src, _ := netProto.ParseAddresses(pkt.NetworkHeader().View())
- if r := n.getAddress(protocol, src); r != nil {
- r.DecRef()
-
- // The source address is one of our own, so we never should have gotten a
- // packet like this unless handleLocal is false. Loopback also calls this
- // function even though the packets didn't come from the physical interface
- // so don't drop those.
- n.stack.stats.IP.InvalidSourceAddressesReceived.Increment()
- return
- }
- }
-
networkEndpoint.HandlePacket(pkt)
}
// DeliverOutboundPacket implements NetworkDispatcher.DeliverOutboundPacket.
-func (n *NIC) DeliverOutboundPacket(remote, local tcpip.LinkAddress, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) {
+func (n *nic) DeliverOutboundPacket(remote, local tcpip.LinkAddress, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) {
n.mu.RLock()
// We do not deliver to protocol specific packet endpoints as on Linux
// only ETH_P_ALL endpoints get outbound packets.
@@ -831,7 +789,7 @@ func (n *NIC) DeliverOutboundPacket(remote, local tcpip.LinkAddress, protocol tc
// DeliverTransportPacket delivers the packets to the appropriate transport
// protocol endpoint.
-func (n *NIC) DeliverTransportPacket(protocol tcpip.TransportProtocolNumber, pkt *PacketBuffer) TransportPacketDisposition {
+func (n *nic) DeliverTransportPacket(protocol tcpip.TransportProtocolNumber, pkt *PacketBuffer) TransportPacketDisposition {
state, ok := n.stack.transportProtocols[protocol]
if !ok {
n.stack.stats.UnknownProtocolRcvdPackets.Increment()
@@ -912,7 +870,7 @@ func (n *NIC) DeliverTransportPacket(protocol tcpip.TransportProtocolNumber, pkt
}
// DeliverTransportError implements TransportDispatcher.
-func (n *NIC) DeliverTransportError(local, remote tcpip.Address, net tcpip.NetworkProtocolNumber, trans tcpip.TransportProtocolNumber, transErr TransportError, pkt *PacketBuffer) {
+func (n *nic) DeliverTransportError(local, remote tcpip.Address, net tcpip.NetworkProtocolNumber, trans tcpip.TransportProtocolNumber, transErr TransportError, pkt *PacketBuffer) {
state, ok := n.stack.transportProtocols[trans]
if !ok {
return
@@ -940,19 +898,19 @@ func (n *NIC) DeliverTransportError(local, remote tcpip.Address, net tcpip.Netwo
}
// ID implements NetworkInterface.
-func (n *NIC) ID() tcpip.NICID {
+func (n *nic) ID() tcpip.NICID {
return n.id
}
// Name implements NetworkInterface.
-func (n *NIC) Name() string {
+func (n *nic) Name() string {
return n.name
}
// nudConfigs gets the NUD configurations for n.
-func (n *NIC) nudConfigs(protocol tcpip.NetworkProtocolNumber) (NUDConfigurations, tcpip.Error) {
+func (n *nic) nudConfigs(protocol tcpip.NetworkProtocolNumber) (NUDConfigurations, tcpip.Error) {
if linkRes, ok := n.linkAddrResolvers[protocol]; ok {
- return linkRes.neighborTable.nudConfig()
+ return linkRes.neigh.config(), nil
}
return NUDConfigurations{}, &tcpip.ErrNotSupported{}
@@ -962,16 +920,17 @@ func (n *NIC) nudConfigs(protocol tcpip.NetworkProtocolNumber) (NUDConfiguration
//
// Note, if c contains invalid NUD configuration values, it will be fixed to
// use default values for the erroneous values.
-func (n *NIC) setNUDConfigs(protocol tcpip.NetworkProtocolNumber, c NUDConfigurations) tcpip.Error {
+func (n *nic) setNUDConfigs(protocol tcpip.NetworkProtocolNumber, c NUDConfigurations) tcpip.Error {
if linkRes, ok := n.linkAddrResolvers[protocol]; ok {
c.resetInvalidFields()
- return linkRes.neighborTable.setNUDConfig(c)
+ linkRes.neigh.setConfig(c)
+ return nil
}
return &tcpip.ErrNotSupported{}
}
-func (n *NIC) registerPacketEndpoint(netProto tcpip.NetworkProtocolNumber, ep PacketEndpoint) tcpip.Error {
+func (n *nic) registerPacketEndpoint(netProto tcpip.NetworkProtocolNumber, ep PacketEndpoint) tcpip.Error {
n.mu.Lock()
defer n.mu.Unlock()
@@ -984,7 +943,7 @@ func (n *NIC) registerPacketEndpoint(netProto tcpip.NetworkProtocolNumber, ep Pa
return nil
}
-func (n *NIC) unregisterPacketEndpoint(netProto tcpip.NetworkProtocolNumber, ep PacketEndpoint) {
+func (n *nic) unregisterPacketEndpoint(netProto tcpip.NetworkProtocolNumber, ep PacketEndpoint) {
n.mu.Lock()
defer n.mu.Unlock()
@@ -998,7 +957,7 @@ func (n *NIC) unregisterPacketEndpoint(netProto tcpip.NetworkProtocolNumber, ep
// isValidForOutgoing returns true if the endpoint can be used to send out a
// packet. It requires the endpoint to not be marked expired (i.e., its address
// has been removed) unless the NIC is in spoofing mode, or temporary.
-func (n *NIC) isValidForOutgoing(ep AssignableAddressEndpoint) bool {
+func (n *nic) isValidForOutgoing(ep AssignableAddressEndpoint) bool {
n.mu.RLock()
spoofing := n.mu.spoofing
n.mu.RUnlock()
@@ -1006,9 +965,9 @@ func (n *NIC) isValidForOutgoing(ep AssignableAddressEndpoint) bool {
}
// HandleNeighborProbe implements NetworkInterface.
-func (n *NIC) HandleNeighborProbe(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address, linkAddr tcpip.LinkAddress) tcpip.Error {
+func (n *nic) HandleNeighborProbe(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address, linkAddr tcpip.LinkAddress) tcpip.Error {
if l, ok := n.linkAddrResolvers[protocol]; ok {
- l.neighborTable.handleProbe(addr, linkAddr)
+ l.neigh.handleProbe(addr, linkAddr)
return nil
}
@@ -1016,11 +975,34 @@ func (n *NIC) HandleNeighborProbe(protocol tcpip.NetworkProtocolNumber, addr tcp
}
// HandleNeighborConfirmation implements NetworkInterface.
-func (n *NIC) HandleNeighborConfirmation(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address, linkAddr tcpip.LinkAddress, flags ReachabilityConfirmationFlags) tcpip.Error {
+func (n *nic) HandleNeighborConfirmation(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address, linkAddr tcpip.LinkAddress, flags ReachabilityConfirmationFlags) tcpip.Error {
if l, ok := n.linkAddrResolvers[protocol]; ok {
- l.neighborTable.handleConfirmation(addr, linkAddr, flags)
+ l.neigh.handleConfirmation(addr, linkAddr, flags)
return nil
}
return &tcpip.ErrNotSupported{}
}
+
+// CheckLocalAddress implements NetworkInterface.
+func (n *nic) CheckLocalAddress(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) bool {
+ if n.Spoofing() {
+ return true
+ }
+
+ if addressEndpoint := n.getAddressOrCreateTempInner(protocol, addr, false /* createTemp */, NeverPrimaryEndpoint); addressEndpoint != nil {
+ addressEndpoint.DecRef()
+ return true
+ }
+
+ return false
+}
+
+func (n *nic) checkDuplicateAddress(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address, h DADCompletionHandler) (DADCheckAddressDisposition, tcpip.Error) {
+ d, ok := n.duplicateAddressDetectors[protocol]
+ if !ok {
+ return 0, &tcpip.ErrNotSupported{}
+ }
+
+ return d.CheckDuplicateAddress(addr, h), nil
+}