mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
main: add a helper function for starting raft
This commit is contained in:
parent
0851a1fe7f
commit
8c9d7e3e93
53
main.go
53
main.go
@ -49,29 +49,7 @@ func main() {
|
||||
log.Fatalf("main: cannot create data directory: %v", err)
|
||||
}
|
||||
|
||||
waldir := path.Join(*dir, "wal")
|
||||
|
||||
var w *wal.WAL
|
||||
var n raft.Node
|
||||
if wal.Exist(waldir) {
|
||||
// TODO(xiangli): check snapshot; not open from zero
|
||||
w, err = wal.OpenFromIndex(waldir, 0)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
// TODO(xiangli): save/recovery nodeID?
|
||||
_, st, ents, err := w.ReadAll()
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
n = raft.Restart(id, peers.Ids(), 10, 1, st, ents)
|
||||
} else {
|
||||
w, err = wal.Create(waldir)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
n = raft.Start(id, peers.Ids(), 10, 1)
|
||||
}
|
||||
n, w := startRaft(id, peers.Ids(), path.Join(*dir, "wal"))
|
||||
|
||||
tk := time.NewTicker(100 * time.Millisecond)
|
||||
s := &etcdserver.Server{
|
||||
@ -89,3 +67,32 @@ func main() {
|
||||
http.Handle("/", h)
|
||||
log.Fatal(http.ListenAndServe(*laddr, nil))
|
||||
}
|
||||
|
||||
// startRaft starts a raft node from the given wal dir.
|
||||
// If the wal dir does not exist, startRaft will start a new raft node.
|
||||
// If the wal dir exists, startRaft will restart the previous raft node.
|
||||
// startRaft returns the started raft node and the opened wal.
|
||||
func startRaft(id int64, peerIds []int64, waldir string) (raft.Node, *wal.WAL) {
|
||||
if !wal.Exist(waldir) {
|
||||
w, err := wal.Create(waldir)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
n := raft.Start(id, peerIds, 10, 1)
|
||||
return n, w
|
||||
}
|
||||
|
||||
// restart a node from previous wal
|
||||
// TODO(xiangli): check snapshot; not open from zero
|
||||
w, err := wal.OpenFromIndex(waldir, 0)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
// TODO(xiangli): save/recovery nodeID?
|
||||
_, st, ents, err := w.ReadAll()
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
n := raft.Restart(id, peerIds, 10, 1, st, ents)
|
||||
return n, w
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user