ipfs-cluster/consensus/raft/raft.go

286 lines
7.1 KiB
Go
Raw Normal View History

package raft
2016-12-02 18:33:39 +00:00
import (
"context"
"errors"
"os"
2016-12-02 18:33:39 +00:00
"path/filepath"
"strings"
"time"
2016-12-02 18:33:39 +00:00
hashiraft "github.com/hashicorp/raft"
raftboltdb "github.com/hashicorp/raft-boltdb"
host "github.com/libp2p/go-libp2p-host"
peer "github.com/libp2p/go-libp2p-peer"
libp2praft "github.com/libp2p/go-libp2p-raft"
2016-12-02 18:33:39 +00:00
)
// RaftMaxSnapshots indicates how many snapshots to keep in the consensus data
// folder.
var RaftMaxSnapshots = 5
// is this running 64 bits arch? https://groups.google.com/forum/#!topic/golang-nuts/vAckmhUMAdQ
const sixtyfour = uint64(^uint(0)) == ^uint64(0)
// Raft performs all Raft-specific operations which are needed by Cluster but
// are not fulfilled by the consensus interface. It should contain most of the
// Raft-related stuff so it can be easily replaced in the future, if need be.
type Raft struct {
2016-12-02 18:33:39 +00:00
raft *hashiraft.Raft
transport *libp2praft.Libp2pTransport
snapshotStore hashiraft.SnapshotStore
logStore hashiraft.LogStore
stableStore hashiraft.StableStore
peerstore *libp2praft.Peerstore
boltdb *raftboltdb.BoltStore
dataFolder string
2016-12-02 18:33:39 +00:00
}
// NewRaft launches a go-libp2p-raft consensus peer.
Issue #162: Rework configuration format The following commit reimplements ipfs-cluster configuration under the following premises: * Each component is initialized with a configuration object defined by its module * Each component decides how the JSON representation of its configuration looks like * Each component parses and validates its own configuration * Each component exposes its own defaults * Component configurations are make the sections of a central JSON configuration file (which replaces the current JSON format) * Component configurations implement a common interface (config.ComponentConfig) with a set of common operations * The central configuration file is managed by a config.ConfigManager which: * Registers ComponentConfigs * Assigns the correspondent sections from the JSON file to each component and delegates the parsing * Delegates the JSON generation for each section * Can be notified when the configuration is updated and must be saved to disk The new service.json would then look as follows: ```json { "cluster": { "id": "QmTVW8NoRxC5wBhV7WtAYtRn7itipEESfozWN5KmXUQnk2", "private_key": "<...>", "secret": "00224102ae6aaf94f2606abf69a0e278251ecc1d64815b617ff19d6d2841f786", "peers": [], "bootstrap": [], "leave_on_shutdown": false, "listen_multiaddress": "/ip4/0.0.0.0/tcp/9096", "state_sync_interval": "1m0s", "ipfs_sync_interval": "2m10s", "replication_factor": -1, "monitor_ping_interval": "15s" }, "consensus": { "raft": { "heartbeat_timeout": "1s", "election_timeout": "1s", "commit_timeout": "50ms", "max_append_entries": 64, "trailing_logs": 10240, "snapshot_interval": "2m0s", "snapshot_threshold": 8192, "leader_lease_timeout": "500ms" } }, "api": { "restapi": { "listen_multiaddress": "/ip4/127.0.0.1/tcp/9094", "read_timeout": "30s", "read_header_timeout": "5s", "write_timeout": "1m0s", "idle_timeout": "2m0s" } }, "ipfs_connector": { "ipfshttp": { "proxy_listen_multiaddress": "/ip4/127.0.0.1/tcp/9095", "node_multiaddress": "/ip4/127.0.0.1/tcp/5001", "connect_swarms_delay": "7s", "proxy_read_timeout": "10m0s", "proxy_read_header_timeout": "5s", "proxy_write_timeout": "10m0s", "proxy_idle_timeout": "1m0s" } }, "monitor": { "monbasic": { "check_interval": "15s" } }, "informer": { "disk": { "metric_ttl": "30s", "metric_type": "freespace" }, "numpin": { "metric_ttl": "10s" } } } ``` This new format aims to be easily extensible per component. As such, it already surfaces quite a few new options which were hardcoded before. Additionally, since Go API have changed, some redundant methods have been removed and small refactoring has happened to take advantage of the new way. License: MIT Signed-off-by: Hector Sanjuan <hector@protocol.ai>
2017-10-11 18:23:03 +00:00
func NewRaft(peers []peer.ID, host host.Host, cfg *Config, fsm hashiraft.FSM) (*Raft, error) {
logger.Debug("creating libp2p Raft transport")
2016-12-02 18:33:39 +00:00
transport, err := libp2praft.NewLibp2pTransportWithHost(host)
if err != nil {
logger.Error("creating libp2p-raft transport: ", err)
return nil, err
2016-12-02 18:33:39 +00:00
}
pstore := &libp2praft.Peerstore{}
peersStr := make([]string, len(peers), len(peers))
for i, p := range peers {
peersStr[i] = peer.IDB58Encode(p)
}
pstore.SetPeers(peersStr)
logger.Debug("creating file snapshot store")
Issue #162: Rework configuration format The following commit reimplements ipfs-cluster configuration under the following premises: * Each component is initialized with a configuration object defined by its module * Each component decides how the JSON representation of its configuration looks like * Each component parses and validates its own configuration * Each component exposes its own defaults * Component configurations are make the sections of a central JSON configuration file (which replaces the current JSON format) * Component configurations implement a common interface (config.ComponentConfig) with a set of common operations * The central configuration file is managed by a config.ConfigManager which: * Registers ComponentConfigs * Assigns the correspondent sections from the JSON file to each component and delegates the parsing * Delegates the JSON generation for each section * Can be notified when the configuration is updated and must be saved to disk The new service.json would then look as follows: ```json { "cluster": { "id": "QmTVW8NoRxC5wBhV7WtAYtRn7itipEESfozWN5KmXUQnk2", "private_key": "<...>", "secret": "00224102ae6aaf94f2606abf69a0e278251ecc1d64815b617ff19d6d2841f786", "peers": [], "bootstrap": [], "leave_on_shutdown": false, "listen_multiaddress": "/ip4/0.0.0.0/tcp/9096", "state_sync_interval": "1m0s", "ipfs_sync_interval": "2m10s", "replication_factor": -1, "monitor_ping_interval": "15s" }, "consensus": { "raft": { "heartbeat_timeout": "1s", "election_timeout": "1s", "commit_timeout": "50ms", "max_append_entries": 64, "trailing_logs": 10240, "snapshot_interval": "2m0s", "snapshot_threshold": 8192, "leader_lease_timeout": "500ms" } }, "api": { "restapi": { "listen_multiaddress": "/ip4/127.0.0.1/tcp/9094", "read_timeout": "30s", "read_header_timeout": "5s", "write_timeout": "1m0s", "idle_timeout": "2m0s" } }, "ipfs_connector": { "ipfshttp": { "proxy_listen_multiaddress": "/ip4/127.0.0.1/tcp/9095", "node_multiaddress": "/ip4/127.0.0.1/tcp/5001", "connect_swarms_delay": "7s", "proxy_read_timeout": "10m0s", "proxy_read_header_timeout": "5s", "proxy_write_timeout": "10m0s", "proxy_idle_timeout": "1m0s" } }, "monitor": { "monbasic": { "check_interval": "15s" } }, "informer": { "disk": { "metric_ttl": "30s", "metric_type": "freespace" }, "numpin": { "metric_ttl": "10s" } } } ``` This new format aims to be easily extensible per component. As such, it already surfaces quite a few new options which were hardcoded before. Additionally, since Go API have changed, some redundant methods have been removed and small refactoring has happened to take advantage of the new way. License: MIT Signed-off-by: Hector Sanjuan <hector@protocol.ai>
2017-10-11 18:23:03 +00:00
dataFolder := cfg.DataFolder
if dataFolder == "" {
dataFolder = filepath.Join(cfg.BaseDir, DefaultDataSubFolder)
}
err = os.MkdirAll(dataFolder, 0700)
if err != nil {
logger.Errorf("creating cosensus data folder (%s): %s",
dataFolder, err)
return nil, err
}
snapshots, err := hashiraft.NewFileSnapshotStoreWithLogger(dataFolder, RaftMaxSnapshots, raftStdLogger)
2016-12-02 18:33:39 +00:00
if err != nil {
logger.Error("creating file snapshot store: ", err)
return nil, err
2016-12-02 18:33:39 +00:00
}
logger.Debug("creating BoltDB log store")
logStore, err := raftboltdb.NewBoltStore(filepath.Join(dataFolder, "raft.db"))
2016-12-02 18:33:39 +00:00
if err != nil {
logger.Error("creating bolt store: ", err)
return nil, err
2016-12-02 18:33:39 +00:00
}
logger.Debug("creating Raft")
Issue #162: Rework configuration format The following commit reimplements ipfs-cluster configuration under the following premises: * Each component is initialized with a configuration object defined by its module * Each component decides how the JSON representation of its configuration looks like * Each component parses and validates its own configuration * Each component exposes its own defaults * Component configurations are make the sections of a central JSON configuration file (which replaces the current JSON format) * Component configurations implement a common interface (config.ComponentConfig) with a set of common operations * The central configuration file is managed by a config.ConfigManager which: * Registers ComponentConfigs * Assigns the correspondent sections from the JSON file to each component and delegates the parsing * Delegates the JSON generation for each section * Can be notified when the configuration is updated and must be saved to disk The new service.json would then look as follows: ```json { "cluster": { "id": "QmTVW8NoRxC5wBhV7WtAYtRn7itipEESfozWN5KmXUQnk2", "private_key": "<...>", "secret": "00224102ae6aaf94f2606abf69a0e278251ecc1d64815b617ff19d6d2841f786", "peers": [], "bootstrap": [], "leave_on_shutdown": false, "listen_multiaddress": "/ip4/0.0.0.0/tcp/9096", "state_sync_interval": "1m0s", "ipfs_sync_interval": "2m10s", "replication_factor": -1, "monitor_ping_interval": "15s" }, "consensus": { "raft": { "heartbeat_timeout": "1s", "election_timeout": "1s", "commit_timeout": "50ms", "max_append_entries": 64, "trailing_logs": 10240, "snapshot_interval": "2m0s", "snapshot_threshold": 8192, "leader_lease_timeout": "500ms" } }, "api": { "restapi": { "listen_multiaddress": "/ip4/127.0.0.1/tcp/9094", "read_timeout": "30s", "read_header_timeout": "5s", "write_timeout": "1m0s", "idle_timeout": "2m0s" } }, "ipfs_connector": { "ipfshttp": { "proxy_listen_multiaddress": "/ip4/127.0.0.1/tcp/9095", "node_multiaddress": "/ip4/127.0.0.1/tcp/5001", "connect_swarms_delay": "7s", "proxy_read_timeout": "10m0s", "proxy_read_header_timeout": "5s", "proxy_write_timeout": "10m0s", "proxy_idle_timeout": "1m0s" } }, "monitor": { "monbasic": { "check_interval": "15s" } }, "informer": { "disk": { "metric_ttl": "30s", "metric_type": "freespace" }, "numpin": { "metric_ttl": "10s" } } } ``` This new format aims to be easily extensible per component. As such, it already surfaces quite a few new options which were hardcoded before. Additionally, since Go API have changed, some redundant methods have been removed and small refactoring has happened to take advantage of the new way. License: MIT Signed-off-by: Hector Sanjuan <hector@protocol.ai>
2017-10-11 18:23:03 +00:00
r, err := hashiraft.NewRaft(cfg.HashiraftCfg, fsm, logStore, logStore, snapshots, pstore, transport)
2016-12-02 18:33:39 +00:00
if err != nil {
logger.Error("initializing raft: ", err)
return nil, err
2016-12-02 18:33:39 +00:00
}
raft := &Raft{
2016-12-02 18:33:39 +00:00
raft: r,
transport: transport,
snapshotStore: snapshots,
logStore: logStore,
stableStore: logStore,
peerstore: pstore,
boltdb: logStore,
dataFolder: dataFolder,
}
return raft, nil
2016-12-02 18:33:39 +00:00
}
// WaitForLeader holds until Raft says we have a leader.
// Returns an error if we don't.
func (r *Raft) WaitForLeader(ctx context.Context) error {
// Using Raft observers panics on non-64 architectures.
// This is a work around
if sixtyfour {
return r.waitForLeader(ctx)
}
return r.waitForLeaderLegacy(ctx)
}
func (r *Raft) waitForLeader(ctx context.Context) error {
obsCh := make(chan hashiraft.Observation, 1)
filter := func(o *hashiraft.Observation) bool {
switch o.Data.(type) {
case hashiraft.LeaderObservation:
return true
default:
return false
}
}
observer := hashiraft.NewObserver(obsCh, false, filter)
r.raft.RegisterObserver(observer)
defer r.raft.DeregisterObserver(observer)
ticker := time.NewTicker(time.Second)
for {
select {
case obs := <-obsCh:
switch obs.Data.(type) {
case hashiraft.LeaderObservation:
leaderObs := obs.Data.(hashiraft.LeaderObservation)
logger.Infof("Raft Leader elected: %s", leaderObs.Leader)
return nil
}
case <-ticker.C:
if l := r.raft.Leader(); l != "" { //we missed or there was no election
logger.Debug("waitForleaderTimer")
logger.Infof("Raft Leader elected: %s", l)
ticker.Stop()
return nil
}
case <-ctx.Done():
return ctx.Err()
}
}
}
// 32-bit systems should use this.
func (r *Raft) waitForLeaderLegacy(ctx context.Context) error {
for {
leader := r.raft.Leader()
if leader != "" {
logger.Infof("Raft Leader elected: %s", leader)
return nil
}
select {
case <-ctx.Done():
return ctx.Err()
default:
time.Sleep(500 * time.Millisecond)
}
}
}
// WaitForUpdates holds until Raft has synced to the last index in the log
func (r *Raft) WaitForUpdates(ctx context.Context) error {
logger.Debug("Raft state is catching up")
for {
select {
case <-ctx.Done():
return ctx.Err()
default:
lai := r.raft.AppliedIndex()
li := r.raft.LastIndex()
logger.Debugf("current Raft index: %d/%d",
lai, li)
if lai == li {
return nil
}
time.Sleep(500 * time.Millisecond)
}
}
}
// Snapshot tells Raft to take a snapshot.
func (r *Raft) Snapshot() error {
future := r.raft.Snapshot()
err := future.Error()
if err != nil && !strings.Contains(err.Error(), "Nothing new to snapshot") {
return errors.New("could not take snapshot: " + err.Error())
}
return nil
}
// Shutdown shutdown Raft and closes the BoltDB.
func (r *Raft) Shutdown() error {
future := r.raft.Shutdown()
err := future.Error()
errMsgs := ""
if err != nil {
errMsgs += "could not shutdown raft: " + err.Error() + ".\n"
}
err = r.boltdb.Close() // important!
if err != nil {
errMsgs += "could not close boltdb: " + err.Error()
}
if errMsgs != "" {
return errors.New(errMsgs)
}
// If the shutdown worked correctly
// (including snapshot) we can remove the Raft
// database (which traces peers additions
// and removals). It makes re-start of the peer
// way less confusing for Raft while the state
// can be restored from the snapshot.
//os.Remove(filepath.Join(r.dataFolder, "raft.db"))
return nil
}
// AddPeer adds a peer to Raft
func (r *Raft) AddPeer(peer string) error {
if r.hasPeer(peer) {
logger.Debug("skipping raft add as already in peer set")
return nil
}
future := r.raft.AddPeer(peer)
err := future.Error()
if err != nil {
logger.Error("raft cannot add peer: ", err)
return err
}
peers, _ := r.peerstore.Peers()
logger.Debugf("raft peerstore: %s", peers)
return err
}
// RemovePeer removes a peer from Raft
func (r *Raft) RemovePeer(peer string) error {
if !r.hasPeer(peer) {
return nil
}
future := r.raft.RemovePeer(peer)
err := future.Error()
if err != nil {
logger.Error("raft cannot remove peer: ", err)
return err
}
peers, _ := r.peerstore.Peers()
logger.Debugf("raft peerstore: %s", peers)
return err
}
// func (r *Raft) SetPeers(peers []string) error {
Issue #162: Rework configuration format The following commit reimplements ipfs-cluster configuration under the following premises: * Each component is initialized with a configuration object defined by its module * Each component decides how the JSON representation of its configuration looks like * Each component parses and validates its own configuration * Each component exposes its own defaults * Component configurations are make the sections of a central JSON configuration file (which replaces the current JSON format) * Component configurations implement a common interface (config.ComponentConfig) with a set of common operations * The central configuration file is managed by a config.ConfigManager which: * Registers ComponentConfigs * Assigns the correspondent sections from the JSON file to each component and delegates the parsing * Delegates the JSON generation for each section * Can be notified when the configuration is updated and must be saved to disk The new service.json would then look as follows: ```json { "cluster": { "id": "QmTVW8NoRxC5wBhV7WtAYtRn7itipEESfozWN5KmXUQnk2", "private_key": "<...>", "secret": "00224102ae6aaf94f2606abf69a0e278251ecc1d64815b617ff19d6d2841f786", "peers": [], "bootstrap": [], "leave_on_shutdown": false, "listen_multiaddress": "/ip4/0.0.0.0/tcp/9096", "state_sync_interval": "1m0s", "ipfs_sync_interval": "2m10s", "replication_factor": -1, "monitor_ping_interval": "15s" }, "consensus": { "raft": { "heartbeat_timeout": "1s", "election_timeout": "1s", "commit_timeout": "50ms", "max_append_entries": 64, "trailing_logs": 10240, "snapshot_interval": "2m0s", "snapshot_threshold": 8192, "leader_lease_timeout": "500ms" } }, "api": { "restapi": { "listen_multiaddress": "/ip4/127.0.0.1/tcp/9094", "read_timeout": "30s", "read_header_timeout": "5s", "write_timeout": "1m0s", "idle_timeout": "2m0s" } }, "ipfs_connector": { "ipfshttp": { "proxy_listen_multiaddress": "/ip4/127.0.0.1/tcp/9095", "node_multiaddress": "/ip4/127.0.0.1/tcp/5001", "connect_swarms_delay": "7s", "proxy_read_timeout": "10m0s", "proxy_read_header_timeout": "5s", "proxy_write_timeout": "10m0s", "proxy_idle_timeout": "1m0s" } }, "monitor": { "monbasic": { "check_interval": "15s" } }, "informer": { "disk": { "metric_ttl": "30s", "metric_type": "freespace" }, "numpin": { "metric_ttl": "10s" } } } ``` This new format aims to be easily extensible per component. As such, it already surfaces quite a few new options which were hardcoded before. Additionally, since Go API have changed, some redundant methods have been removed and small refactoring has happened to take advantage of the new way. License: MIT Signed-off-by: Hector Sanjuan <hector@protocol.ai>
2017-10-11 18:23:03 +00:00
// logger.Debugf("SetPeers(): %s", peers)
// future := r.raft.SetPeers(peers)
// err := future.Error()
// if err != nil {
// logger.Error(err)
// }
// return err
// }
// Leader returns Raft's leader. It may be an empty string if
// there is no leader or it is unknown.
func (r *Raft) Leader() string {
return r.raft.Leader()
}
func (r *Raft) hasPeer(peer string) bool {
found := false
peers, _ := r.peerstore.Peers()
for _, p := range peers {
if p == peer {
found = true
break
}
}
return found
}