Use embedded mutex pattern in neighbor cache/entry

Also while I'm here, update neighbor cahce/entry tests to use the
stack's RNG instead of creating a neigbor cache/entry specific one.

PiperOrigin-RevId: 356040581
This commit is contained in:
Ghanan Gowripalan 2021-02-06 10:44:59 -08:00 committed by gVisor bot
parent 9530f624e9
commit a83c8585af
5 changed files with 348 additions and 337 deletions

View File

@ -47,17 +47,18 @@ type neighborCache struct {
state *NUDState
linkRes LinkAddressResolver
// mu protects the fields below.
mu sync.RWMutex
mu struct {
sync.RWMutex
cache map[tcpip.Address]*neighborEntry
dynamic struct {
lru neighborEntryList
cache map[tcpip.Address]*neighborEntry
dynamic struct {
lru neighborEntryList
// count tracks the amount of dynamic entries in the cache. This is
// needed since static entries do not count towards the LRU cache
// eviction strategy.
count uint16
// count tracks the amount of dynamic entries in the cache. This is
// needed since static entries do not count towards the LRU cache
// eviction strategy.
count uint16
}
}
}
@ -74,11 +75,11 @@ func (n *neighborCache) getOrCreateEntry(remoteAddr tcpip.Address) *neighborEntr
n.mu.Lock()
defer n.mu.Unlock()
if entry, ok := n.cache[remoteAddr]; ok {
if entry, ok := n.mu.cache[remoteAddr]; ok {
entry.mu.RLock()
if entry.neigh.State != Static {
n.dynamic.lru.Remove(entry)
n.dynamic.lru.PushFront(entry)
if entry.mu.neigh.State != Static {
n.mu.dynamic.lru.Remove(entry)
n.mu.dynamic.lru.PushFront(entry)
}
entry.mu.RUnlock()
return entry
@ -87,20 +88,20 @@ func (n *neighborCache) getOrCreateEntry(remoteAddr tcpip.Address) *neighborEntr
// The entry that needs to be created must be dynamic since all static
// entries are directly added to the cache via addStaticEntry.
entry := newNeighborEntry(n, remoteAddr, n.state)
if n.dynamic.count == neighborCacheSize {
e := n.dynamic.lru.Back()
if n.mu.dynamic.count == neighborCacheSize {
e := n.mu.dynamic.lru.Back()
e.mu.Lock()
delete(n.cache, e.neigh.Addr)
n.dynamic.lru.Remove(e)
n.dynamic.count--
delete(n.mu.cache, e.mu.neigh.Addr)
n.mu.dynamic.lru.Remove(e)
n.mu.dynamic.count--
e.removeLocked()
e.mu.Unlock()
}
n.cache[remoteAddr] = entry
n.dynamic.lru.PushFront(entry)
n.dynamic.count++
n.mu.cache[remoteAddr] = entry
n.mu.dynamic.lru.PushFront(entry)
n.mu.dynamic.count++
return entry
}
@ -128,7 +129,7 @@ func (n *neighborCache) entry(remoteAddr, localAddr tcpip.Address, onResolve fun
entry.mu.Lock()
defer entry.mu.Unlock()
switch s := entry.neigh.State; s {
switch s := entry.mu.neigh.State; s {
case Stale:
entry.handlePacketQueuedLocked(localAddr)
fallthrough
@ -139,19 +140,19 @@ func (n *neighborCache) entry(remoteAddr, localAddr tcpip.Address, onResolve fun
// a node continues sending packets to that neighbor using the cached
// link-layer address."
if onResolve != nil {
onResolve(LinkResolutionResult{LinkAddress: entry.neigh.LinkAddr, Success: true})
onResolve(LinkResolutionResult{LinkAddress: entry.mu.neigh.LinkAddr, Success: true})
}
return entry.neigh, nil, nil
return entry.mu.neigh, nil, nil
case Unknown, Incomplete, Failed:
if onResolve != nil {
entry.onResolve = append(entry.onResolve, onResolve)
entry.mu.onResolve = append(entry.mu.onResolve, onResolve)
}
if entry.done == nil {
if entry.mu.done == nil {
// Address resolution needs to be initiated.
entry.done = make(chan struct{})
entry.mu.done = make(chan struct{})
}
entry.handlePacketQueuedLocked(localAddr)
return entry.neigh, entry.done, &tcpip.ErrWouldBlock{}
return entry.mu.neigh, entry.mu.done, &tcpip.ErrWouldBlock{}
default:
panic(fmt.Sprintf("Invalid cache entry state: %s", s))
}
@ -162,10 +163,10 @@ func (n *neighborCache) entries() []NeighborEntry {
n.mu.RLock()
defer n.mu.RUnlock()
entries := make([]NeighborEntry, 0, len(n.cache))
for _, entry := range n.cache {
entries := make([]NeighborEntry, 0, len(n.mu.cache))
for _, entry := range n.mu.cache {
entry.mu.RLock()
entries = append(entries, entry.neigh)
entries = append(entries, entry.mu.neigh)
entry.mu.RUnlock()
}
return entries
@ -181,19 +182,19 @@ func (n *neighborCache) addStaticEntry(addr tcpip.Address, linkAddr tcpip.LinkAd
n.mu.Lock()
defer n.mu.Unlock()
if entry, ok := n.cache[addr]; ok {
if entry, ok := n.mu.cache[addr]; ok {
entry.mu.Lock()
if entry.neigh.State != Static {
if entry.mu.neigh.State != Static {
// Dynamic entry found with the same address.
n.dynamic.lru.Remove(entry)
n.dynamic.count--
} else if entry.neigh.LinkAddr == linkAddr {
n.mu.dynamic.lru.Remove(entry)
n.mu.dynamic.count--
} else if entry.mu.neigh.LinkAddr == linkAddr {
// Static entry found with the same address and link address.
entry.mu.Unlock()
return
} else {
// Static entry found with the same address but different link address.
entry.neigh.LinkAddr = linkAddr
entry.mu.neigh.LinkAddr = linkAddr
entry.dispatchChangeEventLocked()
entry.mu.Unlock()
return
@ -203,7 +204,12 @@ func (n *neighborCache) addStaticEntry(addr tcpip.Address, linkAddr tcpip.LinkAd
entry.mu.Unlock()
}
n.cache[addr] = newStaticNeighborEntry(n, addr, linkAddr, n.state)
entry := newStaticNeighborEntry(n, addr, linkAddr, n.state)
n.mu.cache[addr] = entry
entry.mu.Lock()
defer entry.mu.Unlock()
entry.dispatchAddEventLocked()
}
// removeEntry removes a dynamic or static entry by address from the neighbor
@ -212,7 +218,7 @@ func (n *neighborCache) removeEntry(addr tcpip.Address) bool {
n.mu.Lock()
defer n.mu.Unlock()
entry, ok := n.cache[addr]
entry, ok := n.mu.cache[addr]
if !ok {
return false
}
@ -220,13 +226,13 @@ func (n *neighborCache) removeEntry(addr tcpip.Address) bool {
entry.mu.Lock()
defer entry.mu.Unlock()
if entry.neigh.State != Static {
n.dynamic.lru.Remove(entry)
n.dynamic.count--
if entry.mu.neigh.State != Static {
n.mu.dynamic.lru.Remove(entry)
n.mu.dynamic.count--
}
entry.removeLocked()
delete(n.cache, entry.neigh.Addr)
delete(n.mu.cache, entry.mu.neigh.Addr)
return true
}
@ -235,15 +241,15 @@ func (n *neighborCache) clear() {
n.mu.Lock()
defer n.mu.Unlock()
for _, entry := range n.cache {
for _, entry := range n.mu.cache {
entry.mu.Lock()
entry.removeLocked()
entry.mu.Unlock()
}
n.dynamic.lru = neighborEntryList{}
n.cache = make(map[tcpip.Address]*neighborEntry)
n.dynamic.count = 0
n.mu.dynamic.lru = neighborEntryList{}
n.mu.cache = make(map[tcpip.Address]*neighborEntry)
n.mu.dynamic.count = 0
}
// config returns the NUD configuration.
@ -300,7 +306,7 @@ func (n *neighborCache) handleProbe(remoteAddr tcpip.Address, remoteLinkAddr tcp
// Validation of the confirmation is expected to be handled by the caller.
func (n *neighborCache) handleConfirmation(addr tcpip.Address, linkAddr tcpip.LinkAddress, flags ReachabilityConfirmationFlags) {
n.mu.RLock()
entry, ok := n.cache[addr]
entry, ok := n.mu.cache[addr]
n.mu.RUnlock()
if ok {
entry.mu.Lock()
@ -316,7 +322,7 @@ func (n *neighborCache) handleConfirmation(addr tcpip.Address, linkAddr tcpip.Li
// some protocol that operates at a layer above the IP/link layer.
func (n *neighborCache) handleUpperLevelConfirmation(addr tcpip.Address) {
n.mu.RLock()
entry, ok := n.cache[addr]
entry, ok := n.mu.cache[addr]
n.mu.RUnlock()
if ok {
entry.mu.Lock()
@ -333,3 +339,15 @@ func (n *neighborCache) setNUDConfig(c NUDConfigurations) tcpip.Error {
n.setConfig(c)
return nil
}
func newNeighborCache(nic *nic, r LinkAddressResolver) *neighborCache {
n := &neighborCache{
nic: nic,
state: NewNUDState(nic.stack.nudConfigs, nic.stack.randomGenerator),
linkRes: r,
}
n.mu.Lock()
n.mu.cache = make(map[tcpip.Address]*neighborEntry, neighborCacheSize)
n.mu.Unlock()
return n
}

View File

@ -84,19 +84,16 @@ func newTestNeighborResolver(nudDisp NUDDispatcher, config NUDConfigurations, cl
entries: newTestEntryStore(),
delay: typicalLatency,
}
linkRes.neigh = &neighborCache{
nic: &nic{
stack: &Stack{
clock: clock,
nudDisp: nudDisp,
},
id: 1,
stats: makeNICStats(),
linkRes.neigh = newNeighborCache(&nic{
stack: &Stack{
clock: clock,
nudDisp: nudDisp,
nudConfigs: config,
randomGenerator: rng,
},
state: NewNUDState(config, rng),
linkRes: linkRes,
cache: make(map[tcpip.Address]*neighborEntry, neighborCacheSize),
}
id: 1,
stats: makeNICStats(),
}, linkRes)
return linkRes
}

View File

@ -82,20 +82,21 @@ type neighborEntry struct {
// nudState points to the Neighbor Unreachability Detection configuration.
nudState *NUDState
// mu protects the fields below.
mu sync.RWMutex
mu struct {
sync.RWMutex
neigh NeighborEntry
neigh NeighborEntry
// 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{}
// 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)
// onResolve is called with the result of address resolution.
onResolve []func(LinkResolutionResult)
isRouter bool
job *tcpip.Job
isRouter bool
job *tcpip.Job
}
}
// newNeighborEntry creates a neighbor cache entry starting at the default
@ -103,14 +104,18 @@ type neighborEntry struct {
// `handlePacketQueuedLocked` or `handleProbeLocked` on the newly created
// neighborEntry.
func newNeighborEntry(cache *neighborCache, remoteAddr tcpip.Address, nudState *NUDState) *neighborEntry {
return &neighborEntry{
n := &neighborEntry{
cache: cache,
nudState: nudState,
neigh: NeighborEntry{
Addr: remoteAddr,
State: Unknown,
},
}
n.mu.Lock()
n.mu.neigh = NeighborEntry{
Addr: remoteAddr,
State: Unknown,
}
n.mu.Unlock()
return n
}
// newStaticNeighborEntry creates a neighbor cache entry starting at the
@ -123,14 +128,14 @@ func newStaticNeighborEntry(cache *neighborCache, addr tcpip.Address, linkAddr t
State: Static,
UpdatedAtNanos: cache.nic.stack.clock.NowNanoseconds(),
}
if nudDisp := cache.nic.stack.nudDisp; nudDisp != nil {
nudDisp.OnNeighborAdded(cache.nic.id, entry)
}
return &neighborEntry{
n := &neighborEntry{
cache: cache,
nudState: state,
neigh: entry,
}
n.mu.Lock()
n.mu.neigh = entry
n.mu.Unlock()
return n
}
// notifyCompletionLocked notifies those waiting for address resolution, with
@ -138,14 +143,14 @@ func newStaticNeighborEntry(cache *neighborCache, addr tcpip.Address, linkAddr t
//
// Precondition: e.mu MUST be locked.
func (e *neighborEntry) notifyCompletionLocked(succeeded bool) {
res := LinkResolutionResult{LinkAddress: e.neigh.LinkAddr, Success: succeeded}
for _, callback := range e.onResolve {
res := LinkResolutionResult{LinkAddress: e.mu.neigh.LinkAddr, Success: succeeded}
for _, callback := range e.mu.onResolve {
callback(res)
}
e.onResolve = nil
if ch := e.done; ch != nil {
e.mu.onResolve = nil
if ch := e.mu.done; ch != nil {
close(ch)
e.done = nil
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.
//
@ -153,7 +158,7 @@ func (e *neighborEntry) notifyCompletionLocked(succeeded bool) {
// 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, e.neigh.LinkAddr, succeeded)
go e.cache.nic.linkResQueue.dequeue(ch, e.mu.neigh.LinkAddr, succeeded)
}
}
@ -163,7 +168,7 @@ func (e *neighborEntry) notifyCompletionLocked(succeeded bool) {
// Precondition: e.mu MUST be locked.
func (e *neighborEntry) dispatchAddEventLocked() {
if nudDisp := e.cache.nic.stack.nudDisp; nudDisp != nil {
nudDisp.OnNeighborAdded(e.cache.nic.id, e.neigh)
nudDisp.OnNeighborAdded(e.cache.nic.id, e.mu.neigh)
}
}
@ -173,7 +178,7 @@ func (e *neighborEntry) dispatchAddEventLocked() {
// Precondition: e.mu MUST be locked.
func (e *neighborEntry) dispatchChangeEventLocked() {
if nudDisp := e.cache.nic.stack.nudDisp; nudDisp != nil {
nudDisp.OnNeighborChanged(e.cache.nic.id, e.neigh)
nudDisp.OnNeighborChanged(e.cache.nic.id, e.mu.neigh)
}
}
@ -183,7 +188,7 @@ func (e *neighborEntry) dispatchChangeEventLocked() {
// Precondition: e.mu MUST be locked.
func (e *neighborEntry) dispatchRemoveEventLocked() {
if nudDisp := e.cache.nic.stack.nudDisp; nudDisp != nil {
nudDisp.OnNeighborRemoved(e.cache.nic.id, e.neigh)
nudDisp.OnNeighborRemoved(e.cache.nic.id, e.mu.neigh)
}
}
@ -192,7 +197,7 @@ func (e *neighborEntry) dispatchRemoveEventLocked() {
//
// Precondition: e.mu MUST be locked.
func (e *neighborEntry) cancelJobLocked() {
if job := e.job; job != nil {
if job := e.mu.job; job != nil {
job.Cancel()
}
}
@ -201,7 +206,7 @@ func (e *neighborEntry) cancelJobLocked() {
//
// Precondition: e.mu MUST be locked.
func (e *neighborEntry) removeLocked() {
e.neigh.UpdatedAtNanos = e.cache.nic.stack.clock.NowNanoseconds()
e.mu.neigh.UpdatedAtNanos = e.cache.nic.stack.clock.NowNanoseconds()
e.dispatchRemoveEventLocked()
e.cancelJobLocked()
e.notifyCompletionLocked(false /* succeeded */)
@ -215,28 +220,28 @@ func (e *neighborEntry) removeLocked() {
func (e *neighborEntry) setStateLocked(next NeighborState) {
e.cancelJobLocked()
prev := e.neigh.State
e.neigh.State = next
e.neigh.UpdatedAtNanos = e.cache.nic.stack.clock.NowNanoseconds()
prev := e.mu.neigh.State
e.mu.neigh.State = next
e.mu.neigh.UpdatedAtNanos = e.cache.nic.stack.clock.NowNanoseconds()
config := e.nudState.Config()
switch next {
case Incomplete:
panic(fmt.Sprintf("should never transition to Incomplete with setStateLocked; neigh = %#v, prev state = %s", e.neigh, prev))
panic(fmt.Sprintf("should never transition to Incomplete with setStateLocked; neigh = %#v, prev state = %s", e.mu.neigh, prev))
case Reachable:
e.job = e.cache.nic.stack.newJob(&e.mu, func() {
e.mu.job = e.cache.nic.stack.newJob(&e.mu, func() {
e.setStateLocked(Stale)
e.dispatchChangeEventLocked()
})
e.job.Schedule(e.nudState.ReachableTime())
e.mu.job.Schedule(e.nudState.ReachableTime())
case Delay:
e.job = e.cache.nic.stack.newJob(&e.mu, func() {
e.mu.job = e.cache.nic.stack.newJob(&e.mu, func() {
e.setStateLocked(Probe)
e.dispatchChangeEventLocked()
})
e.job.Schedule(config.DelayFirstProbeTime)
e.mu.job.Schedule(config.DelayFirstProbeTime)
case Probe:
var retryCounter uint32
@ -249,23 +254,23 @@ func (e *neighborEntry) setStateLocked(next NeighborState) {
return
}
if err := e.cache.linkRes.LinkAddressRequest(e.neigh.Addr, "" /* localAddr */, e.neigh.LinkAddr); err != nil {
if err := e.cache.linkRes.LinkAddressRequest(e.mu.neigh.Addr, "" /* localAddr */, e.mu.neigh.LinkAddr); err != nil {
e.dispatchRemoveEventLocked()
e.setStateLocked(Failed)
return
}
retryCounter++
e.job = e.cache.nic.stack.newJob(&e.mu, sendUnicastProbe)
e.job.Schedule(config.RetransmitTimer)
e.mu.job = e.cache.nic.stack.newJob(&e.mu, sendUnicastProbe)
e.mu.job.Schedule(config.RetransmitTimer)
}
// Send a probe in another gorountine to free this thread of execution
// for finishing the state transition. This is necessary to avoid
// deadlock where sending and processing probes are done synchronously,
// such as loopback and integration tests.
e.job = e.cache.nic.stack.newJob(&e.mu, sendUnicastProbe)
e.job.Schedule(immediateDuration)
e.mu.job = e.cache.nic.stack.newJob(&e.mu, sendUnicastProbe)
e.mu.job.Schedule(immediateDuration)
case Failed:
e.notifyCompletionLocked(false /* succeeded */)
@ -285,14 +290,14 @@ func (e *neighborEntry) setStateLocked(next NeighborState) {
//
// Precondition: e.mu MUST be locked.
func (e *neighborEntry) handlePacketQueuedLocked(localAddr tcpip.Address) {
switch e.neigh.State {
switch e.mu.neigh.State {
case Failed:
e.cache.nic.stats.Neighbor.FailedEntryLookups.Increment()
fallthrough
case Unknown:
e.neigh.State = Incomplete
e.neigh.UpdatedAtNanos = e.cache.nic.stack.clock.NowNanoseconds()
e.mu.neigh.State = Incomplete
e.mu.neigh.UpdatedAtNanos = e.cache.nic.stack.clock.NowNanoseconds()
e.dispatchAddEventLocked()
@ -335,7 +340,7 @@ func (e *neighborEntry) handlePacketQueuedLocked(localAddr tcpip.Address) {
// address SHOULD be placed in the IP Source Address of the outgoing
// solicitation.
//
if err := e.cache.linkRes.LinkAddressRequest(e.neigh.Addr, localAddr, ""); err != nil {
if err := e.cache.linkRes.LinkAddressRequest(e.mu.neigh.Addr, localAddr, ""); err != nil {
// There is no need to log the error here; the NUD implementation may
// assume a working link. A valid link should be the responsibility of
// the NIC/stack.LinkEndpoint.
@ -345,16 +350,16 @@ func (e *neighborEntry) handlePacketQueuedLocked(localAddr tcpip.Address) {
}
retryCounter++
e.job = e.cache.nic.stack.newJob(&e.mu, sendMulticastProbe)
e.job.Schedule(config.RetransmitTimer)
e.mu.job = e.cache.nic.stack.newJob(&e.mu, sendMulticastProbe)
e.mu.job.Schedule(config.RetransmitTimer)
}
// Send a probe in another gorountine to free this thread of execution
// for finishing the state transition. This is necessary to avoid
// deadlock where sending and processing probes are done synchronously,
// such as loopback and integration tests.
e.job = e.cache.nic.stack.newJob(&e.mu, sendMulticastProbe)
e.job.Schedule(immediateDuration)
e.mu.job = e.cache.nic.stack.newJob(&e.mu, sendMulticastProbe)
e.mu.job.Schedule(immediateDuration)
case Stale:
e.setStateLocked(Delay)
@ -363,7 +368,7 @@ func (e *neighborEntry) handlePacketQueuedLocked(localAddr tcpip.Address) {
case Incomplete, Reachable, Delay, Probe, Static:
// Do nothing
default:
panic(fmt.Sprintf("Invalid cache entry state: %s", e.neigh.State))
panic(fmt.Sprintf("Invalid cache entry state: %s", e.mu.neigh.State))
}
}
@ -378,9 +383,9 @@ func (e *neighborEntry) handleProbeLocked(remoteLinkAddr tcpip.LinkAddress) {
// not exist, or not bound to the NIC as per RFC 4861 section 7.2.3. These
// checks MUST be done by the NetworkEndpoint.
switch e.neigh.State {
switch e.mu.neigh.State {
case Unknown, Failed:
e.neigh.LinkAddr = remoteLinkAddr
e.mu.neigh.LinkAddr = remoteLinkAddr
e.setStateLocked(Stale)
e.dispatchAddEventLocked()
@ -390,21 +395,21 @@ func (e *neighborEntry) handleProbeLocked(remoteLinkAddr tcpip.LinkAddress) {
// cached address should be replaced by the received address, and the
// entry's reachability state MUST be set to STALE."
// - RFC 4861 section 7.2.3
e.neigh.LinkAddr = remoteLinkAddr
e.mu.neigh.LinkAddr = remoteLinkAddr
e.setStateLocked(Stale)
e.notifyCompletionLocked(true /* succeeded */)
e.dispatchChangeEventLocked()
case Reachable, Delay, Probe:
if e.neigh.LinkAddr != remoteLinkAddr {
e.neigh.LinkAddr = remoteLinkAddr
if e.mu.neigh.LinkAddr != remoteLinkAddr {
e.mu.neigh.LinkAddr = remoteLinkAddr
e.setStateLocked(Stale)
e.dispatchChangeEventLocked()
}
case Stale:
if e.neigh.LinkAddr != remoteLinkAddr {
e.neigh.LinkAddr = remoteLinkAddr
if e.mu.neigh.LinkAddr != remoteLinkAddr {
e.mu.neigh.LinkAddr = remoteLinkAddr
e.dispatchChangeEventLocked()
}
@ -412,7 +417,7 @@ func (e *neighborEntry) handleProbeLocked(remoteLinkAddr tcpip.LinkAddress) {
// Do nothing
default:
panic(fmt.Sprintf("Invalid cache entry state: %s", e.neigh.State))
panic(fmt.Sprintf("Invalid cache entry state: %s", e.mu.neigh.State))
}
}
@ -430,7 +435,7 @@ func (e *neighborEntry) handleProbeLocked(remoteLinkAddr tcpip.LinkAddress) {
//
// Precondition: e.mu MUST be locked.
func (e *neighborEntry) handleConfirmationLocked(linkAddr tcpip.LinkAddress, flags ReachabilityConfirmationFlags) {
switch e.neigh.State {
switch e.mu.neigh.State {
case Incomplete:
if len(linkAddr) == 0 {
// "If the link layer has addresses and no Target Link-Layer Address
@ -439,35 +444,35 @@ func (e *neighborEntry) handleConfirmationLocked(linkAddr tcpip.LinkAddress, fla
break
}
e.neigh.LinkAddr = linkAddr
e.mu.neigh.LinkAddr = linkAddr
if flags.Solicited {
e.setStateLocked(Reachable)
} else {
e.setStateLocked(Stale)
}
e.dispatchChangeEventLocked()
e.isRouter = flags.IsRouter
e.mu.isRouter = flags.IsRouter
e.notifyCompletionLocked(true /* succeeded */)
// "Note that the Override flag is ignored if the entry is in the
// INCOMPLETE state." - RFC 4861 section 7.2.5
case Reachable, Stale, Delay, Probe:
isLinkAddrDifferent := len(linkAddr) != 0 && e.neigh.LinkAddr != linkAddr
isLinkAddrDifferent := len(linkAddr) != 0 && e.mu.neigh.LinkAddr != linkAddr
if isLinkAddrDifferent {
if !flags.Override {
if e.neigh.State == Reachable {
if e.mu.neigh.State == Reachable {
e.setStateLocked(Stale)
e.dispatchChangeEventLocked()
}
break
}
e.neigh.LinkAddr = linkAddr
e.mu.neigh.LinkAddr = linkAddr
if !flags.Solicited {
if e.neigh.State != Stale {
if e.mu.neigh.State != Stale {
e.setStateLocked(Stale)
e.dispatchChangeEventLocked()
} else {
@ -479,7 +484,7 @@ func (e *neighborEntry) handleConfirmationLocked(linkAddr tcpip.LinkAddress, fla
}
if flags.Solicited && (flags.Override || !isLinkAddrDifferent) {
wasReachable := e.neigh.State == Reachable
wasReachable := e.mu.neigh.State == Reachable
// Set state to Reachable again to refresh timers.
e.setStateLocked(Reachable)
e.notifyCompletionLocked(true /* succeeded */)
@ -488,7 +493,7 @@ func (e *neighborEntry) handleConfirmationLocked(linkAddr tcpip.LinkAddress, fla
}
}
if e.isRouter && !flags.IsRouter && header.IsV6UnicastAddress(e.neigh.Addr) {
if e.mu.isRouter && !flags.IsRouter && header.IsV6UnicastAddress(e.mu.neigh.Addr) {
// "In those cases where the IsRouter flag changes from TRUE to FALSE as
// a result of this update, the node MUST remove that router from the
// Default Router List and update the Destination Cache entries for all
@ -505,16 +510,16 @@ func (e *neighborEntry) handleConfirmationLocked(linkAddr tcpip.LinkAddress, fla
}
if ndpEP, ok := ep.(NDPEndpoint); ok {
ndpEP.InvalidateDefaultRouter(e.neigh.Addr)
ndpEP.InvalidateDefaultRouter(e.mu.neigh.Addr)
}
}
e.isRouter = flags.IsRouter
e.mu.isRouter = flags.IsRouter
case Unknown, Failed, Static:
// Do nothing
default:
panic(fmt.Sprintf("Invalid cache entry state: %s", e.neigh.State))
panic(fmt.Sprintf("Invalid cache entry state: %s", e.mu.neigh.State))
}
}
@ -523,9 +528,9 @@ func (e *neighborEntry) handleConfirmationLocked(linkAddr tcpip.LinkAddress, fla
//
// Precondition: e.mu MUST be locked.
func (e *neighborEntry) handleUpperLevelConfirmationLocked() {
switch e.neigh.State {
switch e.mu.neigh.State {
case Reachable, Stale, Delay, Probe:
wasReachable := e.neigh.State == Reachable
wasReachable := e.mu.neigh.State == Reachable
// Set state to Reachable again to refresh timers.
e.setStateLocked(Reachable)
if !wasReachable {
@ -536,6 +541,6 @@ func (e *neighborEntry) handleUpperLevelConfirmationLocked() {
// Do nothing
default:
panic(fmt.Sprintf("Invalid cache entry state: %s", e.neigh.State))
panic(fmt.Sprintf("Invalid cache entry state: %s", e.mu.neigh.State))
}
}

View File

@ -225,8 +225,10 @@ func entryTestSetup(c NUDConfigurations) (*neighborEntry, *testNUDDispatcher, *e
id: entryTestNICID,
stack: &Stack{
clock: clock,
nudDisp: &disp,
clock: clock,
nudDisp: &disp,
nudConfigs: c,
randomGenerator: rand.New(rand.NewSource(time.Now().UnixNano())),
},
stats: makeNICStats(),
}
@ -235,22 +237,17 @@ func entryTestSetup(c NUDConfigurations) (*neighborEntry, *testNUDDispatcher, *e
header.IPv6ProtocolNumber: netEP,
}
rng := rand.New(rand.NewSource(time.Now().UnixNano()))
nudState := NewNUDState(c, rng)
var linkRes entryTestLinkResolver
// Stub out the neighbor cache to verify deletion from the cache.
neigh := &neighborCache{
nic: &nic,
state: nudState,
linkRes: &linkRes,
cache: make(map[tcpip.Address]*neighborEntry, neighborCacheSize),
}
neigh := newNeighborCache(&nic, &linkRes)
l := linkResolver{
resolver: &linkRes,
neighborTable: neigh,
}
entry := newNeighborEntry(neigh, entryTestAddr1 /* remoteAddr */, nudState)
neigh.cache[entryTestAddr1] = entry
entry := newNeighborEntry(neigh, entryTestAddr1 /* remoteAddr */, neigh.state)
neigh.mu.Lock()
neigh.mu.cache[entryTestAddr1] = entry
neigh.mu.Unlock()
nic.linkAddrResolvers = map[tcpip.NetworkProtocolNumber]linkResolver{
header.IPv6ProtocolNumber: l,
}
@ -265,8 +262,8 @@ func TestEntryInitiallyUnknown(t *testing.T) {
e, nudDisp, linkRes, clock := entryTestSetup(c)
e.mu.Lock()
if e.neigh.State != Unknown {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Unknown)
if e.mu.neigh.State != Unknown {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Unknown)
}
e.mu.Unlock()
@ -298,8 +295,8 @@ func TestEntryUnknownToUnknownWhenConfirmationWithUnknownAddress(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Unknown {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Unknown)
if e.mu.neigh.State != Unknown {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Unknown)
}
e.mu.Unlock()
@ -327,8 +324,8 @@ func TestEntryUnknownToIncomplete(t *testing.T) {
e.mu.Lock()
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Incomplete {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Incomplete)
if e.mu.neigh.State != Incomplete {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Incomplete)
}
e.mu.Unlock()
@ -374,8 +371,8 @@ func TestEntryUnknownToStale(t *testing.T) {
e.mu.Lock()
e.handleProbeLocked(entryTestLinkAddr1)
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -413,10 +410,10 @@ func TestEntryIncompleteToIncompleteDoesNotChangeUpdatedAt(t *testing.T) {
e.mu.Lock()
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Incomplete {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Incomplete)
if e.mu.neigh.State != Incomplete {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Incomplete)
}
updatedAtNanos := e.neigh.UpdatedAtNanos
updatedAtNanos := e.mu.neigh.UpdatedAtNanos
e.mu.Unlock()
clock.Advance(c.RetransmitTimer)
@ -443,8 +440,8 @@ func TestEntryIncompleteToIncompleteDoesNotChangeUpdatedAt(t *testing.T) {
}
e.mu.Lock()
if got, want := e.neigh.UpdatedAtNanos, updatedAtNanos; got != want {
t.Errorf("got e.neigh.UpdatedAt = %q, want = %q", got, want)
if got, want := e.mu.neigh.UpdatedAtNanos, updatedAtNanos; got != want {
t.Errorf("got e.mu.neigh.UpdatedAt = %q, want = %q", got, want)
}
e.mu.Unlock()
@ -497,8 +494,8 @@ func TestEntryIncompleteToIncompleteDoesNotChangeUpdatedAt(t *testing.T) {
nudDisp.mu.Unlock()
e.mu.Lock()
if got, notWant := e.neigh.UpdatedAtNanos, updatedAtNanos; got == notWant {
t.Errorf("expected e.neigh.UpdatedAt to change, got = %q", got)
if got, notWant := e.mu.neigh.UpdatedAtNanos, updatedAtNanos; got == notWant {
t.Errorf("expected e.mu.neigh.UpdatedAt to change, got = %q", got)
}
e.mu.Unlock()
}
@ -509,8 +506,8 @@ func TestEntryIncompleteToReachable(t *testing.T) {
e.mu.Lock()
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Incomplete {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Incomplete)
if e.mu.neigh.State != Incomplete {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Incomplete)
}
e.mu.Unlock()
@ -535,8 +532,8 @@ func TestEntryIncompleteToReachable(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
e.mu.Unlock()
@ -573,8 +570,8 @@ func TestEntryIncompleteToReachableWithRouterFlag(t *testing.T) {
e.mu.Lock()
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Incomplete {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Incomplete)
if e.mu.neigh.State != Incomplete {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Incomplete)
}
e.mu.Unlock()
@ -599,11 +596,11 @@ func TestEntryIncompleteToReachableWithRouterFlag(t *testing.T) {
Override: false,
IsRouter: true,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
if !e.isRouter {
t.Errorf("got e.isRouter = %t, want = true", e.isRouter)
if !e.mu.isRouter {
t.Errorf("got e.mu.isRouter = %t, want = true", e.mu.isRouter)
}
e.mu.Unlock()
@ -640,8 +637,8 @@ func TestEntryIncompleteToStaleWhenUnsolicitedConfirmation(t *testing.T) {
e.mu.Lock()
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Incomplete {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Incomplete)
if e.mu.neigh.State != Incomplete {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Incomplete)
}
e.mu.Unlock()
@ -666,8 +663,8 @@ func TestEntryIncompleteToStaleWhenUnsolicitedConfirmation(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -704,8 +701,8 @@ func TestEntryIncompleteToStaleWhenProbe(t *testing.T) {
e.mu.Lock()
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Incomplete {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Incomplete)
if e.mu.neigh.State != Incomplete {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Incomplete)
}
e.mu.Unlock()
@ -726,8 +723,8 @@ func TestEntryIncompleteToStaleWhenProbe(t *testing.T) {
e.mu.Lock()
e.handleProbeLocked(entryTestLinkAddr1)
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -765,8 +762,8 @@ func TestEntryIncompleteToFailed(t *testing.T) {
e.mu.Lock()
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Incomplete {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Incomplete)
if e.mu.neigh.State != Incomplete {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Incomplete)
}
e.mu.Unlock()
@ -826,8 +823,8 @@ func TestEntryIncompleteToFailed(t *testing.T) {
nudDisp.mu.Unlock()
e.mu.Lock()
if e.neigh.State != Failed {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Failed)
if e.mu.neigh.State != Failed {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Failed)
}
e.mu.Unlock()
}
@ -870,11 +867,11 @@ func TestEntryStaysReachableWhenConfirmationWithRouterFlag(t *testing.T) {
Override: false,
IsRouter: true,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
if got, want := e.isRouter, true; got != want {
t.Errorf("got e.isRouter = %t, want = %t", got, want)
if got, want := e.mu.isRouter, true; got != want {
t.Errorf("got e.mu.isRouter = %t, want = %t", got, want)
}
e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
@ -882,11 +879,11 @@ func TestEntryStaysReachableWhenConfirmationWithRouterFlag(t *testing.T) {
Override: false,
IsRouter: false,
})
if got, want := e.isRouter, false; got != want {
t.Errorf("got e.isRouter = %t, want = %t", got, want)
if got, want := e.mu.isRouter, false; got != want {
t.Errorf("got e.mu.isRouter = %t, want = %t", got, want)
}
if ipv6EP.invalidatedRtr != e.neigh.Addr {
t.Errorf("got ipv6EP.invalidatedRtr = %s, want = %s", ipv6EP.invalidatedRtr, e.neigh.Addr)
if ipv6EP.invalidatedRtr != e.mu.neigh.Addr {
t.Errorf("got ipv6EP.invalidatedRtr = %s, want = %s", ipv6EP.invalidatedRtr, e.mu.neigh.Addr)
}
e.mu.Unlock()
@ -917,8 +914,8 @@ func TestEntryStaysReachableWhenConfirmationWithRouterFlag(t *testing.T) {
nudDisp.mu.Unlock()
e.mu.Lock()
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
e.mu.Unlock()
}
@ -952,15 +949,15 @@ func TestEntryStaysReachableWhenProbeWithSameAddress(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
e.handleProbeLocked(entryTestLinkAddr1)
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
if e.neigh.LinkAddr != entryTestLinkAddr1 {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", e.neigh.LinkAddr, entryTestLinkAddr1)
if e.mu.neigh.LinkAddr != entryTestLinkAddr1 {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", e.mu.neigh.LinkAddr, entryTestLinkAddr1)
}
e.mu.Unlock()
@ -1025,8 +1022,8 @@ func TestEntryReachableToStaleWhenTimeout(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
e.mu.Unlock()
@ -1068,8 +1065,8 @@ func TestEntryReachableToStaleWhenTimeout(t *testing.T) {
nudDisp.mu.Unlock()
e.mu.Lock()
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
}
@ -1103,12 +1100,12 @@ func TestEntryReachableToStaleWhenProbeWithDifferentAddress(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
e.handleProbeLocked(entryTestLinkAddr2)
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -1177,16 +1174,16 @@ func TestEntryReachableToStaleWhenConfirmationWithDifferentAddress(t *testing.T)
Override: false,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
Solicited: false,
Override: false,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -1255,16 +1252,16 @@ func TestEntryReachableToStaleWhenConfirmationWithDifferentAddressAndOverride(t
Override: false,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
Solicited: false,
Override: true,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -1333,15 +1330,15 @@ func TestEntryStaysStaleWhenProbeWithSameAddress(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.handleProbeLocked(entryTestLinkAddr1)
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
if e.neigh.LinkAddr != entryTestLinkAddr1 {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", e.neigh.LinkAddr, entryTestLinkAddr1)
if e.mu.neigh.LinkAddr != entryTestLinkAddr1 {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", e.mu.neigh.LinkAddr, entryTestLinkAddr1)
}
e.mu.Unlock()
@ -1401,19 +1398,19 @@ func TestEntryStaleToReachableWhenSolicitedOverrideConfirmation(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
Solicited: true,
Override: true,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
if e.neigh.LinkAddr != entryTestLinkAddr2 {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", e.neigh.LinkAddr, entryTestLinkAddr2)
if e.mu.neigh.LinkAddr != entryTestLinkAddr2 {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", e.mu.neigh.LinkAddr, entryTestLinkAddr2)
}
e.mu.Unlock()
@ -1482,19 +1479,19 @@ func TestEntryStaleToReachableWhenSolicitedConfirmationWithoutAddress(t *testing
Override: false,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.handleConfirmationLocked("" /* linkAddr */, ReachabilityConfirmationFlags{
Solicited: true,
Override: false,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
if e.neigh.LinkAddr != entryTestLinkAddr1 {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", e.neigh.LinkAddr, entryTestLinkAddr1)
if e.mu.neigh.LinkAddr != entryTestLinkAddr1 {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", e.mu.neigh.LinkAddr, entryTestLinkAddr1)
}
e.mu.Unlock()
@ -1563,19 +1560,19 @@ func TestEntryStaleToStaleWhenOverrideConfirmation(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
Solicited: false,
Override: true,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
if e.neigh.LinkAddr != entryTestLinkAddr2 {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", e.neigh.LinkAddr, entryTestLinkAddr2)
if e.mu.neigh.LinkAddr != entryTestLinkAddr2 {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", e.mu.neigh.LinkAddr, entryTestLinkAddr2)
}
e.mu.Unlock()
@ -1644,15 +1641,15 @@ func TestEntryStaleToStaleWhenProbeUpdateAddress(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.handleProbeLocked(entryTestLinkAddr2)
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
if e.neigh.LinkAddr != entryTestLinkAddr2 {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", e.neigh.LinkAddr, entryTestLinkAddr2)
if e.mu.neigh.LinkAddr != entryTestLinkAddr2 {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", e.mu.neigh.LinkAddr, entryTestLinkAddr2)
}
e.mu.Unlock()
@ -1721,12 +1718,12 @@ func TestEntryStaleToDelay(t *testing.T) {
Override: false,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Delay {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Delay {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -1801,12 +1798,12 @@ func TestEntryDelayToReachableWhenUpperLevelConfirmation(t *testing.T) {
IsRouter: false,
})
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Delay {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Delay)
if e.mu.neigh.State != Delay {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Delay)
}
e.handleUpperLevelConfirmationLocked()
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
e.mu.Unlock()
@ -1901,19 +1898,19 @@ func TestEntryDelayToReachableWhenSolicitedOverrideConfirmation(t *testing.T) {
IsRouter: false,
})
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Delay {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Delay)
if e.mu.neigh.State != Delay {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Delay)
}
e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
Solicited: true,
Override: true,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
if e.neigh.LinkAddr != entryTestLinkAddr2 {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", e.neigh.LinkAddr, entryTestLinkAddr2)
if e.mu.neigh.LinkAddr != entryTestLinkAddr2 {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", e.mu.neigh.LinkAddr, entryTestLinkAddr2)
}
e.mu.Unlock()
@ -2008,19 +2005,19 @@ func TestEntryDelayToReachableWhenSolicitedConfirmationWithoutAddress(t *testing
IsRouter: false,
})
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Delay {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Delay)
if e.mu.neigh.State != Delay {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Delay)
}
e.handleConfirmationLocked("" /* linkAddr */, ReachabilityConfirmationFlags{
Solicited: true,
Override: false,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
if e.neigh.LinkAddr != entryTestLinkAddr1 {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", e.neigh.LinkAddr, entryTestLinkAddr1)
if e.mu.neigh.LinkAddr != entryTestLinkAddr1 {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", e.mu.neigh.LinkAddr, entryTestLinkAddr1)
}
e.mu.Unlock()
@ -2109,19 +2106,19 @@ func TestEntryStaysDelayWhenOverrideConfirmationWithSameAddress(t *testing.T) {
IsRouter: false,
})
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Delay {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Delay)
if e.mu.neigh.State != Delay {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Delay)
}
e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
Solicited: false,
Override: true,
IsRouter: false,
})
if e.neigh.State != Delay {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Delay)
if e.mu.neigh.State != Delay {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Delay)
}
if e.neigh.LinkAddr != entryTestLinkAddr1 {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", e.neigh.LinkAddr, entryTestLinkAddr1)
if e.mu.neigh.LinkAddr != entryTestLinkAddr1 {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", e.mu.neigh.LinkAddr, entryTestLinkAddr1)
}
e.mu.Unlock()
@ -2191,12 +2188,12 @@ func TestEntryDelayToStaleWhenProbeWithDifferentAddress(t *testing.T) {
IsRouter: false,
})
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Delay {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Delay)
if e.mu.neigh.State != Delay {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Delay)
}
e.handleProbeLocked(entryTestLinkAddr2)
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -2275,16 +2272,16 @@ func TestEntryDelayToStaleWhenConfirmationWithDifferentAddress(t *testing.T) {
IsRouter: false,
})
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Delay {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Delay)
if e.mu.neigh.State != Delay {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Delay)
}
e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
Solicited: false,
Override: true,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -2366,8 +2363,8 @@ func TestEntryDelayToProbe(t *testing.T) {
IsRouter: false,
})
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Delay {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Delay)
if e.mu.neigh.State != Delay {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Delay)
}
e.mu.Unlock()
@ -2432,8 +2429,8 @@ func TestEntryDelayToProbe(t *testing.T) {
nudDisp.mu.Unlock()
e.mu.Lock()
if e.neigh.State != Probe {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Probe)
if e.mu.neigh.State != Probe {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Probe)
}
e.mu.Unlock()
}
@ -2490,12 +2487,12 @@ func TestEntryProbeToStaleWhenProbeWithDifferentAddress(t *testing.T) {
}
e.mu.Lock()
if e.neigh.State != Probe {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Probe)
if e.mu.neigh.State != Probe {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Probe)
}
e.handleProbeLocked(entryTestLinkAddr2)
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -2605,16 +2602,16 @@ func TestEntryProbeToStaleWhenConfirmationWithDifferentAddress(t *testing.T) {
}
e.mu.Lock()
if e.neigh.State != Probe {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Probe)
if e.mu.neigh.State != Probe {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Probe)
}
e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
Solicited: false,
Override: true,
IsRouter: false,
})
if e.neigh.State != Stale {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Stale)
if e.mu.neigh.State != Stale {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Stale)
}
e.mu.Unlock()
@ -2725,19 +2722,19 @@ func TestEntryStaysProbeWhenOverrideConfirmationWithSameAddress(t *testing.T) {
}
e.mu.Lock()
if e.neigh.State != Probe {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Probe)
if e.mu.neigh.State != Probe {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Probe)
}
e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
Solicited: false,
Override: true,
IsRouter: false,
})
if e.neigh.State != Probe {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Probe)
if e.mu.neigh.State != Probe {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Probe)
}
if got, want := e.neigh.LinkAddr, entryTestLinkAddr1; got != want {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
if got, want := e.mu.neigh.LinkAddr, entryTestLinkAddr1; got != want {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", got, want)
}
e.mu.Unlock()
@ -2821,19 +2818,19 @@ func TestEntryUnknownToStaleToProbeToReachable(t *testing.T) {
}
e.mu.Lock()
if e.neigh.State != Probe {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Probe)
if e.mu.neigh.State != Probe {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Probe)
}
e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
Solicited: true,
Override: true,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
if got, want := e.neigh.LinkAddr, entryTestLinkAddr2; got != want {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
if got, want := e.mu.neigh.LinkAddr, entryTestLinkAddr2; got != want {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", got, want)
}
e.mu.Unlock()
@ -2949,19 +2946,19 @@ func TestEntryProbeToReachableWhenSolicitedOverrideConfirmation(t *testing.T) {
}
e.mu.Lock()
if e.neigh.State != Probe {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Probe)
if e.mu.neigh.State != Probe {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Probe)
}
e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
Solicited: true,
Override: true,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
if got, want := e.neigh.LinkAddr, entryTestLinkAddr2; got != want {
t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
if got, want := e.mu.neigh.LinkAddr, entryTestLinkAddr2; got != want {
t.Errorf("got e.mu.neigh.LinkAddr = %q, want = %q", got, want)
}
e.mu.Unlock()
@ -3086,16 +3083,16 @@ func TestEntryProbeToReachableWhenSolicitedConfirmationWithSameAddress(t *testin
}
e.mu.Lock()
if e.neigh.State != Probe {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Probe)
if e.mu.neigh.State != Probe {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Probe)
}
e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
Solicited: true,
Override: false,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
e.mu.Unlock()
@ -3220,16 +3217,16 @@ func TestEntryProbeToReachableWhenSolicitedConfirmationWithoutAddress(t *testing
}
e.mu.Lock()
if e.neigh.State != Probe {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Probe)
if e.mu.neigh.State != Probe {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Probe)
}
e.handleConfirmationLocked("" /* linkAddr */, ReachabilityConfirmationFlags{
Solicited: true,
Override: false,
IsRouter: false,
})
if e.neigh.State != Reachable {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Reachable)
if e.mu.neigh.State != Reachable {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Reachable)
}
e.mu.Unlock()
@ -3352,8 +3349,8 @@ func TestEntryProbeToFailed(t *testing.T) {
}
e.mu.Lock()
if e.neigh.State != Probe {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Probe)
if e.mu.neigh.State != Probe {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Probe)
}
e.mu.Unlock()
}
@ -3361,8 +3358,8 @@ func TestEntryProbeToFailed(t *testing.T) {
// Wait for the last probe to expire, causing a transition to Failed.
clock.Advance(c.RetransmitTimer)
e.mu.Lock()
if e.neigh.State != Failed {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Failed)
if e.mu.neigh.State != Failed {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Failed)
}
e.mu.Unlock()
@ -3429,8 +3426,8 @@ func TestEntryFailedToIncomplete(t *testing.T) {
// their expected state.
e.mu.Lock()
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Incomplete {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Incomplete)
if e.mu.neigh.State != Incomplete {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Incomplete)
}
e.mu.Unlock()
@ -3464,15 +3461,15 @@ func TestEntryFailedToIncomplete(t *testing.T) {
}
e.mu.Lock()
if e.neigh.State != Failed {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Failed)
if e.mu.neigh.State != Failed {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Failed)
}
e.mu.Unlock()
e.mu.Lock()
e.handlePacketQueuedLocked(entryTestAddr2)
if e.neigh.State != Incomplete {
t.Errorf("got e.neigh.State = %q, want = %q", e.neigh.State, Incomplete)
if e.mu.neigh.State != Incomplete {
t.Errorf("got e.mu.neigh.State = %q, want = %q", e.mu.neigh.State, Incomplete)
}
e.mu.Unlock()

View File

@ -190,13 +190,7 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICC
}
if stack.useNeighborCache {
l.neighborTable = &neighborCache{
nic: nic,
state: NewNUDState(stack.nudConfigs, stack.randomGenerator),
linkRes: r,
cache: make(map[tcpip.Address]*neighborEntry, neighborCacheSize),
}
l.neighborTable = newNeighborCache(nic, r)
} else {
cache := new(linkAddrCache)
cache.init(nic, ageLimit, resolutionTimeout, resolutionAttempts, r)