Browse Source

vxlan: Statically configure routes, arp, and forwarding tables

This removes the complicated monitoring code and replaces it with three
network configuration entries for each subnet lease that cause the
packets to end up in the right place:

- A route pointing all traffic to the subnet via a gateway that is the
  IP of the flannel interface on the target host.
- A forwarding database entry that maps the network routable IP address
  of the target host to the MAC address of the flannel interface on the
  same host.
- An ARP table entry mapping the IP address of the flannel interface on
  the target host to the MAC address of the same interface.
Jonathan Rudenberg 10 years ago
parent
commit
c06d757521
3 changed files with 60 additions and 213 deletions
  1. 53 80
      backend/vxlan/device.go
  2. 0 69
      backend/vxlan/routes.go
  3. 7 64
      backend/vxlan/vxlan.go

+ 53 - 80
backend/vxlan/device.go

@@ -19,11 +19,9 @@ import (
 	"net"
 	"os"
 	"syscall"
-	"time"
 
 	log "github.com/coreos/flannel/Godeps/_workspace/src/github.com/golang/glog"
 	"github.com/coreos/flannel/Godeps/_workspace/src/github.com/vishvananda/netlink"
-	"github.com/coreos/flannel/Godeps/_workspace/src/github.com/vishvananda/netlink/nl"
 
 	"github.com/coreos/flannel/pkg/ip"
 )
@@ -60,9 +58,7 @@ func newVXLANDevice(devAttrs *vxlanDeviceAttrs) (*vxlanDevice, error) {
 		VtepDevIndex: devAttrs.vtepIndex,
 		SrcAddr:      devAttrs.vtepAddr,
 		Port:         devAttrs.vtepPort,
-		Learning:     true,
-		Proxy:        true,
-		L2miss:       true,
+		Learning:     false,
 	}
 
 	link, err := ensureLink(link)
@@ -70,10 +66,6 @@ func newVXLANDevice(devAttrs *vxlanDeviceAttrs) (*vxlanDevice, error) {
 		return nil, err
 	}
 
-	// this enables ARP requests being sent to userspace via netlink
-	sysctlPath := fmt.Sprintf("/proc/sys/net/ipv4/neigh/%s/app_solicit", devAttrs.name)
-	sysctlSet(sysctlPath, "3")
-
 	return &vxlanDevice{
 		link: link,
 	}, nil
@@ -153,94 +145,75 @@ func (dev *vxlanDevice) MTU() int {
 	return dev.link.MTU
 }
 
-func (dev *vxlanDevice) MonitorMisses(misses chan *netlink.Neigh) {
-	nlsock, err := nl.Subscribe(syscall.NETLINK_ROUTE, syscall.RTNLGRP_NEIGH)
-	if err != nil {
-		log.Error("Failed to subscribe to netlink RTNLGRP_NEIGH messages")
-		return
-	}
-
-	for {
-		msgs, err := nlsock.Receive()
-		if err != nil {
-			log.Errorf("Failed to receive from netlink: %v ", err)
-
-			// wait 1 sec before retrying but honor the cancel channel
-			time.Sleep(1 * time.Second)
-			continue
-		}
-
-		for _, msg := range msgs {
-			dev.processNeighMsg(msg, misses)
-		}
-	}
-}
-
-func isNeighResolving(state int) bool {
-	return (state & (netlink.NUD_INCOMPLETE | netlink.NUD_STALE | netlink.NUD_DELAY | netlink.NUD_PROBE)) != 0
-}
-
-func (dev *vxlanDevice) processNeighMsg(msg syscall.NetlinkMessage, misses chan *netlink.Neigh) {
-	neigh, err := netlink.NeighDeserialize(msg.Data)
-	if err != nil {
-		log.Error("Failed to deserialize netlink ndmsg: %v", err)
-		return
-	}
-
-	if int(neigh.LinkIndex) != dev.link.Index {
-		return
-	}
-
-	if msg.Header.Type != syscall.RTM_GETNEIGH && msg.Header.Type != syscall.RTM_NEWNEIGH {
-		return
-	}
-
-	if !isNeighResolving(neigh.State) {
-		// misses come with NUD_STALE bit set
-		return
-	}
-
-	misses <- neigh
+type neigh struct {
+	MAC net.HardwareAddr
+	IP  ip.IP4
 }
 
-func (dev *vxlanDevice) AddL2(mac net.HardwareAddr, vtep net.IP) error {
-	neigh := netlink.Neigh{
+func (dev *vxlanDevice) AddL2(n neigh) error {
+	log.Infof("calling NeighAdd: %v, %v", n.IP, n.MAC)
+	return netlink.NeighAdd(&netlink.Neigh{
 		LinkIndex:    dev.link.Index,
-		State:        netlink.NUD_REACHABLE,
+		State:        netlink.NUD_PERMANENT,
 		Family:       syscall.AF_BRIDGE,
 		Flags:        netlink.NTF_SELF,
-		IP:           vtep,
-		HardwareAddr: mac,
-	}
-
-	log.Infof("calling NeighAdd: %v, %v", vtep, mac)
-	return netlink.NeighAdd(&neigh)
+		IP:           n.IP.ToIP(),
+		HardwareAddr: n.MAC,
+	})
 }
 
-func (dev *vxlanDevice) DelL2(mac net.HardwareAddr, vtep net.IP) error {
-	neigh := netlink.Neigh{
+func (dev *vxlanDevice) DelL2(n neigh) error {
+	log.Infof("calling NeighDel: %v, %v", n.IP, n.MAC)
+	return netlink.NeighDel(&netlink.Neigh{
 		LinkIndex:    dev.link.Index,
 		Family:       syscall.AF_BRIDGE,
 		Flags:        netlink.NTF_SELF,
-		IP:           vtep,
-		HardwareAddr: mac,
-	}
+		IP:           n.IP.ToIP(),
+		HardwareAddr: n.MAC,
+	})
+}
 
-	log.Infof("calling NeighDel: %v, %v", vtep, mac)
-	return netlink.NeighDel(&neigh)
+func (dev *vxlanDevice) AddL3(n neigh) error {
+	log.Infof("calling NeighSet: %v, %v", n.IP, n.MAC)
+	return netlink.NeighSet(&netlink.Neigh{
+		LinkIndex:    dev.link.Index,
+		State:        netlink.NUD_PERMANENT,
+		Type:         syscall.RTN_UNICAST,
+		IP:           n.IP.ToIP(),
+		HardwareAddr: n.MAC,
+	})
 }
 
-func (dev *vxlanDevice) AddL3(ip net.IP, mac net.HardwareAddr) error {
-	neigh := netlink.Neigh{
+func (dev *vxlanDevice) DelL3(n neigh) error {
+	log.Infof("calling NeighDel: %v, %v", n.IP, n.MAC)
+	return netlink.NeighDel(&netlink.Neigh{
 		LinkIndex:    dev.link.Index,
-		State:        netlink.NUD_REACHABLE,
+		State:        netlink.NUD_PERMANENT,
 		Type:         syscall.RTN_UNICAST,
-		IP:           ip,
-		HardwareAddr: mac,
+		IP:           n.IP.ToIP(),
+		HardwareAddr: n.MAC,
+	})
+}
+
+func (dev *vxlanDevice) AddRoute(subnet ip.IP4Net) error {
+	route := &netlink.Route{
+		Scope: netlink.SCOPE_UNIVERSE,
+		Dst:   subnet.ToIPNet(),
+		Gw:    subnet.IP.ToIP(),
 	}
 
-	log.Infof("calling NeighSet: %v, %v", ip, mac)
-	return netlink.NeighSet(&neigh)
+	log.Infof("calling RouteAdd: %s", subnet)
+	return netlink.RouteAdd(route)
+}
+
+func (dev *vxlanDevice) DelRoute(subnet ip.IP4Net) error {
+	route := &netlink.Route{
+		Scope: netlink.SCOPE_UNIVERSE,
+		Dst:   subnet.ToIPNet(),
+		Gw:    subnet.IP.ToIP(),
+	}
+	log.Infof("calling RouteDel: %s", subnet)
+	return netlink.RouteDel(route)
 }
 
 func vxlanLinksIncompat(l1, l2 netlink.Link) string {

+ 0 - 69
backend/vxlan/routes.go

@@ -1,69 +0,0 @@
-// Copyright 2015 CoreOS, Inc.
-//
-// 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 vxlan
-
-import (
-	"bytes"
-	"net"
-
-	"github.com/coreos/flannel/pkg/ip"
-)
-
-type route struct {
-	network ip.IP4Net
-	vtepIP  net.IP
-	vtepMAC net.HardwareAddr
-}
-
-type routes []route
-
-func (rts *routes) set(nw ip.IP4Net, vtepIP net.IP, vtepMAC net.HardwareAddr) {
-	for i, rt := range *rts {
-		if rt.network.Equal(nw) {
-			(*rts)[i].vtepIP = vtepIP
-			(*rts)[i].vtepMAC = vtepMAC
-			return
-		}
-	}
-	*rts = append(*rts, route{nw, vtepIP, vtepMAC})
-}
-
-func (rts *routes) remove(nw ip.IP4Net) {
-	for i, rt := range *rts {
-		if rt.network.Equal(nw) {
-			(*rts)[i] = (*rts)[len(*rts)-1]
-			(*rts) = (*rts)[0 : len(*rts)-1]
-			return
-		}
-	}
-}
-
-func (rts routes) findByNetwork(ipAddr ip.IP4) *route {
-	for i, rt := range rts {
-		if rt.network.Contains(ipAddr) {
-			return &rts[i]
-		}
-	}
-	return nil
-}
-
-func (rts routes) findByVtepMAC(mac net.HardwareAddr) *route {
-	for i, rt := range rts {
-		if bytes.Equal(rt.vtepMAC, mac) {
-			return &rts[i]
-		}
-	}
-	return nil
-}

+ 7 - 64
backend/vxlan/vxlan.go

@@ -22,8 +22,8 @@ import (
 	"time"
 
 	log "github.com/coreos/flannel/Godeps/_workspace/src/github.com/golang/glog"
-	"github.com/coreos/flannel/Godeps/_workspace/src/github.com/vishvananda/netlink"
 	"github.com/coreos/flannel/Godeps/_workspace/src/golang.org/x/net/context"
+
 	"github.com/coreos/flannel/backend"
 	"github.com/coreos/flannel/pkg/ip"
 	"github.com/coreos/flannel/subnet"
@@ -46,7 +46,6 @@ type VXLANBackend struct {
 	ctx    context.Context
 	cancel context.CancelFunc
 	wg     sync.WaitGroup
-	rts    routes
 }
 
 func New(sm subnet.Manager, network string, config *subnet.Config) backend.Backend {
@@ -148,12 +147,6 @@ func (vb *VXLANBackend) Run() {
 		vb.wg.Done()
 	}()
 
-	log.Info("Watching for L2/L3 misses")
-	misses := make(chan *netlink.Neigh, 100)
-	// Unfortunately MonitorMisses does not take a cancel channel
-	// as there's no wait to interrupt netlink socket recv
-	go vb.dev.MonitorMisses(misses)
-
 	log.Info("Watching for new subnet leases")
 	evts := make(chan []subnet.Event)
 	vb.wg.Add(1)
@@ -167,9 +160,6 @@ func (vb *VXLANBackend) Run() {
 
 	for {
 		select {
-		case miss := <-misses:
-			vb.handleMiss(miss)
-
 		case evtBatch := <-evts:
 			vb.handleSubnetEvents(evtBatch)
 
@@ -230,14 +220,13 @@ func (vb *VXLANBackend) handleSubnetEvents(batch []subnet.Event) {
 				log.Error("Error decoding subnet lease JSON: ", err)
 				continue
 			}
-
-			vb.rts.set(evt.Lease.Subnet, evt.Lease.Attrs.PublicIP.ToIP(), net.HardwareAddr(attrs.VtepMAC))
+			vb.dev.AddL2(neigh{IP: evt.Lease.Attrs.PublicIP, MAC: net.HardwareAddr(attrs.VtepMAC)})
+			vb.dev.AddL3(neigh{IP: evt.Lease.Subnet.IP, MAC: net.HardwareAddr(attrs.VtepMAC)})
+			vb.dev.AddRoute(evt.Lease.Subnet)
 
 		case subnet.SubnetRemoved:
 			log.Info("Subnet removed: ", evt.Lease.Subnet)
 
-			vb.rts.remove(evt.Lease.Subnet)
-
 			if evt.Lease.Attrs.BackendType != "vxlan" {
 				log.Warningf("Ignoring non-vxlan subnet: type=%v", evt.Lease.Attrs.BackendType)
 				continue
@@ -249,8 +238,10 @@ func (vb *VXLANBackend) handleSubnetEvents(batch []subnet.Event) {
 				continue
 			}
 
+			vb.dev.DelRoute(evt.Lease.Subnet)
 			if len(attrs.VtepMAC) > 0 {
-				vb.dev.DelL2(net.HardwareAddr(attrs.VtepMAC), evt.Lease.Attrs.PublicIP.ToIP())
+				vb.dev.DelL2(neigh{IP: evt.Lease.Attrs.PublicIP, MAC: net.HardwareAddr(attrs.VtepMAC)})
+				vb.dev.DelL3(neigh{IP: evt.Lease.Subnet.IP, MAC: net.HardwareAddr(attrs.VtepMAC)})
 			}
 
 		default:
@@ -258,51 +249,3 @@ func (vb *VXLANBackend) handleSubnetEvents(batch []subnet.Event) {
 		}
 	}
 }
-
-func (vb *VXLANBackend) handleMiss(miss *netlink.Neigh) {
-	switch {
-	case len(miss.IP) == 0 && len(miss.HardwareAddr) == 0:
-		log.Info("Ignoring nil miss")
-
-	case len(miss.IP) == 0:
-		vb.handleL2Miss(miss)
-
-	case len(miss.HardwareAddr) == 0:
-		vb.handleL3Miss(miss)
-
-	default:
-		log.Infof("Ignoring not a miss: %v, %v", miss.HardwareAddr, miss.IP)
-	}
-}
-
-func (vb *VXLANBackend) handleL2Miss(miss *netlink.Neigh) {
-	log.Infof("L2 miss: %v", miss.HardwareAddr)
-
-	rt := vb.rts.findByVtepMAC(miss.HardwareAddr)
-	if rt == nil {
-		log.Infof("Route for %v not found", miss.HardwareAddr)
-		return
-	}
-
-	if err := vb.dev.AddL2(miss.HardwareAddr, rt.vtepIP); err != nil {
-		log.Errorf("AddL2 failed: %v", err)
-	} else {
-		log.Info("AddL2 succeeded")
-	}
-}
-
-func (vb *VXLANBackend) handleL3Miss(miss *netlink.Neigh) {
-	log.Infof("L3 miss: %v", miss.IP)
-
-	rt := vb.rts.findByNetwork(ip.FromIP(miss.IP))
-	if rt == nil {
-		log.Infof("Route for %v not found", miss.IP)
-		return
-	}
-
-	if err := vb.dev.AddL3(miss.IP, rt.vtepMAC); err != nil {
-		log.Errorf("AddL3 failed: %v", err)
-	} else {
-		log.Info("AddL3 succeeded")
-	}
-}