diff options
-rw-r--r-- | pkg/tcpip/stack/linkaddrcache.go | 359 | ||||
-rw-r--r-- | pkg/tcpip/stack/linkaddrentry_list.go | 221 | ||||
-rw-r--r-- | pkg/tcpip/stack/neighbor_cache.go | 38 | ||||
-rw-r--r-- | pkg/tcpip/stack/nic.go | 64 | ||||
-rw-r--r-- | pkg/tcpip/stack/route.go | 8 | ||||
-rw-r--r-- | pkg/tcpip/stack/stack.go | 18 | ||||
-rw-r--r-- | pkg/tcpip/stack/stack_state_autogen.go | 54 |
7 files changed, 29 insertions, 733 deletions
diff --git a/pkg/tcpip/stack/linkaddrcache.go b/pkg/tcpip/stack/linkaddrcache.go deleted file mode 100644 index 9ec04dd50..000000000 --- a/pkg/tcpip/stack/linkaddrcache.go +++ /dev/null @@ -1,359 +0,0 @@ -// Copyright 2018 The gVisor Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package stack - -import ( - "fmt" - "time" - - "gvisor.dev/gvisor/pkg/sync" - "gvisor.dev/gvisor/pkg/tcpip" -) - -const linkAddrCacheSize = 512 // max cache entries - -// linkAddrCache is a fixed-sized cache mapping IP addresses to link addresses. -// -// The entries are stored in a ring buffer, oldest entry replaced first. -// -// This struct is safe for concurrent use. -type linkAddrCache struct { - nic *nic - - linkRes LinkAddressResolver - - // ageLimit is how long a cache entry is valid for. - ageLimit time.Duration - - // resolutionTimeout is the amount of time to wait for a link request to - // resolve an address. - resolutionTimeout time.Duration - - // resolutionAttempts is the number of times an address is attempted to be - // resolved before failing. - resolutionAttempts int - - mu struct { - sync.Mutex - table map[tcpip.Address]*linkAddrEntry - lru linkAddrEntryList - } -} - -// entryState controls the state of a single entry in the cache. -type entryState int - -const ( - // incomplete means that there is an outstanding request to resolve the - // address. This is the initial state. - incomplete entryState = iota - // ready means that the address has been resolved and can be used. - ready -) - -// String implements Stringer. -func (s entryState) String() string { - switch s { - case incomplete: - return "incomplete" - case ready: - return "ready" - default: - return fmt.Sprintf("unknown(%d)", s) - } -} - -// A linkAddrEntry is an entry in the linkAddrCache. -// This struct is thread-compatible. -type linkAddrEntry struct { - // linkAddrEntryEntry access is synchronized by the linkAddrCache lock. - linkAddrEntryEntry - - cache *linkAddrCache - - mu struct { - sync.RWMutex - - addr tcpip.Address - linkAddr tcpip.LinkAddress - expiration time.Time - s entryState - - // done is closed when address resolution is complete. It is nil iff s is - // incomplete and resolution is not yet in progress. - done chan struct{} - - // onResolve is called with the result of address resolution. - onResolve []func(LinkResolutionResult) - } -} - -func (e *linkAddrEntry) notifyCompletionLocked(linkAddr tcpip.LinkAddress) { - res := LinkResolutionResult{LinkAddress: linkAddr, Success: len(linkAddr) != 0} - for _, callback := range e.mu.onResolve { - callback(res) - } - e.mu.onResolve = nil - if ch := e.mu.done; ch != nil { - close(ch) - e.mu.done = nil - // Dequeue the pending packets in a new goroutine to not hold up the current - // goroutine as writing packets may be a costly operation. - // - // At the time of writing, when writing packets, a neighbor's link address - // is resolved (which ends up obtaining the entry's lock) while holding the - // link resolution queue's lock. Dequeuing packets in a new goroutine avoids - // a lock ordering violation. - go e.cache.nic.linkResQueue.dequeue(ch, linkAddr, len(linkAddr) != 0) - } -} - -// changeStateLocked sets the entry's state to ns. -// -// The entry's expiration is bumped up to the greater of itself and the passed -// expiration; the zero value indicates immediate expiration, and is set -// unconditionally - this is an implementation detail that allows for entries -// to be reused. -// -// Precondition: e.mu must be locked -func (e *linkAddrEntry) changeStateLocked(ns entryState, expiration time.Time) { - if e.mu.s == incomplete && ns == ready { - e.notifyCompletionLocked(e.mu.linkAddr) - } - - if expiration.IsZero() || expiration.After(e.mu.expiration) { - e.mu.expiration = expiration - } - e.mu.s = ns -} - -// add adds a k -> v mapping to the cache. -func (c *linkAddrCache) add(k tcpip.Address, v tcpip.LinkAddress) { - // Calculate expiration time before acquiring the lock, since expiration is - // relative to the time when information was learned, rather than when it - // happened to be inserted into the cache. - expiration := time.Now().Add(c.ageLimit) - - c.mu.Lock() - entry := c.getOrCreateEntryLocked(k) - entry.mu.Lock() - defer entry.mu.Unlock() - c.mu.Unlock() - - entry.mu.linkAddr = v - entry.changeStateLocked(ready, expiration) -} - -// getOrCreateEntryLocked retrieves a cache entry associated with k. The -// returned entry is always refreshed in the cache (it is reachable via the -// map, and its place is bumped in LRU). -// -// If a matching entry exists in the cache, it is returned. If no matching -// entry exists and the cache is full, an existing entry is evicted via LRU, -// reset to state incomplete, and returned. If no matching entry exists and the -// cache is not full, a new entry with state incomplete is allocated and -// returned. -func (c *linkAddrCache) getOrCreateEntryLocked(k tcpip.Address) *linkAddrEntry { - if entry, ok := c.mu.table[k]; ok { - c.mu.lru.Remove(entry) - c.mu.lru.PushFront(entry) - return entry - } - var entry *linkAddrEntry - if len(c.mu.table) == linkAddrCacheSize { - entry = c.mu.lru.Back() - entry.mu.Lock() - - delete(c.mu.table, entry.mu.addr) - c.mu.lru.Remove(entry) - - // Wake waiters and mark the soon-to-be-reused entry as expired. - entry.notifyCompletionLocked("" /* linkAddr */) - entry.mu.Unlock() - } else { - entry = new(linkAddrEntry) - } - - *entry = linkAddrEntry{ - cache: c, - } - entry.mu.Lock() - entry.mu.addr = k - entry.mu.s = incomplete - entry.mu.Unlock() - c.mu.table[k] = entry - c.mu.lru.PushFront(entry) - return entry -} - -// get reports any known link address for addr. -func (c *linkAddrCache) get(addr, localAddr tcpip.Address, onResolve func(LinkResolutionResult)) (tcpip.LinkAddress, <-chan struct{}, tcpip.Error) { - c.mu.Lock() - entry := c.getOrCreateEntryLocked(addr) - entry.mu.Lock() - defer entry.mu.Unlock() - c.mu.Unlock() - - switch s := entry.mu.s; s { - case ready: - if !time.Now().After(entry.mu.expiration) { - // Not expired. - if onResolve != nil { - onResolve(LinkResolutionResult{LinkAddress: entry.mu.linkAddr, Success: true}) - } - return entry.mu.linkAddr, nil, nil - } - - entry.changeStateLocked(incomplete, time.Time{}) - fallthrough - case incomplete: - if onResolve != nil { - entry.mu.onResolve = append(entry.mu.onResolve, onResolve) - } - if entry.mu.done == nil { - entry.mu.done = make(chan struct{}) - go c.startAddressResolution(addr, localAddr, entry.mu.done) // S/R-SAFE: link non-savable; wakers dropped synchronously. - } - return entry.mu.linkAddr, entry.mu.done, &tcpip.ErrWouldBlock{} - default: - panic(fmt.Sprintf("invalid cache entry state: %s", s)) - } -} - -func (c *linkAddrCache) startAddressResolution(k tcpip.Address, localAddr tcpip.Address, done <-chan struct{}) { - for i := 0; ; i++ { - // Send link request, then wait for the timeout limit and check - // whether the request succeeded. - c.linkRes.LinkAddressRequest(k, localAddr, "" /* linkAddr */) - - select { - case now := <-time.After(c.resolutionTimeout): - if stop := c.checkLinkRequest(now, k, i); stop { - return - } - case <-done: - return - } - } -} - -// checkLinkRequest checks whether previous attempt to resolve address has -// succeeded and mark the entry accordingly. Returns true if request can stop, -// false if another request should be sent. -func (c *linkAddrCache) checkLinkRequest(now time.Time, k tcpip.Address, attempt int) bool { - c.mu.Lock() - defer c.mu.Unlock() - entry, ok := c.mu.table[k] - if !ok { - // Entry was evicted from the cache. - return true - } - entry.mu.Lock() - defer entry.mu.Unlock() - - switch s := entry.mu.s; s { - case ready: - // Entry was made ready by resolver. - case incomplete: - if attempt+1 < c.resolutionAttempts { - // No response yet, need to send another ARP request. - return false - } - // Max number of retries reached, delete entry. - entry.notifyCompletionLocked("" /* linkAddr */) - delete(c.mu.table, k) - default: - panic(fmt.Sprintf("invalid cache entry state: %s", s)) - } - return true -} - -func (c *linkAddrCache) init(nic *nic, ageLimit, resolutionTimeout time.Duration, resolutionAttempts int, linkRes LinkAddressResolver) { - *c = linkAddrCache{ - nic: nic, - linkRes: linkRes, - ageLimit: ageLimit, - resolutionTimeout: resolutionTimeout, - resolutionAttempts: resolutionAttempts, - } - - c.mu.Lock() - c.mu.table = make(map[tcpip.Address]*linkAddrEntry, linkAddrCacheSize) - c.mu.Unlock() -} - -var _ neighborTable = (*linkAddrCache)(nil) - -func (*linkAddrCache) neighbors() ([]NeighborEntry, tcpip.Error) { - return nil, &tcpip.ErrNotSupported{} -} - -func (c *linkAddrCache) addStaticEntry(addr tcpip.Address, linkAddr tcpip.LinkAddress) { - c.add(addr, linkAddr) -} - -func (*linkAddrCache) remove(addr tcpip.Address) tcpip.Error { - return &tcpip.ErrNotSupported{} -} - -func (*linkAddrCache) removeAll() tcpip.Error { - return &tcpip.ErrNotSupported{} -} - -func (c *linkAddrCache) handleProbe(addr tcpip.Address, linkAddr tcpip.LinkAddress) { - if len(linkAddr) != 0 { - // NUD allows probes without a link address but linkAddrCache - // is a simple neighbor table which does not implement NUD. - // - // As per RFC 4861 section 4.3, - // - // Source link-layer address - // The link-layer address for the sender. MUST NOT be - // included when the source IP address is the - // unspecified address. Otherwise, on link layers - // that have addresses this option MUST be included in - // multicast solicitations and SHOULD be included in - // unicast solicitations. - c.add(addr, linkAddr) - } -} - -func (c *linkAddrCache) handleConfirmation(addr tcpip.Address, linkAddr tcpip.LinkAddress, flags ReachabilityConfirmationFlags) { - if len(linkAddr) != 0 { - // NUD allows confirmations without a link address but linkAddrCache - // is a simple neighbor table which does not implement NUD. - // - // As per RFC 4861 section 4.4, - // - // Target link-layer address - // The link-layer address for the target, i.e., the - // sender of the advertisement. This option MUST be - // included on link layers that have addresses when - // responding to multicast solicitations. When - // responding to a unicast Neighbor Solicitation this - // option SHOULD be included. - c.add(addr, linkAddr) - } -} - -func (c *linkAddrCache) handleUpperLevelConfirmation(tcpip.Address) {} - -func (*linkAddrCache) nudConfig() (NUDConfigurations, tcpip.Error) { - return NUDConfigurations{}, &tcpip.ErrNotSupported{} -} - -func (*linkAddrCache) setNUDConfig(NUDConfigurations) tcpip.Error { - return &tcpip.ErrNotSupported{} -} diff --git a/pkg/tcpip/stack/linkaddrentry_list.go b/pkg/tcpip/stack/linkaddrentry_list.go deleted file mode 100644 index 8ae2ff09c..000000000 --- a/pkg/tcpip/stack/linkaddrentry_list.go +++ /dev/null @@ -1,221 +0,0 @@ -package stack - -// ElementMapper provides an identity mapping by default. -// -// This can be replaced to provide a struct that maps elements to linker -// objects, if they are not the same. An ElementMapper is not typically -// required if: Linker is left as is, Element is left as is, or Linker and -// Element are the same type. -type linkAddrEntryElementMapper struct{} - -// linkerFor maps an Element to a Linker. -// -// This default implementation should be inlined. -// -//go:nosplit -func (linkAddrEntryElementMapper) linkerFor(elem *linkAddrEntry) *linkAddrEntry { return elem } - -// List is an intrusive list. Entries can be added to or removed from the list -// in O(1) time and with no additional memory allocations. -// -// The zero value for List is an empty list ready to use. -// -// To iterate over a list (where l is a List): -// for e := l.Front(); e != nil; e = e.Next() { -// // do something with e. -// } -// -// +stateify savable -type linkAddrEntryList struct { - head *linkAddrEntry - tail *linkAddrEntry -} - -// Reset resets list l to the empty state. -func (l *linkAddrEntryList) Reset() { - l.head = nil - l.tail = nil -} - -// Empty returns true iff the list is empty. -// -//go:nosplit -func (l *linkAddrEntryList) Empty() bool { - return l.head == nil -} - -// Front returns the first element of list l or nil. -// -//go:nosplit -func (l *linkAddrEntryList) Front() *linkAddrEntry { - return l.head -} - -// Back returns the last element of list l or nil. -// -//go:nosplit -func (l *linkAddrEntryList) Back() *linkAddrEntry { - return l.tail -} - -// Len returns the number of elements in the list. -// -// NOTE: This is an O(n) operation. -// -//go:nosplit -func (l *linkAddrEntryList) Len() (count int) { - for e := l.Front(); e != nil; e = (linkAddrEntryElementMapper{}.linkerFor(e)).Next() { - count++ - } - return count -} - -// PushFront inserts the element e at the front of list l. -// -//go:nosplit -func (l *linkAddrEntryList) PushFront(e *linkAddrEntry) { - linker := linkAddrEntryElementMapper{}.linkerFor(e) - linker.SetNext(l.head) - linker.SetPrev(nil) - if l.head != nil { - linkAddrEntryElementMapper{}.linkerFor(l.head).SetPrev(e) - } else { - l.tail = e - } - - l.head = e -} - -// PushBack inserts the element e at the back of list l. -// -//go:nosplit -func (l *linkAddrEntryList) PushBack(e *linkAddrEntry) { - linker := linkAddrEntryElementMapper{}.linkerFor(e) - linker.SetNext(nil) - linker.SetPrev(l.tail) - if l.tail != nil { - linkAddrEntryElementMapper{}.linkerFor(l.tail).SetNext(e) - } else { - l.head = e - } - - l.tail = e -} - -// PushBackList inserts list m at the end of list l, emptying m. -// -//go:nosplit -func (l *linkAddrEntryList) PushBackList(m *linkAddrEntryList) { - if l.head == nil { - l.head = m.head - l.tail = m.tail - } else if m.head != nil { - linkAddrEntryElementMapper{}.linkerFor(l.tail).SetNext(m.head) - linkAddrEntryElementMapper{}.linkerFor(m.head).SetPrev(l.tail) - - l.tail = m.tail - } - m.head = nil - m.tail = nil -} - -// InsertAfter inserts e after b. -// -//go:nosplit -func (l *linkAddrEntryList) InsertAfter(b, e *linkAddrEntry) { - bLinker := linkAddrEntryElementMapper{}.linkerFor(b) - eLinker := linkAddrEntryElementMapper{}.linkerFor(e) - - a := bLinker.Next() - - eLinker.SetNext(a) - eLinker.SetPrev(b) - bLinker.SetNext(e) - - if a != nil { - linkAddrEntryElementMapper{}.linkerFor(a).SetPrev(e) - } else { - l.tail = e - } -} - -// InsertBefore inserts e before a. -// -//go:nosplit -func (l *linkAddrEntryList) InsertBefore(a, e *linkAddrEntry) { - aLinker := linkAddrEntryElementMapper{}.linkerFor(a) - eLinker := linkAddrEntryElementMapper{}.linkerFor(e) - - b := aLinker.Prev() - eLinker.SetNext(a) - eLinker.SetPrev(b) - aLinker.SetPrev(e) - - if b != nil { - linkAddrEntryElementMapper{}.linkerFor(b).SetNext(e) - } else { - l.head = e - } -} - -// Remove removes e from l. -// -//go:nosplit -func (l *linkAddrEntryList) Remove(e *linkAddrEntry) { - linker := linkAddrEntryElementMapper{}.linkerFor(e) - prev := linker.Prev() - next := linker.Next() - - if prev != nil { - linkAddrEntryElementMapper{}.linkerFor(prev).SetNext(next) - } else if l.head == e { - l.head = next - } - - if next != nil { - linkAddrEntryElementMapper{}.linkerFor(next).SetPrev(prev) - } else if l.tail == e { - l.tail = prev - } - - linker.SetNext(nil) - linker.SetPrev(nil) -} - -// Entry is a default implementation of Linker. Users can add anonymous fields -// of this type to their structs to make them automatically implement the -// methods needed by List. -// -// +stateify savable -type linkAddrEntryEntry struct { - next *linkAddrEntry - prev *linkAddrEntry -} - -// Next returns the entry that follows e in the list. -// -//go:nosplit -func (e *linkAddrEntryEntry) Next() *linkAddrEntry { - return e.next -} - -// Prev returns the entry that precedes e in the list. -// -//go:nosplit -func (e *linkAddrEntryEntry) Prev() *linkAddrEntry { - return e.prev -} - -// SetNext assigns 'entry' as the entry that follows e in the list. -// -//go:nosplit -func (e *linkAddrEntryEntry) SetNext(elem *linkAddrEntry) { - e.next = elem -} - -// SetPrev assigns 'entry' as the entry that precedes e in the list. -// -//go:nosplit -func (e *linkAddrEntryEntry) SetPrev(elem *linkAddrEntry) { - e.prev = elem -} diff --git a/pkg/tcpip/stack/neighbor_cache.go b/pkg/tcpip/stack/neighbor_cache.go index 1409604df..a77fe575a 100644 --- a/pkg/tcpip/stack/neighbor_cache.go +++ b/pkg/tcpip/stack/neighbor_cache.go @@ -266,30 +266,6 @@ func (n *neighborCache) setConfig(config NUDConfigurations) { n.state.SetConfig(config) } -var _ neighborTable = (*neighborCache)(nil) - -func (n *neighborCache) neighbors() ([]NeighborEntry, tcpip.Error) { - return n.entries(), nil -} - -func (n *neighborCache) get(addr, localAddr tcpip.Address, onResolve func(LinkResolutionResult)) (tcpip.LinkAddress, <-chan struct{}, tcpip.Error) { - entry, ch, err := n.entry(addr, localAddr, onResolve) - return entry.LinkAddr, ch, err -} - -func (n *neighborCache) remove(addr tcpip.Address) tcpip.Error { - if !n.removeEntry(addr) { - return &tcpip.ErrBadAddress{} - } - - return nil -} - -func (n *neighborCache) removeAll() tcpip.Error { - n.clear() - return nil -} - // handleProbe handles a neighbor probe as defined by RFC 4861 section 7.2.3. // // Validation of the probe is expected to be handled by the caller. @@ -331,17 +307,8 @@ func (n *neighborCache) handleUpperLevelConfirmation(addr tcpip.Address) { } } -func (n *neighborCache) nudConfig() (NUDConfigurations, tcpip.Error) { - return n.config(), nil -} - -func (n *neighborCache) setNUDConfig(c NUDConfigurations) tcpip.Error { - n.setConfig(c) - return nil -} - -func newNeighborCache(nic *nic, r LinkAddressResolver) *neighborCache { - n := &neighborCache{ +func (n *neighborCache) init(nic *nic, r LinkAddressResolver) { + *n = neighborCache{ nic: nic, state: NewNUDState(nic.stack.nudConfigs, nic.stack.randomGenerator), linkRes: r, @@ -349,5 +316,4 @@ func newNeighborCache(nic *nic, r LinkAddressResolver) *neighborCache { n.mu.Lock() n.mu.cache = make(map[tcpip.Address]*neighborEntry, neighborCacheSize) n.mu.Unlock() - return n } diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go index f85652093..00cfba35a 100644 --- a/pkg/tcpip/stack/nic.go +++ b/pkg/tcpip/stack/nic.go @@ -24,37 +24,23 @@ 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) } +var _ NetworkInterface = (*nic)(nil) + // nic represents a "network interface card" to which the networking stack is // attached. type nic struct { @@ -70,7 +56,7 @@ 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 + linkAddrResolvers map[tcpip.NetworkProtocolNumber]*linkResolver // enabled is set to 1 when the NIC is enabled and 0 when it is disabled. // @@ -165,7 +151,7 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICC context: ctx, stats: makeNICStats(), networkEndpoints: make(map[tcpip.NetworkProtocolNumber]NetworkEndpoint), - linkAddrResolvers: make(map[tcpip.NetworkProtocolNumber]linkResolver), + linkAddrResolvers: make(map[tcpip.NetworkProtocolNumber]*linkResolver), } nic.linkResQueue.init(nic) nic.mu.packetEPs = make(map[tcpip.NetworkProtocolNumber]*packetEndpointList) @@ -185,17 +171,8 @@ 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 = newNeighborCache(nic, r) - } 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 } } @@ -640,7 +617,7 @@ func (n *nic) getLinkAddress(addr, localAddr tcpip.Address, protocol tcpip.Netwo 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{} @@ -648,7 +625,7 @@ func (n *nic) neighbors(protocol tcpip.NetworkProtocolNumber) ([]NeighborEntry, 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 } @@ -657,7 +634,10 @@ func (n *nic) addStaticNeighbor(addr tcpip.Address, protocol tcpip.NetworkProtoc 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{} @@ -665,7 +645,8 @@ func (n *nic) removeNeighbor(protocol tcpip.NetworkProtocolNumber, addr tcpip.Ad 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{} @@ -923,7 +904,7 @@ func (n *nic) Name() string { // nudConfigs gets the NUD configurations for n. 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{} @@ -936,7 +917,8 @@ func (n *nic) nudConfigs(protocol tcpip.NetworkProtocolNumber) (NUDConfiguration 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{} @@ -979,7 +961,7 @@ 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 { if l, ok := n.linkAddrResolvers[protocol]; ok { - l.neighborTable.handleProbe(addr, linkAddr) + l.neigh.handleProbe(addr, linkAddr) return nil } @@ -989,7 +971,7 @@ 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 { if l, ok := n.linkAddrResolvers[protocol]; ok { - l.neighborTable.handleConfirmation(addr, linkAddr, flags) + l.neigh.handleConfirmation(addr, linkAddr, flags) return nil } diff --git a/pkg/tcpip/stack/route.go b/pkg/tcpip/stack/route.go index 2ce1a65c6..e946f9fe3 100644 --- a/pkg/tcpip/stack/route.go +++ b/pkg/tcpip/stack/route.go @@ -53,7 +53,7 @@ type Route struct { // linkRes is set if link address resolution is enabled for this protocol on // the route's NIC. - linkRes linkResolver + linkRes *linkResolver } type routeInfo struct { @@ -184,7 +184,7 @@ func makeRoute(netProto tcpip.NetworkProtocolNumber, gateway, localAddr, remoteA return r } - if r.linkRes.resolver == nil { + if r.linkRes == nil { return r } @@ -400,7 +400,7 @@ func (r *Route) IsResolutionRequired() bool { } func (r *Route) isResolutionRequiredRLocked() bool { - return len(r.mu.remoteLinkAddress) == 0 && r.linkRes.resolver != nil && r.isValidForOutgoingRLocked() && !r.local() + return len(r.mu.remoteLinkAddress) == 0 && r.linkRes != nil && r.isValidForOutgoingRLocked() && !r.local() } func (r *Route) isValidForOutgoing() bool { @@ -528,7 +528,7 @@ func (r *Route) IsOutboundBroadcast() bool { // "Reachable" is defined as having full-duplex communication between the // local and remote ends of the route. func (r *Route) ConfirmReachable() { - if r.linkRes.resolver != nil { + if r.linkRes != nil { r.linkRes.confirmReachable(r.nextHop()) } } diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index 07cd88a6a..46c1817ea 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -434,12 +434,6 @@ type Stack struct { // nudConfigs is the default NUD configurations used by interfaces. nudConfigs NUDConfigurations - // useNeighborCache indicates whether ARP and NDP packets should be handled - // by the NIC's neighborCache instead of linkAddrCache. - // - // TODO(gvisor.dev/issue/4658): Remove this field. - useNeighborCache bool - // nudDisp is the NUD event dispatcher that is used to send the netstack // integrator NUD related events. nudDisp NUDDispatcher @@ -516,17 +510,6 @@ type Options struct { // NUDConfigs is the default NUD configurations used by interfaces. NUDConfigs NUDConfigurations - // UseNeighborCache is unused. - // - // TODO(gvisor.dev/issue/4658): Remove this field. - UseNeighborCache bool - - // UseLinkAddrCache indicates that the legacy link address cache should be - // used for link resolution. - // - // TODO(gvisor.dev/issue/4658): Remove this field. - UseLinkAddrCache bool - // NUDDisp is the NUD event dispatcher that an integrator can provide to // receive NUD related events. NUDDisp NUDDispatcher @@ -666,7 +649,6 @@ func New(opts Options) *Stack { icmpRateLimiter: NewICMPRateLimiter(), seed: generateRandUint32(), nudConfigs: opts.NUDConfigs, - useNeighborCache: !opts.UseLinkAddrCache, uniqueIDGenerator: opts.UniqueID, nudDisp: opts.NUDDisp, randomGenerator: mathrand.New(randSrc), diff --git a/pkg/tcpip/stack/stack_state_autogen.go b/pkg/tcpip/stack/stack_state_autogen.go index 271e0f1f4..462139b82 100644 --- a/pkg/tcpip/stack/stack_state_autogen.go +++ b/pkg/tcpip/stack/stack_state_autogen.go @@ -346,58 +346,6 @@ func (fl *IPHeaderFilter) StateLoad(stateSourceObject state.Source) { stateSourceObject.Load(13, &fl.OutputInterfaceInvert) } -func (l *linkAddrEntryList) StateTypeName() string { - return "pkg/tcpip/stack.linkAddrEntryList" -} - -func (l *linkAddrEntryList) StateFields() []string { - return []string{ - "head", - "tail", - } -} - -func (l *linkAddrEntryList) beforeSave() {} - -func (l *linkAddrEntryList) StateSave(stateSinkObject state.Sink) { - l.beforeSave() - stateSinkObject.Save(0, &l.head) - stateSinkObject.Save(1, &l.tail) -} - -func (l *linkAddrEntryList) afterLoad() {} - -func (l *linkAddrEntryList) StateLoad(stateSourceObject state.Source) { - stateSourceObject.Load(0, &l.head) - stateSourceObject.Load(1, &l.tail) -} - -func (e *linkAddrEntryEntry) StateTypeName() string { - return "pkg/tcpip/stack.linkAddrEntryEntry" -} - -func (e *linkAddrEntryEntry) StateFields() []string { - return []string{ - "next", - "prev", - } -} - -func (e *linkAddrEntryEntry) beforeSave() {} - -func (e *linkAddrEntryEntry) StateSave(stateSinkObject state.Sink) { - e.beforeSave() - stateSinkObject.Save(0, &e.next) - stateSinkObject.Save(1, &e.prev) -} - -func (e *linkAddrEntryEntry) afterLoad() {} - -func (e *linkAddrEntryEntry) StateLoad(stateSourceObject state.Source) { - stateSourceObject.Load(0, &e.next) - stateSourceObject.Load(1, &e.prev) -} - func (l *neighborEntryList) StateTypeName() string { return "pkg/tcpip/stack.neighborEntryList" } @@ -716,8 +664,6 @@ func init() { state.Register((*Table)(nil)) state.Register((*Rule)(nil)) state.Register((*IPHeaderFilter)(nil)) - state.Register((*linkAddrEntryList)(nil)) - state.Register((*linkAddrEntryEntry)(nil)) state.Register((*neighborEntryList)(nil)) state.Register((*neighborEntryEntry)(nil)) state.Register((*PacketBufferList)(nil)) |