Forráskód Böngészése

Refactoring: single ctx and pull out LeaseRenewer

- Use a single copy of ctx and plumb it throughout.
This fixes the left over artifact that ctx went into
flannel in stages.

- Backends being responsible for lease renewal is
not really valuable and just leads to duplication.
Eugene Yakubovich 9 éve
szülő
commit
178fe9e2b4

+ 5 - 20
backend/alloc/alloc.go

@@ -14,33 +14,27 @@ type AllocBackend struct {
 	sm      subnet.Manager
 	network string
 	lease   *subnet.Lease
-	ctx     context.Context
-	cancel  context.CancelFunc
 }
 
 func New(sm subnet.Manager, network string) backend.Backend {
-	ctx, cancel := context.WithCancel(context.Background())
-
 	return &AllocBackend{
 		sm:      sm,
 		network: network,
-		ctx:     ctx,
-		cancel:  cancel,
 	}
 }
 
-func (m *AllocBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
+func (m *AllocBackend) Init(ctx context.Context, extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
 	attrs := subnet.LeaseAttrs{
 		PublicIP: ip.FromIP(extEaddr),
 	}
 
-	l, err := m.sm.AcquireLease(m.ctx, m.network, &attrs)
+	l, err := m.sm.AcquireLease(ctx, m.network, &attrs)
 	switch err {
 	case nil:
 		m.lease = l
 		return &backend.SubnetDef{
-			Net: l.Subnet,
-			MTU: extIface.MTU,
+			Lease: l,
+			MTU:   extIface.MTU,
 		}, nil
 
 	case context.Canceled, context.DeadlineExceeded:
@@ -51,14 +45,5 @@ func (m *AllocBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr n
 	}
 }
 
-func (m *AllocBackend) Run() {
-	subnet.LeaseRenewer(m.ctx, m.sm, m.network, m.lease)
-}
-
-func (m *AllocBackend) Stop() {
-	m.cancel()
-}
-
-func (m *AllocBackend) Name() string {
-	return "allocation"
+func (m *AllocBackend) Run(ctx context.Context) {
 }

+ 9 - 23
backend/awsvpc/awsvpc.go

@@ -17,15 +17,16 @@ package awsvpc
 import (
 	"encoding/json"
 	"fmt"
+	"net"
+
 	log "github.com/coreos/flannel/Godeps/_workspace/src/github.com/golang/glog"
 	"github.com/coreos/flannel/Godeps/_workspace/src/github.com/mitchellh/goamz/aws"
 	"github.com/coreos/flannel/Godeps/_workspace/src/github.com/mitchellh/goamz/ec2"
 	"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"
-	"net"
-	"sync"
 )
 
 type AwsVpcBackend struct {
@@ -35,26 +36,19 @@ type AwsVpcBackend struct {
 	cfg     struct {
 		RouteTableID string
 	}
-	lease  *subnet.Lease
-	ctx    context.Context
-	cancel context.CancelFunc
-	wg     sync.WaitGroup
+	lease *subnet.Lease
 }
 
 func New(sm subnet.Manager, network string, config *subnet.Config) backend.Backend {
-	ctx, cancel := context.WithCancel(context.Background())
-
 	be := AwsVpcBackend{
 		sm:      sm,
 		network: network,
 		config:  config,
-		ctx:     ctx,
-		cancel:  cancel,
 	}
 	return &be
 }
 
-func (m *AwsVpcBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
+func (m *AwsVpcBackend) Init(ctx context.Context, extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
 	// Parse our configuration
 	if len(m.config.Backend) > 0 {
 		if err := json.Unmarshal(m.config.Backend, &m.cfg); err != nil {
@@ -67,7 +61,7 @@ func (m *AwsVpcBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr
 		PublicIP: ip.FromIP(extEaddr),
 	}
 
-	l, err := m.sm.AcquireLease(m.ctx, m.network, &attrs)
+	l, err := m.sm.AcquireLease(ctx, m.network, &attrs)
 	switch err {
 	case nil:
 		m.lease = l
@@ -142,8 +136,8 @@ func (m *AwsVpcBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr
 	}
 
 	return &backend.SubnetDef{
-		Net: l.Subnet,
-		MTU: extIface.MTU,
+		Lease: l,
+		MTU:   extIface.MTU,
 	}, nil
 }
 
@@ -245,14 +239,6 @@ func (m *AwsVpcBackend) detectRouteTableID(instanceID string, ec2c *ec2.EC2) err
 
 	return nil
 }
-func (m *AwsVpcBackend) Run() {
-	subnet.LeaseRenewer(m.ctx, m.sm, m.network, m.lease)
-}
-
-func (m *AwsVpcBackend) Stop() {
-	m.cancel()
-}
 
-func (m *AwsVpcBackend) Name() string {
-	return "aws-vpc"
+func (m *AwsVpcBackend) Run(ctx context.Context) {
 }

+ 7 - 7
backend/common.go

@@ -17,17 +17,17 @@ package backend
 import (
 	"net"
 
-	"github.com/coreos/flannel/pkg/ip"
+	"github.com/coreos/flannel/Godeps/_workspace/src/golang.org/x/net/context"
+
+	"github.com/coreos/flannel/subnet"
 )
 
 type SubnetDef struct {
-	Net ip.IP4Net
-	MTU int
+	Lease *subnet.Lease
+	MTU   int
 }
 
 type Backend interface {
-	Init(extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*SubnetDef, error)
-	Run()
-	Stop()
-	Name() string
+	Init(ctx context.Context, extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*SubnetDef, error)
+	Run(ctx context.Context)
 }

+ 5 - 24
backend/gce/gce.go

@@ -41,7 +41,6 @@ import (
 	"fmt"
 	"net"
 	"strings"
-	"sync"
 	"time"
 
 	"github.com/coreos/flannel/Godeps/_workspace/src/code.google.com/p/goauth2/compute/serviceaccount"
@@ -65,33 +64,26 @@ type GCEBackend struct {
 	sm             subnet.Manager
 	config         *subnet.Config
 	lease          *subnet.Lease
-	ctx            context.Context
-	cancel         context.CancelFunc
-	wg             sync.WaitGroup
 	computeService *compute.Service
 	gceNetwork     *compute.Network
 	gceInstance    *compute.Instance
 }
 
 func New(sm subnet.Manager, network string, config *subnet.Config) backend.Backend {
-	ctx, cancel := context.WithCancel(context.Background())
-
 	gb := GCEBackend{
 		sm:      sm,
 		config:  config,
-		ctx:     ctx,
-		cancel:  cancel,
 		network: network,
 	}
 	return &gb
 }
 
-func (g *GCEBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
+func (g *GCEBackend) Init(ctx context.Context, extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
 	attrs := subnet.LeaseAttrs{
 		PublicIP: ip.FromIP(extEaddr),
 	}
 
-	l, err := g.sm.AcquireLease(g.ctx, g.network, &attrs)
+	l, err := g.sm.AcquireLease(ctx, g.network, &attrs)
 	switch err {
 	case nil:
 		g.lease = l
@@ -161,23 +153,12 @@ func (g *GCEBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr net
 	}
 
 	return &backend.SubnetDef{
-		Net: l.Subnet,
-		MTU: extIface.MTU,
+		Lease: l,
+		MTU:   extIface.MTU,
 	}, nil
 }
 
-func (g *GCEBackend) Run() {
-	log.Info("GCE backend running")
-	subnet.LeaseRenewer(g.ctx, g.sm, g.network, g.lease)
-}
-
-func (g *GCEBackend) Stop() {
-	log.Info("GCE backend stopping")
-	g.cancel()
-}
-
-func (g *GCEBackend) Name() string {
-	return "gce"
+func (g *GCEBackend) Run(ctx context.Context) {
 }
 
 func (g *GCEBackend) pollOperationStatus(operationName string) error {

+ 14 - 33
backend/hostgw/hostgw.go

@@ -39,25 +39,18 @@ type HostgwBackend struct {
 	lease    *subnet.Lease
 	extIface *net.Interface
 	extIaddr net.IP
-	ctx      context.Context
-	cancel   context.CancelFunc
-	wg       sync.WaitGroup
 	rl       []netlink.Route
 }
 
 func New(sm subnet.Manager, network string) backend.Backend {
-	ctx, cancel := context.WithCancel(context.Background())
-
 	b := &HostgwBackend{
 		sm:      sm,
 		network: network,
-		ctx:     ctx,
-		cancel:  cancel,
 	}
 	return b
 }
 
-func (rb *HostgwBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
+func (rb *HostgwBackend) Init(ctx context.Context, extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
 	rb.extIface = extIface
 	rb.extIaddr = extIaddr
 
@@ -70,7 +63,7 @@ func (rb *HostgwBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr
 		BackendType: "host-gw",
 	}
 
-	l, err := rb.sm.AcquireLease(rb.ctx, rb.network, &attrs)
+	l, err := rb.sm.AcquireLease(ctx, rb.network, &attrs)
 	switch err {
 	case nil:
 		rb.lease = l
@@ -85,54 +78,42 @@ func (rb *HostgwBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr
 	/* NB: docker will create the local route to `sn` */
 
 	return &backend.SubnetDef{
-		Net: l.Subnet,
-		MTU: extIface.MTU,
+		Lease: l,
+		MTU:   extIface.MTU,
 	}, nil
 }
 
-func (rb *HostgwBackend) Run() {
-	rb.wg.Add(1)
-	go func() {
-		subnet.LeaseRenewer(rb.ctx, rb.sm, rb.network, rb.lease)
-		rb.wg.Done()
-	}()
+func (rb *HostgwBackend) Run(ctx context.Context) {
+	wg := sync.WaitGroup{}
 
 	log.Info("Watching for new subnet leases")
 	evts := make(chan []subnet.Event)
-	rb.wg.Add(1)
+	wg.Add(1)
 	go func() {
-		subnet.WatchLeases(rb.ctx, rb.sm, rb.network, rb.lease, evts)
-		rb.wg.Done()
+		subnet.WatchLeases(ctx, rb.sm, rb.network, rb.lease, evts)
+		wg.Done()
 	}()
 
 	rb.rl = make([]netlink.Route, 0, 10)
-	rb.wg.Add(1)
+	wg.Add(1)
 	go func() {
-		rb.routeCheck(rb.ctx)
-		rb.wg.Done()
+		rb.routeCheck(ctx)
+		wg.Done()
 	}()
 
-	defer rb.wg.Wait()
+	defer wg.Wait()
 
 	for {
 		select {
 		case evtBatch := <-evts:
 			rb.handleSubnetEvents(evtBatch)
 
-		case <-rb.ctx.Done():
+		case <-ctx.Done():
 			return
 		}
 	}
 }
 
-func (rb *HostgwBackend) Stop() {
-	rb.cancel()
-}
-
-func (rb *HostgwBackend) Name() string {
-	return "host-gw"
-}
-
 func (rb *HostgwBackend) handleSubnetEvents(batch []subnet.Event) {
 	for _, evt := range batch {
 		switch evt.Type {

+ 24 - 50
backend/udp/udp.go

@@ -50,26 +50,19 @@ type UdpBackend struct {
 	conn   *net.UDPConn
 	mtu    int
 	tunNet ip.IP4Net
-	ctx    context.Context
-	cancel context.CancelFunc
-	wg     sync.WaitGroup
 }
 
 func New(sm subnet.Manager, network string, config *subnet.Config) backend.Backend {
-	ctx, cancel := context.WithCancel(context.Background())
-
 	be := UdpBackend{
 		sm:      sm,
 		network: network,
 		config:  config,
-		ctx:     ctx,
-		cancel:  cancel,
 	}
 	be.cfg.Port = defaultPort
 	return &be
 }
 
-func (m *UdpBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
+func (m *UdpBackend) Init(ctx context.Context, extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
 	// Parse our configuration
 	if len(m.config.Backend) > 0 {
 		if err := json.Unmarshal(m.config.Backend, &m.cfg); err != nil {
@@ -82,7 +75,7 @@ func (m *UdpBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr net
 		PublicIP: ip.FromIP(extEaddr),
 	}
 
-	l, err := m.sm.AcquireLease(m.ctx, m.network, &attrs)
+	l, err := m.sm.AcquireLease(ctx, m.network, &attrs)
 	switch err {
 	case nil:
 		m.lease = l
@@ -119,40 +112,43 @@ func (m *UdpBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr net
 	}
 
 	return &backend.SubnetDef{
-		Net: l.Subnet,
-		MTU: m.mtu,
+		Lease: l,
+		MTU:   m.mtu,
 	}, nil
 }
 
-func (m *UdpBackend) Run() {
+func (m *UdpBackend) Run(ctx context.Context) {
 	// one for each goroutine below
-	m.wg.Add(2)
+	wg := sync.WaitGroup{}
 
+	wg.Add(1)
 	go func() {
 		runCProxy(m.tun, m.conn, m.ctl2, m.tunNet.IP, m.mtu)
-		m.wg.Done()
+		wg.Done()
 	}()
 
+	log.Info("Watching for new subnet leases")
+
+	evts := make(chan []subnet.Event)
+
+	wg.Add(1)
 	go func() {
-		subnet.LeaseRenewer(m.ctx, m.sm, m.network, m.lease)
-		m.wg.Done()
+		subnet.WatchLeases(ctx, m.sm, m.network, m.lease, evts)
+		wg.Done()
 	}()
 
-	m.monitorEvents()
-
-	m.wg.Wait()
-}
+	for {
+		select {
+		case evtBatch := <-evts:
+			m.processSubnetEvents(evtBatch)
 
-func (m *UdpBackend) Stop() {
-	if m.ctl != nil {
-		stopProxy(m.ctl)
+		case <-ctx.Done():
+			stopProxy(m.ctl)
+			break
+		}
 	}
 
-	m.cancel()
-}
-
-func (m *UdpBackend) Name() string {
-	return "UDP"
+	wg.Wait()
 }
 
 func newCtlSockets() (*os.File, *os.File, error) {
@@ -218,28 +214,6 @@ func configureIface(ifname string, ipn ip.IP4Net, mtu int) error {
 	return nil
 }
 
-func (m *UdpBackend) monitorEvents() {
-	log.Info("Watching for new subnet leases")
-
-	evts := make(chan []subnet.Event)
-
-	m.wg.Add(1)
-	go func() {
-		subnet.WatchLeases(m.ctx, m.sm, m.network, m.lease, evts)
-		m.wg.Done()
-	}()
-
-	for {
-		select {
-		case evtBatch := <-evts:
-			m.processSubnetEvents(evtBatch)
-
-		case <-m.ctx.Done():
-			return
-		}
-	}
-}
-
 func (m *UdpBackend) processSubnetEvents(batch []subnet.Event) {
 	for _, evt := range batch {
 		switch evt.Type {

+ 15 - 35
backend/vxlan/vxlan.go

@@ -43,23 +43,16 @@ type VXLANBackend struct {
 		VNI  int
 		Port int
 	}
-	lease  *subnet.Lease
-	dev    *vxlanDevice
-	ctx    context.Context
-	cancel context.CancelFunc
-	wg     sync.WaitGroup
-	rts    routes
+	lease *subnet.Lease
+	dev   *vxlanDevice
+	rts   routes
 }
 
 func New(sm subnet.Manager, network string, config *subnet.Config) backend.Backend {
-	ctx, cancel := context.WithCancel(context.Background())
-
 	vb := &VXLANBackend{
 		sm:      sm,
 		network: network,
 		config:  config,
-		ctx:     ctx,
-		cancel:  cancel,
 	}
 	vb.cfg.VNI = defaultVNI
 
@@ -79,7 +72,7 @@ func newSubnetAttrs(extEaddr net.IP, mac net.HardwareAddr) (*subnet.LeaseAttrs,
 	}, nil
 }
 
-func (vb *VXLANBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
+func (vb *VXLANBackend) Init(ctx context.Context, extIface *net.Interface, extIaddr net.IP, extEaddr net.IP) (*backend.SubnetDef, error) {
 	// Parse our configuration
 	if len(vb.config.Backend) > 0 {
 		if err := json.Unmarshal(vb.config.Backend, &vb.cfg); err != nil {
@@ -114,7 +107,7 @@ func (vb *VXLANBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr
 		return nil, err
 	}
 
-	l, err := vb.sm.AcquireLease(vb.ctx, vb.network, sa)
+	l, err := vb.sm.AcquireLease(ctx, vb.network, sa)
 	switch err {
 	case nil:
 		vb.lease = l
@@ -137,35 +130,30 @@ func (vb *VXLANBackend) Init(extIface *net.Interface, extIaddr net.IP, extEaddr
 	}
 
 	return &backend.SubnetDef{
-		Net: l.Subnet,
-		MTU: vb.dev.MTU(),
+		Lease: l,
+		MTU:   vb.dev.MTU(),
 	}, nil
 }
 
-func (vb *VXLANBackend) Run() {
-	vb.wg.Add(1)
-	go func() {
-		subnet.LeaseRenewer(vb.ctx, vb.sm, vb.network, vb.lease)
-		log.Info("LeaseRenewer exited")
-		vb.wg.Done()
-	}()
-
+func (vb *VXLANBackend) Run(ctx context.Context) {
 	log.Info("Watching for L3 misses")
 	misses := make(chan *netlink.Neigh, 100)
 	// Unfrtunately MonitorMisses does not take a cancel channel
 	// as there's no wait to interrupt netlink socket recv
 	go vb.dev.MonitorMisses(misses)
 
+	wg := sync.WaitGroup{}
+
 	log.Info("Watching for new subnet leases")
 	evts := make(chan []subnet.Event)
-	vb.wg.Add(1)
+	wg.Add(1)
 	go func() {
-		subnet.WatchLeases(vb.ctx, vb.sm, vb.network, vb.lease, evts)
+		subnet.WatchLeases(ctx, vb.sm, vb.network, vb.lease, evts)
 		log.Info("WatchLeases exited")
-		vb.wg.Done()
+		wg.Done()
 	}()
 
-	defer vb.wg.Wait()
+	defer wg.Wait()
 	initialEvtsBatch := <-evts
 	for {
 		err := vb.handleInitialSubnetEvents(initialEvtsBatch)
@@ -184,20 +172,12 @@ func (vb *VXLANBackend) Run() {
 		case evtBatch := <-evts:
 			vb.handleSubnetEvents(evtBatch)
 
-		case <-vb.ctx.Done():
+		case <-ctx.Done():
 			return
 		}
 	}
 }
 
-func (vb *VXLANBackend) Stop() {
-	vb.cancel()
-}
-
-func (vb *VXLANBackend) Name() string {
-	return "VXLAN"
-}
-
 // So we can make it JSON (un)marshalable
 type hardwareAddr net.HardwareAddr
 

+ 5 - 4
main.go

@@ -81,7 +81,7 @@ func init() {
 	flag.BoolVar(&opts.version, "version", false, "print version and exit")
 }
 
-func writeSubnetFile(path string, nw ip.IP4Net, sn *backend.SubnetDef) error {
+func writeSubnetFile(path string, nw ip.IP4Net, sd *backend.SubnetDef) error {
 	dir, name := filepath.Split(path)
 	os.MkdirAll(dir, 0755)
 
@@ -93,11 +93,12 @@ func writeSubnetFile(path string, nw ip.IP4Net, sn *backend.SubnetDef) error {
 
 	// Write out the first usable IP by incrementing
 	// sn.IP by one
-	sn.Net.IP += 1
+	sn := sd.Lease.Subnet
+	sn.IP += 1
 
 	fmt.Fprintf(f, "FLANNEL_NETWORK=%s\n", nw)
-	fmt.Fprintf(f, "FLANNEL_SUBNET=%s\n", sn.Net)
-	fmt.Fprintf(f, "FLANNEL_MTU=%d\n", sn.MTU)
+	fmt.Fprintf(f, "FLANNEL_SUBNET=%s\n", sn)
+	fmt.Fprintf(f, "FLANNEL_MTU=%d\n", sd.MTU)
 	_, err = fmt.Fprintf(f, "FLANNEL_IPMASQ=%v\n", opts.ipMasq)
 	f.Close()
 	if err != nil {

+ 2 - 15
network/backend.go

@@ -1,7 +1,6 @@
 package network
 
 import (
-	"encoding/json"
 	"fmt"
 	"strings"
 
@@ -16,19 +15,7 @@ import (
 )
 
 func newBackend(sm subnet.Manager, network string, config *subnet.Config) (backend.Backend, error) {
-	var bt struct {
-		Type string
-	}
-
-	if len(config.Backend) == 0 {
-		bt.Type = "udp"
-	} else {
-		if err := json.Unmarshal(config.Backend, &bt); err != nil {
-			return nil, fmt.Errorf("Error decoding Backend property of config: %v", err)
-		}
-	}
-
-	switch strings.ToLower(bt.Type) {
+	switch strings.ToLower(config.BackendType) {
 	case "udp":
 		return udp.New(sm, network, config), nil
 	case "alloc":
@@ -42,6 +29,6 @@ func newBackend(sm subnet.Manager, network string, config *subnet.Config) (backe
 	case "gce":
 		return gce.New(sm, network, config), nil
 	default:
-		return nil, fmt.Errorf("%v: '%v': unknown backend type", network, bt.Type)
+		return nil, fmt.Errorf("%v: '%v': unknown backend type", network, config.BackendType)
 	}
 }

+ 11 - 5
network/network.go

@@ -32,6 +32,7 @@ type Network struct {
 	sm     subnet.Manager
 	ipMasq bool
 	be     backend.Backend
+	lease  *subnet.Lease
 }
 
 func New(sm subnet.Manager, name string, ipMasq bool) *Network {
@@ -66,10 +67,11 @@ func (n *Network) Init(ctx context.Context, iface *net.Interface, iaddr net.IP,
 		},
 
 		func() (err error) {
-			sn, err = be.Init(iface, iaddr, eaddr)
+			sn, err = be.Init(ctx, iface, iaddr, eaddr)
 			if err != nil {
-				log.Errorf("Failed to initialize network %v (type %v): %v", n.Name, be.Name(), err)
+				log.Errorf("Failed to initialize network %v (type %v): %v", n.Name, n.Config.BackendType, err)
 			}
+			n.lease = sn.Lease
 			return
 		},
 
@@ -106,12 +108,16 @@ func (n *Network) Run(ctx context.Context) {
 	wg := sync.WaitGroup{}
 	wg.Add(1)
 	go func() {
-		n.be.Run()
+		n.be.Run(ctx)
 		wg.Done()
 	}()
 
-	<-ctx.Done()
-	n.be.Stop()
+	wg.Add(1)
+	go func() {
+		subnet.LeaseRenewer(ctx, n.sm, n.Name, n.lease)
+		wg.Done()
+	}()
 
+	<-ctx.Done()
 	wg.Wait()
 }

+ 29 - 5
subnet/config.go

@@ -17,16 +17,34 @@ package subnet
 import (
 	"encoding/json"
 	"errors"
+	"fmt"
 
 	"github.com/coreos/flannel/pkg/ip"
 )
 
 type Config struct {
-	Network   ip.IP4Net
-	SubnetMin ip.IP4
-	SubnetMax ip.IP4
-	SubnetLen uint
-	Backend   json.RawMessage `json:",omitempty"`
+	Network     ip.IP4Net
+	SubnetMin   ip.IP4
+	SubnetMax   ip.IP4
+	SubnetLen   uint
+	BackendType string          `json:"-"`
+	Backend     json.RawMessage `json:",omitempty"`
+}
+
+func parseBackendType(be json.RawMessage) (string, error) {
+	var bt struct {
+		Type string
+	}
+
+	if len(be) == 0 {
+		return "udp", nil
+	} else {
+		if err := json.Unmarshal(be, &bt); err != nil {
+			return "", fmt.Errorf("error decoding Backend property of config: %v", err)
+		}
+	}
+
+	return bt.Type, nil
 }
 
 func ParseConfig(s string) (*Config, error) {
@@ -67,5 +85,11 @@ func ParseConfig(s string) (*Config, error) {
 		return nil, errors.New("SubnetMax is not in the range of the Network")
 	}
 
+	bt, err := parseBackendType(cfg.Backend)
+	if err != nil {
+		return nil, err
+	}
+	cfg.BackendType = bt
+
 	return cfg, nil
 }