diff options
Diffstat (limited to 'pkg/tcpip/stack')
-rw-r--r-- | pkg/tcpip/stack/BUILD | 4 | ||||
-rw-r--r-- | pkg/tcpip/stack/ndp.go | 279 | ||||
-rw-r--r-- | pkg/tcpip/stack/ndp_test.go | 352 | ||||
-rw-r--r-- | pkg/tcpip/stack/nic.go | 103 | ||||
-rw-r--r-- | pkg/tcpip/stack/stack.go | 48 |
5 files changed, 775 insertions, 11 deletions
diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD index 6a78432c9..bfc03e90b 100644 --- a/pkg/tcpip/stack/BUILD +++ b/pkg/tcpip/stack/BUILD @@ -22,6 +22,7 @@ go_library( "icmp_rate_limit.go", "linkaddrcache.go", "linkaddrentry_list.go", + "ndp.go", "nic.go", "registration.go", "route.go", @@ -53,6 +54,7 @@ go_test( name = "stack_x_test", size = "small", srcs = [ + "ndp_test.go", "stack_test.go", "transport_demuxer_test.go", "transport_test.go", @@ -61,12 +63,14 @@ go_test( ":stack", "//pkg/tcpip", "//pkg/tcpip/buffer", + "//pkg/tcpip/checker", "//pkg/tcpip/header", "//pkg/tcpip/iptables", "//pkg/tcpip/link/channel", "//pkg/tcpip/link/loopback", "//pkg/tcpip/network/ipv4", "//pkg/tcpip/network/ipv6", + "//pkg/tcpip/transport/icmp", "//pkg/tcpip/transport/udp", "//pkg/waiter", ], diff --git a/pkg/tcpip/stack/ndp.go b/pkg/tcpip/stack/ndp.go new file mode 100644 index 000000000..bed60d7b1 --- /dev/null +++ b/pkg/tcpip/stack/ndp.go @@ -0,0 +1,279 @@ +// Copyright 2019 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stack + +import ( + "fmt" + "log" + "time" + + "gvisor.dev/gvisor/pkg/tcpip" + "gvisor.dev/gvisor/pkg/tcpip/buffer" + "gvisor.dev/gvisor/pkg/tcpip/header" +) + +const ( + // defaultDupAddrDetectTransmits is the default number of NDP Neighbor + // Solicitation messages to send when doing Duplicate Address Detection + // for a tentative address. + // + // Default = 1 (from RFC 4862 section 5.1) + defaultDupAddrDetectTransmits = 1 + + // defaultRetransmitTimer is the default amount of time to wait between + // sending NDP Neighbor solicitation messages. + // + // Default = 1s (from RFC 4861 section 10). + defaultRetransmitTimer = time.Second + + // minimumRetransmitTimer is the minimum amount of time to wait between + // sending NDP Neighbor solicitation messages. Note, RFC 4861 does + // not impose a minimum Retransmit Timer, but we do here to make sure + // the messages are not sent all at once. We also come to this value + // because in the RetransmitTimer field of a Router Advertisement, a + // value of 0 means unspecified, so the smallest valid value is 1. + // Note, the unit of the RetransmitTimer field in the Router + // Advertisement is milliseconds. + // + // Min = 1ms. + minimumRetransmitTimer = time.Millisecond +) + +// NDPConfigurations is the NDP configurations for the netstack. +type NDPConfigurations struct { + // The number of Neighbor Solicitation messages to send when doing + // Duplicate Address Detection for a tentative address. + // + // Note, a value of zero effectively disables DAD. + DupAddrDetectTransmits uint8 + + // The amount of time to wait between sending Neighbor solicitation + // messages. + // + // Must be greater than 0.5s. + RetransmitTimer time.Duration +} + +// DefaultNDPConfigurations returns an NDPConfigurations populated with +// default values. +func DefaultNDPConfigurations() NDPConfigurations { + return NDPConfigurations{ + DupAddrDetectTransmits: defaultDupAddrDetectTransmits, + RetransmitTimer: defaultRetransmitTimer, + } +} + +// validate modifies an NDPConfigurations with valid values. If invalid values +// are present in c, the corresponding default values will be used instead. +// +// If RetransmitTimer is less than minimumRetransmitTimer, then a value of +// defaultRetransmitTimer will be used. +func (c *NDPConfigurations) validate() { + if c.RetransmitTimer < minimumRetransmitTimer { + c.RetransmitTimer = defaultRetransmitTimer + } +} + +// ndpState is the per-interface NDP state. +type ndpState struct { + // The DAD state to send the next NS message, or resolve the address. + dad map[tcpip.Address]dadState +} + +// dadState holds the Duplicate Address Detection timer and channel to signal +// to the DAD goroutine that DAD should stop. +type dadState struct { + // The DAD timer to send the next NS message, or resolve the address. + timer *time.Timer + + // Used to let the DAD timer know that it has been stopped. + // + // Must only be read from or written to while protected by the lock of + // the NIC this dadState is associated with. + done *bool +} + +// startDuplicateAddressDetection performs Duplicate Address Detection. +// +// 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 { + // addr must be a valid unicast IPv6 address. + if !header.IsV6UnicastAddress(addr) { + return tcpip.ErrAddressFamilyNotSupported + } + + // Should not attempt to perform DAD on an address that is currently in + // the DAD process. + if _, ok := ndp.dad[addr]; ok { + // Should never happen because we should only ever call this + // function for newly created addresses. If we attemped to + // "add" an address that already existed, we would returned an + // error since we attempted to add a duplicate address, or its + // 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())) + } + + remaining := n.stack.ndpConfigs.DupAddrDetectTransmits + + { + done, err := ndp.doDuplicateAddressDetection(n, addr, remaining, ref) + if err != nil { + return err + } + if done { + return nil + } + } + + remaining-- + + var done bool + var timer *time.Timer + timer = time.AfterFunc(n.stack.ndpConfigs.RetransmitTimer, func() { + n.mu.Lock() + defer n.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 it this function + // obtained the NIC lock. Simply return here and do + // nothing further. + return + } + + 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())) + } + + 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) + } + + ndp.stopDuplicateAddressDetection(addr) + return + } + + timer.Reset(n.stack.ndpConfigs.RetransmitTimer) + remaining-- + + }) + + ndp.dad[addr] = dadState{ + timer: timer, + done: &done, + } + + return nil +} + +// doDuplicateAddressDetection is called on every iteration of the timer, and +// when DAD starts. +// +// It handles resolving the address (if there are no more NS to send), or +// sending the next NS if there are more NS to send. +// +// This function must only be called by IPv6 addresses that are currently +// tentative. +// +// 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) { + 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())) + } + + if remaining == 0 { + // DAD has resolved. + ref.setKind(permanent) + return true, nil + } + + // Send a new NS. + snmc := header.SolicitedNodeAddr(addr) + snmcRef, ok := n.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)) + } + + // Use the unspecified address as the source address when performing + // DAD. + r := makeRoute(header.IPv6ProtocolNumber, header.IPv6Any, snmc, n.linkEP.LinkAddress(), snmcRef, false, false) + + hdr := buffer.NewPrependable(int(r.MaxHeaderLength()) + header.ICMPv6NeighborSolicitMinimumSize) + pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6NeighborSolicitMinimumSize)) + pkt.SetType(header.ICMPv6NeighborSolicit) + ns := header.NDPNeighborSolicit(pkt.NDPPayload()) + ns.SetTargetAddress(addr) + pkt.SetChecksum(header.ICMPv6Checksum(pkt, r.LocalAddress, r.RemoteAddress, buffer.VectorisedView{})) + + sent := r.Stats().ICMP.V6PacketsSent + if err := r.WritePacket(nil, hdr, buffer.VectorisedView{}, NetworkHeaderParams{Protocol: header.ICMPv6ProtocolNumber, TTL: r.DefaultTTL(), TOS: DefaultTOS}); err != nil { + sent.Dropped.Increment() + return false, err + } + sent.NeighborSolicit.Increment() + + return false, nil +} + +// stopDuplicateAddressDetection ends a running Duplicate Address Detection +// process. Note, this may leave the DAD process for a tentative address in +// such a state forever, unless some other external event resolves the DAD +// process (receiving an NA from the true owner of addr, or an NS for addr +// (implying another node is attempting to use addr)). It is up to the caller +// of this function to handle such a scenario. Normally, addr will be removed +// from n right after this function returns or the address successfully +// resolved. +// +// The NIC that ndp belongs to MUST be locked. +func (ndp *ndpState) stopDuplicateAddressDetection(addr tcpip.Address) { + dad, ok := ndp.dad[addr] + if !ok { + // Not currently performing DAD on addr, just return. + return + } + + if dad.timer != nil { + dad.timer.Stop() + dad.timer = nil + + *dad.done = true + dad.done = nil + } + + delete(ndp.dad, addr) + + return +} diff --git a/pkg/tcpip/stack/ndp_test.go b/pkg/tcpip/stack/ndp_test.go new file mode 100644 index 000000000..cc789d70f --- /dev/null +++ b/pkg/tcpip/stack/ndp_test.go @@ -0,0 +1,352 @@ +// Copyright 2019 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stack_test + +import ( + "testing" + "time" + + "gvisor.dev/gvisor/pkg/tcpip" + "gvisor.dev/gvisor/pkg/tcpip/buffer" + "gvisor.dev/gvisor/pkg/tcpip/checker" + "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/link/channel" + "gvisor.dev/gvisor/pkg/tcpip/network/ipv6" + "gvisor.dev/gvisor/pkg/tcpip/stack" + "gvisor.dev/gvisor/pkg/tcpip/transport/icmp" +) + +const ( + addr1 = "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01" + addr2 = "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02" + linkAddr1 = "\x01\x02\x03\x04\x05\x06" + linkAddr2 = "\x01\x02\x03\x04\x05\x07" +) + +// TestDADDisabled tests that an address successfully resolves immediately +// when DAD is not enabled (the default for an empty stack.Options). +func TestDADDisabled(t *testing.T) { + opts := stack.Options{ + NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + } + + e := channel.New(10, 1280, linkAddr1) + s := stack.New(opts) + if err := s.CreateNIC(1, e); err != nil { + t.Fatalf("CreateNIC(_) = %s", err) + } + + if err := s.AddAddress(1, header.IPv6ProtocolNumber, addr1); err != nil { + t.Fatalf("AddAddress(_, %d, %s) = %s", header.IPv6ProtocolNumber, addr1, err) + } + + // Should get the address immediately since we should not have performed + // DAD on it. + addr, err := s.GetMainNICAddress(1, header.IPv6ProtocolNumber) + if err != nil { + t.Fatalf("stack.GetMainNICAddress(_, _) err = %s", err) + } + if addr.Address != addr1 { + t.Fatalf("got stack.GetMainNICAddress(_, _) = %s, want = %s", addr, addr1) + } + + // We should not have sent any NDP NS messages. + if got := s.Stats().ICMP.V6PacketsSent.NeighborSolicit.Value(); got != 0 { + t.Fatalf("got NeighborSolicit = %d, want = 0", got) + } +} + +// 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 +// RetransmitTimer (<1ms) values get fixed to the default RetransmitTimer of 1s. +func TestDADResolve(t *testing.T) { + tests := []struct { + name string + dupAddrDetectTransmits uint8 + retransTimer time.Duration + expectedRetransmitTimer time.Duration + }{ + {"1:1s:1s", 1, time.Second, time.Second}, + {"2:1s:1s", 2, time.Second, time.Second}, + {"1:2s:2s", 1, 2 * time.Second, 2 * time.Second}, + // 0s is an invalid RetransmitTimer timer and will be fixed to + // the default RetransmitTimer value of 1s. + {"1:0s:1s", 1, 0, time.Second}, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + opts := stack.Options{ + NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + } + opts.NDPConfigs.RetransmitTimer = test.retransTimer + opts.NDPConfigs.DupAddrDetectTransmits = test.dupAddrDetectTransmits + + e := channel.New(10, 1280, linkAddr1) + s := stack.New(opts) + if err := s.CreateNIC(1, e); err != nil { + t.Fatalf("CreateNIC(_) = %s", err) + } + + if err := s.AddAddress(1, header.IPv6ProtocolNumber, addr1); err != nil { + t.Fatalf("AddAddress(_, %d, %s) = %s", header.IPv6ProtocolNumber, addr1, err) + } + + stat := s.Stats().ICMP.V6PacketsSent.NeighborSolicit + + // Should have sent an NDP NS almost immediately. + time.Sleep(100 * time.Millisecond) + if got := stat.Value(); got != 1 { + t.Fatalf("got NeighborSolicit = %d, want = 1", got) + + } + + // Address should not be considered bound to the NIC yet + // (DAD ongoing). + addr, err := s.GetMainNICAddress(1, header.IPv6ProtocolNumber) + if err != nil { + t.Fatalf("got stack.GetMainNICAddress(_, _) = (_, %v), want = (_, nil)", err) + } + if want := (tcpip.AddressWithPrefix{}); addr != want { + 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) + addr, err = s.GetMainNICAddress(1, header.IPv6ProtocolNumber) + if err != nil { + t.Fatalf("got stack.GetMainNICAddress(_, _) = (_, %v), want = (_, nil)", err) + } + if want := (tcpip.AddressWithPrefix{}); addr != want { + 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) + addr, err = s.GetMainNICAddress(1, header.IPv6ProtocolNumber) + if err != nil { + t.Fatalf("stack.GetMainNICAddress(_, _) err = %s", err) + } + if addr.Address != addr1 { + t.Fatalf("got stack.GetMainNICAddress(_, _) = %s, want = %s", addr, addr1) + } + + // Should not have sent any more NS messages. + if got := stat.Value(); got != uint64(test.dupAddrDetectTransmits) { + t.Fatalf("got NeighborSolicit = %d, want = %d", got, test.dupAddrDetectTransmits) + } + + // Validate the sent Neighbor Solicitation messages. + for i := uint8(0); i < test.dupAddrDetectTransmits; i++ { + p := <-e.C + + // Make sure its an IPv6 packet. + if p.Proto != header.IPv6ProtocolNumber { + t.Fatalf("got Proto = %d, want = %d", p.Proto, header.IPv6ProtocolNumber) + } + + // Check NDP packet. + checker.IPv6(t, p.Header.ToVectorisedView().First(), + checker.NDPNS( + checker.NDPNSTargetAddress(addr1))) + } + }) + } + +} + +// TestDADFail tests to make sure that the DAD process fails if another node is +// detected to be performing DAD on the same address (receive an NS message from +// a node doing DAD for the same address), or if another node is detected to own +// the address already (receive an NA message for the tentative address). +func TestDADFail(t *testing.T) { + tests := []struct { + name string + makeBuf func(tgt tcpip.Address) buffer.Prependable + getStat func(s tcpip.ICMPv6ReceivedPacketStats) *tcpip.StatCounter + }{ + { + "RxSolicit", + func(tgt tcpip.Address) buffer.Prependable { + hdr := buffer.NewPrependable(header.IPv6MinimumSize + header.ICMPv6NeighborSolicitMinimumSize) + pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6NeighborSolicitMinimumSize)) + pkt.SetType(header.ICMPv6NeighborSolicit) + ns := header.NDPNeighborSolicit(pkt.NDPPayload()) + ns.SetTargetAddress(tgt) + snmc := header.SolicitedNodeAddr(tgt) + pkt.SetChecksum(header.ICMPv6Checksum(pkt, header.IPv6Any, snmc, buffer.VectorisedView{})) + payloadLength := hdr.UsedLength() + ip := header.IPv6(hdr.Prepend(header.IPv6MinimumSize)) + ip.Encode(&header.IPv6Fields{ + PayloadLength: uint16(payloadLength), + NextHeader: uint8(icmp.ProtocolNumber6), + HopLimit: 255, + SrcAddr: header.IPv6Any, + DstAddr: snmc, + }) + + return hdr + + }, + func(s tcpip.ICMPv6ReceivedPacketStats) *tcpip.StatCounter { + return s.NeighborSolicit + }, + }, + { + "RxAdvert", + func(tgt tcpip.Address) buffer.Prependable { + hdr := buffer.NewPrependable(header.IPv6MinimumSize + header.ICMPv6NeighborAdvertSize) + pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6NeighborAdvertSize)) + pkt.SetType(header.ICMPv6NeighborAdvert) + na := header.NDPNeighborAdvert(pkt.NDPPayload()) + na.SetSolicitedFlag(true) + na.SetOverrideFlag(true) + na.SetTargetAddress(tgt) + pkt.SetChecksum(header.ICMPv6Checksum(pkt, tgt, header.IPv6AllNodesMulticastAddress, buffer.VectorisedView{})) + payloadLength := hdr.UsedLength() + ip := header.IPv6(hdr.Prepend(header.IPv6MinimumSize)) + ip.Encode(&header.IPv6Fields{ + PayloadLength: uint16(payloadLength), + NextHeader: uint8(icmp.ProtocolNumber6), + HopLimit: 255, + SrcAddr: tgt, + DstAddr: header.IPv6AllNodesMulticastAddress, + }) + + return hdr + + }, + func(s tcpip.ICMPv6ReceivedPacketStats) *tcpip.StatCounter { + return s.NeighborAdvert + }, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + opts := stack.Options{ + NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NDPConfigs: stack.DefaultNDPConfigurations(), + } + opts.NDPConfigs.RetransmitTimer = time.Second * 2 + + e := channel.New(10, 1280, linkAddr1) + s := stack.New(opts) + if err := s.CreateNIC(1, e); err != nil { + t.Fatalf("CreateNIC(_) = %s", err) + } + + if err := s.AddAddress(1, header.IPv6ProtocolNumber, addr1); err != nil { + t.Fatalf("AddAddress(_, %d, %s) = %s", header.IPv6ProtocolNumber, addr1, err) + } + + // Address should not be considered bound to the NIC yet + // (DAD ongoing). + addr, err := s.GetMainNICAddress(1, header.IPv6ProtocolNumber) + if err != nil { + t.Fatalf("got stack.GetMainNICAddress(_, _) = (_, %v), want = (_, nil)", err) + } + if want := (tcpip.AddressWithPrefix{}); addr != want { + t.Fatalf("got stack.GetMainNICAddress(_, _) = (%s, nil), want = (%s, nil)", addr, want) + } + + // Receive a packet to simulate multiple nodes owning or + // attempting to own the same address. + hdr := test.makeBuf(addr1) + e.Inject(header.IPv6ProtocolNumber, hdr.View().ToVectorisedView()) + + stat := test.getStat(s.Stats().ICMP.V6PacketsReceived) + if got := stat.Value(); got != 1 { + t.Fatalf("got stat = %d, want = 1", got) + } + + // Wait 3 seconds to make sure that DAD did not resolve + time.Sleep(3 * time.Second) + addr, err = s.GetMainNICAddress(1, header.IPv6ProtocolNumber) + if err != nil { + t.Fatalf("got stack.GetMainNICAddress(_, _) = (_, %v), want = (_, nil)", err) + } + if want := (tcpip.AddressWithPrefix{}); addr != want { + t.Fatalf("got stack.GetMainNICAddress(_, _) = (%s, nil), want = (%s, nil)", addr, want) + } + }) + } +} + +// TestDADStop tests to make sure that the DAD process stops when an address is +// removed. +func TestDADStop(t *testing.T) { + opts := stack.Options{ + NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + } + opts.NDPConfigs.RetransmitTimer = time.Second + opts.NDPConfigs.DupAddrDetectTransmits = 2 + + e := channel.New(10, 1280, linkAddr1) + s := stack.New(opts) + if err := s.CreateNIC(1, e); err != nil { + t.Fatalf("CreateNIC(_) = %s", err) + } + + if err := s.AddAddress(1, header.IPv6ProtocolNumber, addr1); err != nil { + t.Fatalf("AddAddress(_, %d, %s) = %s", header.IPv6ProtocolNumber, addr1, err) + } + + // Address should not be considered bound to the NIC yet (DAD ongoing). + addr, err := s.GetMainNICAddress(1, header.IPv6ProtocolNumber) + if err != nil { + t.Fatalf("got stack.GetMainNICAddress(_, _) = (_, %v), want = (_, nil)", err) + } + if want := (tcpip.AddressWithPrefix{}); addr != want { + t.Fatalf("got stack.GetMainNICAddress(_, _) = (%s, nil), want = (%s, nil)", addr, want) + } + + // Remove the address. This should stop DAD. + if err := s.RemoveAddress(1, addr1); err != nil { + 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) + addr, err = s.GetMainNICAddress(1, header.IPv6ProtocolNumber) + if err != nil { + t.Fatalf("got stack.GetMainNICAddress(_, _) = (_, %v), want = (_, nil)", err) + } + if want := (tcpip.AddressWithPrefix{}); addr != want { + t.Fatalf("got stack.GetMainNICAddress(_, _) = (%s, nil), want = (%s, nil)", addr, want) + } + + // Should not have sent more than 1 NS message. + if got := s.Stats().ICMP.V6PacketsSent.NeighborSolicit.Value(); got > 1 { + t.Fatalf("got NeighborSolicit = %d, want <= 1", got) + } +} diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go index f64bbf6eb..43f4ad91e 100644 --- a/pkg/tcpip/stack/nic.go +++ b/pkg/tcpip/stack/nic.go @@ -43,6 +43,8 @@ type NIC struct { mcastJoins map[NetworkEndpointID]int32 stats NICStats + + ndp ndpState } // NICStats includes transmitted and received stats. @@ -96,6 +98,9 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, loopback Bytes: &tcpip.StatCounter{}, }, }, + ndp: ndpState{ + dad: make(map[tcpip.Address]dadState), + }, } } @@ -282,7 +287,7 @@ func (n *NIC) addPermanentAddressLocked(protocolAddress tcpip.ProtocolAddress, p id := NetworkEndpointID{protocolAddress.AddressWithPrefix.Address} if ref, ok := n.endpoints[id]; ok { switch ref.getKind() { - case permanent: + case permanentTentative, permanent: // The NIC already have a permanent endpoint with that address. return nil, tcpip.ErrDuplicateAddress case permanentExpired, temporary: @@ -298,6 +303,7 @@ func (n *NIC) addPermanentAddressLocked(protocolAddress tcpip.ProtocolAddress, p n.removeEndpointLocked(ref) } } + return n.addAddressLocked(protocolAddress, peb, permanent) } @@ -321,6 +327,15 @@ func (n *NIC) addAddressLocked(protocolAddress tcpip.ProtocolAddress, peb Primar if err != nil { return nil, err } + + isIPv6Unicast := protocolAddress.Protocol == header.IPv6ProtocolNumber && header.IsV6UnicastAddress(protocolAddress.AddressWithPrefix.Address) + + // If the address is an IPv6 address and it is a permanent address, + // mark it as tentative so it goes through the DAD process. + if isIPv6Unicast && kind == permanent { + kind = permanentTentative + } + ref := &referencedNetworkEndpoint{ refs: 1, ep: ep, @@ -338,7 +353,7 @@ func (n *NIC) addAddressLocked(protocolAddress tcpip.ProtocolAddress, peb Primar // If we are adding an IPv6 unicast address, join the solicited-node // multicast address. - if protocolAddress.Protocol == header.IPv6ProtocolNumber && header.IsV6UnicastAddress(protocolAddress.AddressWithPrefix.Address) { + if isIPv6Unicast { snmc := header.SolicitedNodeAddr(protocolAddress.AddressWithPrefix.Address) if err := n.joinGroupLocked(protocolAddress.Protocol, snmc); err != nil { return nil, err @@ -360,6 +375,13 @@ func (n *NIC) addAddressLocked(protocolAddress tcpip.ProtocolAddress, peb Primar l.PushFront(ref) } + // 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 { + return nil, err + } + } + return ref, nil } @@ -382,10 +404,12 @@ func (n *NIC) AllAddresses() []tcpip.ProtocolAddress { addrs := make([]tcpip.ProtocolAddress, 0, len(n.endpoints)) for nid, ref := range n.endpoints { - // Don't include expired or temporary endpoints to avoid confusion and - // prevent the caller from using those. + // Don't include tentative, expired or temporary endpoints to + // avoid confusion and prevent the caller from using those. switch ref.getKind() { - case permanentExpired, temporary: + case permanentTentative, permanentExpired, temporary: + // TODO(b/140898488): Should tentative addresses be + // returned? continue } addrs = append(addrs, tcpip.ProtocolAddress{ @@ -408,10 +432,11 @@ func (n *NIC) PrimaryAddresses() []tcpip.ProtocolAddress { for proto, list := range n.primary { for e := list.Front(); e != nil; e = e.Next() { ref := e.(*referencedNetworkEndpoint) - // Don't include expired or tempory endpoints to avoid confusion and - // prevent the caller from using those. + // Don't include tentative, expired or tempory endpoints + // to avoid confusion and prevent the caller from using + // those. switch ref.getKind() { - case permanentExpired, temporary: + case permanentTentative, permanentExpired, temporary: continue } @@ -504,10 +529,22 @@ func (n *NIC) removeEndpoint(r *referencedNetworkEndpoint) { func (n *NIC) removePermanentAddressLocked(addr tcpip.Address) *tcpip.Error { r, ok := n.endpoints[NetworkEndpointID{addr}] - if !ok || r.getKind() != permanent { + if !ok { return tcpip.ErrBadLocalAddress } + kind := r.getKind() + if kind != permanent && kind != permanentTentative { + return tcpip.ErrBadLocalAddress + } + + isIPv6Unicast := r.protocol == header.IPv6ProtocolNumber && header.IsV6UnicastAddress(addr) + + // If we are removing a tentative IPv6 unicast address, stop DAD. + if isIPv6Unicast && kind == permanentTentative { + n.ndp.stopDuplicateAddressDetection(addr) + } + r.setKind(permanentExpired) if !r.decRefLocked() { // The endpoint still has references to it. @@ -518,7 +555,7 @@ func (n *NIC) removePermanentAddressLocked(addr tcpip.Address) *tcpip.Error { // If we are removing an IPv6 unicast address, leave the solicited-node // multicast address. - if r.protocol == header.IPv6ProtocolNumber && header.IsV6UnicastAddress(addr) { + if isIPv6Unicast { snmc := header.SolicitedNodeAddr(addr) if err := n.leaveGroupLocked(snmc); err != nil { return err @@ -767,14 +804,58 @@ func (n *NIC) Stack() *Stack { return n.stack } +// isAddrTentative returns true if addr is tentative on n. +// +// Note that if addr is not associated with n, then this function will return +// false. It will only return true if the address is associated with the NIC +// AND it is tentative. +func (n *NIC) isAddrTentative(addr tcpip.Address) bool { + ref, ok := n.endpoints[NetworkEndpointID{addr}] + if !ok { + return false + } + + return ref.getKind() == permanentTentative +} + +// dupTentativeAddrDetected attempts to inform n that a tentative addr +// is a duplicate on a link. +// +// dupTentativeAddrDetected will delete the tentative address if it exists. +func (n *NIC) dupTentativeAddrDetected(addr tcpip.Address) *tcpip.Error { + n.mu.Lock() + defer n.mu.Unlock() + + ref, ok := n.endpoints[NetworkEndpointID{addr}] + if !ok { + return tcpip.ErrBadAddress + } + + if ref.getKind() != permanentTentative { + return tcpip.ErrInvalidEndpointState + } + + return n.removePermanentAddressLocked(addr) +} + type networkEndpointKind int32 const ( + // A permanentTentative endpoint is a permanent address that is not yet + // considered to be fully bound to an interface in the traditional + // sense. That is, the address is associated with a NIC, but packets + // destined to the address MUST NOT be accepted and MUST be silently + // dropped, and the address MUST NOT be used as a source address for + // outgoing packets. For IPv6, addresses will be of this kind until + // NDP's Duplicate Address Detection has resolved, or be deleted if + // the process results in detecting a duplicate address. + permanentTentative networkEndpointKind = iota + // A permanent endpoint is created by adding a permanent address (vs. a // temporary one) to the NIC. Its reference count is biased by 1 to avoid // removal when no route holds a reference to it. It is removed by explicitly // removing the permanent address from the NIC. - permanent networkEndpointKind = iota + permanent // An expired permanent endoint is a permanent endoint that had its address // removed from the NIC, and it is waiting to be removed once no more routes diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index f67975525..a199bc1cc 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -399,6 +399,9 @@ type Stack struct { // // TODO(gvisor.dev/issue/940): S/R this field. portSeed uint32 + + // ndpConfigs is the NDP configurations used by interfaces. + ndpConfigs NDPConfigurations } // Options contains optional Stack configuration. @@ -425,6 +428,13 @@ type Options struct { // UnassociatedFactory produces unassociated endpoints raw endpoints. // Raw endpoints are enabled only if this is non-nil. UnassociatedFactory UnassociatedEndpointFactory + + // NDPConfigs is the NDP configurations used by interfaces. + // + // By default, NDPConfigs will have a zero value for its + // DupAddrDetectTransmits field, implying that DAD will not be performed + // before assigning an address to a NIC. + NDPConfigs NDPConfigurations } // TransportEndpointInfo holds useful information about a transport endpoint @@ -458,6 +468,9 @@ func (*TransportEndpointInfo) IsEndpointInfo() {} // New allocates a new networking stack with only the requested networking and // transport protocols configured with default options. // +// Note, NDPConfigurations will be fixed before being used by the Stack. That +// is, if an invalid value was provided, it will be reset to the default value. +// // Protocol options can be changed by calling the // SetNetworkProtocolOption/SetTransportProtocolOption methods provided by the // stack. Please refer to individual protocol implementations as to what options @@ -468,6 +481,9 @@ func New(opts Options) *Stack { clock = &tcpip.StdClock{} } + // Make sure opts.NDPConfigs contains valid values only. + opts.NDPConfigs.validate() + s := &Stack{ transportProtocols: make(map[tcpip.TransportProtocolNumber]*transportProtocolState), networkProtocols: make(map[tcpip.NetworkProtocolNumber]NetworkProtocol), @@ -480,6 +496,7 @@ func New(opts Options) *Stack { handleLocal: opts.HandleLocal, icmpRateLimiter: NewICMPRateLimiter(), portSeed: generateRandUint32(), + ndpConfigs: opts.NDPConfigs, } // Add specified network protocols. @@ -1238,6 +1255,37 @@ func (s *Stack) AllowICMPMessage() bool { return s.icmpRateLimiter.Allow() } +// IsAddrTentative returns true if addr is tentative on the NIC with ID id. +// +// Note that if addr is not associated with a NIC with id ID, then this +// function will return false. It will only return true if the address is +// associated with the NIC AND it is tentative. +func (s *Stack) IsAddrTentative(id tcpip.NICID, addr tcpip.Address) (bool, *tcpip.Error) { + s.mu.RLock() + defer s.mu.RUnlock() + + nic, ok := s.nics[id] + if !ok { + return false, tcpip.ErrUnknownNICID + } + + return nic.isAddrTentative(addr), nil +} + +// DupTentativeAddrDetected attempts to inform the NIC with ID id that a +// tentative addr on it is a duplicate on a link. +func (s *Stack) DupTentativeAddrDetected(id tcpip.NICID, addr tcpip.Address) *tcpip.Error { + s.mu.Lock() + defer s.mu.Unlock() + + nic, ok := s.nics[id] + if !ok { + return tcpip.ErrUnknownNICID + } + + return nic.dupTentativeAddrDetected(addr) +} + // PortSeed returns a 32 bit value that can be used as a seed value for port // picking. // |