Browse Source

Merge pull request #842 from chenchun/iptun

Add new ipip backend
Tom Denham 7 years ago
parent
commit
9ac598819f

+ 16 - 0
Documentation/backends.md

@@ -105,3 +105,19 @@ Route Limits: GCE [limits](https://cloud.google.com/compute/docs/resource-quotas
 [alicloud-vpc]: https://github.com/coreos/flannel/blob/master/Documentation/alicloud-vpc-backend.md
 [amazon-vpc]: https://github.com/coreos/flannel/blob/master/Documentation/aws-vpc-backend.md
 [gce-backend]: https://github.com/coreos/flannel/blob/master/Documentation/gce-backend.md
+
+
+### IPIP
+
+Use in-kernel IPIP to encapsulate the packets.
+
+IPIP kind of tunnels is the simplest one. It has the lowest overhead, but can incapsulate only IPv4 unicast traffic, so you will not be able to setup OSPF, RIP or any other multicast-based protocol.
+
+Type:
+* `Type` (string): `ipip`
+* `DirectRouting` (Boolean): Enable direct routes (like `host-gw`) when the hosts are on the same subnet. IPIP will only be used to encapsulate packets to hosts on different subnets. Defaults to `false`.
+
+Note that there may exist two ipip tunnel device `tunl0` and `flannel.ipip`, this is expected and it's not a bug.
+`tunl0` is automatically created per network namespace by ipip kernel module on modprobe ipip module. It is the namespace default IPIP device with attributes local=any and remote=any.
+When receiving IPIP protocol packets, kernel will forward them to tunl0 as a fallback device if it can't find an option whose local/remote attribute matches their src/dst ip address more precisely.
+`flannel.ipip` is created by flannel to achieve one to many ipip network.

+ 4 - 2
Makefile

@@ -22,7 +22,7 @@ K8S_VERSION=v1.6.6
 GOARM=7
 
 # These variables can be overridden by setting an environment variable.
-TEST_PACKAGES?=pkg/ip subnet subnet/etcdv2 network backend/hostgw
+TEST_PACKAGES?=pkg/ip subnet subnet/etcdv2 network backend
 TEST_PACKAGES_EXPANDED=$(TEST_PACKAGES:%=github.com/coreos/flannel/%)
 PACKAGES?=$(TEST_PACKAGES) network
 PACKAGES_EXPANDED=$(PACKAGES:%=github.com/coreos/flannel/%)
@@ -65,7 +65,9 @@ endif
 ### TESTING
 test: license-check gofmt
 	# Run the unit tests
-	docker run --cap-add=NET_ADMIN --rm -v $(shell pwd):/go/src/github.com/coreos/flannel golang:1.8.3 go test -v -cover $(TEST_PACKAGES_EXPANDED)
+	# NET_ADMIN capacity is required to do some network operation
+	# SYS_ADMIN capacity is required to create network namespace
+	docker run --cap-add=NET_ADMIN --cap-add=SYS_ADMIN --rm -v $(shell pwd):/go/src/github.com/coreos/flannel golang:1.8.3 go test -v -cover $(TEST_PACKAGES_EXPANDED)
 
 	# Test the docker-opts script
 	cd dist; ./mk-docker-opts_tests.sh

+ 0 - 17
backend/common.go

@@ -44,20 +44,3 @@ type Network interface {
 }
 
 type BackendCtor func(sm subnet.Manager, ei *ExternalInterface) (Backend, error)
-
-type SimpleNetwork struct {
-	SubnetLease *subnet.Lease
-	ExtIface    *ExternalInterface
-}
-
-func (n *SimpleNetwork) Lease() *subnet.Lease {
-	return n.SubnetLease
-}
-
-func (n *SimpleNetwork) MTU() int {
-	return n.ExtIface.Iface.MTU
-}
-
-func (_ *SimpleNetwork) Run(ctx context.Context) {
-	<-ctx.Done()
-}

+ 17 - 11
backend/hostgw/hostgw.go

@@ -21,6 +21,7 @@ import (
 	"github.com/coreos/flannel/backend"
 	"github.com/coreos/flannel/pkg/ip"
 	"github.com/coreos/flannel/subnet"
+	"github.com/vishvananda/netlink"
 	"golang.org/x/net/context"
 )
 
@@ -28,14 +29,9 @@ func init() {
 	backend.Register("host-gw", New)
 }
 
-const (
-	routeCheckRetries = 10
-)
-
 type HostgwBackend struct {
 	sm       subnet.Manager
 	extIface *backend.ExternalInterface
-	networks map[string]*network
 }
 
 func New(sm subnet.Manager, extIface *backend.ExternalInterface) (backend.Backend, error) {
@@ -46,16 +42,26 @@ func New(sm subnet.Manager, extIface *backend.ExternalInterface) (backend.Backen
 	be := &HostgwBackend{
 		sm:       sm,
 		extIface: extIface,
-		networks: make(map[string]*network),
 	}
-
 	return be, nil
 }
 
 func (be *HostgwBackend) RegisterNetwork(ctx context.Context, config *subnet.Config) (backend.Network, error) {
-	n := &network{
-		extIface: be.extIface,
-		sm:       be.sm,
+	n := &backend.RouteNetwork{
+		SimpleNetwork: backend.SimpleNetwork{
+			ExtIface: be.extIface,
+		},
+		SM:          be.sm,
+		BackendType: "host-gw",
+		Mtu:         be.extIface.Iface.MTU,
+		LinkIndex:   be.extIface.Iface.Index,
+	}
+	n.GetRoute = func(lease *subnet.Lease) *netlink.Route {
+		return &netlink.Route{
+			Dst:       lease.Subnet.ToIPNet(),
+			Gw:        lease.Attrs.PublicIP.ToIP(),
+			LinkIndex: n.LinkIndex,
+		}
 	}
 
 	attrs := subnet.LeaseAttrs{
@@ -66,7 +72,7 @@ func (be *HostgwBackend) RegisterNetwork(ctx context.Context, config *subnet.Con
 	l, err := be.sm.AcquireLease(ctx, &attrs)
 	switch err {
 	case nil:
-		n.lease = l
+		n.SubnetLease = l
 
 	case context.Canceled, context.DeadlineExceeded:
 		return nil, err

+ 204 - 0
backend/ipip/ipip.go

@@ -0,0 +1,204 @@
+// Copyright 2017 flannel 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 ipip
+
+import (
+	"encoding/json"
+	"fmt"
+	"syscall"
+
+	"github.com/coreos/flannel/backend"
+	"github.com/coreos/flannel/pkg/ip"
+	"github.com/coreos/flannel/subnet"
+	log "github.com/golang/glog"
+	"github.com/vishvananda/netlink"
+	"golang.org/x/net/context"
+)
+
+const (
+	backendType = "ipip"
+	tunnelName  = "flannel.ipip"
+)
+
+func init() {
+	backend.Register(backendType, New)
+}
+
+type IPIPBackend struct {
+	sm       subnet.Manager
+	extIface *backend.ExternalInterface
+}
+
+func New(sm subnet.Manager, extIface *backend.ExternalInterface) (backend.Backend, error) {
+	be := &IPIPBackend{
+		sm:       sm,
+		extIface: extIface,
+	}
+	return be, nil
+}
+
+func (be *IPIPBackend) RegisterNetwork(ctx context.Context, config *subnet.Config) (backend.Network, error) {
+	cfg := struct {
+		DirectRouting bool
+	}{}
+
+	if len(config.Backend) > 0 {
+		if err := json.Unmarshal(config.Backend, &cfg); err != nil {
+			return nil, fmt.Errorf("error decoding IPIP backend config: %v", err)
+		}
+	}
+
+	log.Infof("IPIP config: DirectRouting=%v", cfg.DirectRouting)
+
+	n := &backend.RouteNetwork{
+		SimpleNetwork: backend.SimpleNetwork{
+			ExtIface: be.extIface,
+		},
+		SM:          be.sm,
+		BackendType: backendType,
+	}
+
+	attrs := &subnet.LeaseAttrs{
+		PublicIP:    ip.FromIP(be.extIface.ExtAddr),
+		BackendType: backendType,
+	}
+
+	l, err := be.sm.AcquireLease(ctx, attrs)
+	switch err {
+	case nil:
+		n.SubnetLease = l
+	case context.Canceled, context.DeadlineExceeded:
+		return nil, err
+	default:
+		return nil, fmt.Errorf("failed to acquire lease: %v", err)
+	}
+
+	link, err := be.configureIPIPDevice(n.SubnetLease)
+
+	if err != nil {
+		return nil, err
+	}
+
+	n.Mtu = link.MTU
+	n.LinkIndex = link.Index
+	n.GetRoute = func(lease *subnet.Lease) *netlink.Route {
+		route := netlink.Route{
+			Dst:       lease.Subnet.ToIPNet(),
+			Gw:        lease.Attrs.PublicIP.ToIP(),
+			LinkIndex: n.LinkIndex,
+			Flags:     int(netlink.FLAG_ONLINK),
+		}
+
+		if cfg.DirectRouting {
+			dr, err := ip.DirectRouting(lease.Attrs.PublicIP.ToIP())
+
+			if err != nil {
+				log.Error(err)
+			}
+
+			if dr {
+				log.V(2).Infof("configure route to %v via direct routing", lease.Attrs.PublicIP.String())
+				route.LinkIndex = n.ExtIface.Iface.Index
+			}
+		}
+
+		return &route
+	}
+
+	return n, nil
+}
+
+func (be *IPIPBackend) configureIPIPDevice(lease *subnet.Lease) (*netlink.Iptun, error) {
+	// When modprobe ipip module, a tunl0 ipip device is created automatically per network namespace by ipip kernel module.
+	// It is the namespace default IPIP device with attributes local=any and remote=any.
+	// When receiving IPIP protocol packets, kernel will forward them to tunl0 as a fallback device
+	// if it can't find an option whose local/remote attribute matches their src/dst ip address more precisely.
+	// See https://github.com/torvalds/linux/blob/v4.13/net/ipv4/ip_tunnel.c#L85-L95 .
+
+	// So we have two options of creating ipip device, either rename tunl0 to flannel.ipip or create an new ipip device
+	// and set local attribute of flannel.ipip to distinguish these two devices.
+	// Considering tunl0 might be used by users, so choose the later option.
+	link := &netlink.Iptun{LinkAttrs: netlink.LinkAttrs{Name: tunnelName}, Local: be.extIface.IfaceAddr}
+
+	if err := netlink.LinkAdd(link); err != nil {
+		if err != syscall.EEXIST {
+			return nil, err
+		}
+
+		// The link already exists, so check existing link attributes.
+		existing, err := netlink.LinkByName(tunnelName)
+		if err != nil {
+			return nil, err
+		}
+
+		// If there's an exists device but it's not an ipip/IpTun device then get the user to fix it (flannel shouldn't
+		// delete a user's device)
+		if existing.Type() != "ipip" {
+			return nil, fmt.Errorf("%v isn't an ipip mode device, please remove device and try again", tunnelName)
+		}
+		ipip, ok := existing.(*netlink.Iptun)
+		if !ok {
+			return nil, fmt.Errorf("%s isn't an iptun device (%#v), please remove device and try again", tunnelName, link)
+		}
+
+		// local attribute may change if a user changes iface configuration, we need to recreate the device to ensure
+		// local and remote attribute is expected.
+		// local should be equal to the extIface.IfaceAddr and remote should be nil (or equal to 0.0.0.0)
+		if ipip.Local == nil || !ipip.Local.Equal(be.extIface.IfaceAddr) || (ipip.Remote != nil && ipip.Remote.String() != "0.0.0.0") {
+			log.Warningf("%q already exists with incompatable attributes: local=%v remote=%v; recreating device",
+				tunnelName, ipip.Local, ipip.Remote)
+
+			if err = netlink.LinkDel(existing); err != nil {
+				return nil, fmt.Errorf("failed to delete interface: %v", err)
+			}
+
+			if err = netlink.LinkAdd(link); err != nil {
+				return nil, fmt.Errorf("failed to create ipip interface: %v", err)
+			}
+		}
+	}
+
+	// Due to the extra 20 byte IP header that the tunnel will add to each packet,
+	// MTU size for both the workload and tunnel interfaces should be 20 bytes less than the selected iface (specified with the --iface option).
+	expectMTU := be.extIface.Iface.MTU - 20
+	if expectMTU <= 0 {
+		return nil, fmt.Errorf("MTU %d of iface %s is too small for ipip mode to work", be.extIface.Iface.MTU, be.extIface.Iface.Name)
+	}
+
+	oldMTU := link.Attrs().MTU
+	if oldMTU > expectMTU || oldMTU == 0 {
+		log.Infof("current MTU of %s is %d, setting it to %d", tunnelName, oldMTU, expectMTU)
+		err := netlink.LinkSetMTU(link, expectMTU)
+
+		if err != nil {
+			return nil, fmt.Errorf("failed to set %v MTU to %d: %v", tunnelName, expectMTU, err)
+		}
+		// change MTU as it will be written into /run/flannel/subnet.env
+		link.Attrs().MTU = expectMTU
+	}
+
+	// Ensure that the device has a /32 address so that no broadcast routes are created.
+	// This IP is just used as a source address for host to workload traffic (so
+	// the return path for the traffic has an address on the flannel network to use as the destination)
+	if err := ip.EnsureV4AddressOnLink(ip.IP4Net{IP: lease.Subnet.IP, PrefixLen: 32}, link); err != nil {
+		return nil, fmt.Errorf("failed to ensure address of interface %s: %s", link.Attrs().Name, err)
+	}
+
+	if err := netlink.LinkSetUp(link); err != nil {
+		return nil, fmt.Errorf("failed to set %v UP: %v", tunnelName, err)
+	}
+
+	return link, nil
+}

+ 24 - 0
backend/ipip/ipip_windows.go

@@ -0,0 +1,24 @@
+// Copyright 2017 flannel 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.
+// +build windows
+
+package ipip
+
+import (
+	log "github.com/golang/glog"
+)
+
+func init() {
+	log.Infof("ipip is not supported on this platform")
+}

+ 52 - 69
backend/hostgw/hostgw_network.go → backend/route_network.go

@@ -1,4 +1,4 @@
-// Copyright 2015 flannel authors
+// Copyright 2017 flannel authors
 //
 // Licensed under the Apache License, Version 2.0 (the "License");
 // you may not use this file except in compliance with the License.
@@ -11,9 +11,8 @@
 // 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.
-// +build !windows
 
-package hostgw
+package backend
 
 import (
 	"bytes"
@@ -22,49 +21,43 @@ import (
 	"time"
 
 	log "github.com/golang/glog"
-	"github.com/vishvananda/netlink"
 	"golang.org/x/net/context"
 
-	"github.com/coreos/flannel/backend"
 	"github.com/coreos/flannel/subnet"
+	"github.com/vishvananda/netlink"
 )
 
-type network struct {
-	name     string
-	extIface *backend.ExternalInterface
-	rl       []netlink.Route
-	lease    *subnet.Lease
-	sm       subnet.Manager
-}
-
-func (n *network) Lease() *subnet.Lease {
-	return n.lease
-}
+const (
+	routeCheckRetries = 10
+)
 
-func (n *network) MTU() int {
-	return n.extIface.Iface.MTU
+type RouteNetwork struct {
+	SimpleNetwork
+	BackendType string
+	routes      []netlink.Route
+	SM          subnet.Manager
+	GetRoute    func(lease *subnet.Lease) *netlink.Route
+	Mtu         int
+	LinkIndex   int
 }
 
-func (n *network) LinkIndex() int {
-	return n.extIface.Iface.Index
+func (n *RouteNetwork) MTU() int {
+	return n.Mtu
 }
 
-func (n *network) Run(ctx context.Context) {
+func (n *RouteNetwork) Run(ctx context.Context) {
 	wg := sync.WaitGroup{}
 
 	log.Info("Watching for new subnet leases")
 	evts := make(chan []subnet.Event)
 	wg.Add(1)
 	go func() {
-		subnet.WatchLeases(ctx, n.sm, n.lease, evts)
+		subnet.WatchLeases(ctx, n.SM, n.SubnetLease, evts)
 		wg.Done()
 	}()
 
-	// Store a list of routes, initialized to capacity of 10.
-	n.rl = make([]netlink.Route, 0, 10)
+	n.routes = make([]netlink.Route, 0, 10)
 	wg.Add(1)
-
-	// Start a goroutine which periodically checks that the right routes are created
 	go func() {
 		n.routeCheck(ctx)
 		wg.Done()
@@ -83,69 +76,56 @@ func (n *network) Run(ctx context.Context) {
 	}
 }
 
-func (n *network) handleSubnetEvents(batch []subnet.Event) {
+func (n *RouteNetwork) handleSubnetEvents(batch []subnet.Event) {
 	for _, evt := range batch {
 		switch evt.Type {
 		case subnet.EventAdded:
 			log.Infof("Subnet added: %v via %v", evt.Lease.Subnet, evt.Lease.Attrs.PublicIP)
 
-			if evt.Lease.Attrs.BackendType != "host-gw" {
-				log.Warningf("Ignoring non-host-gw subnet: type=%v", evt.Lease.Attrs.BackendType)
+			if evt.Lease.Attrs.BackendType != n.BackendType {
+				log.Warningf("Ignoring non-%v subnet: type=%v", n.BackendType, evt.Lease.Attrs.BackendType)
 				continue
 			}
+			route := n.GetRoute(&evt.Lease)
 
-			route := netlink.Route{
-				Dst:       evt.Lease.Subnet.ToIPNet(),
-				Gw:        evt.Lease.Attrs.PublicIP.ToIP(),
-				LinkIndex: n.LinkIndex(),
-			}
-
-			// Always add the route to the route list.
-			n.addToRouteList(route)
-
+			n.addToRouteList(*route)
 			// Check if route exists before attempting to add it
-			routeList, err := netlink.RouteListFiltered(netlink.FAMILY_V4, &netlink.Route{
-				Dst: route.Dst,
-			}, netlink.RT_FILTER_DST)
+			routeList, err := netlink.RouteListFiltered(netlink.FAMILY_V4, &netlink.Route{Dst: route.Dst}, netlink.RT_FILTER_DST)
 			if err != nil {
 				log.Warningf("Unable to list routes: %v", err)
 			}
-			//   Check match on Dst for match on Gw
-			if len(routeList) > 0 && !routeList[0].Gw.Equal(route.Gw) {
-				// Same Dst different Gw. Remove it, correct route will be added below.
-				log.Warningf("Replacing existing route to %v via %v with %v via %v.", evt.Lease.Subnet, routeList[0].Gw, evt.Lease.Subnet, evt.Lease.Attrs.PublicIP)
+
+			if len(routeList) > 0 && !routeEqual(routeList[0], *route) {
+				// Same Dst different Gw or different link index. Remove it, correct route will be added below.
+				log.Warningf("Replacing existing route to %v via %v dev index %d with %v via %v dev index %d.", evt.Lease.Subnet, routeList[0].Gw, routeList[0].LinkIndex, evt.Lease.Subnet, evt.Lease.Attrs.PublicIP, route.LinkIndex)
 				if err := netlink.RouteDel(&routeList[0]); err != nil {
 					log.Errorf("Error deleting route to %v: %v", evt.Lease.Subnet, err)
 					continue
 				}
 				n.removeFromRouteList(routeList[0])
 			}
-			if len(routeList) > 0 && routeList[0].Gw.Equal(route.Gw) {
+
+			if len(routeList) > 0 && routeEqual(routeList[0], *route) {
 				// Same Dst and same Gw, keep it and do not attempt to add it.
-				log.Infof("Route to %v via %v already exists, skipping.", evt.Lease.Subnet, evt.Lease.Attrs.PublicIP)
-			} else if err := netlink.RouteAdd(&route); err != nil {
-				log.Errorf("Error adding route to %v via %v: %v", evt.Lease.Subnet, evt.Lease.Attrs.PublicIP, err)
+				log.Infof("Route to %v via %v dev index %d already exists, skipping.", evt.Lease.Subnet, evt.Lease.Attrs.PublicIP, routeList[0].LinkIndex)
+			} else if err := netlink.RouteAdd(route); err != nil {
+				log.Errorf("Error adding route to %v via %v dev index %d: %v", evt.Lease.Subnet, evt.Lease.Attrs.PublicIP, route.LinkIndex, err)
 				continue
 			}
 
 		case subnet.EventRemoved:
 			log.Info("Subnet removed: ", evt.Lease.Subnet)
 
-			if evt.Lease.Attrs.BackendType != "host-gw" {
-				log.Warningf("Ignoring non-host-gw subnet: type=%v", evt.Lease.Attrs.BackendType)
+			if evt.Lease.Attrs.BackendType != n.BackendType {
+				log.Warningf("Ignoring non-%v subnet: type=%v", n.BackendType, evt.Lease.Attrs.BackendType)
 				continue
 			}
 
-			route := netlink.Route{
-				Dst:       evt.Lease.Subnet.ToIPNet(),
-				Gw:        evt.Lease.Attrs.PublicIP.ToIP(),
-				LinkIndex: n.LinkIndex(),
-			}
-
+			route := n.GetRoute(&evt.Lease)
 			// Always remove the route from the route list.
-			n.removeFromRouteList(route)
+			n.removeFromRouteList(*route)
 
-			if err := netlink.RouteDel(&route); err != nil {
+			if err := netlink.RouteDel(route); err != nil {
 				log.Errorf("Error deleting route to %v: %v", evt.Lease.Subnet, err)
 				continue
 			}
@@ -156,25 +136,25 @@ func (n *network) handleSubnetEvents(batch []subnet.Event) {
 	}
 }
 
-func (n *network) addToRouteList(route netlink.Route) {
-	for _, r := range n.rl {
+func (n *RouteNetwork) addToRouteList(route netlink.Route) {
+	for _, r := range n.routes {
 		if routeEqual(r, route) {
 			return
 		}
 	}
-	n.rl = append(n.rl, route)
+	n.routes = append(n.routes, route)
 }
 
-func (n *network) removeFromRouteList(route netlink.Route) {
-	for index, r := range n.rl {
+func (n *RouteNetwork) removeFromRouteList(route netlink.Route) {
+	for index, r := range n.routes {
 		if routeEqual(r, route) {
-			n.rl = append(n.rl[:index], n.rl[index+1:]...)
+			n.routes = append(n.routes[:index], n.routes[index+1:]...)
 			return
 		}
 	}
 }
 
-func (n *network) routeCheck(ctx context.Context) {
+func (n *RouteNetwork) routeCheck(ctx context.Context) {
 	for {
 		select {
 		case <-ctx.Done():
@@ -185,10 +165,10 @@ func (n *network) routeCheck(ctx context.Context) {
 	}
 }
 
-func (n *network) checkSubnetExistInRoutes() {
+func (n *RouteNetwork) checkSubnetExistInRoutes() {
 	routeList, err := netlink.RouteList(nil, netlink.FAMILY_V4)
 	if err == nil {
-		for _, route := range n.rl {
+		for _, route := range n.routes {
 			exist := false
 			for _, r := range routeList {
 				if r.Dst == nil {
@@ -199,6 +179,7 @@ func (n *network) checkSubnetExistInRoutes() {
 					break
 				}
 			}
+
 			if !exist {
 				if err := netlink.RouteAdd(&route); err != nil {
 					if nerr, ok := err.(net.Error); !ok {
@@ -216,7 +197,9 @@ func (n *network) checkSubnetExistInRoutes() {
 }
 
 func routeEqual(x, y netlink.Route) bool {
-	if x.Dst.IP.Equal(y.Dst.IP) && x.Gw.Equal(y.Gw) && bytes.Equal(x.Dst.Mask, y.Dst.Mask) {
+	// For ipip backend, when enabling directrouting, link index of some routes may change
+	// For both ipip and host-gw backend, link index may also change if updating ExtIface
+	if x.Dst.IP.Equal(y.Dst.IP) && x.Gw.Equal(y.Gw) && bytes.Equal(x.Dst.Mask, y.Dst.Mask) && x.LinkIndex == y.LinkIndex {
 		return true
 	}
 	return false

+ 29 - 22
backend/hostgw/hostgw_network_test.go → backend/route_network_test.go

@@ -1,4 +1,4 @@
-// Copyright 2015 flannel authors
+// Copyright 2017 flannel authors
 //
 // Licensed under the Apache License, Version 2.0 (the "License");
 // you may not use this file except in compliance with the License.
@@ -13,28 +13,21 @@
 // limitations under the License.
 // +build !windows
 
-package hostgw
+package backend
 
 import (
 	"net"
-	"runtime"
 	"testing"
 
-	"github.com/coreos/flannel/backend"
 	"github.com/coreos/flannel/pkg/ip"
+	"github.com/coreos/flannel/pkg/ns"
 	"github.com/coreos/flannel/subnet"
 	"github.com/vishvananda/netlink"
-	"github.com/vishvananda/netns"
 )
 
 func TestRouteCache(t *testing.T) {
-	runtime.LockOSThread()
-	defer runtime.UnlockOSThread()
-	origns, _ := netns.Get()
-	defer origns.Close()
-	newns, _ := netns.New()
-	netns.Set(newns)
-	defer newns.Close()
+	teardown := ns.SetUpNetlinkTest(t)
+	defer teardown()
 
 	lo, err := netlink.LinkByName("lo")
 	if err != nil {
@@ -46,26 +39,40 @@ func TestRouteCache(t *testing.T) {
 	if err := netlink.LinkSetUp(lo); err != nil {
 		t.Fatal(err)
 	}
-	nw := network{extIface: &backend.ExternalInterface{Iface: &net.Interface{Index: lo.Attrs().Index}}}
+	nw := RouteNetwork{
+		SimpleNetwork: SimpleNetwork{
+			ExtIface: &ExternalInterface{Iface: &net.Interface{Index: lo.Attrs().Index}},
+		},
+		BackendType: "host-gw",
+		LinkIndex:   lo.Attrs().Index,
+	}
+	nw.GetRoute = func(lease *subnet.Lease) *netlink.Route {
+		return &netlink.Route{
+			Dst:       lease.Subnet.ToIPNet(),
+			Gw:        lease.Attrs.PublicIP.ToIP(),
+			LinkIndex: nw.LinkIndex,
+		}
+	}
 	gw1, gw2 := ip.FromIP(net.ParseIP("127.0.0.1")), ip.FromIP(net.ParseIP("127.0.0.2"))
 	subnet1 := ip.IP4Net{IP: ip.FromIP(net.ParseIP("192.168.0.0")), PrefixLen: 24}
 	nw.handleSubnetEvents([]subnet.Event{
-		{Type: subnet.EventAdded, Lease: subnet.Lease{Subnet: subnet1, Attrs: subnet.LeaseAttrs{PublicIP: gw1, BackendType: "host-gw"}}},
+		{Type: subnet.EventAdded, Lease: subnet.Lease{
+			Subnet: subnet1, Attrs: subnet.LeaseAttrs{PublicIP: gw1, BackendType: "host-gw"}}},
 	})
-	if len(nw.rl) != 1 {
-		t.Fatal(nw.rl)
+	if len(nw.routes) != 1 {
+		t.Fatal(nw.routes)
 	}
-	if !routeEqual(nw.rl[0], netlink.Route{Dst: subnet1.ToIPNet(), Gw: gw1.ToIP()}) {
-		t.Fatal(nw.rl[0])
+	if !routeEqual(nw.routes[0], netlink.Route{Dst: subnet1.ToIPNet(), Gw: gw1.ToIP(), LinkIndex: lo.Attrs().Index}) {
+		t.Fatal(nw.routes[0])
 	}
 	// change gateway of previous route
 	nw.handleSubnetEvents([]subnet.Event{
 		{Type: subnet.EventAdded, Lease: subnet.Lease{
 			Subnet: subnet1, Attrs: subnet.LeaseAttrs{PublicIP: gw2, BackendType: "host-gw"}}}})
-	if len(nw.rl) != 1 {
-		t.Fatal(nw.rl)
+	if len(nw.routes) != 1 {
+		t.Fatal(nw.routes)
 	}
-	if !routeEqual(nw.rl[0], netlink.Route{Dst: subnet1.ToIPNet(), Gw: gw2.ToIP()}) {
-		t.Fatal(nw.rl[0])
+	if !routeEqual(nw.routes[0], netlink.Route{Dst: subnet1.ToIPNet(), Gw: gw2.ToIP(), LinkIndex: lo.Attrs().Index}) {
+		t.Fatal(nw.routes[0])
 	}
 }

+ 11 - 19
backend/hostgw/hostgw_network_windows.go → backend/simple_network.go

@@ -1,4 +1,4 @@
-// Copyright 2015 flannel authors
+// Copyright 2017 flannel authors
 //
 // Licensed under the Apache License, Version 2.0 (the "License");
 // you may not use this file except in compliance with the License.
@@ -11,36 +11,28 @@
 // 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.
-// +build windows
 
-package hostgw
+package backend
 
 import (
 	"golang.org/x/net/context"
 
-	"github.com/coreos/flannel/backend"
 	"github.com/coreos/flannel/subnet"
-
-	netroute "github.com/rakelkar/gonetsh/netroute"
 )
 
-type network struct {
-	name      string
-	extIface  *backend.ExternalInterface
-	linkIndex int
-	rl        []netroute.Route
-	lease     *subnet.Lease
-	sm        subnet.Manager
+type SimpleNetwork struct {
+	SubnetLease *subnet.Lease
+	ExtIface    *ExternalInterface
 }
 
-func (n *network) Lease() *subnet.Lease {
-	return n.lease
+func (n *SimpleNetwork) Lease() *subnet.Lease {
+	return n.SubnetLease
 }
 
-func (n *network) MTU() int {
-	return n.extIface.Iface.MTU
+func (n *SimpleNetwork) MTU() int {
+	return n.ExtIface.Iface.MTU
 }
 
-func (n *network) Run(ctx context.Context) {
-
+func (_ *SimpleNetwork) Run(ctx context.Context) {
+	<-ctx.Done()
 }

+ 2 - 24
backend/vxlan/device.go

@@ -107,30 +107,8 @@ func ensureLink(vxlan *netlink.Vxlan) (*netlink.Vxlan, error) {
 }
 
 func (dev *vxlanDevice) Configure(ipn ip.IP4Net) error {
-	addr := netlink.Addr{IPNet: ipn.ToIPNet()}
-	existingAddrs, err := netlink.AddrList(dev.link, netlink.FAMILY_V4)
-	if err != nil {
-		return err
-	}
-
-	// flannel will never make this happen. This situation can only be caused by a user, so get them to sort it out.
-	if len(existingAddrs) > 1 {
-		return fmt.Errorf("link has incompatible addresses. Remove additional addresses and try again. %#v", dev.link)
-	}
-
-	// If the device has an incompatible address then delete it. This can happen if the lease changes for example.
-	if len(existingAddrs) == 1 && !existingAddrs[0].Equal(addr) {
-		if err := netlink.AddrDel(dev.link, &existingAddrs[0]); err != nil {
-			return fmt.Errorf("failed to remove IP address %s from %s: %s", ipn.String(), dev.link.Attrs().Name, err)
-		}
-		existingAddrs = []netlink.Addr{}
-	}
-
-	// Actually add the desired address to the interface if needed.
-	if len(existingAddrs) == 0 {
-		if err := netlink.AddrAdd(dev.link, &addr); err != nil {
-			return fmt.Errorf("failed to add IP address %s to %s: %s", ipn.String(), dev.link.Attrs().Name, err)
-		}
+	if err := ip.EnsureV4AddressOnLink(ipn, dev.link); err != nil {
+		return fmt.Errorf("failed to ensure address of interface %s: %s", dev.link.Attrs().Name, err)
 	}
 
 	if err := netlink.LinkSetUp(dev.link); err != nil {

+ 4 - 8
backend/vxlan/vxlan_network.go

@@ -118,14 +118,10 @@ func (nw *network) handleSubnetEvents(batch []subnet.Event) {
 		}
 		var directRoutingOK = false
 		if nw.dev.directRouting {
-			routes, err := netlink.RouteGet(attrs.PublicIP.ToIP())
-			if err != nil {
-				log.Errorf("Couldn't lookup route to %v: %v", attrs.PublicIP, err)
-				continue
-			}
-			if len(routes) == 1 && routes[0].Gw == nil {
-				// There is only a single route and there's no gateway (i.e. it's directly connected)
-				directRoutingOK = true
+			if dr, err := ip.DirectRouting(attrs.PublicIP.ToIP()); err != nil {
+				log.Error(err)
+			} else {
+				directRoutingOK = dr
 			}
 		}
 

+ 6 - 0
dist/functional-test-k8s.sh

@@ -121,6 +121,12 @@ test_host-gw() {
     pings
 }
 
+test_ipip() {
+    start_flannel ipip
+    create_ping_dest # creates ping_dest1 and ping_dest2 variables
+    pings
+}
+
 test_public-ip-overwrite(){
   docker exec flannel-e2e-k8s-apiserver kubectl annotate node flannel1 \
     flannel.alpha.coreos.com/public-ip-overwrite=172.18.0.2 >/dev/null 2>&1

+ 12 - 0
dist/functional-test.sh

@@ -98,6 +98,12 @@ test_host-gw_ping() {
     pings
 }
 
+test_ipip_ping() {
+    write_config_etcd ipip
+    create_ping_dest # creates ping_dest1 and ping_dest2 variables
+    pings
+}
+
 pings() {
     # ping in both directions
 	assert "docker exec --privileged flannel-e2e-test-flannel1 /bin/ping -c 3 $ping_dest2" "Host 1 cannot ping host 2"
@@ -125,6 +131,12 @@ test_udp_perf() {
 }
 fi
 
+test_ipip_perf() {
+    write_config_etcd ipip
+    create_ping_dest
+    perf
+}
+
 perf() {
     # Perf test - run iperf server on flannel1 and client on flannel2
     docker rm -f flannel-e2e-test-flannel1-iperf 2>/dev/null

+ 1 - 0
main.go

@@ -53,6 +53,7 @@ import (
 	_ "github.com/coreos/flannel/backend/extension"
 	_ "github.com/coreos/flannel/backend/gce"
 	_ "github.com/coreos/flannel/backend/hostgw"
+	_ "github.com/coreos/flannel/backend/ipip"
 	_ "github.com/coreos/flannel/backend/udp"
 	_ "github.com/coreos/flannel/backend/vxlan"
 	"github.com/coreos/go-systemd/daemon"

+ 45 - 0
pkg/ip/iface.go

@@ -17,6 +17,7 @@ package ip
 
 import (
 	"errors"
+	"fmt"
 	"net"
 	"syscall"
 
@@ -116,3 +117,47 @@ func GetInterfaceByIP(ip net.IP) (*net.Interface, error) {
 
 	return nil, errors.New("No interface with given IP found")
 }
+
+func DirectRouting(ip net.IP) (bool, error) {
+	routes, err := netlink.RouteGet(ip)
+	if err != nil {
+		return false, fmt.Errorf("couldn't lookup route to %v: %v", ip, err)
+	}
+	if len(routes) == 1 && routes[0].Gw == nil {
+		// There is only a single route and there's no gateway (i.e. it's directly connected)
+		return true, nil
+	}
+	return false, nil
+}
+
+// EnsureV4AddressOnLink ensures that there is only one v4 Addr on `link` and it equals `ipn`.
+// If there exist multiple addresses on link, it returns an error message to tell callers to remove additional address.
+func EnsureV4AddressOnLink(ipn IP4Net, link netlink.Link) error {
+	addr := netlink.Addr{IPNet: ipn.ToIPNet()}
+	existingAddrs, err := netlink.AddrList(link, netlink.FAMILY_V4)
+	if err != nil {
+		return err
+	}
+
+	// flannel will never make this happen. This situation can only be caused by a user, so get them to sort it out.
+	if len(existingAddrs) > 1 {
+		return fmt.Errorf("link has incompatible addresses. Remove additional addresses and try again. %#v", link)
+	}
+
+	// If the device has an incompatible address then delete it. This can happen if the lease changes for example.
+	if len(existingAddrs) == 1 && !existingAddrs[0].Equal(addr) {
+		if err := netlink.AddrDel(link, &existingAddrs[0]); err != nil {
+			return fmt.Errorf("failed to remove IP address %s from %s: %s", ipn.String(), link.Attrs().Name, err)
+		}
+		existingAddrs = []netlink.Addr{}
+	}
+
+	// Actually add the desired address to the interface if needed.
+	if len(existingAddrs) == 0 {
+		if err := netlink.AddrAdd(link, &addr); err != nil {
+			return fmt.Errorf("failed to add IP address %s to %s: %s", ipn.String(), link.Attrs().Name, err)
+		}
+	}
+
+	return nil
+}

+ 54 - 0
pkg/ip/iface_test.go

@@ -0,0 +1,54 @@
+// Copyright 2017 flannel 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 ip
+
+import (
+	"net"
+	"testing"
+
+	"github.com/coreos/flannel/pkg/ns"
+	"github.com/vishvananda/netlink"
+)
+
+func TestEnsureV4AddressOnLink(t *testing.T) {
+	teardown := ns.SetUpNetlinkTest(t)
+	defer teardown()
+	lo, err := netlink.LinkByName("lo")
+	if err != nil {
+		t.Fatal(err)
+	}
+	if err := netlink.LinkSetUp(lo); err != nil {
+		t.Fatal(err)
+	}
+	// check changing address
+	if err := EnsureV4AddressOnLink(IP4Net{IP: FromIP(net.ParseIP("127.0.0.2")), PrefixLen: 24}, lo); err != nil {
+		t.Fatal(err)
+	}
+	addrs, err := netlink.AddrList(lo, netlink.FAMILY_V4)
+	if err != nil {
+		t.Fatal(err)
+	}
+	if len(addrs) != 1 || addrs[0].String() != "127.0.0.2/24 lo" {
+		t.Fatalf("addrs %v is not expected", addrs)
+	}
+
+	// check changing address if there exist multiple addresses
+	if err := netlink.AddrAdd(lo, &netlink.Addr{IPNet: &net.IPNet{IP: net.ParseIP("127.0.0.3"), Mask: net.CIDRMask(24, 32)}}); err != nil {
+		t.Fatal(err)
+	}
+	if err := EnsureV4AddressOnLink(IP4Net{IP: FromIP(net.ParseIP("127.0.0.2")), PrefixLen: 24}, lo); err == nil {
+		t.Fatal("EnsureV4AddressOnLink should return error if there exist multiple address on link")
+	}
+}

+ 38 - 0
pkg/ns/ns.go

@@ -0,0 +1,38 @@
+// Copyright 2017 flannel 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 ns
+
+import (
+	"runtime"
+	"testing"
+
+	"github.com/vishvananda/netns"
+)
+
+func SetUpNetlinkTest(t *testing.T) func() {
+	// new temporary namespace so we don't pollute the host
+	// lock thread since the namespace is thread local
+	runtime.LockOSThread()
+	var err error
+	ns, err := netns.New()
+	if err != nil {
+		t.Fatalf("Failed to create newns: %v", err)
+	}
+
+	return func() {
+		ns.Close()
+		runtime.UnlockOSThread()
+	}
+}