summaryrefslogtreecommitdiffhomepage
path: root/pkg
diff options
context:
space:
mode:
Diffstat (limited to 'pkg')
-rw-r--r--pkg/tcpip/network/ipv6/icmp.go8
-rw-r--r--pkg/tcpip/stack/ndp.go118
-rw-r--r--pkg/tcpip/stack/ndp_test.go149
-rw-r--r--pkg/tcpip/stack/nic.go3
-rw-r--r--pkg/tcpip/stack/stack.go9
-rw-r--r--pkg/tcpip/stack/stack_test.go46
6 files changed, 245 insertions, 88 deletions
diff --git a/pkg/tcpip/network/ipv6/icmp.go b/pkg/tcpip/network/ipv6/icmp.go
index 6c14b4aae..b289e902f 100644
--- a/pkg/tcpip/network/ipv6/icmp.go
+++ b/pkg/tcpip/network/ipv6/icmp.go
@@ -246,15 +246,15 @@ func (e *endpoint) handleICMP(r *stack.Route, netHeader buffer.View, vv buffer.V
return
}
- // At this point we know that the targetAddress is not tentaive
+ // At this point we know that the targetAddress is not tentative
// on rxNICID. However, targetAddr may still be assigned to
// rxNICID but not tentative (it could be permanent). Such a
// scenario is beyond the scope of RFC 4862. As such, we simply
// ignore such a scenario for now and proceed as normal.
//
- // TODO(b/140896005): Handle the scenario described above
- // (inform the netstack integration that a duplicate address was
- // was detected)
+ // TODO(b/143147598): Handle the scenario described above. Also
+ // inform the netstack integration that a duplicate address was
+ // detected outside of DAD.
e.linkAddrCache.AddLinkAddress(e.nicid, targetAddr, r.RemoteLinkAddress)
if targetAddr != r.RemoteAddress {
diff --git a/pkg/tcpip/stack/ndp.go b/pkg/tcpip/stack/ndp.go
index 921d1c9c7..ea2dbed2e 100644
--- a/pkg/tcpip/stack/ndp.go
+++ b/pkg/tcpip/stack/ndp.go
@@ -51,6 +51,22 @@ const (
minimumRetransmitTimer = time.Millisecond
)
+// NDPDispatcher is the interface integrators of netstack must implement to
+// receive and handle NDP related events.
+type NDPDispatcher interface {
+ // OnDuplicateAddressDetectionStatus will be called when the DAD process
+ // for an address (addr) on a NIC (with ID nicid) completes. resolved
+ // will be set to true if DAD completed successfully (no duplicate addr
+ // detected); false otherwise (addr was detected to be a duplicate on
+ // the link the NIC is a part of, or it was stopped for some other
+ // reason, such as the address being removed). If an error occured
+ // during DAD, err will be set and resolved must be ignored.
+ //
+ // This function is permitted to block indefinitely without interfering
+ // with the stack's operation.
+ OnDuplicateAddressDetectionStatus(nicid tcpip.NICID, addr tcpip.Address, resolved bool, err *tcpip.Error)
+}
+
// NDPConfigurations is the NDP configurations for the netstack.
type NDPConfigurations struct {
// The number of Neighbor Solicitation messages to send when doing
@@ -88,6 +104,9 @@ func (c *NDPConfigurations) validate() {
// ndpState is the per-interface NDP state.
type ndpState struct {
+ // The NIC this ndpState is for.
+ nic *NIC
+
// The DAD state to send the next NS message, or resolve the address.
dad map[tcpip.Address]dadState
}
@@ -110,8 +129,8 @@ type dadState struct {
// This function must only be called by IPv6 addresses that are currently
// tentative.
//
-// The NIC that ndp belongs to (n) MUST be locked.
-func (ndp *ndpState) startDuplicateAddressDetection(n *NIC, addr tcpip.Address, ref *referencedNetworkEndpoint) *tcpip.Error {
+// The NIC that ndp belongs to MUST be locked.
+func (ndp *ndpState) startDuplicateAddressDetection(addr tcpip.Address, ref *referencedNetworkEndpoint) *tcpip.Error {
// addr must be a valid unicast IPv6 address.
if !header.IsV6UnicastAddress(addr) {
return tcpip.ErrAddressFamilyNotSupported
@@ -127,13 +146,13 @@ func (ndp *ndpState) startDuplicateAddressDetection(n *NIC, addr tcpip.Address,
// reference count would have been increased without doing the
// work that would have been done for an address that was brand
// new. See NIC.addPermanentAddressLocked.
- panic(fmt.Sprintf("ndpdad: already performing DAD for addr %s on NIC(%d)", addr, n.ID()))
+ panic(fmt.Sprintf("ndpdad: already performing DAD for addr %s on NIC(%d)", addr, ndp.nic.ID()))
}
- remaining := n.stack.ndpConfigs.DupAddrDetectTransmits
+ remaining := ndp.nic.stack.ndpConfigs.DupAddrDetectTransmits
{
- done, err := ndp.doDuplicateAddressDetection(n, addr, remaining, ref)
+ done, err := ndp.doDuplicateAddressDetection(addr, remaining, ref)
if err != nil {
return err
}
@@ -146,41 +165,59 @@ func (ndp *ndpState) startDuplicateAddressDetection(n *NIC, addr tcpip.Address,
var done bool
var timer *time.Timer
- timer = time.AfterFunc(n.stack.ndpConfigs.RetransmitTimer, func() {
- n.mu.Lock()
- defer n.mu.Unlock()
+ timer = time.AfterFunc(ndp.nic.stack.ndpConfigs.RetransmitTimer, func() {
+ var d bool
+ var err *tcpip.Error
+
+ // doDadIteration does a single iteration of the DAD loop.
+ //
+ // Returns true if the integrator needs to be informed of DAD
+ // completing.
+ doDadIteration := func() bool {
+ ndp.nic.mu.Lock()
+ defer ndp.nic.mu.Unlock()
+
+ if done {
+ // If we reach this point, it means that the DAD
+ // timer fired after another goroutine already
+ // obtained the NIC lock and stopped DAD before
+ // this function obtained the NIC lock. Simply
+ // return here and do nothing further.
+ return false
+ }
- if done {
- // If we reach this point, it means that the DAD timer
- // fired after another goroutine already obtained the
- // NIC lock and stopped DAD before it this function
- // obtained the NIC lock. Simply return here and do
- // nothing further.
- return
- }
+ ref, ok := ndp.nic.endpoints[NetworkEndpointID{addr}]
+ if !ok {
+ // This should never happen.
+ // We should have an endpoint for addr since we
+ // are still performing DAD on it. If the
+ // endpoint does not exist, but we are doing DAD
+ // on it, then we started DAD at some point, but
+ // forgot to stop it when the endpoint was
+ // deleted.
+ panic(fmt.Sprintf("ndpdad: unrecognized addr %s for NIC(%d)", addr, ndp.nic.ID()))
+ }
- ref, ok := n.endpoints[NetworkEndpointID{addr}]
- if !ok {
- // This should never happen.
- // We should have an endpoint for addr since we are
- // still performing DAD on it. If the endpoint does not
- // exist, but we are doing DAD on it, then we started
- // DAD at some point, but forgot to stop it when the
- // endpoint was deleted.
- panic(fmt.Sprintf("ndpdad: unrecognized addr %s for NIC(%d)", addr, n.ID()))
- }
+ d, err = ndp.doDuplicateAddressDetection(addr, remaining, ref)
+ if err != nil || d {
+ delete(ndp.dad, addr)
- if done, err := ndp.doDuplicateAddressDetection(n, addr, remaining, ref); err != nil || done {
- if err != nil {
- log.Printf("ndpdad: Error occured during DAD iteration for addr (%s) on NIC(%d); err = %s", addr, n.ID(), err)
+ if err != nil {
+ log.Printf("ndpdad: Error occured during DAD iteration for addr (%s) on NIC(%d); err = %s", addr, ndp.nic.ID(), err)
+ }
+
+ // Let the integrator know DAD has completed.
+ return true
}
- ndp.stopDuplicateAddressDetection(addr)
- return
+ remaining--
+ timer.Reset(ndp.nic.stack.ndpConfigs.RetransmitTimer)
+ return false
}
- timer.Reset(n.stack.ndpConfigs.RetransmitTimer)
- remaining--
+ if doDadIteration() && ndp.nic.stack.ndpDisp != nil {
+ ndp.nic.stack.ndpDisp.OnDuplicateAddressDetectionStatus(ndp.nic.ID(), addr, d, err)
+ }
})
@@ -204,11 +241,11 @@ func (ndp *ndpState) startDuplicateAddressDetection(n *NIC, addr tcpip.Address,
// The NIC that ndp belongs to (n) MUST be locked.
//
// Returns true if DAD has resolved; false if DAD is still ongoing.
-func (ndp *ndpState) doDuplicateAddressDetection(n *NIC, addr tcpip.Address, remaining uint8, ref *referencedNetworkEndpoint) (bool, *tcpip.Error) {
+func (ndp *ndpState) doDuplicateAddressDetection(addr tcpip.Address, remaining uint8, ref *referencedNetworkEndpoint) (bool, *tcpip.Error) {
if ref.getKind() != permanentTentative {
// The endpoint should still be marked as tentative
// since we are still performing DAD on it.
- panic(fmt.Sprintf("ndpdad: addr %s is not tentative on NIC(%d)", addr, n.ID()))
+ panic(fmt.Sprintf("ndpdad: addr %s is not tentative on NIC(%d)", addr, ndp.nic.ID()))
}
if remaining == 0 {
@@ -219,17 +256,17 @@ func (ndp *ndpState) doDuplicateAddressDetection(n *NIC, addr tcpip.Address, rem
// Send a new NS.
snmc := header.SolicitedNodeAddr(addr)
- snmcRef, ok := n.endpoints[NetworkEndpointID{snmc}]
+ snmcRef, ok := ndp.nic.endpoints[NetworkEndpointID{snmc}]
if !ok {
// This should never happen as if we have the
// address, we should have the solicited-node
// address.
- panic(fmt.Sprintf("ndpdad: NIC(%d) is not in the solicited-node multicast group (%s) but it has addr %s", n.ID(), snmc, addr))
+ panic(fmt.Sprintf("ndpdad: NIC(%d) is not in the solicited-node multicast group (%s) but it has addr %s", ndp.nic.ID(), snmc, addr))
}
// Use the unspecified address as the source address when performing
// DAD.
- r := makeRoute(header.IPv6ProtocolNumber, header.IPv6Any, snmc, n.linkEP.LinkAddress(), snmcRef, false, false)
+ r := makeRoute(header.IPv6ProtocolNumber, header.IPv6Any, snmc, ndp.nic.linkEP.LinkAddress(), snmcRef, false, false)
hdr := buffer.NewPrependable(int(r.MaxHeaderLength()) + header.ICMPv6NeighborSolicitMinimumSize)
pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6NeighborSolicitMinimumSize))
@@ -275,5 +312,8 @@ func (ndp *ndpState) stopDuplicateAddressDetection(addr tcpip.Address) {
delete(ndp.dad, addr)
- return
+ // Let the integrator know DAD did not resolve.
+ if ndp.nic.stack.ndpDisp != nil {
+ go ndp.nic.stack.ndpDisp.OnDuplicateAddressDetectionStatus(ndp.nic.ID(), addr, false, nil)
+ }
}
diff --git a/pkg/tcpip/stack/ndp_test.go b/pkg/tcpip/stack/ndp_test.go
index 8995fbfc3..b089ce2ae 100644
--- a/pkg/tcpip/stack/ndp_test.go
+++ b/pkg/tcpip/stack/ndp_test.go
@@ -67,6 +67,35 @@ func TestDADDisabled(t *testing.T) {
}
}
+// ndpDADEvent is a set of parameters that was passed to
+// ndpDispatcher.OnDuplicateAddressDetectionStatus.
+type ndpDADEvent struct {
+ nicid tcpip.NICID
+ addr tcpip.Address
+ resolved bool
+ err *tcpip.Error
+}
+
+var _ stack.NDPDispatcher = (*ndpDispatcher)(nil)
+
+// ndpDispatcher implements NDPDispatcher so tests can know when various NDP
+// related events happen for test purposes.
+type ndpDispatcher struct {
+ dadC chan ndpDADEvent
+}
+
+// Implements stack.NDPDispatcher.OnDuplicateAddressDetectionStatus.
+//
+// If the DAD event matches what we are expecting, send signal on n.dadC.
+func (n *ndpDispatcher) OnDuplicateAddressDetectionStatus(nicid tcpip.NICID, addr tcpip.Address, resolved bool, err *tcpip.Error) {
+ n.dadC <- ndpDADEvent{
+ nicid,
+ addr,
+ resolved,
+ err,
+ }
+}
+
// TestDADResolve tests that an address successfully resolves after performing
// DAD for various values of DupAddrDetectTransmits and RetransmitTimer.
// Included in the subtests is a test to make sure that an invalid
@@ -88,8 +117,12 @@ func TestDADResolve(t *testing.T) {
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
+ ndpDisp := ndpDispatcher{
+ dadC: make(chan ndpDADEvent),
+ }
opts := stack.Options{
NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NDPDisp: &ndpDisp,
}
opts.NDPConfigs.RetransmitTimer = test.retransTimer
opts.NDPConfigs.DupAddrDetectTransmits = test.dupAddrDetectTransmits
@@ -106,8 +139,7 @@ func TestDADResolve(t *testing.T) {
stat := s.Stats().ICMP.V6PacketsSent.NeighborSolicit
- // Should have sent an NDP NS almost immediately.
- time.Sleep(100 * time.Millisecond)
+ // Should have sent an NDP NS immediately.
if got := stat.Value(); got != 1 {
t.Fatalf("got NeighborSolicit = %d, want = 1", got)
@@ -123,16 +155,10 @@ func TestDADResolve(t *testing.T) {
t.Fatalf("got stack.GetMainNICAddress(_, _) = (%s, nil), want = (%s, nil)", addr, want)
}
- // Wait for the remaining time - 500ms, to make sure
- // the address is still not resolved. Note, we subtract
- // 600ms because we already waited for 100ms earlier,
- // so our remaining time is 100ms less than the expected
- // time.
- // (X - 100ms) - 500ms = X - 600ms
- //
- // TODO(b/140896005): Use events from the netstack to
- // be signalled before DAD resolves.
- time.Sleep(test.expectedRetransmitTimer*time.Duration(test.dupAddrDetectTransmits) - 600*time.Millisecond)
+ // Wait for the remaining time - some delta (500ms), to
+ // make sure the address is still not resolved.
+ const delta = 500 * time.Millisecond
+ time.Sleep(test.expectedRetransmitTimer*time.Duration(test.dupAddrDetectTransmits) - delta)
addr, err = s.GetMainNICAddress(1, header.IPv6ProtocolNumber)
if err != nil {
t.Fatalf("got stack.GetMainNICAddress(_, _) = (_, %v), want = (_, nil)", err)
@@ -141,13 +167,30 @@ func TestDADResolve(t *testing.T) {
t.Fatalf("got stack.GetMainNICAddress(_, _) = (%s, nil), want = (%s, nil)", addr, want)
}
- // Wait for the remaining time + 250ms, at which point
- // the address should be resolved. Note, the remaining
- // time is 500ms. See above comments.
- //
- // TODO(b/140896005): Use events from the netstack to
- // know immediately when DAD completes.
- time.Sleep(750 * time.Millisecond)
+ // Wait for DAD to resolve.
+ select {
+ case <-time.After(2 * delta):
+ // We should get a resolution event after 500ms
+ // (delta) since we wait for 500ms less than the
+ // expected resolution time above to make sure
+ // that the address did not yet resolve. Waiting
+ // for 1s (2x delta) without a resolution event
+ // means something is wrong.
+ t.Fatal("timed out waiting for DAD resolution")
+ case e := <-ndpDisp.dadC:
+ if e.err != nil {
+ t.Fatal("got DAD error: ", e.err)
+ }
+ if e.nicid != 1 {
+ t.Fatalf("got DAD event w/ nicid = %d, want = 1", e.nicid)
+ }
+ if e.addr != addr1 {
+ t.Fatalf("got DAD event w/ addr = %s, want = %s", addr, addr1)
+ }
+ if !e.resolved {
+ t.Fatal("got DAD event w/ resolved = false, want = true")
+ }
+ }
addr, err = s.GetMainNICAddress(1, header.IPv6ProtocolNumber)
if err != nil {
t.Fatalf("stack.GetMainNICAddress(_, _) err = %s", err)
@@ -250,9 +293,14 @@ func TestDADFail(t *testing.T) {
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
+ ndpDisp := ndpDispatcher{
+ dadC: make(chan ndpDADEvent),
+ }
+ ndpConfigs := stack.DefaultNDPConfigurations()
opts := stack.Options{
NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
- NDPConfigs: stack.DefaultNDPConfigurations(),
+ NDPConfigs: ndpConfigs,
+ NDPDisp: &ndpDisp,
}
opts.NDPConfigs.RetransmitTimer = time.Second * 2
@@ -286,8 +334,28 @@ func TestDADFail(t *testing.T) {
t.Fatalf("got stat = %d, want = 1", got)
}
- // Wait 3 seconds to make sure that DAD did not resolve
- time.Sleep(3 * time.Second)
+ // Wait for DAD to fail and make sure the address did
+ // not get resolved.
+ select {
+ case <-time.After(time.Duration(ndpConfigs.DupAddrDetectTransmits)*ndpConfigs.RetransmitTimer + time.Second):
+ // If we don't get a failure event after the
+ // expected resolution time + extra 1s buffer,
+ // something is wrong.
+ t.Fatal("timed out waiting for DAD failure")
+ case e := <-ndpDisp.dadC:
+ if e.err != nil {
+ t.Fatal("got DAD error: ", e.err)
+ }
+ if e.nicid != 1 {
+ t.Fatalf("got DAD event w/ nicid = %d, want = 1", e.nicid)
+ }
+ if e.addr != addr1 {
+ t.Fatalf("got DAD event w/ addr = %s, want = %s", addr, addr1)
+ }
+ if e.resolved {
+ t.Fatal("got DAD event w/ resolved = true, want = false")
+ }
+ }
addr, err = s.GetMainNICAddress(1, header.IPv6ProtocolNumber)
if err != nil {
t.Fatalf("got stack.GetMainNICAddress(_, _) = (_, %v), want = (_, nil)", err)
@@ -302,11 +370,18 @@ func TestDADFail(t *testing.T) {
// TestDADStop tests to make sure that the DAD process stops when an address is
// removed.
func TestDADStop(t *testing.T) {
+ ndpDisp := ndpDispatcher{
+ dadC: make(chan ndpDADEvent),
+ }
+ ndpConfigs := stack.NDPConfigurations{
+ RetransmitTimer: time.Second,
+ DupAddrDetectTransmits: 2,
+ }
opts := stack.Options{
NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NDPDisp: &ndpDisp,
+ NDPConfigs: ndpConfigs,
}
- opts.NDPConfigs.RetransmitTimer = time.Second
- opts.NDPConfigs.DupAddrDetectTransmits = 2
e := channel.New(10, 1280, linkAddr1)
s := stack.New(opts)
@@ -332,11 +407,27 @@ func TestDADStop(t *testing.T) {
t.Fatalf("RemoveAddress(_, %s) = %s", addr1, err)
}
- // Wait for the time to normally resolve
- // DupAddrDetectTransmits(2) * RetransmitTimer(1s) = 2s.
- // An extra 250ms is added to make sure that if DAD was still running
- // it resolves and the check below fails.
- time.Sleep(2*time.Second + 250*time.Millisecond)
+ // Wait for DAD to fail (since the address was removed during DAD).
+ select {
+ case <-time.After(time.Duration(ndpConfigs.DupAddrDetectTransmits)*ndpConfigs.RetransmitTimer + time.Second):
+ // If we don't get a failure event after the expected resolution
+ // time + extra 1s buffer, something is wrong.
+ t.Fatal("timed out waiting for DAD failure")
+ case e := <-ndpDisp.dadC:
+ if e.err != nil {
+ t.Fatal("got DAD error: ", e.err)
+ }
+ if e.nicid != 1 {
+ t.Fatalf("got DAD event w/ nicid = %d, want = 1", e.nicid)
+ }
+ if e.addr != addr1 {
+ t.Fatalf("got DAD event w/ addr = %s, want = %s", addr, addr1)
+ }
+ if e.resolved {
+ t.Fatal("got DAD event w/ resolved = true, want = false")
+ }
+
+ }
addr, err = s.GetMainNICAddress(1, header.IPv6ProtocolNumber)
if err != nil {
t.Fatalf("got stack.GetMainNICAddress(_, _) = (_, %v), want = (_, nil)", err)
diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go
index e456e05f4..2d29fa88e 100644
--- a/pkg/tcpip/stack/nic.go
+++ b/pkg/tcpip/stack/nic.go
@@ -108,6 +108,7 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, loopback
dad: make(map[tcpip.Address]dadState),
},
}
+ nic.ndp.nic = nic
// Register supported packet endpoint protocols.
for _, netProto := range header.Ethertypes {
@@ -432,7 +433,7 @@ func (n *NIC) addAddressLocked(protocolAddress tcpip.ProtocolAddress, peb Primar
// If we are adding a tentative IPv6 address, start DAD.
if isIPv6Unicast && kind == permanentTentative {
- if err := n.ndp.startDuplicateAddressDetection(n, protocolAddress.AddressWithPrefix.Address, ref); err != nil {
+ if err := n.ndp.startDuplicateAddressDetection(protocolAddress.AddressWithPrefix.Address, ref); err != nil {
return nil, err
}
}
diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go
index 284280917..5ea432a24 100644
--- a/pkg/tcpip/stack/stack.go
+++ b/pkg/tcpip/stack/stack.go
@@ -406,6 +406,10 @@ type Stack struct {
// to auto-generate an IPv6 link-local address for newly enabled NICs.
// See the AutoGenIPv6LinkLocal field of Options for more details.
autoGenIPv6LinkLocal bool
+
+ // ndpDisp is the NDP event dispatcher that is used to send the netstack
+ // integrator NDP related events.
+ ndpDisp NDPDispatcher
}
// Options contains optional Stack configuration.
@@ -448,6 +452,10 @@ type Options struct {
// guidelines.
AutoGenIPv6LinkLocal bool
+ // NDPDisp is the NDP event dispatcher that an integrator can provide to
+ // receive NDP related events.
+ NDPDisp NDPDispatcher
+
// RawFactory produces raw endpoints. Raw endpoints are enabled only if
// this is non-nil.
RawFactory RawFactory
@@ -514,6 +522,7 @@ func New(opts Options) *Stack {
portSeed: generateRandUint32(),
ndpConfigs: opts.NDPConfigs,
autoGenIPv6LinkLocal: opts.AutoGenIPv6LinkLocal,
+ ndpDisp: opts.NDPDisp,
}
// Add specified network protocols.
diff --git a/pkg/tcpip/stack/stack_test.go b/pkg/tcpip/stack/stack_test.go
index 9a8906a0d..9dae853d0 100644
--- a/pkg/tcpip/stack/stack_test.go
+++ b/pkg/tcpip/stack/stack_test.go
@@ -1971,13 +1971,15 @@ func TestNICAutoGenAddr(t *testing.T) {
// TestNICAutoGenAddrDoesDAD tests that the successful auto-generation of IPv6
// link-local addresses will only be assigned after the DAD process resolves.
func TestNICAutoGenAddrDoesDAD(t *testing.T) {
+ ndpDisp := ndpDispatcher{
+ dadC: make(chan ndpDADEvent),
+ }
+ ndpConfigs := stack.DefaultNDPConfigurations()
opts := stack.Options{
- NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
- NDPConfigs: stack.NDPConfigurations{
- RetransmitTimer: time.Second,
- DupAddrDetectTransmits: 1,
- },
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NDPConfigs: ndpConfigs,
AutoGenIPv6LinkLocal: true,
+ NDPDisp: &ndpDisp,
}
e := channel.New(10, 1280, linkAddr1)
@@ -1996,21 +1998,35 @@ func TestNICAutoGenAddrDoesDAD(t *testing.T) {
t.Fatalf("got stack.GetMainNICAddress(_, _) = (%s, nil), want = (%s, nil)", addr, want)
}
- // Wait for the address to resolve (an extra
- // 250ms to make sure the address resolves).
- //
- // TODO(b/140896005): Use events from the
- // netstack to know immediately when DAD
- // completes.
- time.Sleep(time.Second + 250*time.Millisecond)
+ linkLocalAddr := header.LinkLocalAddr(linkAddr1)
- // Should have auto-generated an address and
- // resolved (if DAD).
+ // Wait for DAD to resolve.
+ select {
+ case <-time.After(time.Duration(ndpConfigs.DupAddrDetectTransmits)*ndpConfigs.RetransmitTimer + time.Second):
+ // We should get a resolution event after 1s (default time to
+ // resolve as per default NDP configurations). Waiting for that
+ // resolution time + an extra 1s without a resolution event
+ // means something is wrong.
+ t.Fatal("timed out waiting for DAD resolution")
+ case e := <-ndpDisp.dadC:
+ if e.err != nil {
+ t.Fatal("got DAD error: ", e.err)
+ }
+ if e.nicid != 1 {
+ t.Fatalf("got DAD event w/ nicid = %d, want = 1", e.nicid)
+ }
+ if e.addr != linkLocalAddr {
+ t.Fatalf("got DAD event w/ addr = %s, want = %s", addr, linkLocalAddr)
+ }
+ if !e.resolved {
+ t.Fatal("got DAD event w/ resolved = false, want = true")
+ }
+ }
addr, err = s.GetMainNICAddress(1, header.IPv6ProtocolNumber)
if err != nil {
t.Fatalf("stack.GetMainNICAddress(_, _) err = %s", err)
}
- if want := (tcpip.AddressWithPrefix{Address: header.LinkLocalAddr(linkAddr1), PrefixLen: header.IPv6LinkLocalPrefix.PrefixLen}); addr != want {
+ if want := (tcpip.AddressWithPrefix{Address: linkLocalAddr, PrefixLen: header.IPv6LinkLocalPrefix.PrefixLen}); addr != want {
t.Fatalf("got stack.GetMainNICAddress(_, _) = %s, want = %s", addr, want)
}
}