ipfs-cluster/consensus/raft/consensus.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

469 lines
12 KiB
Go

// Package raft implements a Consensus component for IPFS Cluster which uses
// Raft (go-libp2p-raft).
package raft
import (
"context"
"errors"
"fmt"
"sort"
"sync"
"time"
"github.com/ipfs/ipfs-cluster/api"
"github.com/ipfs/ipfs-cluster/state"
rpc "github.com/hsanjuan/go-libp2p-gorpc"
logging "github.com/ipfs/go-log"
consensus "github.com/libp2p/go-libp2p-consensus"
host "github.com/libp2p/go-libp2p-host"
peer "github.com/libp2p/go-libp2p-peer"
libp2praft "github.com/libp2p/go-libp2p-raft"
ma "github.com/multiformats/go-multiaddr"
)
var logger = logging.Logger("consensus")
// Consensus handles the work of keeping a shared-state between
// the peers of an IPFS Cluster, as well as modifying that state and
// applying any updates in a thread-safe manner.
type Consensus struct {
ctx context.Context
cancel func()
config *Config
host host.Host
consensus consensus.OpLogConsensus
actor consensus.Actor
baseOp *LogOp
raft *raftWrapper
rpcClient *rpc.Client
rpcReady chan struct{}
readyCh chan struct{}
shutdownLock sync.Mutex
shutdown bool
}
// NewConsensus builds a new ClusterConsensus component using Raft. The state
// is used to initialize the Consensus system, so any information
// in it is discarded once the raft state is loaded.
// The singlePeer parameter controls whether this Raft peer is be expected to
// join a cluster or it should run on its own.
func NewConsensus(
host host.Host,
cfg *Config,
state state.State,
staging bool, // this peer must not be bootstrapped if no state exists
) (*Consensus, error) {
err := cfg.Validate()
if err != nil {
return nil, err
}
baseOp := &LogOp{}
logger.Debug("starting Consensus and waiting for a leader...")
consensus := libp2praft.NewOpLog(state, baseOp)
raft, err := newRaftWrapper(host, cfg, consensus.FSM(), staging)
if err != nil {
logger.Error("error creating raft: ", err)
return nil, err
}
actor := libp2praft.NewActor(raft.raft)
consensus.SetActor(actor)
ctx, cancel := context.WithCancel(context.Background())
cc := &Consensus{
ctx: ctx,
cancel: cancel,
config: cfg,
host: host,
consensus: consensus,
actor: actor,
baseOp: baseOp,
raft: raft,
rpcReady: make(chan struct{}, 1),
readyCh: make(chan struct{}, 1),
}
baseOp.consensus = cc
go cc.finishBootstrap()
return cc, nil
}
// WaitForSync waits for a leader and for the state to be up to date, then returns.
func (cc *Consensus) WaitForSync() error {
leaderCtx, cancel := context.WithTimeout(
cc.ctx,
cc.config.WaitForLeaderTimeout)
defer cancel()
// 1 - wait for leader
// 2 - wait until we are a Voter
// 3 - wait until last index is applied
// From raft docs:
// once a staging server receives enough log entries to be sufficiently
// caught up to the leader's log, the leader will invoke a membership
// change to change the Staging server to a Voter
// Thus, waiting to be a Voter is a guarantee that we have a reasonable
// up to date state. Otherwise, we might return too early (see
// https://github.com/ipfs/ipfs-cluster/issues/378)
_, err := cc.raft.WaitForLeader(leaderCtx)
if err != nil {
return errors.New("error waiting for leader: " + err.Error())
}
err = cc.raft.WaitForVoter(cc.ctx)
if err != nil {
return errors.New("error waiting to become a Voter: " + err.Error())
}
err = cc.raft.WaitForUpdates(cc.ctx)
if err != nil {
return errors.New("error waiting for consensus updates: " + err.Error())
}
return nil
}
// waits until there is a consensus leader and syncs the state
// to the tracker. If errors happen, this will return and never
// signal the component as Ready.
func (cc *Consensus) finishBootstrap() {
// wait until we have RPC to perform any actions.
if cc.rpcClient == nil {
select {
case <-cc.ctx.Done():
return
case <-cc.rpcReady:
}
}
// Sometimes bootstrap is a no-op. It only applies when
// no state exists and staging=false.
_, err := cc.raft.Bootstrap()
if err != nil {
return
}
err = cc.WaitForSync()
if err != nil {
return
}
logger.Debug("Raft state is now up to date")
logger.Debug("consensus ready")
cc.readyCh <- struct{}{}
}
// Shutdown stops the component so it will not process any
// more updates. The underlying consensus is permanently
// shutdown, along with the libp2p transport.
func (cc *Consensus) Shutdown() error {
cc.shutdownLock.Lock()
defer cc.shutdownLock.Unlock()
if cc.shutdown {
logger.Debug("already shutdown")
return nil
}
logger.Info("stopping Consensus component")
// Raft Shutdown
err := cc.raft.Shutdown()
if err != nil {
logger.Error(err)
}
if cc.config.hostShutdown {
cc.host.Close()
}
cc.shutdown = true
cc.cancel()
close(cc.rpcReady)
return nil
}
// SetClient makes the component ready to perform RPC requets
func (cc *Consensus) SetClient(c *rpc.Client) {
cc.rpcClient = c
cc.rpcReady <- struct{}{}
}
// Ready returns a channel which is signaled when the Consensus
// algorithm has finished bootstrapping and is ready to use
func (cc *Consensus) Ready() <-chan struct{} {
return cc.readyCh
}
func (cc *Consensus) op(pin api.Pin, t LogOpType) *LogOp {
return &LogOp{
Cid: pin.ToSerial(),
Type: t,
}
}
// returns true if the operation was redirected to the leader
// note that if the leader just dissappeared, the rpc call will
// fail because we haven't heard that it's gone.
func (cc *Consensus) redirectToLeader(method string, arg interface{}) (bool, error) {
var finalErr error
// Retry redirects
for i := 0; i <= cc.config.CommitRetries; i++ {
logger.Debugf("redirect try %d", i)
leader, err := cc.Leader()
// No leader, wait for one
if err != nil {
logger.Warning("there seems to be no leader. Waiting for one")
rctx, cancel := context.WithTimeout(
cc.ctx,
cc.config.WaitForLeaderTimeout)
defer cancel()
pidstr, err := cc.raft.WaitForLeader(rctx)
// means we timed out waiting for a leader
// we don't retry in this case
if err != nil {
return false, fmt.Errorf("timed out waiting for leader: %s", err)
}
leader, err = peer.IDB58Decode(pidstr)
if err != nil {
return false, err
}
}
// We are the leader. Do not redirect
if leader == cc.host.ID() {
return false, nil
}
logger.Debugf("redirecting %s to leader: %s", method, leader.Pretty())
finalErr = cc.rpcClient.Call(
leader,
"Cluster",
method,
arg,
&struct{}{})
if finalErr != nil {
logger.Error(finalErr)
logger.Error("retrying to redirect request to leader")
time.Sleep(2 * cc.config.RaftConfig.HeartbeatTimeout)
continue
}
break
}
// We tried to redirect, but something happened
return true, finalErr
}
// commit submits a cc.consensus commit. It retries upon failures.
func (cc *Consensus) commit(op *LogOp, rpcOp string, redirectArg interface{}) error {
var finalErr error
for i := 0; i <= cc.config.CommitRetries; i++ {
logger.Debugf("attempt #%d: committing %+v", i, op)
// this means we are retrying
if finalErr != nil {
logger.Errorf("retrying upon failed commit (retry %d): %s ",
i, finalErr)
}
// try to send it to the leader
// redirectToLeader has it's own retry loop. If this fails
// we're done here.
ok, err := cc.redirectToLeader(rpcOp, redirectArg)
if err != nil || ok {
return err
}
// Being here means we are the LEADER. We can commit.
// now commit the changes to our state
cc.shutdownLock.Lock() // do not shut down while committing
_, finalErr = cc.consensus.CommitOp(op)
cc.shutdownLock.Unlock()
if finalErr != nil {
goto RETRY
}
switch op.Type {
case LogOpPin:
logger.Infof("pin committed to global state: %s", op.Cid.Cid)
case LogOpUnpin:
logger.Infof("unpin committed to global state: %s", op.Cid.Cid)
}
break
RETRY:
time.Sleep(cc.config.CommitRetryDelay)
}
return finalErr
}
// LogPin submits a Cid to the shared state of the cluster. It will forward
// the operation to the leader if this is not it.
func (cc *Consensus) LogPin(pin api.Pin) error {
op := cc.op(pin, LogOpPin)
err := cc.commit(op, "ConsensusLogPin", pin.ToSerial())
if err != nil {
return err
}
return nil
}
// LogUnpin removes a Cid from the shared state of the cluster.
func (cc *Consensus) LogUnpin(pin api.Pin) error {
op := cc.op(pin, LogOpUnpin)
err := cc.commit(op, "ConsensusLogUnpin", pin.ToSerial())
if err != nil {
return err
}
return nil
}
// AddPeer adds a new peer to participate in this consensus. It will
// forward the operation to the leader if this is not it.
func (cc *Consensus) AddPeer(pid peer.ID) error {
var finalErr error
for i := 0; i <= cc.config.CommitRetries; i++ {
logger.Debugf("attempt #%d: AddPeer %s", i, pid.Pretty())
if finalErr != nil {
logger.Errorf("retrying to add peer. Attempt #%d failed: %s", i, finalErr)
}
ok, err := cc.redirectToLeader("ConsensusAddPeer", pid)
if err != nil || ok {
return err
}
// Being here means we are the leader and can commit
cc.shutdownLock.Lock() // do not shutdown while committing
finalErr = cc.raft.AddPeer(peer.IDB58Encode(pid))
cc.shutdownLock.Unlock()
if finalErr != nil {
time.Sleep(cc.config.CommitRetryDelay)
continue
}
logger.Infof("peer added to Raft: %s", pid.Pretty())
break
}
return finalErr
}
// RmPeer removes a peer from this consensus. It will
// forward the operation to the leader if this is not it.
func (cc *Consensus) RmPeer(pid peer.ID) error {
var finalErr error
for i := 0; i <= cc.config.CommitRetries; i++ {
logger.Debugf("attempt #%d: RmPeer %s", i, pid.Pretty())
if finalErr != nil {
logger.Errorf("retrying to remove peer. Attempt #%d failed: %s", i, finalErr)
}
ok, err := cc.redirectToLeader("ConsensusRmPeer", pid)
if err != nil || ok {
return err
}
// Being here means we are the leader and can commit
cc.shutdownLock.Lock() // do not shutdown while committing
finalErr = cc.raft.RemovePeer(peer.IDB58Encode(pid))
cc.shutdownLock.Unlock()
if finalErr != nil {
time.Sleep(cc.config.CommitRetryDelay)
continue
}
logger.Infof("peer removed from Raft: %s", pid.Pretty())
break
}
return finalErr
}
// State retrieves the current consensus State. It may error
// if no State has been agreed upon or the state is not
// consistent. The returned State is the last agreed-upon
// State known by this node.
func (cc *Consensus) State() (state.State, error) {
st, err := cc.consensus.GetLogHead()
if err != nil {
return nil, err
}
state, ok := st.(state.State)
if !ok {
return nil, errors.New("wrong state type")
}
return state, nil
}
// Leader returns the peerID of the Leader of the
// cluster. It returns an error when there is no leader.
func (cc *Consensus) Leader() (peer.ID, error) {
// Note the hard-dependency on raft here...
raftactor := cc.actor.(*libp2praft.Actor)
return raftactor.Leader()
}
// Clean removes all raft data from disk. Next time
// a full new peer will be bootstrapped.
func (cc *Consensus) Clean() error {
if !cc.shutdown {
return errors.New("consensus component is not shutdown")
}
err := cc.raft.Clean()
if err != nil {
return err
}
return nil
}
// Rollback replaces the current agreed-upon
// state with the state provided. Only the consensus leader
// can perform this operation.
func (cc *Consensus) Rollback(state state.State) error {
// This is unused. It *might* be used for upgrades.
// There is rather untested magic in libp2p-raft's FSM()
// to make this possible.
return cc.consensus.Rollback(state)
}
// Peers return the current list of peers in the consensus.
// The list will be sorted alphabetically.
func (cc *Consensus) Peers() ([]peer.ID, error) {
if cc.shutdown { // things hang a lot in this case
return nil, errors.New("consensus is shutdown")
}
peers := []peer.ID{}
raftPeers, err := cc.raft.Peers()
if err != nil {
return nil, fmt.Errorf("cannot retrieve list of peers: %s", err)
}
sort.Strings(raftPeers)
for _, p := range raftPeers {
id, err := peer.IDB58Decode(p)
if err != nil {
panic("could not decode peer")
}
peers = append(peers, id)
}
return peers, nil
}
func parsePIDFromMultiaddr(addr ma.Multiaddr) string {
pidstr, err := addr.ValueForProtocol(ma.P_IPFS)
if err != nil {
panic("peer badly encoded")
}
return pidstr
}