Browse Source

Merge pull request #4438 from adamwg/raftexample-add-nodes

Reconfiguration for raftexample (#4018)
Anthony Romano 10 years ago
parent
commit
2a261e1b03

+ 30 - 1
contrib/raftexample/README.md

@@ -65,6 +65,36 @@ goreman run start raftexample2
 curl -L http://127.0.0.1:22380/my-key
 ```
 
+### Dynamic cluster reconfiguration
+
+Nodes can be added to or removed from a running cluster using requests to the REST API.
+
+For example, suppose we have a 3-node cluster that was started with the commands:
+```sh
+raftexample --id 1 --cluster http://127.0.0.1:12379,http://127.0.0.1:22379,http://127.0.0.1:32379 --port 12380
+raftexample --id 2 --cluster http://127.0.0.1:12379,http://127.0.0.1:22379,http://127.0.0.1:32379 --port 22380
+raftexample --id 3 --cluster http://127.0.0.1:12379,http://127.0.0.1:22379,http://127.0.0.1:32379 --port 32380
+```
+
+A fourth node with ID 4 can be added by issuing a POST:
+```sh
+curl -L http://127.0.0.1:12380/4 -XPOST -d http://127.0.0.1:42379
+```
+
+Then the new node can be started as the others were, using the --join option:
+```sh
+raftexample --id 4 --cluster http://127.0.0.1:12379,http://127.0.0.1:22379,http://127.0.0.1:32379,http://127.0.0.1:42379 --port 42380 --join
+```
+
+The new node should join the cluster and be able to service key/value requests.
+
+We can remove a node using a DELETE request:
+```sh
+curl -L http://127.0.0.1:12380/3 -XDELETE
+```
+
+Node 3 should shut itself down once the cluster has processed this request.
+
 ## Design
 
 The raftexample consists of three components: a raft-backed key-value store, a REST API server, and a raft consensus server based on etcd's raft implementation.
@@ -87,4 +117,3 @@ For raftexample, this commit channel is consumed by the key-value store.
 
 ### TODO
 - Snapshot support
-- Dynamic reconfiguration

+ 54 - 4
contrib/raftexample/httpapi.go

@@ -19,11 +19,14 @@ import (
 	"log"
 	"net/http"
 	"strconv"
+
+	"github.com/coreos/etcd/raft/raftpb"
 )
 
 // Handler for a http based key-value store backed by raft
 type httpKVAPI struct {
-	store *kvstore
+	store       *kvstore
+	confChangeC chan<- raftpb.ConfChange
 }
 
 func (h *httpKVAPI) ServeHTTP(w http.ResponseWriter, r *http.Request) {
@@ -48,18 +51,65 @@ func (h *httpKVAPI) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 		} else {
 			http.Error(w, "Failed to GET", http.StatusNotFound)
 		}
+	case r.Method == "POST":
+		url, err := ioutil.ReadAll(r.Body)
+		if err != nil {
+			log.Printf("Failed to read on POST (%v)\n", err)
+			http.Error(w, "Failed on POST", http.StatusBadRequest)
+			return
+		}
+
+		nodeId, err := strconv.ParseUint(key[1:], 0, 64)
+		if err != nil {
+			log.Printf("Failed to convert ID for conf change (%v)\n", err)
+			http.Error(w, "Failed on POST", http.StatusBadRequest)
+			return
+		}
+
+		cc := raftpb.ConfChange{
+			Type:    raftpb.ConfChangeAddNode,
+			NodeID:  nodeId,
+			Context: url,
+		}
+		h.confChangeC <- cc
+
+		// As above, optimistic that raft will apply the conf change
+		w.WriteHeader(http.StatusNoContent)
+	case r.Method == "DELETE":
+		nodeId, err := strconv.ParseUint(key[1:], 0, 64)
+		if err != nil {
+			log.Printf("Failed to convert ID for conf change (%v)\n", err)
+			http.Error(w, "Failed on DELETE", http.StatusBadRequest)
+			return
+		}
+
+		cc := raftpb.ConfChange{
+			Type:   raftpb.ConfChangeRemoveNode,
+			NodeID: nodeId,
+		}
+		h.confChangeC <- cc
+
+		// As above, optimistic that raft will apply the conf change
+		w.WriteHeader(http.StatusNoContent)
 	default:
 		w.Header().Set("Allow", "PUT")
 		w.Header().Add("Allow", "GET")
+		w.Header().Add("Allow", "POST")
+		w.Header().Add("Allow", "DELETE")
 		http.Error(w, "Method not allowed", http.StatusMethodNotAllowed)
 	}
 }
 
 // serveHttpKVAPI starts a key-value server with a GET/PUT API and listens.
-func serveHttpKVAPI(port int, proposeC chan<- string, commitC <-chan *string, errorC <-chan error) {
+func serveHttpKVAPI(port int, proposeC chan<- string, confChangeC chan<- raftpb.ConfChange,
+	commitC <-chan *string, errorC <-chan error) {
+
 	srv := http.Server{
-		Addr:    ":" + strconv.Itoa(port),
-		Handler: &httpKVAPI{newKVStore(proposeC, commitC, errorC)},
+		Addr: ":" + strconv.Itoa(port),
+		Handler: &httpKVAPI{
+			store:       newKVStore(proposeC, commitC, errorC),
+			confChangeC: confChangeC,
+		},
 	}
 	if err := srv.ListenAndServe(); err != nil {
 		log.Fatal(err)

+ 7 - 2
contrib/raftexample/main.go

@@ -17,20 +17,25 @@ package main
 import (
 	"flag"
 	"strings"
+
+	"github.com/coreos/etcd/raft/raftpb"
 )
 
 func main() {
 	cluster := flag.String("cluster", "http://127.0.0.1:9021", "comma separated cluster peers")
 	id := flag.Int("id", 1, "node ID")
 	kvport := flag.Int("port", 9121, "key-value server port")
+	join := flag.Bool("join", false, "join an existing cluster")
 	flag.Parse()
 
 	proposeC := make(chan string)
 	defer close(proposeC)
+	confChangeC := make(chan raftpb.ConfChange)
+	defer close(confChangeC)
 
 	// raft provides a commit stream for the proposals from the http api
-	commitC, errorC := newRaftNode(*id, strings.Split(*cluster, ","), proposeC)
+	commitC, errorC := newRaftNode(*id, strings.Split(*cluster, ","), *join, proposeC, confChangeC)
 
 	// the key-value http handler will propose updates to raft
-	serveHttpKVAPI(*kvport, proposeC, commitC, errorC)
+	serveHttpKVAPI(*kvport, proposeC, confChangeC, commitC, errorC)
 }

+ 69 - 18
contrib/raftexample/raft.go

@@ -36,12 +36,14 @@ import (
 
 // A key-value stream backed by raft
 type raftNode struct {
-	proposeC <-chan string // proposed messages (k,v)
-	commitC  chan *string  // entries committed to log (k,v)
-	errorC   chan error    // errors from raft session
+	proposeC    <-chan string            // proposed messages (k,v)
+	confChangeC <-chan raftpb.ConfChange // proposed cluster config changes
+	commitC     chan *string             // entries committed to log (k,v)
+	errorC      chan error               // errors from raft session
 
 	id     int      // client ID for raft session
 	peers  []string // raft peer URLs
+	join   bool     // node is joining an existing cluster
 	waldir string   // path to WAL directory
 
 	// raft backing for the commit/error channel
@@ -59,13 +61,17 @@ type raftNode struct {
 // provided the proposal channel. All log entries are replayed over the
 // commit channel, followed by a nil message (to indicate the channel is
 // current), then new log entries. To shutdown, close proposeC and read errorC.
-func newRaftNode(id int, peers []string, proposeC <-chan string) (<-chan *string, <-chan error) {
+func newRaftNode(id int, peers []string, join bool, proposeC <-chan string,
+	confChangeC <-chan raftpb.ConfChange) (<-chan *string, <-chan error) {
+
 	rc := &raftNode{
 		proposeC:    proposeC,
+		confChangeC: confChangeC,
 		commitC:     make(chan *string),
 		errorC:      make(chan error),
 		id:          id,
 		peers:       peers,
+		join:        join,
 		waldir:      fmt.Sprintf("raftexample-%d", id),
 		raftStorage: raft.NewMemoryStorage(),
 		stopc:       make(chan struct{}),
@@ -81,15 +87,36 @@ func newRaftNode(id int, peers []string, proposeC <-chan string) (<-chan *string
 // whether all entries could be published.
 func (rc *raftNode) publishEntries(ents []raftpb.Entry) bool {
 	for i := range ents {
-		if ents[i].Type != raftpb.EntryNormal || len(ents[i].Data) == 0 {
-			// ignore conf changes and empty messages
-			continue
-		}
-		s := string(ents[i].Data)
-		select {
-		case rc.commitC <- &s:
-		case <-rc.stopc:
-			return false
+		switch ents[i].Type {
+		case raftpb.EntryNormal:
+			if len(ents[i].Data) == 0 {
+				// ignore conf changes and empty messages
+				continue
+			}
+			s := string(ents[i].Data)
+			select {
+			case rc.commitC <- &s:
+			case <-rc.stopc:
+				return false
+			}
+
+		case raftpb.EntryConfChange:
+			var cc raftpb.ConfChange
+			cc.Unmarshal(ents[i].Data)
+
+			rc.node.ApplyConfChange(cc)
+			switch cc.Type {
+			case raftpb.ConfChangeAddNode:
+				if len(cc.Context) > 0 {
+					rc.transport.AddPeer(types.ID(cc.NodeID), []string{string(cc.Context)})
+				}
+			case raftpb.ConfChangeRemoveNode:
+				if cc.NodeID == uint64(rc.id) {
+					log.Println("I've been removed from the cluster! Shutting down.")
+					return false
+				}
+				rc.transport.RemovePeer(types.ID(cc.NodeID))
+			}
 		}
 	}
 	return true
@@ -161,7 +188,11 @@ func (rc *raftNode) startRaft() {
 	if oldwal {
 		rc.node = raft.RestartNode(c)
 	} else {
-		rc.node = raft.StartNode(c, rpeers)
+		startPeers := rpeers
+		if rc.join {
+			startPeers = nil
+		}
+		rc.node = raft.StartNode(c, startPeers)
 	}
 
 	ss := &stats.ServerStats{}
@@ -193,6 +224,8 @@ func (rc *raftNode) stop() {
 	close(rc.commitC)
 	close(rc.errorC)
 	rc.node.Stop()
+
+	os.Exit(0)
 }
 
 func (rc *raftNode) stopHTTP() {
@@ -209,9 +242,27 @@ func (rc *raftNode) serveChannels() {
 
 	// send proposals over raft
 	go func() {
-		for prop := range rc.proposeC {
-			// blocks until accepted by raft state machine
-			rc.node.Propose(context.TODO(), []byte(prop))
+		var confChangeCount uint64 = 0
+
+		for rc.proposeC != nil && rc.confChangeC != nil {
+			select {
+			case prop, ok := <-rc.proposeC:
+				if !ok {
+					rc.proposeC = nil
+				} else {
+					// blocks until accepted by raft state machine
+					rc.node.Propose(context.TODO(), []byte(prop))
+				}
+
+			case cc, ok := <-rc.confChangeC:
+				if !ok {
+					rc.confChangeC = nil
+				} else {
+					confChangeCount += 1
+					cc.ID = confChangeCount
+					rc.node.ProposeConfChange(context.TODO(), cc)
+				}
+			}
 		}
 		// client closed channel; shutdown raft if not already
 		close(rc.stopc)
@@ -228,7 +279,7 @@ func (rc *raftNode) serveChannels() {
 			rc.wal.Save(rd.HardState, rd.Entries)
 			rc.raftStorage.Append(rd.Entries)
 			rc.transport.Send(rd.Messages)
-			if ok := rc.publishEntries(rd.Entries); !ok {
+			if ok := rc.publishEntries(rd.CommittedEntries); !ok {
 				rc.stop()
 				return
 			}

+ 15 - 9
contrib/raftexample/raftexample_test.go

@@ -18,13 +18,16 @@ import (
 	"fmt"
 	"os"
 	"testing"
+
+	"github.com/coreos/etcd/raft/raftpb"
 )
 
 type cluster struct {
-	peers    []string
-	commitC  []<-chan *string
-	errorC   []<-chan error
-	proposeC []chan string
+	peers       []string
+	commitC     []<-chan *string
+	errorC      []<-chan error
+	proposeC    []chan string
+	confChangeC []chan raftpb.ConfChange
 }
 
 // newCluster creates a cluster of n nodes
@@ -35,15 +38,18 @@ func newCluster(n int) *cluster {
 	}
 
 	clus := &cluster{
-		peers:    peers,
-		commitC:  make([]<-chan *string, len(peers)),
-		errorC:   make([]<-chan error, len(peers)),
-		proposeC: make([]chan string, len(peers))}
+		peers:       peers,
+		commitC:     make([]<-chan *string, len(peers)),
+		errorC:      make([]<-chan error, len(peers)),
+		proposeC:    make([]chan string, len(peers)),
+		confChangeC: make([]chan raftpb.ConfChange, len(peers)),
+	}
 
 	for i := range clus.peers {
 		os.RemoveAll(fmt.Sprintf("raftexample-%d", i+1))
 		clus.proposeC[i] = make(chan string, 1)
-		clus.commitC[i], clus.errorC[i] = newRaftNode(i+1, clus.peers, clus.proposeC[i])
+		clus.confChangeC[i] = make(chan raftpb.ConfChange, 1)
+		clus.commitC[i], clus.errorC[i] = newRaftNode(i+1, clus.peers, false, clus.proposeC[i], clus.confChangeC[i])
 	}
 
 	return clus