ソースを参照

Add GCE backend

No configuration parameter is required other than the Backend=gce.
On startup, flannel deletes any conflicting entries from the route table and adds an entry setting the next hop, for the subnet it has acquired,
to itself. Instead of NextHopIP we use NextHopInstance. In the rare case
that a conflicting rule does exist, flannel waits for the write to be
implemented, which takes an average of 20 seconds, before inserting the
new entry.

Fixes #176
Mohammad Ahmad 9 年 前
コミット
e775876fea
4 ファイル変更352 行追加0 行削除
  1. 9 0
      README.md
  2. 265 0
      backend/gce/gce.go
  3. 75 0
      backend/gce/metadata.go
  4. 3 0
      network/backend.go

+ 9 - 0
README.md

@@ -81,6 +81,15 @@ This is the only mandatory key.
   Authentication is handled via either environment variables or the node's IAM role.
   If the node has insufficient privileges to modify the VPC routing table specified, ensure that appropriate `AWS_ACCESS_KEY_ID`, `AWS_SECRET_ACCESS_KEY`, and optionally `AWS_SECURITY_TOKEN` environment variables are set when running the flanneld process.
 
+* gce: create IP routes in a [Google Compute Engine Network](https://cloud.google.com/compute/docs/networking#networks)
+  * Requirements:
+    * [Enable IP forwarding for the instances](https://cloud.google.com/compute/docs/networking#canipforward).
+    * [Instance service account](https://cloud.google.com/compute/docs/authentication#using) with read-write compute permissions. 
+  * `Type` (string): `gce`  
+  
+  Command to create a compute instance with the correct permissions and IP forwarding enabled:  
+  `$ gcloud compute instances create INSTANCE --can-ip-forward --scopes compute-rw`
+
 * alloc: only perform subnet allocation (no forwarding of data packets).
   * `Type` (string): `alloc`
 

+ 265 - 0
backend/gce/gce.go

@@ -0,0 +1,265 @@
+// Copyright 2015 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.
+
+// This work borrows from the https://github.com/kelseyhightower/flannel-route-manager
+// project which has the following license agreement.
+
+// Copyright (c) 2014 Kelsey Hightower
+
+// Permission is hereby granted, free of charge, to any person obtaining a copy of
+// this software and associated documentation files (the "Software"), to deal in
+// the Software without restriction, including without limitation the rights to
+// use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies
+// of the Software, and to permit persons to whom the Software is furnished to do
+// so, subject to the following conditions:
+
+// The above copyright notice and this permission notice shall be included in all
+// copies or substantial portions of the Software.
+
+// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+// SOFTWARE.
+
+package gce
+
+import (
+	"fmt"
+	"net"
+	"strings"
+	"sync"
+	"time"
+
+	"github.com/coreos/flannel/Godeps/_workspace/src/code.google.com/p/goauth2/compute/serviceaccount"
+	"github.com/coreos/flannel/Godeps/_workspace/src/code.google.com/p/google-api-go-client/compute/v1"
+	"github.com/coreos/flannel/Godeps/_workspace/src/code.google.com/p/google-api-go-client/googleapi"
+	log "github.com/coreos/flannel/Godeps/_workspace/src/github.com/golang/glog"
+	"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"
+)
+
+var metadataEndpoint = "http://169.254.169.254/computeMetadata/v1"
+
+var replacer = strings.NewReplacer(".", "-", "/", "-")
+
+type GCEBackend struct {
+	network        string
+	project        string
+	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, extIP net.IP) (*backend.SubnetDef, error) {
+	attrs := subnet.LeaseAttrs{
+		PublicIP: ip.FromIP(extIP),
+	}
+
+	l, err := g.sm.AcquireLease(g.ctx, g.network, &attrs)
+	switch err {
+	case nil:
+		g.lease = l
+
+	case context.Canceled, context.DeadlineExceeded:
+		return nil, err
+
+	default:
+		return nil, fmt.Errorf("failed to acquire lease: %v", err)
+	}
+
+	client, err := serviceaccount.NewClient(&serviceaccount.Options{})
+	if err != nil {
+		return nil, fmt.Errorf("error creating client: %v", err)
+	}
+
+	g.computeService, err = compute.New(client)
+	if err != nil {
+		return nil, fmt.Errorf("error creating compute service: %v", err)
+	}
+
+	networkName, err := networkFromMetadata()
+	if err != nil {
+		return nil, fmt.Errorf("error getting network metadata: %v", err)
+	}
+
+	g.project, err = projectFromMetadata()
+	if err != nil {
+		return nil, fmt.Errorf("error getting project: %v", err)
+	}
+
+	instanceName, err := instanceNameFromMetadata()
+	if err != nil {
+		return nil, fmt.Errorf("error getting instance name: %v", err)
+	}
+
+	instanceZone, err := instanceZoneFromMetadata()
+	if err != nil {
+		return nil, fmt.Errorf("error getting instance zone: %v", err)
+	}
+
+	g.gceNetwork, err = g.computeService.Networks.Get(g.project, networkName).Do()
+	if err != nil {
+		return nil, fmt.Errorf("error getting network from compute service: %v", err)
+	}
+
+	g.gceInstance, err = g.computeService.Instances.Get(g.project, instanceZone, instanceName).Do()
+	if err != nil {
+		return nil, fmt.Errorf("error getting instance from compute service: %v", err)
+	}
+
+	found, err := g.handleMatchingRoute(l.Subnet.String())
+	if err != nil {
+		return nil, fmt.Errorf("error handling matching route: %v", err)
+	}
+
+	if !found {
+		operation, err := g.insertRoute(l.Subnet.String())
+		if err != nil {
+			return nil, fmt.Errorf("error inserting route: %v", err)
+		}
+
+		err = g.pollOperationStatus(operation.Name)
+		if err != nil {
+			return nil, fmt.Errorf("insert operaiton failed: ", err)
+		}
+	}
+
+	return &backend.SubnetDef{
+		Net: l.Subnet,
+		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) pollOperationStatus(operationName string) error {
+	for i := 0; i < 100; i++ {
+		operation, err := g.computeService.GlobalOperations.Get(g.project, operationName).Do()
+		if err != nil {
+			return fmt.Errorf("error fetching operation status: %v", err)
+		}
+
+		if operation.Error != nil {
+			return fmt.Errorf("error running operation: %v", operation.Error)
+		}
+
+		if i%5 == 0 {
+			log.Infof("%v operation status: %v waiting for completion...", operation.OperationType, operation.Status)
+		}
+
+		if operation.Status == "DONE" {
+			return nil
+		}
+		time.Sleep(time.Second)
+	}
+
+	return fmt.Errorf("timeout waiting for operation to finish")
+}
+
+//returns true if an exact matching rule is found
+func (g *GCEBackend) handleMatchingRoute(subnet string) (bool, error) {
+	matchingRoute, err := g.getRoute(subnet)
+	if err != nil {
+		if apiError, ok := err.(*googleapi.Error); ok {
+			if apiError.Code != 404 {
+				return false, fmt.Errorf("error getting the route err: %v", err)
+			}
+			return false, nil
+		}
+		return false, fmt.Errorf("error getting googleapi: %v", err)
+	}
+
+	if matchingRoute.NextHopInstance == g.gceInstance.SelfLink {
+		log.Info("Exact pre-existing route found")
+		return true, nil
+	}
+
+	log.Info("Deleting conflicting route")
+	operation, err := g.deleteRoute(subnet)
+	if err != nil {
+		return false, fmt.Errorf("error deleting conflicting route : %v", err)
+	}
+
+	err = g.pollOperationStatus(operation.Name)
+	if err != nil {
+		return false, fmt.Errorf("delete operation failed: %v", err)
+	}
+
+	return false, nil
+
+}
+
+func (g *GCEBackend) getRoute(subnet string) (*compute.Route, error) {
+	routeName := formatRouteName(g.gceNetwork.Name, subnet)
+	return g.computeService.Routes.Get(g.project, routeName).Do()
+}
+
+func (g *GCEBackend) deleteRoute(subnet string) (*compute.Operation, error) {
+	routeName := formatRouteName(g.gceNetwork.Name, subnet)
+	return g.computeService.Routes.Delete(g.project, routeName).Do()
+}
+
+func (g *GCEBackend) insertRoute(subnet string) (*compute.Operation, error) {
+	log.Infof("Inserting route for subnet: %v", subnet)
+	route := &compute.Route{
+		Name:            formatRouteName(g.gceNetwork.Name, subnet),
+		DestRange:       subnet,
+		Network:         g.gceNetwork.SelfLink,
+		NextHopInstance: g.gceInstance.SelfLink,
+		Priority:        1000,
+		Tags:            []string{},
+	}
+	return g.computeService.Routes.Insert(g.project, route).Do()
+}
+
+func formatRouteName(network, subnet string) string {
+	return fmt.Sprintf("flannel-%s-%s", network, replacer.Replace(subnet))
+}

+ 75 - 0
backend/gce/metadata.go

@@ -0,0 +1,75 @@
+// Copyright 2015 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 gce
+
+import (
+	"io/ioutil"
+	"net/http"
+	"path"
+	"strings"
+)
+
+func networkFromMetadata() (string, error) {
+	network, err := metadataGet("/instance/network-interfaces/0/network")
+	if err != nil {
+		return "", err
+	}
+	return path.Base(network), nil
+}
+
+func projectFromMetadata() (string, error) {
+	projectName, err := metadataGet("/project/project-id")
+	if err != nil {
+		return "", err
+	}
+	return path.Base(projectName), nil
+}
+
+func instanceZoneFromMetadata() (string, error) {
+	zone, err := metadataGet("/instance/zone")
+
+	if err != nil {
+		return "", err
+	}
+	return path.Base(zone), nil
+}
+
+func instanceNameFromMetadata() (string, error) {
+	hostname, err := metadataGet("/instance/hostname")
+	if err != nil {
+		return "", err
+	}
+	//works because we can't have . in the instance name
+	return strings.SplitN(hostname, ".", 2)[0], nil
+}
+
+func metadataGet(path string) (string, error) {
+	req, err := http.NewRequest("GET", metadataEndpoint+path, nil)
+	if err != nil {
+		return "", err
+	}
+	req.Header.Add("Metadata-Flavor", "Google")
+	client := &http.Client{}
+	resp, err := client.Do(req)
+	if err != nil {
+		return "", err
+	}
+	defer resp.Body.Close()
+	data, err := ioutil.ReadAll(resp.Body)
+	if err != nil {
+		return "", err
+	}
+	return string(data), nil
+}

+ 3 - 0
network/backend.go

@@ -8,6 +8,7 @@ import (
 	"github.com/coreos/flannel/backend"
 	"github.com/coreos/flannel/backend/alloc"
 	"github.com/coreos/flannel/backend/awsvpc"
+	"github.com/coreos/flannel/backend/gce"
 	"github.com/coreos/flannel/backend/hostgw"
 	"github.com/coreos/flannel/backend/udp"
 	"github.com/coreos/flannel/backend/vxlan"
@@ -38,6 +39,8 @@ func newBackend(sm subnet.Manager, network string, config *subnet.Config) (backe
 		return vxlan.New(sm, network, config), nil
 	case "aws-vpc":
 		return awsvpc.New(sm, network, config), nil
+	case "gce":
+		return gce.New(sm, network, config), nil
 	default:
 		return nil, fmt.Errorf("%v: '%v': unknown backend type", network, bt.Type)
 	}