summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/stack
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2019-10-17 05:58:27 +0000
committergVisor bot <gvisor-bot@google.com>2019-10-17 05:58:27 +0000
commit1993a36b013fd7d6786b23ed8fd85f77b65e6c38 (patch)
tree4bf60bd669e61ed150548eee8e7aeed8acc17504 /pkg/tcpip/stack
parent61ea1714d2b2ae34af11f1eb1e45f22efa6fc157 (diff)
parent06ed9e329d46268cf12fe07bd3974c5d54ae7e0b (diff)
Merge release-20190806.1-284-g06ed9e3 (automated)
Diffstat (limited to 'pkg/tcpip/stack')
-rwxr-xr-xpkg/tcpip/stack/ndp.go279
-rw-r--r--pkg/tcpip/stack/nic.go103
-rw-r--r--pkg/tcpip/stack/stack.go48
3 files changed, 419 insertions, 11 deletions
diff --git a/pkg/tcpip/stack/ndp.go b/pkg/tcpip/stack/ndp.go
new file mode 100755
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/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.
//