ipfs-cluster/consensus/raft/raft.go
Hector Sanjuan 33d9cdd3c4 Feat: emancipate Consensus from the Cluster component
This commit promotes the Consensus component (and Raft) to become a fully
independent thing like other components, passed to NewCluster during
initialization. Cluster (main component) no longer creates the consensus
layer internally. This has triggered a number of breaking changes
that I will explain below.

Motivation: Future work will require the possibility of running Cluster
with a consensus layer that is not Raft. The "consensus" layer is in charge
of maintaining two things:
  * The current cluster peerset, as required by the implementation
  * The current cluster pinset (shared state)

While the pinset maintenance has always been in the consensus layer, the
peerset maintenance was handled by the main component (starting by the "peers"
key in the configuration) AND the Raft component (internally)
and this generated lots of confusion: if the user edited the peers in the
configuration they would be greeted with an error.

The bootstrap process (adding a peer to an existing cluster) and configuration
key also complicated many things, since the main component did it, but only
when the consensus was initialized and in single peer mode.

In all this we also mixed the peerstore (list of peer addresses in the libp2p
host) with the peerset, when they need not to be linked.

By initializing the consensus layer before calling NewCluster, all the
difficulties in maintaining the current implementation in the same way
have come to light. Thus, the following changes have been introduced:

* Remove "peers" and "bootstrap" keys from the configuration: we no longer
edit or save the configuration files. This was a very bad practice, requiring
write permissions by the process to the file containing the private key and
additionally made things like Puppet deployments of cluster difficult as
configuration would mutate from its initial version. Needless to say all the
maintenance associated to making sure peers and bootstrap had correct values
when peers are bootstrapped or removed. A loud and detailed error message has
been added when staring cluster with an old config, along with instructions on
how to move forward.

* Introduce a PeerstoreFile ("peerstore") which stores peer addresses: in
ipfs, the peerstore is not persisted because it can be re-built from the
network bootstrappers and the DHT. Cluster should probably also allow
discoverability of peers addresses (when not bootstrapping, as in that case
we have it), but in the meantime, we will read and persist the peerstore
addresses for cluster peers in this file, different from the configuration.
Note that dns multiaddresses are now fully supported and no IPs are saved
when we have DNS multiaddresses for a peer.

* The former "peer_manager" code is now a pstoremgr module, providing utilities
to parse, add, list and generally maintain the libp2p host peerstore, including
operations on the PeerstoreFile. This "pstoremgr" can now also be extended to
perform address autodiscovery and other things indepedently from Cluster.

* Create and initialize Raft outside of the main Cluster component: since we
can now launch Raft independently from Cluster, we have more degrees of
freedom. A new "staging" option when creating the object allows a raft peer to
be launched in Staging mode, waiting to be added to a running consensus, and
thus, not electing itself as leader or doing anything like we were doing
before. This additionally allows us to track when the peer has become a
Voter, which only happens when it's caught up with the state, something that
was wonky previously.

* The raft configuration now includes an InitPeerset key, which allows to
provide a peerset for new peers and which is ignored when staging==true. The
whole Raft initialization code is way cleaner and stronger now.

* Cluster peer bootsrapping is now an ipfs-cluster-service feature. The
--bootstrap flag works as before (additionally allowing comma-separated-list
of entries). What bootstrap does, is to initialize Raft with staging == true,
and then call Join in the main cluster component. Only when the Raft peer
transitions to Voter, consensus becomes ready, and cluster becomes Ready.
This is cleaner, works better and is less complex than before (supporting
both flags and config values). We also backup and clean the state whenever
we are boostrapping, automatically

* ipfs-cluster-service no longer runs the daemon. Starting cluster needs
now "ipfs-cluster-service daemon". The daemon specific flags (bootstrap,
alloc) are now flags for the daemon subcommand. Here we mimic ipfs ("ipfs"
does not start the daemon but print help) and pave the path for merging both
service and ctl in the future.

While this brings some breaking changes, it significantly reduces the
complexity of the configuration, the code and most importantly, the
documentation. It should be easier now to explain the user what is the
right way to launch a cluster peer, and more difficult to make mistakes.

As a side effect, the PR also:

* Fixes #381 - peers with dynamic addresses
* Fixes #371 - peers should be Raft configuration option
* Fixes #378 - waitForUpdates may return before state fully synced
* Fixes #235 - config option shadowing (no cfg saves, no need to shadow)

License: MIT
Signed-off-by: Hector Sanjuan <code@hector.link>
2018-05-07 07:39:41 +02:00

663 lines
17 KiB
Go

package raft
import (
"context"
"errors"
"fmt"
"io"
"os"
"path/filepath"
"time"
hraft "github.com/hashicorp/raft"
raftboltdb "github.com/hashicorp/raft-boltdb"
host "github.com/libp2p/go-libp2p-host"
peer "github.com/libp2p/go-libp2p-peer"
p2praft "github.com/libp2p/go-libp2p-raft"
"github.com/ipfs/ipfs-cluster/state"
)
// errBadRaftState is returned when the consensus component cannot start
// because the cluster peers do not match the raft peers.
var errBadRaftState = errors.New("cluster peers do not match raft peers")
// ErrWaitingForSelf is returned when we are waiting for ourselves to depart
// the peer set, which won't happen
var errWaitingForSelf = errors.New("waiting for ourselves to depart")
// RaftMaxSnapshots indicates how many snapshots to keep in the consensus data
// folder.
// TODO: Maybe include this in Config. Not sure how useful it is to touch
// this anyways.
var RaftMaxSnapshots = 5
// RaftLogCacheSize is the maximum number of logs to cache in-memory.
// This is used to reduce disk I/O for the recently committed entries.
var RaftLogCacheSize = 512
// Are we compiled on a 64-bit architecture?
// https://groups.google.com/forum/#!topic/golang-nuts/vAckmhUMAdQ
// This is used below because raft Observers panic on 32-bit.
const sixtyfour = uint64(^uint(0)) == ^uint64(0)
// How long we wait for updates during shutdown before snapshotting
var waitForUpdatesShutdownTimeout = 5 * time.Second
var waitForUpdatesInterval = 100 * time.Millisecond
// How many times to retry snapshotting when shutting down
var maxShutdownSnapshotRetries = 5
// raftWrapper wraps the hraft.Raft object and related things like the
// different stores used or the hraft.Configuration.
// Its methods provide functionality for working with Raft.
type raftWrapper struct {
raft *hraft.Raft
config *Config
host host.Host
serverConfig hraft.Configuration
transport *hraft.NetworkTransport
snapshotStore hraft.SnapshotStore
logStore hraft.LogStore
stableStore hraft.StableStore
boltdb *raftboltdb.BoltStore
staging bool
}
// newRaftWrapper creates a Raft instance and initializes
// everything leaving it ready to use. Note, that Bootstrap() should be called
// to make sure the raft instance is usable.
func newRaftWrapper(
host host.Host,
cfg *Config,
fsm hraft.FSM,
staging bool,
) (*raftWrapper, error) {
raftW := &raftWrapper{}
raftW.config = cfg
raftW.host = host
raftW.staging = staging
// Set correct LocalID
cfg.RaftConfig.LocalID = hraft.ServerID(peer.IDB58Encode(host.ID()))
df := cfg.GetDataFolder()
err := makeDataFolder(df)
if err != nil {
return nil, err
}
raftW.makeServerConfig()
err = raftW.makeTransport()
if err != nil {
return nil, err
}
err = raftW.makeStores()
if err != nil {
return nil, err
}
logger.Debug("creating Raft")
raftW.raft, err = hraft.NewRaft(
cfg.RaftConfig,
fsm,
raftW.logStore,
raftW.stableStore,
raftW.snapshotStore,
raftW.transport,
)
if err != nil {
logger.Error("initializing raft: ", err)
return nil, err
}
return raftW, nil
}
// makeDataFolder creates the folder that is meant
// to store Raft data.
func makeDataFolder(folder string) error {
err := os.MkdirAll(folder, 0700)
if err != nil {
return err
}
return nil
}
func (rw *raftWrapper) makeTransport() (err error) {
logger.Debug("creating libp2p Raft transport")
rw.transport, err = p2praft.NewLibp2pTransport(
rw.host,
rw.config.NetworkTimeout,
)
return err
}
func (rw *raftWrapper) makeStores() error {
logger.Debug("creating BoltDB store")
df := rw.config.GetDataFolder()
store, err := raftboltdb.NewBoltStore(filepath.Join(df, "raft.db"))
if err != nil {
return err
}
// wraps the store in a LogCache to improve performance.
// See consul/agent/consul/server.go
cacheStore, err := hraft.NewLogCache(RaftLogCacheSize, store)
if err != nil {
return err
}
logger.Debug("creating raft snapshot store")
snapstore, err := hraft.NewFileSnapshotStoreWithLogger(
df,
RaftMaxSnapshots,
raftStdLogger,
)
if err != nil {
return err
}
rw.logStore = cacheStore
rw.stableStore = store
rw.snapshotStore = snapstore
rw.boltdb = store
return nil
}
// Bootstrap calls BootstrapCluster on the Raft instance with a valid
// Configuration (generated from InitPeerset) when Raft has no state
// and we are not setting up a staging peer. It returns if Raft
// was boostrapped (true) and an error.
func (rw *raftWrapper) Bootstrap() (bool, error) {
logger.Debug("checking for existing raft states")
hasState, err := hraft.HasExistingState(
rw.logStore,
rw.stableStore,
rw.snapshotStore,
)
if err != nil {
return false, err
}
if hasState {
logger.Debug("raft cluster is already initialized")
// Inform the user that we are working with a pre-existing peerset
logger.Info("existing Raft state found! raft.InitPeerset will be ignored")
cf := rw.raft.GetConfiguration()
if err := cf.Error(); err != nil {
logger.Debug(err)
return false, err
}
currentCfg := cf.Configuration()
srvs := ""
for _, s := range currentCfg.Servers {
srvs += fmt.Sprintf(" %s\n", s.ID)
}
logger.Debugf("Current Raft Peerset:\n%s\n", srvs)
return false, nil
}
if rw.staging {
logger.Debug("staging servers do not need initialization")
logger.Info("peer is ready to join a cluster")
return false, nil
}
voters := ""
for _, s := range rw.serverConfig.Servers {
voters += fmt.Sprintf(" %s\n", s.ID)
}
logger.Infof("initializing raft cluster with the following voters:\n%s\n", voters)
future := rw.raft.BootstrapCluster(rw.serverConfig)
if err := future.Error(); err != nil {
logger.Error("bootstrapping cluster: ", err)
return true, err
}
return true, nil
}
// create Raft servers configuration. The result is used
// by Bootstrap() when it proceeds to Bootstrap.
func (rw *raftWrapper) makeServerConfig() {
rw.serverConfig = makeServerConf(append(rw.config.InitPeerset, rw.host.ID()))
}
// creates a server configuration with all peers as Voters.
func makeServerConf(peers []peer.ID) hraft.Configuration {
sm := make(map[string]struct{})
servers := make([]hraft.Server, 0)
// Servers are peers + self. We avoid duplicate entries below
for _, pid := range peers {
p := peer.IDB58Encode(pid)
_, ok := sm[p]
if !ok { // avoid dups
sm[p] = struct{}{}
servers = append(servers, hraft.Server{
Suffrage: hraft.Voter,
ID: hraft.ServerID(p),
Address: hraft.ServerAddress(p),
})
}
}
return hraft.Configuration{Servers: servers}
}
// WaitForLeader holds until Raft says we have a leader.
// Returns uf ctx is cancelled.
func (rw *raftWrapper) WaitForLeader(ctx context.Context) (string, error) {
obsCh := make(chan hraft.Observation, 1)
if sixtyfour { // 32-bit systems don't support observers
observer := hraft.NewObserver(obsCh, false, nil)
rw.raft.RegisterObserver(observer)
defer rw.raft.DeregisterObserver(observer)
}
ticker := time.NewTicker(time.Second / 2)
for {
select {
case obs := <-obsCh:
_ = obs
// See https://github.com/hashicorp/raft/issues/254
// switch obs.Data.(type) {
// case hraft.LeaderObservation:
// lObs := obs.Data.(hraft.LeaderObservation)
// logger.Infof("Raft Leader elected: %s",
// lObs.Leader)
// return string(lObs.Leader), nil
// }
case <-ticker.C:
if l := rw.raft.Leader(); l != "" {
logger.Debug("waitForleaderTimer")
logger.Infof("Current Raft Leader: %s", l)
ticker.Stop()
return string(l), nil
}
case <-ctx.Done():
return "", ctx.Err()
}
}
}
func (rw *raftWrapper) WaitForVoter(ctx context.Context) error {
logger.Debug("waiting until we are promoted to a voter")
pid := hraft.ServerID(peer.IDB58Encode(rw.host.ID()))
for {
select {
case <-ctx.Done():
return ctx.Err()
default:
configFuture := rw.raft.GetConfiguration()
if err := configFuture.Error(); err != nil {
return err
}
if isVoter(pid, configFuture.Configuration()) {
return nil
}
time.Sleep(waitForUpdatesInterval)
}
}
}
func isVoter(srvID hraft.ServerID, cfg hraft.Configuration) bool {
for _, server := range cfg.Servers {
if server.ID == srvID && server.Suffrage == hraft.Voter {
return true
}
}
return false
}
// WaitForUpdates holds until Raft has synced to the last index in the log
func (rw *raftWrapper) WaitForUpdates(ctx context.Context) error {
logger.Debug("Raft state is catching up to the latest known version. Please wait...")
for {
select {
case <-ctx.Done():
return ctx.Err()
default:
lai := rw.raft.AppliedIndex()
li := rw.raft.LastIndex()
logger.Debugf("current Raft index: %d/%d",
lai, li)
if lai == li {
return nil
}
time.Sleep(waitForUpdatesInterval)
}
}
}
func (rw *raftWrapper) WaitForPeer(ctx context.Context, pid string, depart bool) error {
for {
select {
case <-ctx.Done():
return ctx.Err()
default:
peers, err := rw.Peers()
if err != nil {
return err
}
if len(peers) == 1 && pid == peers[0] && depart {
return errWaitingForSelf
}
found := find(peers, pid)
// departing
if depart && !found {
return nil
}
// joining
if !depart && found {
return nil
}
time.Sleep(50 * time.Millisecond)
}
}
}
// Snapshot tells Raft to take a snapshot.
func (rw *raftWrapper) Snapshot() error {
future := rw.raft.Snapshot()
err := future.Error()
if err != nil && err.Error() != hraft.ErrNothingNewToSnapshot.Error() {
return err
}
return nil
}
// snapshotOnShutdown attempts to take a snapshot before a shutdown.
// Snapshotting might fail if the raft applied index is not the last index.
// This waits for the updates and tries to take a snapshot when the
// applied index is up to date.
// It will retry if the snapshot still fails, in case more updates have arrived.
// If waiting for updates times-out, it will not try anymore, since something
// is wrong. This is a best-effort solution as there is no way to tell Raft
// to stop processing entries because we want to take a snapshot before
// shutting down.
func (rw *raftWrapper) snapshotOnShutdown() error {
var err error
for i := 0; i < maxShutdownSnapshotRetries; i++ {
done := false
ctx, cancel := context.WithTimeout(context.Background(), waitForUpdatesShutdownTimeout)
err := rw.WaitForUpdates(ctx)
cancel()
if err != nil {
logger.Warning("timed out waiting for state updates before shutdown. Snapshotting may fail")
done = true // let's not wait for updates again
}
err = rw.Snapshot()
if err != nil {
err = errors.New("could not snapshot raft: " + err.Error())
} else {
err = nil
done = true
}
if done {
break
}
logger.Warningf("retrying to snapshot (%d/%d)...", i+1, maxShutdownSnapshotRetries)
}
return err
}
// Shutdown shutdown Raft and closes the BoltDB.
func (rw *raftWrapper) Shutdown() error {
errMsgs := ""
err := rw.snapshotOnShutdown()
if err != nil {
errMsgs += err.Error() + ".\n"
}
future := rw.raft.Shutdown()
err = future.Error()
if err != nil {
errMsgs += "could not shutdown raft: " + err.Error() + ".\n"
}
err = rw.boltdb.Close() // important!
if err != nil {
errMsgs += "could not close boltdb: " + err.Error()
}
if errMsgs != "" {
return errors.New(errMsgs)
}
return nil
}
// AddPeer adds a peer to Raft
func (rw *raftWrapper) AddPeer(peer string) error {
// Check that we don't have it to not waste
// log entries if so.
peers, err := rw.Peers()
if err != nil {
return err
}
if find(peers, peer) {
logger.Infof("%s is already a raft peer", peer)
return nil
}
future := rw.raft.AddVoter(
hraft.ServerID(peer),
hraft.ServerAddress(peer),
0,
0) // TODO: Extra cfg value?
err = future.Error()
if err != nil {
logger.Error("raft cannot add peer: ", err)
}
return err
}
// RemovePeer removes a peer from Raft
func (rw *raftWrapper) RemovePeer(peer string) error {
// Check that we have it to not waste
// log entries if we don't.
peers, err := rw.Peers()
if err != nil {
return err
}
if !find(peers, peer) {
logger.Infof("%s is not among raft peers", peer)
return nil
}
if len(peers) == 1 && peers[0] == peer {
return errors.New("cannot remove ourselves from a 1-peer cluster")
}
rmFuture := rw.raft.RemoveServer(
hraft.ServerID(peer),
0,
0) // TODO: Extra cfg value?
err = rmFuture.Error()
if err != nil {
logger.Error("raft cannot remove peer: ", err)
return err
}
return nil
}
// Leader returns Raft's leader. It may be an empty string if
// there is no leader or it is unknown.
func (rw *raftWrapper) Leader() string {
return string(rw.raft.Leader())
}
func (rw *raftWrapper) Peers() ([]string, error) {
ids := make([]string, 0)
configFuture := rw.raft.GetConfiguration()
if err := configFuture.Error(); err != nil {
return nil, err
}
for _, server := range configFuture.Configuration().Servers {
ids = append(ids, string(server.ID))
}
return ids, nil
}
// latestSnapshot looks for the most recent raft snapshot stored at the
// provided basedir. It returns the snapshot's metadata, and a reader
// to the snapshot's bytes
func latestSnapshot(raftDataFolder string) (*hraft.SnapshotMeta, io.ReadCloser, error) {
store, err := hraft.NewFileSnapshotStore(raftDataFolder, RaftMaxSnapshots, nil)
if err != nil {
return nil, nil, err
}
snapMetas, err := store.List()
if err != nil {
return nil, nil, err
}
if len(snapMetas) == 0 { // no error if snapshot isn't found
return nil, nil, nil
}
meta, r, err := store.Open(snapMetas[0].ID)
if err != nil {
return nil, nil, err
}
return meta, r, nil
}
// LastStateRaw returns the bytes of the last snapshot stored, its metadata,
// and a flag indicating whether any snapshot was found.
func LastStateRaw(cfg *Config) (io.Reader, bool, error) {
// Read most recent snapshot
dataFolder := cfg.GetDataFolder()
if _, err := os.Stat(dataFolder); os.IsNotExist(err) {
// nothing to read
return nil, false, nil
}
meta, r, err := latestSnapshot(dataFolder)
if err != nil {
return nil, false, err
}
if meta == nil { // no snapshots could be read
return nil, false, nil
}
return r, true, nil
}
// SnapshotSave saves the provided state to a snapshot in the
// raft data path. Old raft data is backed up and replaced
// by the new snapshot. pids contains the config-specified
// peer ids to include in the snapshot metadata if no snapshot exists
// from which to copy the raft metadata
func SnapshotSave(cfg *Config, newState state.State, pids []peer.ID) error {
newStateBytes, err := p2praft.EncodeSnapshot(newState)
if err != nil {
return err
}
dataFolder := cfg.GetDataFolder()
err = makeDataFolder(dataFolder)
if err != nil {
return err
}
meta, _, err := latestSnapshot(dataFolder)
if err != nil {
return err
}
// make a new raft snapshot
var raftSnapVersion hraft.SnapshotVersion
raftSnapVersion = 1 // As of hraft v1.0.0 this is always 1
configIndex := uint64(1)
var raftIndex uint64
var raftTerm uint64
var srvCfg hraft.Configuration
if meta != nil {
raftIndex = meta.Index
raftTerm = meta.Term
srvCfg = meta.Configuration
CleanupRaft(dataFolder, cfg.BackupsRotate)
} else {
// Begin the log after the index of a fresh start so that
// the snapshot's state propagate's during bootstrap
raftIndex = uint64(2)
raftTerm = uint64(1)
srvCfg = makeServerConf(pids)
}
snapshotStore, err := hraft.NewFileSnapshotStoreWithLogger(dataFolder, RaftMaxSnapshots, nil)
if err != nil {
return err
}
_, dummyTransport := hraft.NewInmemTransport("")
sink, err := snapshotStore.Create(raftSnapVersion, raftIndex, raftTerm, srvCfg, configIndex, dummyTransport)
if err != nil {
return err
}
_, err = sink.Write(newStateBytes)
if err != nil {
sink.Cancel()
return err
}
err = sink.Close()
if err != nil {
return err
}
return nil
}
// CleanupRaft moves the current data folder to a backup location
func CleanupRaft(dataFolder string, keep int) error {
meta, _, err := latestSnapshot(dataFolder)
if meta == nil && err == nil {
// no snapshots at all. Avoid creating backups
// from empty state folders.
logger.Infof("cleaning empty Raft data folder (%s)", dataFolder)
os.RemoveAll(dataFolder)
return nil
}
logger.Infof("cleaning and backing up Raft data folder (%s)", dataFolder)
dbh := newDataBackupHelper(dataFolder, keep)
err = dbh.makeBackup()
if err != nil {
logger.Warning(err)
logger.Warning("the state could not be cleaned properly")
logger.Warning("manual intervention may be needed before starting cluster again")
}
return nil
}
// only call when Raft is shutdown
func (rw *raftWrapper) Clean() error {
return CleanupRaft(rw.config.GetDataFolder(), rw.config.BackupsRotate)
}
func find(s []string, elem string) bool {
for _, selem := range s {
if selem == elem {
return true
}
}
return false
}