add state sync reactor (#4705)

Fixes #828. Adds state sync, as outlined in [ADR-053](https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-053-state-sync-prototype.md). See related PRs in Cosmos SDK (https://github.com/cosmos/cosmos-sdk/pull/5803) and Gaia (https://github.com/cosmos/gaia/pull/327).

This is split out of the previous PR #4645, and branched off of the ABCI interface in #4704. 

* Adds a new P2P reactor which exchanges snapshots with peers, and bootstraps an empty local node from remote snapshots when requested.

* Adds a new configuration section `[statesync]` that enables state sync and configures the light client. Also enables `statesync:info` logging by default.

* Integrates state sync into node startup. Does not support the v2 blockchain reactor, since it needs some reorganization to defer startup.
This commit is contained in:
Erik Grinaker
2020-04-29 10:47:00 +02:00
committed by GitHub
parent 569981325a
commit 511ab6717c
32 changed files with 4145 additions and 106 deletions

View File

@@ -30,6 +30,7 @@ import (
"github.com/tendermint/tendermint/libs/log"
tmpubsub "github.com/tendermint/tendermint/libs/pubsub"
"github.com/tendermint/tendermint/libs/service"
lite "github.com/tendermint/tendermint/lite2"
mempl "github.com/tendermint/tendermint/mempool"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/p2p/pex"
@@ -43,6 +44,7 @@ import (
"github.com/tendermint/tendermint/state/txindex"
"github.com/tendermint/tendermint/state/txindex/kv"
"github.com/tendermint/tendermint/state/txindex/null"
"github.com/tendermint/tendermint/statesync"
"github.com/tendermint/tendermint/store"
"github.com/tendermint/tendermint/types"
tmtime "github.com/tendermint/tendermint/types/time"
@@ -123,6 +125,12 @@ func DefaultMetricsProvider(config *cfg.InstrumentationConfig) MetricsProvider {
// Option sets a parameter for the node.
type Option func(*Node)
// Temporary interface for switching to fast sync, we should get rid of v0 and v1 reactors.
// See: https://github.com/tendermint/tendermint/issues/4595
type fastSyncReactor interface {
SwitchToFastSync(sm.State) error
}
// CustomReactors allows you to add custom reactors (name -> p2p.Reactor) to
// the node's Switch.
//
@@ -134,6 +142,7 @@ type Option func(*Node)
// - CONSENSUS
// - EVIDENCE
// - PEX
// - STATESYNC
func CustomReactors(reactors map[string]p2p.Reactor) Option {
return func(n *Node) {
for name, reactor := range reactors {
@@ -147,6 +156,15 @@ func CustomReactors(reactors map[string]p2p.Reactor) Option {
}
}
// StateProvider overrides the state provider used by state sync to retrieve trusted app hashes and
// build a State object for bootstrapping the node.
// WARNING: this interface is considered unstable and subject to change.
func StateProvider(stateProvider statesync.StateProvider) Option {
return func(n *Node) {
n.stateSyncProvider = stateProvider
}
}
//------------------------------------------------------------------------------
// Node is the highest level interface to a full Tendermint node.
@@ -168,21 +186,24 @@ type Node struct {
isListening bool
// services
eventBus *types.EventBus // pub/sub for services
stateDB dbm.DB
blockStore *store.BlockStore // store the blockchain to disk
bcReactor p2p.Reactor // for fast-syncing
mempoolReactor *mempl.Reactor // for gossipping transactions
mempool mempl.Mempool
consensusState *cs.State // latest consensus state
consensusReactor *cs.Reactor // for participating in the consensus
pexReactor *pex.Reactor // for exchanging peer addresses
evidencePool *evidence.Pool // tracking evidence
proxyApp proxy.AppConns // connection to the application
rpcListeners []net.Listener // rpc servers
txIndexer txindex.TxIndexer
indexerService *txindex.IndexerService
prometheusSrv *http.Server
eventBus *types.EventBus // pub/sub for services
stateDB dbm.DB
blockStore *store.BlockStore // store the blockchain to disk
bcReactor p2p.Reactor // for fast-syncing
mempoolReactor *mempl.Reactor // for gossipping transactions
mempool mempl.Mempool
stateSync bool // whether the node should state sync on startup
stateSyncReactor *statesync.Reactor // for hosting and restoring state sync snapshots
stateSyncProvider statesync.StateProvider // provides state data for bootstrapping a node
consensusState *cs.State // latest consensus state
consensusReactor *cs.Reactor // for participating in the consensus
pexReactor *pex.Reactor // for exchanging peer addresses
evidencePool *evidence.Pool // tracking evidence
proxyApp proxy.AppConns // connection to the application
rpcListeners []net.Listener // rpc servers
txIndexer txindex.TxIndexer
indexerService *txindex.IndexerService
prometheusSrv *http.Server
}
func initDBs(config *cfg.Config, dbProvider DBProvider) (blockStore *store.BlockStore, stateDB dbm.DB, err error) {
@@ -369,7 +390,7 @@ func createConsensusReactor(config *cfg.Config,
evidencePool *evidence.Pool,
privValidator types.PrivValidator,
csMetrics *cs.Metrics,
fastSync bool,
waitSync bool,
eventBus *types.EventBus,
consensusLogger log.Logger) (*consensus.Reactor, *consensus.State) {
@@ -386,7 +407,7 @@ func createConsensusReactor(config *cfg.Config,
if privValidator != nil {
consensusState.SetPrivValidator(privValidator)
}
consensusReactor := cs.NewReactor(consensusState, fastSync, cs.ReactorMetrics(csMetrics))
consensusReactor := cs.NewReactor(consensusState, waitSync, cs.ReactorMetrics(csMetrics))
consensusReactor.SetLogger(consensusLogger)
// services which will be publishing and/or subscribing for messages (events)
// consensusReactor will set it on consensusState and blockExecutor
@@ -469,6 +490,7 @@ func createSwitch(config *cfg.Config,
peerFilters []p2p.PeerFilterFunc,
mempoolReactor *mempl.Reactor,
bcReactor p2p.Reactor,
stateSyncReactor *statesync.Reactor,
consensusReactor *consensus.Reactor,
evidenceReactor *evidence.Reactor,
nodeInfo p2p.NodeInfo,
@@ -486,6 +508,7 @@ func createSwitch(config *cfg.Config,
sw.AddReactor("BLOCKCHAIN", bcReactor)
sw.AddReactor("CONSENSUS", consensusReactor)
sw.AddReactor("EVIDENCE", evidenceReactor)
sw.AddReactor("STATESYNC", stateSyncReactor)
sw.SetNodeInfo(nodeInfo)
sw.SetNodeKey(nodeKey)
@@ -542,6 +565,59 @@ func createPEXReactorAndAddToSwitch(addrBook pex.AddrBook, config *cfg.Config,
return pexReactor
}
// startStateSync starts an asynchronous state sync process, then switches to fast sync mode.
func startStateSync(ssR *statesync.Reactor, bcR fastSyncReactor, conR *consensus.Reactor,
stateProvider statesync.StateProvider, config *cfg.StateSyncConfig, fastSync bool,
stateDB dbm.DB, blockStore *store.BlockStore) error {
ssR.Logger.Info("Starting state sync")
state := sm.LoadState(stateDB)
if stateProvider == nil {
var err error
stateProvider, err = statesync.NewLightClientStateProvider(state.ChainID, state.Version,
config.RPCServers, lite.TrustOptions{
Period: config.TrustPeriod,
Height: config.TrustHeight,
Hash: config.TrustHashBytes(),
}, ssR.Logger.With("module", "lite"))
if err != nil {
return fmt.Errorf("failed to set up light client state provider: %w", err)
}
}
go func() {
state, commit, err := ssR.Sync(stateProvider)
if err != nil {
ssR.Logger.Error("State sync failed", "err", err)
return
}
err = sm.BootstrapState(stateDB, state)
if err != nil {
ssR.Logger.Error("Failed to bootstrap node with new state", "err", err)
return
}
err = blockStore.SaveSeenCommit(state.LastBlockHeight, commit)
if err != nil {
ssR.Logger.Error("Failed to store last seen commit", "err", err)
return
}
if fastSync {
// FIXME Very ugly to have these metrics bleed through here.
conR.Metrics.StateSyncing.Set(0)
conR.Metrics.FastSyncing.Set(1)
err = bcR.SwitchToFastSync(state)
if err != nil {
ssR.Logger.Error("Failed to switch to fast sync", "err", err)
return
}
} else {
conR.SwitchToConsensus(state, true)
}
}()
return nil
}
// NewNode returns a new, ready to go, Tendermint Node.
func NewNode(config *cfg.Config,
privValidator types.PrivValidator,
@@ -584,18 +660,6 @@ func NewNode(config *cfg.Config,
return nil, err
}
// Create the handshaker, which calls RequestInfo, sets the AppVersion on the state,
// and replays any blocks as necessary to sync tendermint with the app.
consensusLogger := logger.With("module", "consensus")
if err := doHandshake(stateDB, state, blockStore, genDoc, eventBus, proxyApp, consensusLogger); err != nil {
return nil, err
}
// Reload the state. It will have the Version.Consensus.App set by the
// Handshake, and may have other modifications as well (ie. depending on
// what happened during block replay).
state = sm.LoadState(stateDB)
// If an address is provided, listen on the socket for a connection from an
// external signing process.
if config.PrivValidatorListenAddr != "" {
@@ -611,11 +675,34 @@ func NewNode(config *cfg.Config,
return nil, errors.Wrap(err, "can't get pubkey")
}
logNodeStartupInfo(state, pubKey, logger, consensusLogger)
// Decide whether to fast-sync or not
// Determine whether we should do state and/or fast sync.
// We don't fast-sync when the only validator is us.
fastSync := config.FastSyncMode && !onlyValidatorIsUs(state, pubKey)
stateSync := config.StateSync.Enable && !onlyValidatorIsUs(state, pubKey)
if stateSync && state.LastBlockHeight > 0 {
logger.Info("Found local state with non-zero height, skipping state sync")
stateSync = false
}
// Don't check fastSync == true, since the v2 reactor has a bug where it fast syncs regardless.
if stateSync && config.FastSync.Version == "v2" {
return nil, errors.New("state sync is not supported with blockchain v2 reactor")
}
// Create the handshaker, which calls RequestInfo, sets the AppVersion on the state,
// and replays any blocks as necessary to sync tendermint with the app.
consensusLogger := logger.With("module", "consensus")
if !stateSync {
if err := doHandshake(stateDB, state, blockStore, genDoc, eventBus, proxyApp, consensusLogger); err != nil {
return nil, err
}
// Reload the state. It will have the Version.Consensus.App set by the
// Handshake, and may have other modifications as well (ie. depending on
// what happened during block replay).
state = sm.LoadState(stateDB)
}
logNodeStartupInfo(state, pubKey, logger, consensusLogger)
csMetrics, p2pMetrics, memplMetrics, smMetrics := metricsProvider(genDoc.ChainID)
@@ -638,18 +725,32 @@ func NewNode(config *cfg.Config,
sm.BlockExecutorWithMetrics(smMetrics),
)
// Make BlockchainReactor
bcReactor, err := createBlockchainReactor(config, state, blockExec, blockStore, fastSync, logger)
// Make BlockchainReactor. Don't start fast sync if we're doing a state sync first.
bcReactor, err := createBlockchainReactor(config, state, blockExec, blockStore, fastSync && !stateSync, logger)
if err != nil {
return nil, errors.Wrap(err, "could not create blockchain reactor")
}
// Make ConsensusReactor
// Make ConsensusReactor. Don't enable fully if doing a state sync and/or fast sync first.
// FIXME We need to update metrics here, since other reactors don't have access to them.
if stateSync {
csMetrics.StateSyncing.Set(1)
} else if fastSync {
csMetrics.FastSyncing.Set(1)
}
consensusReactor, consensusState := createConsensusReactor(
config, state, blockExec, blockStore, mempool, evidencePool,
privValidator, csMetrics, fastSync, eventBus, consensusLogger,
privValidator, csMetrics, stateSync || fastSync, eventBus, consensusLogger,
)
// Set up state sync reactor, and schedule a sync if requested.
// FIXME The way we do phased startups (e.g. replay -> fast sync -> consensus) is very messy,
// we should clean this whole thing up. See:
// https://github.com/tendermint/tendermint/issues/4644
stateSyncReactor := statesync.NewReactor(proxyApp.Snapshot(), proxyApp.Query(),
config.StateSync.TempDir)
stateSyncReactor.SetLogger(logger.With("module", "statesync"))
nodeInfo, err := makeNodeInfo(config, nodeKey, txIndexer, genDoc, state)
if err != nil {
return nil, err
@@ -662,7 +763,7 @@ func NewNode(config *cfg.Config,
p2pLogger := logger.With("module", "p2p")
sw := createSwitch(
config, transport, p2pMetrics, peerFilters, mempoolReactor, bcReactor,
consensusReactor, evidenceReactor, nodeInfo, nodeKey, p2pLogger,
stateSyncReactor, consensusReactor, evidenceReactor, nodeInfo, nodeKey, p2pLogger,
)
err = sw.AddPersistentPeers(splitAndTrimEmpty(config.P2P.PersistentPeers, ",", " "))
@@ -721,6 +822,8 @@ func NewNode(config *cfg.Config,
mempool: mempool,
consensusState: consensusState,
consensusReactor: consensusReactor,
stateSyncReactor: stateSyncReactor,
stateSync: stateSync,
pexReactor: pexReactor,
evidencePool: evidencePool,
proxyApp: proxyApp,
@@ -791,6 +894,19 @@ func (n *Node) OnStart() error {
return errors.Wrap(err, "could not dial peers from persistent_peers field")
}
// Run state sync
if n.stateSync {
bcR, ok := n.bcReactor.(fastSyncReactor)
if !ok {
return fmt.Errorf("this blockchain reactor does not support switching from state sync")
}
err := startStateSync(n.stateSyncReactor, bcR, n.consensusReactor, n.stateSyncProvider,
n.config.StateSync, n.config.FastSyncMode, n.stateDB, n.blockStore)
if err != nil {
return fmt.Errorf("failed to start state sync: %w", err)
}
}
return nil
}
@@ -1106,6 +1222,7 @@ func makeNodeInfo(
cs.StateChannel, cs.DataChannel, cs.VoteChannel, cs.VoteSetBitsChannel,
mempl.MempoolChannel,
evidence.EvidenceChannel,
statesync.SnapshotChannel, statesync.ChunkChannel,
},
Moniker: config.Moniker,
Other: p2p.DefaultNodeInfoOther{