summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/network
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/tcpip/network')
-rw-r--r--pkg/tcpip/network/arp/arp.go13
-rw-r--r--pkg/tcpip/network/internal/ip/duplicate_address_detection.go139
-rw-r--r--pkg/tcpip/network/internal/ip/duplicate_address_detection_test.go124
-rw-r--r--pkg/tcpip/network/ipv6/icmp.go55
-rw-r--r--pkg/tcpip/network/ipv6/ipv6.go89
-rw-r--r--pkg/tcpip/network/ipv6/mld_test.go18
-rw-r--r--pkg/tcpip/network/ipv6/ndp.go14
-rw-r--r--pkg/tcpip/network/ipv6/ndp_test.go72
8 files changed, 424 insertions, 100 deletions
diff --git a/pkg/tcpip/network/arp/arp.go b/pkg/tcpip/network/arp/arp.go
index 43a4b7cac..7ae38d684 100644
--- a/pkg/tcpip/network/arp/arp.go
+++ b/pkg/tcpip/network/arp/arp.go
@@ -38,6 +38,7 @@ const (
var _ stack.DuplicateAddressDetector = (*endpoint)(nil)
var _ stack.LinkAddressResolver = (*endpoint)(nil)
+var _ ip.DADProtocol = (*endpoint)(nil)
// ARP endpoints need to implement stack.NetworkEndpoint because the stack
// considers the layer above the link-layer a network layer; the only
@@ -82,7 +83,8 @@ func (*endpoint) DuplicateAddressProtocol() tcpip.NetworkProtocolNumber {
return header.IPv4ProtocolNumber
}
-func (e *endpoint) SendDADMessage(addr tcpip.Address) tcpip.Error {
+// SendDADMessage implements ip.DADProtocol.
+func (e *endpoint) SendDADMessage(addr tcpip.Address, _ []byte) tcpip.Error {
return e.sendARPRequest(header.IPv4Any, addr, header.EthernetBroadcastAddress)
}
@@ -284,9 +286,12 @@ func (p *protocol) NewEndpoint(nic stack.NetworkInterface, dispatcher stack.Tran
e.mu.Lock()
e.mu.dad.Init(&e.mu, p.options.DADConfigs, ip.DADOptions{
- Clock: p.stack.Clock(),
- Protocol: e,
- NICID: nic.ID(),
+ Clock: p.stack.Clock(),
+ SecureRNG: p.stack.SecureRNG(),
+ // ARP does not support sending nonce values.
+ NonceSize: 0,
+ Protocol: e,
+ NICID: nic.ID(),
})
e.mu.Unlock()
diff --git a/pkg/tcpip/network/internal/ip/duplicate_address_detection.go b/pkg/tcpip/network/internal/ip/duplicate_address_detection.go
index 0053646ee..eed49f5d2 100644
--- a/pkg/tcpip/network/internal/ip/duplicate_address_detection.go
+++ b/pkg/tcpip/network/internal/ip/duplicate_address_detection.go
@@ -16,14 +16,27 @@
package ip
import (
+ "bytes"
"fmt"
+ "io"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/tcpip"
"gvisor.dev/gvisor/pkg/tcpip/stack"
)
+type extendRequest int
+
+const (
+ notRequested extendRequest = iota
+ requested
+ extended
+)
+
type dadState struct {
+ nonce []byte
+ extendRequest extendRequest
+
done *bool
timer tcpip.Timer
@@ -33,14 +46,17 @@ type dadState struct {
// DADProtocol is a protocol whose core state machine can be represented by DAD.
type DADProtocol interface {
// SendDADMessage attempts to send a DAD probe message.
- SendDADMessage(tcpip.Address) tcpip.Error
+ SendDADMessage(tcpip.Address, []byte) tcpip.Error
}
// DADOptions holds options for DAD.
type DADOptions struct {
- Clock tcpip.Clock
- Protocol DADProtocol
- NICID tcpip.NICID
+ Clock tcpip.Clock
+ SecureRNG io.Reader
+ NonceSize uint8
+ ExtendDADTransmits uint8
+ Protocol DADProtocol
+ NICID tcpip.NICID
}
// DAD performs duplicate address detection for addresses.
@@ -63,6 +79,10 @@ func (d *DAD) Init(protocolMU sync.Locker, configs stack.DADConfigurations, opts
panic("attempted to initialize DAD state twice")
}
+ if opts.NonceSize != 0 && opts.ExtendDADTransmits == 0 {
+ panic(fmt.Sprintf("given a non-zero value for NonceSize (%d) but zero for ExtendDADTransmits", opts.NonceSize))
+ }
+
*d = DAD{
opts: opts,
configs: configs,
@@ -96,10 +116,55 @@ func (d *DAD) CheckDuplicateAddressLocked(addr tcpip.Address, h stack.DADComplet
s = dadState{
done: &done,
timer: d.opts.Clock.AfterFunc(0, func() {
- var err tcpip.Error
dadDone := remaining == 0
+
+ nonce, earlyReturn := func() ([]byte, bool) {
+ d.protocolMU.Lock()
+ defer d.protocolMU.Unlock()
+
+ if done {
+ return nil, true
+ }
+
+ s, ok := d.addresses[addr]
+ if !ok {
+ panic(fmt.Sprintf("dad: timer fired but missing state for %s on NIC(%d)", addr, d.opts.NICID))
+ }
+
+ // As per RFC 7527 section 4
+ //
+ // If any probe is looped back within RetransTimer milliseconds
+ // after having sent DupAddrDetectTransmits NS(DAD) messages, the
+ // interface continues with another MAX_MULTICAST_SOLICIT number of
+ // NS(DAD) messages transmitted RetransTimer milliseconds apart.
+ if dadDone && s.extendRequest == requested {
+ dadDone = false
+ remaining = d.opts.ExtendDADTransmits
+ s.extendRequest = extended
+ }
+
+ if !dadDone && d.opts.NonceSize != 0 {
+ if s.nonce == nil {
+ s.nonce = make([]byte, d.opts.NonceSize)
+ }
+
+ if n, err := io.ReadFull(d.opts.SecureRNG, s.nonce); err != nil {
+ panic(fmt.Sprintf("SecureRNG.Read(...): %s", err))
+ } else if n != len(s.nonce) {
+ panic(fmt.Sprintf("expected to read %d bytes from secure RNG, only read %d bytes", len(s.nonce), n))
+ }
+ }
+
+ d.addresses[addr] = s
+ return s.nonce, false
+ }()
+ if earlyReturn {
+ return
+ }
+
+ var err tcpip.Error
if !dadDone {
- err = d.opts.Protocol.SendDADMessage(addr)
+ err = d.opts.Protocol.SendDADMessage(addr, nonce)
}
d.protocolMU.Lock()
@@ -142,6 +207,68 @@ func (d *DAD) CheckDuplicateAddressLocked(addr tcpip.Address, h stack.DADComplet
return ret
}
+// ExtendIfNonceEqualLockedDisposition enumerates the possible results from
+// ExtendIfNonceEqualLocked.
+type ExtendIfNonceEqualLockedDisposition int
+
+const (
+ // Extended indicates that the DAD process was extended.
+ Extended ExtendIfNonceEqualLockedDisposition = iota
+
+ // AlreadyExtended indicates that the DAD process was already extended.
+ AlreadyExtended
+
+ // NoDADStateFound indicates that DAD state was not found for the address.
+ NoDADStateFound
+
+ // NonceDisabled indicates that nonce values are not sent with DAD messages.
+ NonceDisabled
+
+ // NonceNotEqual indicates that the nonce value passed and the nonce in the
+ // last send DAD message are not equal.
+ NonceNotEqual
+)
+
+// ExtendIfNonceEqualLocked extends the DAD process if the provided nonce is the
+// same as the nonce sent in the last DAD message.
+//
+// Precondition: d.protocolMU must be locked.
+func (d *DAD) ExtendIfNonceEqualLocked(addr tcpip.Address, nonce []byte) ExtendIfNonceEqualLockedDisposition {
+ s, ok := d.addresses[addr]
+ if !ok {
+ return NoDADStateFound
+ }
+
+ if d.opts.NonceSize == 0 {
+ return NonceDisabled
+ }
+
+ if s.extendRequest != notRequested {
+ return AlreadyExtended
+ }
+
+ // As per RFC 7527 section 4
+ //
+ // If any probe is looped back within RetransTimer milliseconds after having
+ // sent DupAddrDetectTransmits NS(DAD) messages, the interface continues
+ // with another MAX_MULTICAST_SOLICIT number of NS(DAD) messages transmitted
+ // RetransTimer milliseconds apart.
+ //
+ // If a DAD message has already been sent and the nonce value we observed is
+ // the same as the nonce value we last sent, then we assume our probe was
+ // looped back and request an extension to the DAD process.
+ //
+ // Note, the first DAD message is sent asynchronously so we need to make sure
+ // that we sent a DAD message by checking if we have a nonce value set.
+ if s.nonce != nil && bytes.Equal(s.nonce, nonce) {
+ s.extendRequest = requested
+ d.addresses[addr] = s
+ return Extended
+ }
+
+ return NonceNotEqual
+}
+
// StopLocked stops a currently running DAD process.
//
// Precondition: d.protocolMU must be locked.
diff --git a/pkg/tcpip/network/internal/ip/duplicate_address_detection_test.go b/pkg/tcpip/network/internal/ip/duplicate_address_detection_test.go
index e00aa4678..a22b712c6 100644
--- a/pkg/tcpip/network/internal/ip/duplicate_address_detection_test.go
+++ b/pkg/tcpip/network/internal/ip/duplicate_address_detection_test.go
@@ -15,6 +15,7 @@
package ip_test
import (
+ "bytes"
"testing"
"time"
@@ -32,8 +33,8 @@ type mockDADProtocol struct {
mu struct {
sync.Mutex
- dad ip.DAD
- sendCount map[tcpip.Address]int
+ dad ip.DAD
+ sentNonces map[tcpip.Address][][]byte
}
}
@@ -48,26 +49,30 @@ func (m *mockDADProtocol) init(t *testing.T, c stack.DADConfigurations, opts ip.
}
func (m *mockDADProtocol) initLocked() {
- m.mu.sendCount = make(map[tcpip.Address]int)
+ m.mu.sentNonces = make(map[tcpip.Address][][]byte)
}
-func (m *mockDADProtocol) SendDADMessage(addr tcpip.Address) tcpip.Error {
+func (m *mockDADProtocol) SendDADMessage(addr tcpip.Address, nonce []byte) tcpip.Error {
m.mu.Lock()
defer m.mu.Unlock()
- m.mu.sendCount[addr]++
+ m.mu.sentNonces[addr] = append(m.mu.sentNonces[addr], nonce)
return nil
}
func (m *mockDADProtocol) check(addrs []tcpip.Address) string {
- m.mu.Lock()
- defer m.mu.Unlock()
-
- sendCount := make(map[tcpip.Address]int)
+ sentNonces := make(map[tcpip.Address][][]byte)
for _, a := range addrs {
- sendCount[a]++
+ sentNonces[a] = append(sentNonces[a], nil)
}
- diff := cmp.Diff(sendCount, m.mu.sendCount)
+ return m.checkWithNonce(sentNonces)
+}
+
+func (m *mockDADProtocol) checkWithNonce(expectedSentNonces map[tcpip.Address][][]byte) string {
+ m.mu.Lock()
+ defer m.mu.Unlock()
+
+ diff := cmp.Diff(expectedSentNonces, m.mu.sentNonces)
m.initLocked()
return diff
}
@@ -84,6 +89,12 @@ func (m *mockDADProtocol) stop(addr tcpip.Address, reason stack.DADResult) {
m.mu.dad.StopLocked(addr, reason)
}
+func (m *mockDADProtocol) extendIfNonceEqual(addr tcpip.Address, nonce []byte) ip.ExtendIfNonceEqualLockedDisposition {
+ m.mu.Lock()
+ defer m.mu.Unlock()
+ return m.mu.dad.ExtendIfNonceEqualLocked(addr, nonce)
+}
+
func (m *mockDADProtocol) setConfigs(c stack.DADConfigurations) {
m.mu.Lock()
defer m.mu.Unlock()
@@ -277,3 +288,94 @@ func TestDADStop(t *testing.T) {
default:
}
}
+
+func TestNonce(t *testing.T) {
+ const (
+ nonceSize = 2
+
+ extendRequestAttempts = 2
+
+ dupAddrDetectTransmits = 2
+ extendTransmits = 5
+ )
+
+ var secureRNGBytes [nonceSize * (dupAddrDetectTransmits + extendTransmits)]byte
+ for i := range secureRNGBytes {
+ secureRNGBytes[i] = byte(i)
+ }
+
+ tests := []struct {
+ name string
+ mockedReceivedNonce []byte
+ expectedResults [extendRequestAttempts]ip.ExtendIfNonceEqualLockedDisposition
+ expectedTransmits int
+ }{
+ {
+ name: "not matching",
+ mockedReceivedNonce: []byte{0, 0},
+ expectedResults: [extendRequestAttempts]ip.ExtendIfNonceEqualLockedDisposition{ip.NonceNotEqual, ip.NonceNotEqual},
+ expectedTransmits: dupAddrDetectTransmits,
+ },
+ {
+ name: "matching nonce",
+ mockedReceivedNonce: secureRNGBytes[:nonceSize],
+ expectedResults: [extendRequestAttempts]ip.ExtendIfNonceEqualLockedDisposition{ip.Extended, ip.AlreadyExtended},
+ expectedTransmits: dupAddrDetectTransmits + extendTransmits,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ var dad mockDADProtocol
+ clock := faketime.NewManualClock()
+ dadConfigs := stack.DADConfigurations{
+ DupAddrDetectTransmits: dupAddrDetectTransmits,
+ RetransmitTimer: time.Second,
+ }
+
+ var secureRNG bytes.Reader
+ secureRNG.Reset(secureRNGBytes[:])
+ dad.init(t, dadConfigs, ip.DADOptions{
+ Clock: clock,
+ SecureRNG: &secureRNG,
+ NonceSize: nonceSize,
+ ExtendDADTransmits: extendTransmits,
+ })
+
+ ch := make(chan dadResult, 1)
+ if res := dad.checkDuplicateAddress(addr1, handler(ch, addr1)); res != stack.DADStarting {
+ t.Errorf("got dad.checkDuplicateAddress(%s, _) = %d, want = %d", addr1, res, stack.DADStarting)
+ }
+
+ clock.Advance(0)
+ for i, want := range test.expectedResults {
+ if got := dad.extendIfNonceEqual(addr1, test.mockedReceivedNonce); got != want {
+ t.Errorf("(i=%d) got dad.extendIfNonceEqual(%s, _) = %d, want = %d", i, addr1, got, want)
+ }
+ }
+
+ for i := 0; i < test.expectedTransmits; i++ {
+ if diff := dad.checkWithNonce(map[tcpip.Address][][]byte{
+ addr1: {
+ secureRNGBytes[nonceSize*i:][:nonceSize],
+ },
+ }); diff != "" {
+ t.Errorf("(i=%d) dad check mismatch (-want +got):\n%s", i, diff)
+ }
+
+ clock.Advance(dadConfigs.RetransmitTimer)
+ }
+
+ if diff := cmp.Diff(dadResult{Addr: addr1, R: &stack.DADSucceeded{}}, <-ch); diff != "" {
+ t.Errorf("dad result mismatch (-want +got):\n%s", diff)
+ }
+
+ // Should not have anymore updates.
+ select {
+ case r := <-ch:
+ t.Fatalf("unexpectedly got an extra DAD result; r = %#v", r)
+ default:
+ }
+ })
+ }
+}
diff --git a/pkg/tcpip/network/ipv6/icmp.go b/pkg/tcpip/network/ipv6/icmp.go
index 8059e0690..2afa856dc 100644
--- a/pkg/tcpip/network/ipv6/icmp.go
+++ b/pkg/tcpip/network/ipv6/icmp.go
@@ -369,6 +369,18 @@ func (e *endpoint) handleICMP(pkt *stack.PacketBuffer, hasFragmentHeader bool, r
return
}
+ var it header.NDPOptionIterator
+ {
+ var err error
+ it, err = ns.Options().Iter(false /* check */)
+ if err != nil {
+ // Options are not valid as per the wire format, silently drop the
+ // packet.
+ received.invalid.Increment()
+ return
+ }
+ }
+
if e.hasTentativeAddr(targetAddr) {
// If the target address is tentative and the source of the packet is a
// unicast (specified) address, then the source of the packet is
@@ -382,6 +394,22 @@ func (e *endpoint) handleICMP(pkt *stack.PacketBuffer, hasFragmentHeader bool, r
// stack know so it can handle such a scenario and do nothing further with
// the NS.
if srcAddr == header.IPv6Any {
+ var nonce []byte
+ for {
+ opt, done, err := it.Next()
+ if err != nil {
+ received.invalid.Increment()
+ return
+ }
+ if done {
+ break
+ }
+ if n, ok := opt.(header.NDPNonceOption); ok {
+ nonce = n.Nonce()
+ break
+ }
+ }
+
// Since this is a DAD message we know the sender does not actually hold
// the target address so there is no "holder".
var holderLinkAddress tcpip.LinkAddress
@@ -397,7 +425,7 @@ func (e *endpoint) handleICMP(pkt *stack.PacketBuffer, hasFragmentHeader bool, r
//
// TODO(gvisor.dev/issue/4046): Handle the scenario when a duplicate
// address is detected for an assigned address.
- switch err := e.dupTentativeAddrDetected(targetAddr, holderLinkAddress); err.(type) {
+ switch err := e.dupTentativeAddrDetected(targetAddr, holderLinkAddress, nonce); err.(type) {
case nil, *tcpip.ErrBadAddress, *tcpip.ErrInvalidEndpointState:
default:
panic(fmt.Sprintf("unexpected error handling duplicate tentative address: %s", err))
@@ -418,21 +446,10 @@ func (e *endpoint) handleICMP(pkt *stack.PacketBuffer, hasFragmentHeader bool, r
return
}
- var sourceLinkAddr tcpip.LinkAddress
- {
- it, err := ns.Options().Iter(false /* check */)
- if err != nil {
- // Options are not valid as per the wire format, silently drop the
- // packet.
- received.invalid.Increment()
- return
- }
-
- sourceLinkAddr, ok = getSourceLinkAddr(it)
- if !ok {
- received.invalid.Increment()
- return
- }
+ sourceLinkAddr, ok := getSourceLinkAddr(it)
+ if !ok {
+ received.invalid.Increment()
+ return
}
// As per RFC 4861 section 4.3, the Source Link-Layer Address Option MUST
@@ -586,6 +603,10 @@ func (e *endpoint) handleICMP(pkt *stack.PacketBuffer, hasFragmentHeader bool, r
e.dad.mu.Unlock()
if e.hasTentativeAddr(targetAddr) {
+ // We only send a nonce value in DAD messages to check for loopedback
+ // messages so we use the empty nonce value here.
+ var nonce []byte
+
// We just got an NA from a node that owns an address we are performing
// DAD on, implying the address is not unique. In this case we let the
// stack know so it can handle such a scenario and do nothing furthur with
@@ -602,7 +623,7 @@ func (e *endpoint) handleICMP(pkt *stack.PacketBuffer, hasFragmentHeader bool, r
//
// TODO(gvisor.dev/issue/4046): Handle the scenario when a duplicate
// address is detected for an assigned address.
- switch err := e.dupTentativeAddrDetected(targetAddr, targetLinkAddr); err.(type) {
+ switch err := e.dupTentativeAddrDetected(targetAddr, targetLinkAddr, nonce); err.(type) {
case nil, *tcpip.ErrBadAddress, *tcpip.ErrInvalidEndpointState:
return
default:
diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go
index 46b6cc41a..350493958 100644
--- a/pkg/tcpip/network/ipv6/ipv6.go
+++ b/pkg/tcpip/network/ipv6/ipv6.go
@@ -348,7 +348,7 @@ func (e *endpoint) hasTentativeAddr(addr tcpip.Address) bool {
// dupTentativeAddrDetected removes the tentative address if it exists. If the
// address was generated via SLAAC, an attempt is made to generate a new
// address.
-func (e *endpoint) dupTentativeAddrDetected(addr tcpip.Address, holderLinkAddr tcpip.LinkAddress) tcpip.Error {
+func (e *endpoint) dupTentativeAddrDetected(addr tcpip.Address, holderLinkAddr tcpip.LinkAddress, nonce []byte) tcpip.Error {
e.mu.Lock()
defer e.mu.Unlock()
@@ -361,27 +361,48 @@ func (e *endpoint) dupTentativeAddrDetected(addr tcpip.Address, holderLinkAddr t
return &tcpip.ErrInvalidEndpointState{}
}
- // If the address is a SLAAC address, do not invalidate its SLAAC prefix as an
- // attempt will be made to generate a new address for it.
- if err := e.removePermanentEndpointLocked(addressEndpoint, false /* allowSLAACInvalidation */, &stack.DADDupAddrDetected{HolderLinkAddress: holderLinkAddr}); err != nil {
- return err
- }
+ switch result := e.mu.ndp.dad.ExtendIfNonceEqualLocked(addr, nonce); result {
+ case ip.Extended:
+ // The nonce we got back was the same we sent so we know the message
+ // indicating a duplicate address was likely ours so do not consider
+ // the address duplicate here.
+ return nil
+ case ip.AlreadyExtended:
+ // See Extended.
+ //
+ // Our DAD message was looped back already.
+ return nil
+ case ip.NoDADStateFound:
+ panic(fmt.Sprintf("expected DAD state for tentative address %s", addr))
+ case ip.NonceDisabled:
+ // If nonce is disabled then we have no way to know if the packet was
+ // looped-back so we have to assume it indicates a duplicate address.
+ fallthrough
+ case ip.NonceNotEqual:
+ // If the address is a SLAAC address, do not invalidate its SLAAC prefix as an
+ // attempt will be made to generate a new address for it.
+ if err := e.removePermanentEndpointLocked(addressEndpoint, false /* allowSLAACInvalidation */, &stack.DADDupAddrDetected{HolderLinkAddress: holderLinkAddr}); err != nil {
+ return err
+ }
- prefix := addressEndpoint.Subnet()
+ prefix := addressEndpoint.Subnet()
- switch t := addressEndpoint.ConfigType(); t {
- case stack.AddressConfigStatic:
- case stack.AddressConfigSlaac:
- e.mu.ndp.regenerateSLAACAddr(prefix)
- case stack.AddressConfigSlaacTemp:
- // Do not reset the generation attempts counter for the prefix as the
- // temporary address is being regenerated in response to a DAD conflict.
- e.mu.ndp.regenerateTempSLAACAddr(prefix, false /* resetGenAttempts */)
+ switch t := addressEndpoint.ConfigType(); t {
+ case stack.AddressConfigStatic:
+ case stack.AddressConfigSlaac:
+ e.mu.ndp.regenerateSLAACAddr(prefix)
+ case stack.AddressConfigSlaacTemp:
+ // Do not reset the generation attempts counter for the prefix as the
+ // temporary address is being regenerated in response to a DAD conflict.
+ e.mu.ndp.regenerateTempSLAACAddr(prefix, false /* resetGenAttempts */)
+ default:
+ panic(fmt.Sprintf("unrecognized address config type = %d", t))
+ }
+
+ return nil
default:
- panic(fmt.Sprintf("unrecognized address config type = %d", t))
+ panic(fmt.Sprintf("unhandled result = %d", result))
}
-
- return nil
}
// transitionForwarding transitions the endpoint's forwarding status to
@@ -1797,16 +1818,36 @@ func (p *protocol) NewEndpoint(nic stack.NetworkInterface, dispatcher stack.Tran
dispatcher: dispatcher,
protocol: p,
}
+
+ // NDP options must be 8 octet aligned and the first 2 bytes are used for
+ // the type and length fields leaving 6 octets as the minimum size for a
+ // nonce option without padding.
+ const nonceSize = 6
+
+ // As per RFC 7527 section 4.1,
+ //
+ // If any probe is looped back within RetransTimer milliseconds after
+ // having sent DupAddrDetectTransmits NS(DAD) messages, the interface
+ // continues with another MAX_MULTICAST_SOLICIT number of NS(DAD)
+ // messages transmitted RetransTimer milliseconds apart.
+ //
+ // Value taken from RFC 4861 section 10.
+ const maxMulticastSolicit = 3
+ dadOptions := ip.DADOptions{
+ Clock: p.stack.Clock(),
+ SecureRNG: p.stack.SecureRNG(),
+ NonceSize: nonceSize,
+ ExtendDADTransmits: maxMulticastSolicit,
+ Protocol: &e.mu.ndp,
+ NICID: nic.ID(),
+ }
+
e.mu.Lock()
e.mu.addressableEndpointState.Init(e)
- e.mu.ndp.init(e)
+ e.mu.ndp.init(e, dadOptions)
e.mu.mld.init(e)
e.dad.mu.Lock()
- e.dad.mu.dad.Init(&e.dad.mu, p.options.DADConfigs, ip.DADOptions{
- Clock: p.stack.Clock(),
- Protocol: &e.mu.ndp,
- NICID: nic.ID(),
- })
+ e.dad.mu.dad.Init(&e.dad.mu, p.options.DADConfigs, dadOptions)
e.dad.mu.Unlock()
e.mu.Unlock()
diff --git a/pkg/tcpip/network/ipv6/mld_test.go b/pkg/tcpip/network/ipv6/mld_test.go
index 9a425e50a..85a8f9944 100644
--- a/pkg/tcpip/network/ipv6/mld_test.go
+++ b/pkg/tcpip/network/ipv6/mld_test.go
@@ -15,6 +15,7 @@
package ipv6_test
import (
+ "bytes"
"testing"
"time"
@@ -119,11 +120,26 @@ func TestSendQueuedMLDReports(t *testing.T) {
},
}
+ nonce := [...]byte{
+ 1, 2, 3, 4, 5, 6,
+ }
+
+ const maxNSMessages = 2
+ secureRNGBytes := make([]byte, len(nonce)*maxNSMessages)
+ for b := secureRNGBytes[:]; len(b) > 0; b = b[len(nonce):] {
+ if n := copy(b, nonce[:]); n != len(nonce) {
+ t.Fatalf("got copy(...) = %d, want = %d", n, len(nonce))
+ }
+ }
+
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
dadResolutionTime := test.retransmitTimer * time.Duration(test.dadTransmits)
clock := faketime.NewManualClock()
+ var secureRNG bytes.Reader
+ secureRNG.Reset(secureRNGBytes[:])
s := stack.New(stack.Options{
+ SecureRNG: &secureRNG,
NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
DADConfigs: stack.DADConfigurations{
DupAddrDetectTransmits: test.dadTransmits,
@@ -154,7 +170,7 @@ func TestSendQueuedMLDReports(t *testing.T) {
checker.TTL(header.NDPHopLimit),
checker.NDPNS(
checker.NDPNSTargetAddress(addr),
- checker.NDPNSOptions(nil),
+ checker.NDPNSOptions([]header.NDPOption{header.NDPNonceOption(nonce[:])}),
))
}
}
diff --git a/pkg/tcpip/network/ipv6/ndp.go b/pkg/tcpip/network/ipv6/ndp.go
index d9b728878..536493f87 100644
--- a/pkg/tcpip/network/ipv6/ndp.go
+++ b/pkg/tcpip/network/ipv6/ndp.go
@@ -1789,18 +1789,14 @@ func (ndp *ndpState) stopSolicitingRouters() {
ndp.rtrSolicitTimer = timer{}
}
-func (ndp *ndpState) init(ep *endpoint) {
+func (ndp *ndpState) init(ep *endpoint, dadOptions ip.DADOptions) {
if ndp.defaultRouters != nil {
panic("attempted to initialize NDP state twice")
}
ndp.ep = ep
ndp.configs = ep.protocol.options.NDPConfigs
- ndp.dad.Init(&ndp.ep.mu, ep.protocol.options.DADConfigs, ip.DADOptions{
- Clock: ep.protocol.stack.Clock(),
- Protocol: ndp,
- NICID: ep.nic.ID(),
- })
+ ndp.dad.Init(&ndp.ep.mu, ep.protocol.options.DADConfigs, dadOptions)
ndp.defaultRouters = make(map[tcpip.Address]defaultRouterState)
ndp.onLinkPrefixes = make(map[tcpip.Subnet]onLinkPrefixState)
ndp.slaacPrefixes = make(map[tcpip.Subnet]slaacPrefixState)
@@ -1811,9 +1807,11 @@ func (ndp *ndpState) init(ep *endpoint) {
}
}
-func (ndp *ndpState) SendDADMessage(addr tcpip.Address) tcpip.Error {
+func (ndp *ndpState) SendDADMessage(addr tcpip.Address, nonce []byte) tcpip.Error {
snmc := header.SolicitedNodeAddr(addr)
- return ndp.ep.sendNDPNS(header.IPv6Any, snmc, addr, header.EthernetAddressFromMulticastIPv6Address(snmc), nil /* opts */)
+ return ndp.ep.sendNDPNS(header.IPv6Any, snmc, addr, header.EthernetAddressFromMulticastIPv6Address(snmc), header.NDPOptionsSerializer{
+ header.NDPNonceOption(nonce),
+ })
}
func (e *endpoint) sendNDPNS(srcAddr, dstAddr, targetAddr tcpip.Address, remoteLinkAddr tcpip.LinkAddress, opts header.NDPOptionsSerializer) tcpip.Error {
diff --git a/pkg/tcpip/network/ipv6/ndp_test.go b/pkg/tcpip/network/ipv6/ndp_test.go
index 6e850fd46..52b9a200c 100644
--- a/pkg/tcpip/network/ipv6/ndp_test.go
+++ b/pkg/tcpip/network/ipv6/ndp_test.go
@@ -15,6 +15,7 @@
package ipv6
import (
+ "bytes"
"context"
"strings"
"testing"
@@ -1264,8 +1265,21 @@ func TestCheckDuplicateAddress(t *testing.T) {
DupAddrDetectTransmits: 1,
RetransmitTimer: time.Second,
}
+
+ nonces := [...][]byte{
+ {1, 2, 3, 4, 5, 6},
+ {7, 8, 9, 10, 11, 12},
+ }
+
+ var secureRNGBytes []byte
+ for _, n := range nonces {
+ secureRNGBytes = append(secureRNGBytes, n...)
+ }
+ var secureRNG bytes.Reader
+ secureRNG.Reset(secureRNGBytes[:])
s := stack.New(stack.Options{
- Clock: clock,
+ SecureRNG: &secureRNG,
+ Clock: clock,
NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocolWithOptions(Options{
DADConfigs: dadConfigs,
})},
@@ -1278,10 +1292,36 @@ func TestCheckDuplicateAddress(t *testing.T) {
t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
}
- dadPacketsSent := 1
+ dadPacketsSent := 0
+ snmc := header.SolicitedNodeAddr(lladdr0)
+ remoteLinkAddr := header.EthernetAddressFromMulticastIPv6Address(snmc)
+ checkDADMsg := func() {
+ p, ok := e.ReadContext(context.Background())
+ if !ok {
+ t.Fatalf("expected %d-th DAD message", dadPacketsSent)
+ }
+
+ if p.Proto != header.IPv6ProtocolNumber {
+ t.Errorf("(i=%d) got p.Proto = %d, want = %d", dadPacketsSent, p.Proto, header.IPv6ProtocolNumber)
+ }
+
+ if p.Route.RemoteLinkAddress != remoteLinkAddr {
+ t.Errorf("(i=%d) got p.Route.RemoteLinkAddress = %s, want = %s", dadPacketsSent, p.Route.RemoteLinkAddress, remoteLinkAddr)
+ }
+
+ checker.IPv6(t, stack.PayloadSince(p.Pkt.NetworkHeader()),
+ checker.SrcAddr(header.IPv6Any),
+ checker.DstAddr(snmc),
+ checker.TTL(header.NDPHopLimit),
+ checker.NDPNS(
+ checker.NDPNSTargetAddress(lladdr0),
+ checker.NDPNSOptions([]header.NDPOption{header.NDPNonceOption(nonces[dadPacketsSent])}),
+ ))
+ }
if err := s.AddAddress(nicID, ProtocolNumber, lladdr0); err != nil {
t.Fatalf("AddAddress(%d, %d, %s) = %s", nicID, ProtocolNumber, lladdr0, err)
}
+ checkDADMsg()
// Start DAD for the address we just added.
//
@@ -1297,6 +1337,7 @@ func TestCheckDuplicateAddress(t *testing.T) {
} else if res != stack.DADStarting {
t.Fatalf("got s.CheckDuplicateAddress(%d, %d, %s, _) = %d, want = %d", nicID, ProtocolNumber, lladdr0, res, stack.DADStarting)
}
+ checkDADMsg()
// Remove the address and make sure our DAD request was not stopped.
if err := s.RemoveAddress(nicID, lladdr0); err != nil {
@@ -1328,33 +1369,6 @@ func TestCheckDuplicateAddress(t *testing.T) {
default:
}
- snmc := header.SolicitedNodeAddr(lladdr0)
- remoteLinkAddr := header.EthernetAddressFromMulticastIPv6Address(snmc)
-
- for i := 0; i < dadPacketsSent; i++ {
- p, ok := e.Read()
- if !ok {
- t.Fatalf("expected %d-th DAD message", i)
- }
-
- if p.Proto != header.IPv6ProtocolNumber {
- t.Errorf("(i=%d) got p.Proto = %d, want = %d", i, p.Proto, header.IPv6ProtocolNumber)
- }
-
- if p.Route.RemoteLinkAddress != remoteLinkAddr {
- t.Errorf("(i=%d) got p.Route.RemoteLinkAddress = %s, want = %s", i, p.Route.RemoteLinkAddress, remoteLinkAddr)
- }
-
- checker.IPv6(t, stack.PayloadSince(p.Pkt.NetworkHeader()),
- checker.SrcAddr(header.IPv6Any),
- checker.DstAddr(snmc),
- checker.TTL(header.NDPHopLimit),
- checker.NDPNS(
- checker.NDPNSTargetAddress(lladdr0),
- checker.NDPNSOptions(nil),
- ))
- }
-
// Should have no more packets.
if p, ok := e.Read(); ok {
t.Errorf("got unexpected packet = %#v", p)