瀏覽代碼

Lots of fixes getting it ready to merge (tests/docs)

- Add e2e tests
- Documentation
- Comments and logging tweaks
- Remove the "initial events" code, which means that stale polciy won't
  be removed.
- Various other small changes
- Get it compiling on windows
Tom Denham 7 年之前
父節點
當前提交
59ab6afa56

+ 23 - 0
Documentation/backends.md

@@ -121,3 +121,26 @@ Note that there may exist two ipip tunnel device `tunl0` and `flannel.ipip`, thi
 `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.
+
+### IPSec
+
+Use in-kernel IPSec to encapsulate and encrypt the packets.
+
+[Strongswan](https://www.strongswan.org) is used at the IKEv2 daemon. A single pre-shared key is used for the initial key exchange between hosts and then Strongswan ensures that keys are rotated at regular intervals. 
+
+Type:
+* `Type` (string): `ipsec`
+* `PSK` (Boolean): Required. The pre shared key to use. It needs to be at least 96 characters long. One method for generating this key is to run `dd if=/dev/urandom count=48 bs=1 status=none | xxd -p -c 48`
+* `UDPEncap` (string): Optional, defaults to false. Forces the use UDP encapsulation of packets which can help with some NAT gateways.
+* `ESPProposal` (string): Optional, defaults to `aes128gcm16-sha256-prfsha256-ecp256`. Change this string to choose another ESP Proposal.
+
+#### Troubleshooting
+Logging
+* When flannel is run from a container, the Strongswan tools are installed. `swanctl` can be used for interacting with the charon and it provides a logs command.. 
+* Charon logs are also written to the stdout of the flannel process. 
+
+Troubleshooting
+* `ip xfrm state` can be used to interact with the kernel's security association database. This can be used to show the current security associations (SA) and whether a host is successfully establishing ipsec connections to other hosts.
+* `ip xfrm policy` can be used to show the installed polcies. Flannel installs three policies for each host it connects to. 
+
+Flannel will not restore policies that are manually deleted (unless flannel is restarted). It will also not delete stale policies on startup. They can be removed by rebooting your host or by removing all ipsec state with `ip xfrm state flush && ip xfrm policy flush` and restarting flannel.

+ 28 - 37
backend/ipsec/handle_charon.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,15 +11,12 @@
 // 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 ipsec
 
 import (
 	"fmt"
-	"github.com/bronze1man/goStrongswanVici"
-	"github.com/coreos/flannel/subnet"
-	log "github.com/golang/glog"
-	"golang.org/x/net/context"
 	"net"
 	"os"
 	"os/exec"
@@ -27,9 +24,12 @@ import (
 	"sync"
 	"syscall"
 	"time"
-)
 
-const defaultViciUri = "unix:///var/run/charon.vici"
+	"github.com/bronze1man/goStrongswanVici"
+	"github.com/coreos/flannel/subnet"
+	log "github.com/golang/glog"
+	"golang.org/x/net/context"
+)
 
 type Uri struct {
 	network, address string
@@ -41,45 +41,35 @@ type CharonIKEDaemon struct {
 	ctx         context.Context
 }
 
-func NewCharonIKEDaemon(ctx context.Context, wg sync.WaitGroup, charonExecutablePath string,
-	charonViciUri string, espProposal string) (*CharonIKEDaemon, error) {
+func NewCharonIKEDaemon(ctx context.Context, wg sync.WaitGroup, espProposal string) (*CharonIKEDaemon, error) {
 
 	charon := &CharonIKEDaemon{ctx: ctx, espProposal: espProposal}
 
-	if charonViciUri == "" {
-		charonViciUri = defaultViciUri
-	}
-
-	log.Infof("Using charon at: %s", charonViciUri)
-	addr := strings.Split(charonViciUri, "://")
+	addr := strings.Split("unix:///var/run/charon.vici", "://")
 	charon.viciUri = Uri{addr[0], addr[1]}
 
-	log.Infof("Using ESP proposal: %s", espProposal)
-	if charonExecutablePath != "" {
-		cmd, err := charon.runBundled(charonExecutablePath)
+	cmd, err := charon.runBundled("/usr/lib/strongswan/charon")
 
-		if err != nil {
-			log.Errorf("Error starting charon daemon: %v", err)
-			return nil, err
-		} else {
-			log.Info("Charon daemon started")
+	if err != nil {
+		log.Errorf("Error starting charon daemon: %v", err)
+		return nil, err
+	} else {
+		log.Info("Charon daemon started")
+	}
+	wg.Add(1)
+	go func() {
+		select {
+		case <-ctx.Done():
+			cmd.Process.Signal(syscall.SIGTERM)
+			cmd.Wait()
+			log.Infof("Stopped charon daemon")
+			wg.Done()
 		}
-		wg.Add(1)
-		go func() {
-			select {
-			case <-ctx.Done():
-				cmd.Process.Signal(syscall.SIGTERM)
-				log.Infof("Stopped charon daemon")
-				wg.Done()
-				return
-			}
-		}()
-	}
+	}()
 	return charon, nil
 }
 
-func (charon *CharonIKEDaemon) getClient(wait bool) (
-	client *goStrongswanVici.ClientConn, err error) {
+func (charon *CharonIKEDaemon) getClient(wait bool) (client *goStrongswanVici.ClientConn, err error) {
 	for {
 		socket_conn, err := net.Dial(charon.viciUri.network, charon.viciUri.address)
 		if err == nil {
@@ -113,8 +103,9 @@ func (charon *CharonIKEDaemon) runBundled(execPath string) (cmd *exec.Cmd, err e
 		SysProcAttr: &syscall.SysProcAttr{
 			Pdeathsig: syscall.SIGTERM,
 		},
+		Stdout: os.Stdout,
+		Stderr: os.Stderr,
 	}
-	cmd.Stderr = os.Stderr
 	err = cmd.Start()
 	return
 }

+ 2 - 6
backend/ipsec/handle_xfrm.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,13 +11,13 @@
 // 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 ipsec
 
 import (
 	"fmt"
 	"net"
-	"syscall"
 
 	log "github.com/golang/glog"
 	"github.com/vishvananda/netlink"
@@ -89,7 +89,3 @@ func DeleteXFRMPolicy(localSubnet, remoteSubnet *net.IPNet, localPublicIP, remot
 
 	return nil
 }
-
-func GetIPSECPolicies() ([]netlink.XfrmPolicy, error) {
-	return netlink.XfrmPolicyList(syscall.AF_INET)
-}

+ 24 - 14
backend/ipsec/ipsec.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,23 +11,39 @@
 // 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 ipsec
 
 import (
 	"encoding/json"
 	"fmt"
+	"sync"
+
 	log "github.com/golang/glog"
 	"golang.org/x/net/context"
-	"sync"
 
 	"github.com/coreos/flannel/backend"
 	"github.com/coreos/flannel/pkg/ip"
 	"github.com/coreos/flannel/subnet"
 )
 
-var CharonExecutablePath string
-var CharonViciUri string
+/*
+	Flannel's approach to IPSec uses Strongswan to handle the key exchange (using IKEv2) and the kernel to handle the
+	actual encryption.
+
+	Strongswan's "charon" is bundled in the flannel container. Flannel runs it as a child process when the ipsec backend
+	is selected and communicates with it using the "VICI" interface. Strongswan ships a utility "swanctl" which also
+	uses the VICI interface. This utility is bundled in the flannel container and can help with debugging.
+
+	The file "handle_charon.go" contains the logic for working with the charon. It supports creating a "CharonIKEDaemon"
+	which supports loading the PSK into the charon and adding and removing connections.
+
+	The file "handle_xfrm.go" contains functions for adding and removing the ipsec polcies.
+
+	ipsec_network.go ties it all together, loading the PSK for current host on startu and as new hosts are added and
+	removed it, adds/removes the PSK and connection details to strongswan and adds/remove the policy to the kernel.
+*/
 
 const (
 	defaultESPProposal = "aes128gcm16-sha256-prfsha256-ecp256"
@@ -66,7 +82,6 @@ func (be *IPSECBackend) RegisterNetwork(
 	}
 
 	if len(config.Backend) > 0 {
-		log.Info("i.config.backend length > 0")
 		if err := json.Unmarshal(config.Backend, &cfg); err != nil {
 			return nil, fmt.Errorf("error decoding IPSEC backend config: %v", err)
 		}
@@ -74,10 +89,12 @@ func (be *IPSECBackend) RegisterNetwork(
 
 	if len(cfg.PSK) < minPasswordLength {
 		return nil, fmt.Errorf(
-			"config error, password should be at least %s characters long",
+			"config error, password should be at least %d characters long",
 			minPasswordLength)
 	}
 
+	log.Infof("IPSec config: UDPEncap=%v ESPProposal=%s", cfg.UDPEncap, cfg.ESPProposal)
+
 	attrs := subnet.LeaseAttrs{
 		PublicIP:    ip.FromIP(be.extIface.ExtAddr),
 		BackendType: "ipsec",
@@ -95,17 +112,10 @@ func (be *IPSECBackend) RegisterNetwork(
 		return nil, fmt.Errorf("failed to acquire lease: %v", err)
 	}
 
-	ikeDaemon, err := NewCharonIKEDaemon(ctx, wg, CharonExecutablePath, CharonViciUri,
-		cfg.ESPProposal)
+	ikeDaemon, err := NewCharonIKEDaemon(ctx, wg, cfg.ESPProposal)
 	if err != nil {
 		return nil, fmt.Errorf("error creating CharonIKEDaemon struct: %v", err)
 	}
 
-	log.Info("UDPEncap: ", cfg.UDPEncap)
-
 	return newNetwork(be.sm, be.extIface, cfg.UDPEncap, cfg.PSK, ikeDaemon, l)
 }
-
-func (be *IPSECBackend) Run(ctx context.Context) {
-	<-ctx.Done()
-}

+ 6 - 88
backend/ipsec/ipsec_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,6 +11,7 @@
 // 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 ipsec
 
@@ -19,7 +20,6 @@ import (
 	"net"
 	"strconv"
 	"sync"
-	"time"
 
 	log "github.com/golang/glog"
 	"github.com/vishvananda/netlink"
@@ -42,7 +42,8 @@ const (
 	*/
 	ipsecOverhead    = 77
 	udpEncapOverhead = 8
-	defaultReqID     = 11
+
+	defaultReqID = 11
 )
 
 type network struct {
@@ -92,101 +93,18 @@ func (n *network) Run(ctx context.Context) {
 		wg.Done()
 	}()
 
-	initialEvtsBatch := <-evts
-	for {
-		err := n.handleInitialSubnetEvents(initialEvtsBatch)
-		if err == nil {
-			break
-		}
-
-		log.Error(err, " Retrying")
-		time.Sleep(time.Second)
-	}
-
 	for {
 		select {
 		case evtsBatch := <-evts:
+			log.Info("Handling event")
 			n.handleSubnetEvents(evtsBatch)
 		case <-ctx.Done():
+			log.Info("Received DONE")
 			return
 		}
 	}
 }
 
-func (n *network) handleInitialSubnetEvents(batch []subnet.Event) error {
-	log.Infof("Handling initial subnet events \n")
-
-	installedPolicies, err := GetIPSECPolicies()
-	if err != nil {
-		return fmt.Errorf("error getting ipsec policies: %v", err)
-	}
-
-	evtMarker := make([]bool, len(batch))
-	policyMarker := make([]bool, len(installedPolicies))
-
-	for k, evt := range batch {
-		if evt.Lease.Attrs.BackendType != "ipsec" {
-			log.Warningf("Ignoring non-ipsec subnet event type:%v", evt.Lease.Attrs.BackendType)
-			evtMarker[k] = true
-			continue
-		}
-
-		for j, policy := range installedPolicies {
-			if (policy.Src.String() == n.SubnetLease.Subnet.ToIPNet().String()) &&
-				(policy.Dst.String() == evt.Lease.Subnet.ToIPNet().String()) {
-
-				if policy.Dir != netlink.XFRM_DIR_OUT {
-					continue
-				}
-
-				if (policy.Tmpls[0].Src.Equal(n.SubnetLease.Attrs.PublicIP.ToIP())) &&
-					(policy.Tmpls[0].Dst.Equal(evt.Lease.Attrs.PublicIP.ToIP())) {
-
-					evtMarker[k] = true
-					policyMarker[j] = true
-				}
-			}
-		}
-	}
-
-	for k, marker := range evtMarker {
-		if !marker {
-			if err := n.AddIPSECPolicies(&batch[k].Lease, defaultReqID); err != nil {
-				log.Errorf("error adding initial ipsec policy: %v", err)
-			}
-		}
-	}
-
-	for _, evt := range batch {
-		if err := n.iked.LoadSharedKey(evt.Lease.Attrs.PublicIP.String(), n.password); err != nil {
-			log.Errorf("error loading initial shared key: %v", err)
-		}
-
-		if err := n.iked.LoadConnection(n.SubnetLease, &evt.Lease, strconv.Itoa(defaultReqID),
-			strconv.FormatBool(n.UDPEncap)); err != nil {
-
-			log.Errorf("error loading initial connection into IKE daemon: %v", err)
-		}
-	}
-
-	for j, marker := range policyMarker {
-		if !marker {
-			if installedPolicies[j].Dir != netlink.XFRM_DIR_OUT {
-				continue
-			}
-
-			if err := n.DeleteIPSECPolicies(installedPolicies[j].Src, installedPolicies[j].Dst,
-				installedPolicies[j].Tmpls[0].Src, installedPolicies[j].Tmpls[0].Dst,
-				installedPolicies[j].Tmpls[0].Reqid); err != nil {
-
-				log.Errorf("error deleting installed policy")
-			}
-		}
-	}
-
-	return nil
-}
-
 func (n *network) handleSubnetEvents(batch []subnet.Event) {
 	for _, evt := range batch {
 		switch evt.Type {

+ 21 - 0
backend/ipsec/ipsec_windows.go

@@ -0,0 +1,21 @@
+// 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 ipsec
+
+import log "github.com/golang/glog"
+
+func init() {
+	log.Infof("ipsec is not supported on this platform")
+}

+ 18 - 6
dist/functional-test.sh

@@ -98,7 +98,7 @@ test_udp_ping() {
 }
 fi
 
-test_host-gw_ping() {
+test_hostgw_ping() {
     write_config_etcd host-gw
     create_ping_dest # creates ping_dest1 and ping_dest2 variables
     pings
@@ -110,14 +110,20 @@ test_ipip_ping() {
     pings
 }
 
+test_ipsec_ping() {
+    write_config_etcd ipsec
+    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"
-    assert "docker exec --privileged flannel-e2e-test-flannel2 /bin/ping -c 3 $ping_dest1" "Host 2 cannot ping host 1"
+    assert "docker exec --privileged flannel-e2e-test-flannel1 /bin/ping -I $ping_dest1 -c 3 $ping_dest2" "Host 1 cannot ping host 2"
+    assert "docker exec --privileged flannel-e2e-test-flannel2 /bin/ping -I $ping_dest2 -c 3 $ping_dest1" "Host 2 cannot ping host 1"
 }
 
 # These perf tests don't actually assert on anything
-test_host-gw-perf() {
+test_hostgw_perf() {
     write_config_etcd host-gw
     create_ping_dest
     perf
@@ -143,6 +149,12 @@ test_ipip_perf() {
     perf
 }
 
+test_ipsec_perf() {
+    write_config_etcd ipsec
+    create_ping_dest
+    perf
+}
+
 #test_wireguard_perf() {
 #    write_config_etcd extension-wireguard
 #    create_ping_dest
@@ -152,8 +164,8 @@ test_ipip_perf() {
 perf() {
     # Perf test - run iperf server on flannel1 and client on flannel2
     docker rm -f flannel-e2e-test-flannel1-iperf 2>/dev/null
-    docker run -d --name flannel-e2e-test-flannel1-iperf --net=container:flannel-e2e-test-flannel1 iperf3:latest
-    docker run --rm --net=container:flannel-e2e-test-flannel2 iperf3:latest -c $ping_dest1
+    docker run -d --name flannel-e2e-test-flannel1-iperf --net=container:flannel-e2e-test-flannel1 iperf3:latest >/dev/null
+    docker run --rm --net=container:flannel-e2e-test-flannel2 iperf3:latest -c $ping_dest1 -B $ping_dest2
 }
 
 test_multi() {

+ 7 - 0
dist/ipsec

@@ -0,0 +1,7 @@
+{
+  "Network": "10.50.0.0/16",
+  "Backend": {
+    "Type": "ipsec",
+    "PSK":"4bc1e570ff249cce3cc8cef5e2f8625bac76c7b02532f8bde9747196eb15a742480e265bbc0c60c265a8fe4eb6380cd1"
+  }
+}

+ 1 - 10
main.go

@@ -32,7 +32,6 @@ import (
 	log "github.com/golang/glog"
 	"golang.org/x/net/context"
 
-	"github.com/coreos/flannel/backend/ipsec"
 	"github.com/coreos/flannel/network"
 	"github.com/coreos/flannel/pkg/ip"
 	"github.com/coreos/flannel/subnet"
@@ -55,6 +54,7 @@ import (
 	_ "github.com/coreos/flannel/backend/gce"
 	_ "github.com/coreos/flannel/backend/hostgw"
 	_ "github.com/coreos/flannel/backend/ipip"
+	_ "github.com/coreos/flannel/backend/ipsec"
 	_ "github.com/coreos/flannel/backend/udp"
 	_ "github.com/coreos/flannel/backend/vxlan"
 	"github.com/coreos/go-systemd/daemon"
@@ -124,8 +124,6 @@ func init() {
 	flannelFlags.BoolVar(&opts.version, "version", false, "print version and exit")
 	flannelFlags.StringVar(&opts.healthzIP, "healthz-ip", "0.0.0.0", "the IP address for healthz server to listen")
 	flannelFlags.IntVar(&opts.healthzPort, "healthz-port", 0, "the port for healthz server to listen(0 to disable)")
-	flannelFlags.StringVar(&opts.charonExecutablePath, "charon-exec-path", "", "Path to charon executable. Setting it will make flannel attempt to start charon.")
-	flannelFlags.StringVar(&opts.charonViciUri, "charon-vici-uri", "", "Charon vici URI (default: unix:///var/run/charon.vici")
 
 	// glog will log to tmp files by default. override so all entries
 	// can flow into journald (if running under systemd)
@@ -232,13 +230,6 @@ func main() {
 		}
 	}
 
-	if opts.charonViciUri != "" {
-		ipsec.CharonViciUri = opts.charonViciUri
-	}
-	if opts.charonExecutablePath != "" {
-		ipsec.CharonExecutablePath = opts.charonExecutablePath
-	}
-
 	sm, err := newSubnetManager()
 	if err != nil {
 		log.Error("Failed to create SubnetManager: ", err)