From 511ab6717c3690c045315534d6680bf579b72b80 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Wed, 29 Apr 2020 10:47:00 +0200 Subject: [PATCH] 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. --- CHANGELOG_PENDING.md | 1 + blockchain/v0/reactor.go | 22 +- blockchain/v1/reactor.go | 20 +- blockchain/v1/reactor_test.go | 2 +- blockchain/v2/io.go | 8 +- blockchain/v2/reactor.go | 2 +- blockchain/v2/reactor_test.go | 2 +- config/config.go | 93 ++++- config/config_test.go | 6 + config/toml.go | 24 ++ consensus/byzantine_test.go | 8 +- consensus/metrics.go | 9 + consensus/reactor.go | 61 ++- consensus/reactor_test.go | 2 +- node/node.go | 191 +++++++-- p2p/mocks/peer.go | 331 ++++++++++++++++ p2p/peer.go | 2 + rpc/core/status.go | 4 +- state/store.go | 10 + statesync/chunks.go | 321 +++++++++++++++ statesync/chunks_test.go | 551 ++++++++++++++++++++++++++ statesync/messages.go | 129 ++++++ statesync/messages_test.go | 106 +++++ statesync/mocks/state_provider.go | 82 ++++ statesync/reactor.go | 261 ++++++++++++ statesync/reactor_test.go | 148 +++++++ statesync/snapshots.go | 263 ++++++++++++ statesync/snapshots_test.go | 326 +++++++++++++++ statesync/stateprovider.go | 179 +++++++++ statesync/syncer.go | 442 +++++++++++++++++++++ statesync/syncer_test.go | 639 ++++++++++++++++++++++++++++++ store/store.go | 6 + 32 files changed, 4145 insertions(+), 106 deletions(-) create mode 100644 p2p/mocks/peer.go create mode 100644 statesync/chunks.go create mode 100644 statesync/chunks_test.go create mode 100644 statesync/messages.go create mode 100644 statesync/messages_test.go create mode 100644 statesync/mocks/state_provider.go create mode 100644 statesync/reactor.go create mode 100644 statesync/reactor_test.go create mode 100644 statesync/snapshots.go create mode 100644 statesync/snapshots_test.go create mode 100644 statesync/stateprovider.go create mode 100644 statesync/syncer.go create mode 100644 statesync/syncer_test.go diff --git a/CHANGELOG_PENDING.md b/CHANGELOG_PENDING.md index daec02e88..bb86caff4 100644 --- a/CHANGELOG_PENDING.md +++ b/CHANGELOG_PENDING.md @@ -27,6 +27,7 @@ Friendly reminder, we have a [bug bounty program](https://hackerone.com/tendermi ### FEATURES: +- [statesync] Add state sync support, where a new node can be rapidly bootstrapped by fetching state snapshots from peers instead of replaying blocks. See the `[statesync]` config section. - [evidence] [\#4532](https://github.com/tendermint/tendermint/pull/4532) Handle evidence from light clients (@melekes) - [lite2] [\#4532](https://github.com/tendermint/tendermint/pull/4532) Submit conflicting headers, if any, to a full node & all witnesses (@melekes) diff --git a/blockchain/v0/reactor.go b/blockchain/v0/reactor.go index c82457bee..1adde6bd5 100644 --- a/blockchain/v0/reactor.go +++ b/blockchain/v0/reactor.go @@ -41,7 +41,7 @@ const ( type consensusReactor interface { // for when we switch from blockchain reactor and fast sync to // the consensus machine - SwitchToConsensus(sm.State, uint64) + SwitchToConsensus(state sm.State, skipWAL bool) } type peerError struct { @@ -115,11 +115,25 @@ func (bcR *BlockchainReactor) OnStart() error { if err != nil { return err } - go bcR.poolRoutine() + go bcR.poolRoutine(false) } return nil } +// SwitchToFastSync is called by the state sync reactor when switching to fast sync. +func (bcR *BlockchainReactor) SwitchToFastSync(state sm.State) error { + bcR.fastSync = true + bcR.initialState = state + + bcR.pool.height = state.LastBlockHeight + 1 + err := bcR.pool.Start() + if err != nil { + return err + } + go bcR.poolRoutine(true) + return nil +} + // OnStop implements service.Service. func (bcR *BlockchainReactor) OnStop() { bcR.pool.Stop() @@ -213,7 +227,7 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) // Handle messages from the poolReactor telling the reactor what to do. // NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down! -func (bcR *BlockchainReactor) poolRoutine() { +func (bcR *BlockchainReactor) poolRoutine(stateSynced bool) { trySyncTicker := time.NewTicker(trySyncIntervalMS * time.Millisecond) statusUpdateTicker := time.NewTicker(statusUpdateIntervalSeconds * time.Second) @@ -273,7 +287,7 @@ FOR_LOOP: bcR.pool.Stop() conR, ok := bcR.Switch.Reactor("CONSENSUS").(consensusReactor) if ok { - conR.SwitchToConsensus(state, blocksSynced) + conR.SwitchToConsensus(state, blocksSynced > 0 || stateSynced) } // else { // should only happen during testing diff --git a/blockchain/v1/reactor.go b/blockchain/v1/reactor.go index 315dbffbf..970825d14 100644 --- a/blockchain/v1/reactor.go +++ b/blockchain/v1/reactor.go @@ -44,7 +44,7 @@ var ( type consensusReactor interface { // for when we switch from blockchain reactor and fast sync to // the consensus machine - SwitchToConsensus(sm.State, uint64) + SwitchToConsensus(state sm.State, skipWAL bool) } // BlockchainReactor handles long-term catchup syncing. @@ -57,7 +57,8 @@ type BlockchainReactor struct { blockExec *sm.BlockExecutor store *store.BlockStore - fastSync bool + fastSync bool + stateSynced bool fsm *BcReactorFSM blocksSynced uint64 @@ -154,6 +155,19 @@ func (bcR *BlockchainReactor) OnStop() { _ = bcR.Stop() } +// SwitchToFastSync is called by the state sync reactor when switching to fast sync. +func (bcR *BlockchainReactor) SwitchToFastSync(state sm.State) error { + bcR.fastSync = true + bcR.initialState = state + bcR.state = state + bcR.stateSynced = true + + bcR.fsm = NewFSM(state.LastBlockHeight+1, bcR) + bcR.fsm.SetLogger(bcR.Logger) + go bcR.poolRoutine() + return nil +} + // GetChannels implements Reactor func (bcR *BlockchainReactor) GetChannels() []*p2p.ChannelDescriptor { return []*p2p.ChannelDescriptor{ @@ -474,7 +488,7 @@ func (bcR *BlockchainReactor) sendBlockRequest(peerID p2p.ID, height int64) erro func (bcR *BlockchainReactor) switchToConsensus() { conR, ok := bcR.Switch.Reactor("CONSENSUS").(consensusReactor) if ok { - conR.SwitchToConsensus(bcR.state, bcR.blocksSynced) + conR.SwitchToConsensus(bcR.state, bcR.blocksSynced > 0 || bcR.stateSynced) bcR.eventsFromFSMCh <- bcFsmMessage{event: syncFinishedEv} } // else { diff --git a/blockchain/v1/reactor_test.go b/blockchain/v1/reactor_test.go index e0b3472bf..0e6f21222 100644 --- a/blockchain/v1/reactor_test.go +++ b/blockchain/v1/reactor_test.go @@ -166,7 +166,7 @@ type consensusReactorTest struct { mtx sync.Mutex } -func (conR *consensusReactorTest) SwitchToConsensus(state sm.State, blocksSynced uint64) { +func (conR *consensusReactorTest) SwitchToConsensus(state sm.State, blocksSynced bool) { conR.mtx.Lock() defer conR.mtx.Unlock() conR.switchedToConsensus = true diff --git a/blockchain/v2/io.go b/blockchain/v2/io.go index 32cf3aeaf..29850d3e5 100644 --- a/blockchain/v2/io.go +++ b/blockchain/v2/io.go @@ -16,7 +16,7 @@ type iIO interface { broadcastStatusRequest(base int64, height int64) - trySwitchToConsensus(state state.State, blocksSynced int) + trySwitchToConsensus(state state.State, skipWAL bool) } type switchIO struct { @@ -37,7 +37,7 @@ const ( type consensusReactor interface { // for when we switch from blockchain reactor and fast sync to // the consensus machine - SwitchToConsensus(state.State, int) + SwitchToConsensus(state state.State, skipWAL bool) } func (sio *switchIO) sendBlockRequest(peerID p2p.ID, height int64) error { @@ -97,10 +97,10 @@ func (sio *switchIO) sendBlockNotFound(height int64, peerID p2p.ID) error { return nil } -func (sio *switchIO) trySwitchToConsensus(state state.State, blocksSynced int) { +func (sio *switchIO) trySwitchToConsensus(state state.State, skipWAL bool) { conR, ok := sio.sw.Reactor("CONSENSUS").(consensusReactor) if ok { - conR.SwitchToConsensus(state, blocksSynced) + conR.SwitchToConsensus(state, skipWAL) } } diff --git a/blockchain/v2/reactor.go b/blockchain/v2/reactor.go index d68b27a41..1fb8ada1a 100644 --- a/blockchain/v2/reactor.go +++ b/blockchain/v2/reactor.go @@ -395,7 +395,7 @@ func (r *BlockchainReactor) demux() { case pcBlockVerificationFailure: r.scheduler.send(event) case pcFinished: - r.io.trySwitchToConsensus(event.tmState, event.blocksSynced) + r.io.trySwitchToConsensus(event.tmState, event.blocksSynced > 0) r.processor.stop() } diff --git a/blockchain/v2/reactor_test.go b/blockchain/v2/reactor_test.go index 10b1d23df..e782c35ac 100644 --- a/blockchain/v2/reactor_test.go +++ b/blockchain/v2/reactor_test.go @@ -117,7 +117,7 @@ func (sio *mockSwitchIo) sendBlockNotFound(height int64, peerID p2p.ID) error { return nil } -func (sio *mockSwitchIo) trySwitchToConsensus(state sm.State, blocksSynced int) { +func (sio *mockSwitchIo) trySwitchToConsensus(state sm.State, skipWAL bool) { sio.mtx.Lock() defer sio.mtx.Unlock() sio.switchedToConsensus = true diff --git a/config/config.go b/config/config.go index 173713279..49ccf186a 100644 --- a/config/config.go +++ b/config/config.go @@ -1,13 +1,13 @@ package config import ( + "encoding/hex" + "errors" "fmt" "net/http" "os" "path/filepath" "time" - - "github.com/pkg/errors" ) const ( @@ -65,6 +65,7 @@ type Config struct { RPC *RPCConfig `mapstructure:"rpc"` P2P *P2PConfig `mapstructure:"p2p"` Mempool *MempoolConfig `mapstructure:"mempool"` + StateSync *StateSyncConfig `mapstructure:"statesync"` FastSync *FastSyncConfig `mapstructure:"fastsync"` Consensus *ConsensusConfig `mapstructure:"consensus"` TxIndex *TxIndexConfig `mapstructure:"tx_index"` @@ -78,6 +79,7 @@ func DefaultConfig() *Config { RPC: DefaultRPCConfig(), P2P: DefaultP2PConfig(), Mempool: DefaultMempoolConfig(), + StateSync: DefaultStateSyncConfig(), FastSync: DefaultFastSyncConfig(), Consensus: DefaultConsensusConfig(), TxIndex: DefaultTxIndexConfig(), @@ -92,6 +94,7 @@ func TestConfig() *Config { RPC: TestRPCConfig(), P2P: TestP2PConfig(), Mempool: TestMempoolConfig(), + StateSync: TestStateSyncConfig(), FastSync: TestFastSyncConfig(), Consensus: TestConsensusConfig(), TxIndex: TestTxIndexConfig(), @@ -116,24 +119,27 @@ func (cfg *Config) ValidateBasic() error { return err } if err := cfg.RPC.ValidateBasic(); err != nil { - return errors.Wrap(err, "Error in [rpc] section") + return fmt.Errorf("error in [rpc] section: %w", err) } if err := cfg.P2P.ValidateBasic(); err != nil { - return errors.Wrap(err, "Error in [p2p] section") + return fmt.Errorf("error in [p2p] section: %w", err) } if err := cfg.Mempool.ValidateBasic(); err != nil { - return errors.Wrap(err, "Error in [mempool] section") + return fmt.Errorf("error in [mempool] section: %w", err) + } + if err := cfg.StateSync.ValidateBasic(); err != nil { + return fmt.Errorf("error in [statesync] section: %w", err) } if err := cfg.FastSync.ValidateBasic(); err != nil { - return errors.Wrap(err, "Error in [fastsync] section") + return fmt.Errorf("error in [fastsync] section: %w", err) } if err := cfg.Consensus.ValidateBasic(); err != nil { - return errors.Wrap(err, "Error in [consensus] section") + return fmt.Errorf("error in [consensus] section: %w", err) } - return errors.Wrap( - cfg.Instrumentation.ValidateBasic(), - "Error in [instrumentation] section", - ) + if err := cfg.Instrumentation.ValidateBasic(); err != nil { + return fmt.Errorf("error in [instrumentation] section: %w", err) + } + return nil } //----------------------------------------------------------------------------- @@ -292,7 +298,7 @@ func DefaultLogLevel() string { // DefaultPackageLogLevels returns a default log level setting so all packages // log at "error", while the `state` and `main` packages log at "info" func DefaultPackageLogLevels() string { - return fmt.Sprintf("main:info,state:info,*:%s", DefaultLogLevel()) + return fmt.Sprintf("main:info,state:info,statesync:info,*:%s", DefaultLogLevel()) } //----------------------------------------------------------------------------- @@ -701,6 +707,69 @@ func (cfg *MempoolConfig) ValidateBasic() error { return nil } +//----------------------------------------------------------------------------- +// StateSyncConfig + +// StateSyncConfig defines the configuration for the Tendermint state sync service +type StateSyncConfig struct { + Enable bool `mapstructure:"enable"` + TempDir string `mapstructure:"temp_dir"` + RPCServers []string `mapstructure:"rpc_servers"` + TrustPeriod time.Duration `mapstructure:"trust_period"` + TrustHeight int64 `mapstructure:"trust_height"` + TrustHash string `mapstructure:"trust_hash"` +} + +func (cfg *StateSyncConfig) TrustHashBytes() []byte { + // validated in ValidateBasic, so we can safely panic here + bytes, err := hex.DecodeString(cfg.TrustHash) + if err != nil { + panic(err) + } + return bytes +} + +// DefaultStateSyncConfig returns a default configuration for the state sync service +func DefaultStateSyncConfig() *StateSyncConfig { + return &StateSyncConfig{} +} + +// TestFastSyncConfig returns a default configuration for the state sync service +func TestStateSyncConfig() *StateSyncConfig { + return DefaultStateSyncConfig() +} + +// ValidateBasic performs basic validation. +func (cfg *StateSyncConfig) ValidateBasic() error { + if cfg.Enable { + if len(cfg.RPCServers) == 0 { + return errors.New("rpc_servers is required") + } + if len(cfg.RPCServers) < 2 { + return errors.New("at least two rpc_servers entries is required") + } + for _, server := range cfg.RPCServers { + if len(server) == 0 { + return errors.New("found empty rpc_servers entry") + } + } + if cfg.TrustPeriod <= 0 { + return errors.New("trusted_period is required") + } + if cfg.TrustHeight <= 0 { + return errors.New("trusted_height is required") + } + if len(cfg.TrustHash) == 0 { + return errors.New("trusted_hash is required") + } + _, err := hex.DecodeString(cfg.TrustHash) + if err != nil { + return fmt.Errorf("invalid trusted_hash: %w", err) + } + } + return nil +} + //----------------------------------------------------------------------------- // FastSyncConfig diff --git a/config/config_test.go b/config/config_test.go index c83f1c3f5..4cd161125 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -6,6 +6,7 @@ import ( "time" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestDefaultConfig(t *testing.T) { @@ -122,6 +123,11 @@ func TestMempoolConfigValidateBasic(t *testing.T) { } } +func TestStateSyncConfigValidateBasic(t *testing.T) { + cfg := TestStateSyncConfig() + require.NoError(t, cfg.ValidateBasic()) +} + func TestFastSyncConfigValidateBasic(t *testing.T) { cfg := TestFastSyncConfig() assert.NoError(t, cfg.ValidateBasic()) diff --git a/config/toml.go b/config/toml.go index 3fe4d1aac..5fac85cf8 100644 --- a/config/toml.go +++ b/config/toml.go @@ -315,6 +315,30 @@ cache_size = {{ .Mempool.CacheSize }} # NOTE: the max size of a tx transmitted over the network is {max_tx_bytes} + {amino overhead}. max_tx_bytes = {{ .Mempool.MaxTxBytes }} +##### state sync configuration options ##### +[statesync] +# State sync rapidly bootstraps a new node by discovering, fetching, and restoring a state machine +# snapshot from peers instead of fetching and replaying historical blocks. Requires some peers in +# the network to take and serve state machine snapshots. State sync is not attempted if the node +# has any local state (LastBlockHeight > 0). The node will have a truncated block history, +# starting from the height of the snapshot. +enable = {{ .StateSync.Enable }} + +# RPC servers (comma-separated) for light client verification of the synced state machine and +# retrieval of state data for node bootstrapping. Also needs a trusted height and corresponding +# header hash obtained from a trusted source, and a period during which validators can be trusted. +# +# For Cosmos SDK-based chains, trust_period should usually be about 2/3 of the unbonding time (~2 +# weeks) during which they can be financially punished (slashed) for misbehavior. +rpc_servers = "" +trust_height = {{ .StateSync.TrustHeight }} +trust_hash = "{{ .StateSync.TrustHash }}" +trust_period = "{{ .StateSync.TrustPeriod }}" + +# Temporary directory for state sync snapshot chunks, defaults to the OS tempdir (typically /tmp). +# Will create a new, randomly named directory within, and remove it when done. +temp_dir = "{{ .StateSync.TempDir }}" + ##### fast sync configuration options ##### [fastsync] diff --git a/consensus/byzantine_test.go b/consensus/byzantine_test.go index 649d47fbb..5b9ec1768 100644 --- a/consensus/byzantine_test.go +++ b/consensus/byzantine_test.go @@ -112,13 +112,13 @@ func TestByzantine(t *testing.T) { // note these must be started before the byz for i := 1; i < N; i++ { cr := reactors[i].(*Reactor) - cr.SwitchToConsensus(cr.conS.GetState(), 0) + cr.SwitchToConsensus(cr.conS.GetState(), false) } // start the byzantine state machine byzR := reactors[0].(*ByzantineReactor) s := byzR.reactor.conS.GetState() - byzR.reactor.SwitchToConsensus(s, 0) + byzR.reactor.SwitchToConsensus(s, false) // byz proposer sends one block to peers[0] // and the other block to peers[1] and peers[2]. @@ -268,8 +268,8 @@ func (br *ByzantineReactor) AddPeer(peer p2p.Peer) { peer.Set(types.PeerStateKey, peerState) // Send our state to peer. - // If we're fast_syncing, broadcast a RoundStepMessage later upon SwitchToConsensus(). - if !br.reactor.fastSync { + // If we're syncing, broadcast a RoundStepMessage later upon SwitchToConsensus(). + if !br.reactor.waitSync { br.reactor.sendNewRoundStepMessage(peer) } } diff --git a/consensus/metrics.go b/consensus/metrics.go index 5fa27118a..b4c6458dc 100644 --- a/consensus/metrics.go +++ b/consensus/metrics.go @@ -55,6 +55,8 @@ type Metrics struct { CommittedHeight metrics.Gauge // Whether or not a node is fast syncing. 1 if yes, 0 if no. FastSyncing metrics.Gauge + // Whether or not a node is state syncing. 1 if yes, 0 if no. + StateSyncing metrics.Gauge // Number of blockparts transmitted by peer. BlockParts metrics.Counter @@ -174,6 +176,12 @@ func PrometheusMetrics(namespace string, labelsAndValues ...string) *Metrics { Name: "fast_syncing", Help: "Whether or not a node is fast syncing. 1 if yes, 0 if no.", }, labels).With(labelsAndValues...), + StateSyncing: prometheus.NewGaugeFrom(stdprometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: MetricsSubsystem, + Name: "state_syncing", + Help: "Whether or not a node is state syncing. 1 if yes, 0 if no.", + }, labels).With(labelsAndValues...), BlockParts: prometheus.NewCounterFrom(stdprometheus.CounterOpts{ Namespace: namespace, Subsystem: MetricsSubsystem, @@ -208,6 +216,7 @@ func NopMetrics() *Metrics { TotalTxs: discard.NewGauge(), CommittedHeight: discard.NewGauge(), FastSyncing: discard.NewGauge(), + StateSyncing: discard.NewGauge(), BlockParts: discard.NewCounter(), } } diff --git a/consensus/reactor.go b/consensus/reactor.go index 99f40ac0b..eb9dd0871 100644 --- a/consensus/reactor.go +++ b/consensus/reactor.go @@ -41,23 +41,22 @@ type Reactor struct { conS *State mtx sync.RWMutex - fastSync bool + waitSync bool eventBus *types.EventBus - metrics *Metrics + Metrics *Metrics } type ReactorOption func(*Reactor) // NewReactor returns a new Reactor with the given // consensusState. -func NewReactor(consensusState *State, fastSync bool, options ...ReactorOption) *Reactor { +func NewReactor(consensusState *State, waitSync bool, options ...ReactorOption) *Reactor { conR := &Reactor{ conS: consensusState, - fastSync: fastSync, - metrics: NopMetrics(), + waitSync: waitSync, + Metrics: NopMetrics(), } - conR.updateFastSyncingMetric() conR.BaseReactor = *p2p.NewBaseReactor("Consensus", conR) for _, option := range options { @@ -70,14 +69,14 @@ func NewReactor(consensusState *State, fastSync bool, options ...ReactorOption) // OnStart implements BaseService by subscribing to events, which later will be // broadcasted to other peers and starting state if we're not in fast sync. func (conR *Reactor) OnStart() error { - conR.Logger.Info("Reactor ", "fastSync", conR.FastSync()) + conR.Logger.Info("Reactor ", "waitSync", conR.WaitSync()) // start routine that computes peer statistics for evaluating peer quality go conR.peerStatsRoutine() conR.subscribeToBroadcastEvents() - if !conR.FastSync() { + if !conR.WaitSync() { err := conR.conS.Start() if err != nil { return err @@ -92,14 +91,14 @@ func (conR *Reactor) OnStart() error { func (conR *Reactor) OnStop() { conR.unsubscribeFromBroadcastEvents() conR.conS.Stop() - if !conR.FastSync() { + if !conR.WaitSync() { conR.conS.Wait() } } // SwitchToConsensus switches from fast_sync mode to consensus mode. // It resets the state, turns off fast_sync, and starts the consensus state-machine -func (conR *Reactor) SwitchToConsensus(state sm.State, blocksSynced uint64) { +func (conR *Reactor) SwitchToConsensus(state sm.State, skipWAL bool) { conR.Logger.Info("SwitchToConsensus") conR.conS.reconstructLastCommit(state) // NOTE: The line below causes broadcastNewRoundStepRoutine() to @@ -107,12 +106,12 @@ func (conR *Reactor) SwitchToConsensus(state sm.State, blocksSynced uint64) { conR.conS.updateToState(state) conR.mtx.Lock() - conR.fastSync = false + conR.waitSync = false conR.mtx.Unlock() - conR.metrics.FastSyncing.Set(0) + conR.Metrics.FastSyncing.Set(0) + conR.Metrics.StateSyncing.Set(0) - if blocksSynced > 0 { - // dont bother with the WAL if we fast synced + if skipWAL { conR.conS.doWALCatchup = false } err := conR.conS.Start() @@ -187,7 +186,7 @@ func (conR *Reactor) AddPeer(peer p2p.Peer) { // Send our state to peer. // If we're fast_syncing, broadcast a RoundStepMessage later upon SwitchToConsensus(). - if !conR.FastSync() { + if !conR.WaitSync() { conR.sendNewRoundStepMessage(peer) } } @@ -284,8 +283,8 @@ func (conR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { } case DataChannel: - if conR.FastSync() { - conR.Logger.Info("Ignoring message received during fastSync", "msg", msg) + if conR.WaitSync() { + conR.Logger.Info("Ignoring message received during sync", "msg", msg) return } switch msg := msg.(type) { @@ -296,15 +295,15 @@ func (conR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { ps.ApplyProposalPOLMessage(msg) case *BlockPartMessage: ps.SetHasProposalBlockPart(msg.Height, msg.Round, msg.Part.Index) - conR.metrics.BlockParts.With("peer_id", string(src.ID())).Add(1) + conR.Metrics.BlockParts.With("peer_id", string(src.ID())).Add(1) conR.conS.peerMsgQueue <- msgInfo{msg, src.ID()} default: conR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg))) } case VoteChannel: - if conR.FastSync() { - conR.Logger.Info("Ignoring message received during fastSync", "msg", msg) + if conR.WaitSync() { + conR.Logger.Info("Ignoring message received during sync", "msg", msg) return } switch msg := msg.(type) { @@ -325,8 +324,8 @@ func (conR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { } case VoteSetBitsChannel: - if conR.FastSync() { - conR.Logger.Info("Ignoring message received during fastSync", "msg", msg) + if conR.WaitSync() { + conR.Logger.Info("Ignoring message received during sync", "msg", msg) return } switch msg := msg.(type) { @@ -366,11 +365,11 @@ func (conR *Reactor) SetEventBus(b *types.EventBus) { conR.conS.SetEventBus(b) } -// FastSync returns whether the consensus reactor is in fast-sync mode. -func (conR *Reactor) FastSync() bool { +// WaitSync returns whether the consensus reactor is waiting for state/fast sync. +func (conR *Reactor) WaitSync() bool { conR.mtx.RLock() defer conR.mtx.RUnlock() - return conR.fastSync + return conR.waitSync } //-------------------------------------- @@ -886,19 +885,9 @@ func (conR *Reactor) StringIndented(indent string) string { return s } -func (conR *Reactor) updateFastSyncingMetric() { - var fastSyncing float64 - if conR.fastSync { - fastSyncing = 1 - } else { - fastSyncing = 0 - } - conR.metrics.FastSyncing.Set(fastSyncing) -} - // ReactorMetrics sets the metrics func ReactorMetrics(metrics *Metrics) ReactorOption { - return func(conR *Reactor) { conR.metrics = metrics } + return func(conR *Reactor) { conR.Metrics = metrics } } //----------------------------------------------------------------------------- diff --git a/consensus/reactor_test.go b/consensus/reactor_test.go index fcf22ddd6..aee222bbd 100644 --- a/consensus/reactor_test.go +++ b/consensus/reactor_test.go @@ -75,7 +75,7 @@ func startConsensusNet(t *testing.T, css []*State, n int) ( // TODO: is this still true with new pubsub? for i := 0; i < n; i++ { s := reactors[i].conS.GetState() - reactors[i].SwitchToConsensus(s, 0) + reactors[i].SwitchToConsensus(s, false) } return reactors, blocksSubs, eventBuses } diff --git a/node/node.go b/node/node.go index a59a3d19e..7623563d2 100644 --- a/node/node.go +++ b/node/node.go @@ -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{ diff --git a/p2p/mocks/peer.go b/p2p/mocks/peer.go new file mode 100644 index 000000000..f874e0405 --- /dev/null +++ b/p2p/mocks/peer.go @@ -0,0 +1,331 @@ +// Code generated by mockery v1.0.0. DO NOT EDIT. + +package mocks + +import ( + log "github.com/tendermint/tendermint/libs/log" + conn "github.com/tendermint/tendermint/p2p/conn" + + mock "github.com/stretchr/testify/mock" + + net "net" + + p2p "github.com/tendermint/tendermint/p2p" +) + +// Peer is an autogenerated mock type for the Peer type +type Peer struct { + mock.Mock +} + +// CloseConn provides a mock function with given fields: +func (_m *Peer) CloseConn() error { + ret := _m.Called() + + var r0 error + if rf, ok := ret.Get(0).(func() error); ok { + r0 = rf() + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// FlushStop provides a mock function with given fields: +func (_m *Peer) FlushStop() { + _m.Called() +} + +// Get provides a mock function with given fields: _a0 +func (_m *Peer) Get(_a0 string) interface{} { + ret := _m.Called(_a0) + + var r0 interface{} + if rf, ok := ret.Get(0).(func(string) interface{}); ok { + r0 = rf(_a0) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(interface{}) + } + } + + return r0 +} + +// ID provides a mock function with given fields: +func (_m *Peer) ID() p2p.ID { + ret := _m.Called() + + var r0 p2p.ID + if rf, ok := ret.Get(0).(func() p2p.ID); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(p2p.ID) + } + + return r0 +} + +// IsOutbound provides a mock function with given fields: +func (_m *Peer) IsOutbound() bool { + ret := _m.Called() + + var r0 bool + if rf, ok := ret.Get(0).(func() bool); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// IsPersistent provides a mock function with given fields: +func (_m *Peer) IsPersistent() bool { + ret := _m.Called() + + var r0 bool + if rf, ok := ret.Get(0).(func() bool); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// IsRunning provides a mock function with given fields: +func (_m *Peer) IsRunning() bool { + ret := _m.Called() + + var r0 bool + if rf, ok := ret.Get(0).(func() bool); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// NodeInfo provides a mock function with given fields: +func (_m *Peer) NodeInfo() p2p.NodeInfo { + ret := _m.Called() + + var r0 p2p.NodeInfo + if rf, ok := ret.Get(0).(func() p2p.NodeInfo); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(p2p.NodeInfo) + } + } + + return r0 +} + +// OnReset provides a mock function with given fields: +func (_m *Peer) OnReset() error { + ret := _m.Called() + + var r0 error + if rf, ok := ret.Get(0).(func() error); ok { + r0 = rf() + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// OnStart provides a mock function with given fields: +func (_m *Peer) OnStart() error { + ret := _m.Called() + + var r0 error + if rf, ok := ret.Get(0).(func() error); ok { + r0 = rf() + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// OnStop provides a mock function with given fields: +func (_m *Peer) OnStop() { + _m.Called() +} + +// Quit provides a mock function with given fields: +func (_m *Peer) Quit() <-chan struct{} { + ret := _m.Called() + + var r0 <-chan struct{} + if rf, ok := ret.Get(0).(func() <-chan struct{}); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(<-chan struct{}) + } + } + + return r0 +} + +// RemoteAddr provides a mock function with given fields: +func (_m *Peer) RemoteAddr() net.Addr { + ret := _m.Called() + + var r0 net.Addr + if rf, ok := ret.Get(0).(func() net.Addr); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(net.Addr) + } + } + + return r0 +} + +// RemoteIP provides a mock function with given fields: +func (_m *Peer) RemoteIP() net.IP { + ret := _m.Called() + + var r0 net.IP + if rf, ok := ret.Get(0).(func() net.IP); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(net.IP) + } + } + + return r0 +} + +// Reset provides a mock function with given fields: +func (_m *Peer) Reset() error { + ret := _m.Called() + + var r0 error + if rf, ok := ret.Get(0).(func() error); ok { + r0 = rf() + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// Send provides a mock function with given fields: _a0, _a1 +func (_m *Peer) Send(_a0 byte, _a1 []byte) bool { + ret := _m.Called(_a0, _a1) + + var r0 bool + if rf, ok := ret.Get(0).(func(byte, []byte) bool); ok { + r0 = rf(_a0, _a1) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// Set provides a mock function with given fields: _a0, _a1 +func (_m *Peer) Set(_a0 string, _a1 interface{}) { + _m.Called(_a0, _a1) +} + +// SetLogger provides a mock function with given fields: _a0 +func (_m *Peer) SetLogger(_a0 log.Logger) { + _m.Called(_a0) +} + +// SocketAddr provides a mock function with given fields: +func (_m *Peer) SocketAddr() *p2p.NetAddress { + ret := _m.Called() + + var r0 *p2p.NetAddress + if rf, ok := ret.Get(0).(func() *p2p.NetAddress); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*p2p.NetAddress) + } + } + + return r0 +} + +// Start provides a mock function with given fields: +func (_m *Peer) Start() error { + ret := _m.Called() + + var r0 error + if rf, ok := ret.Get(0).(func() error); ok { + r0 = rf() + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// Status provides a mock function with given fields: +func (_m *Peer) Status() conn.ConnectionStatus { + ret := _m.Called() + + var r0 conn.ConnectionStatus + if rf, ok := ret.Get(0).(func() conn.ConnectionStatus); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(conn.ConnectionStatus) + } + + return r0 +} + +// Stop provides a mock function with given fields: +func (_m *Peer) Stop() error { + ret := _m.Called() + + var r0 error + if rf, ok := ret.Get(0).(func() error); ok { + r0 = rf() + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// String provides a mock function with given fields: +func (_m *Peer) String() string { + ret := _m.Called() + + var r0 string + if rf, ok := ret.Get(0).(func() string); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(string) + } + + return r0 +} + +// TrySend provides a mock function with given fields: _a0, _a1 +func (_m *Peer) TrySend(_a0 byte, _a1 []byte) bool { + ret := _m.Called(_a0, _a1) + + var r0 bool + if rf, ok := ret.Get(0).(func(byte, []byte) bool); ok { + r0 = rf(_a0, _a1) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} diff --git a/p2p/peer.go b/p2p/peer.go index 7a6d6f868..35727cbf6 100644 --- a/p2p/peer.go +++ b/p2p/peer.go @@ -12,6 +12,8 @@ import ( tmconn "github.com/tendermint/tendermint/p2p/conn" ) +//go:generate mockery -case underscore -name Peer + const metricsTickerDuration = 10 * time.Second // Peer is an interface representing a peer connected on a reactor. diff --git a/rpc/core/status.go b/rpc/core/status.go index 4e950d4a3..67c43ea0d 100644 --- a/rpc/core/status.go +++ b/rpc/core/status.go @@ -31,7 +31,7 @@ func Status(ctx *rpctypes.Context) (*ctypes.ResultStatus, error) { } var latestHeight int64 - if consensusReactor.FastSync() { + if consensusReactor.WaitSync() { latestHeight = blockStore.Height() } else { latestHeight = consensusState.GetLastHeight() @@ -66,7 +66,7 @@ func Status(ctx *rpctypes.Context) (*ctypes.ResultStatus, error) { EarliestAppHash: earliestAppHash, EarliestBlockHeight: earliestBlockHeight, EarliestBlockTime: time.Unix(0, earliestBlockTimeNano), - CatchingUp: consensusReactor.FastSync(), + CatchingUp: consensusReactor.WaitSync(), }, ValidatorInfo: ctypes.ValidatorInfo{ Address: pubKey.Address(), diff --git a/state/store.go b/state/store.go index 149fd595b..35e239731 100644 --- a/state/store.go +++ b/state/store.go @@ -117,6 +117,16 @@ func saveState(db dbm.DB, state State, key []byte) { } } +// BootstrapState saves a new state, used e.g. by state sync when starting from non-zero height. +func BootstrapState(db dbm.DB, state State) error { + height := state.LastBlockHeight + saveValidatorsInfo(db, height, height, state.LastValidators) + saveValidatorsInfo(db, height+1, height+1, state.Validators) + saveValidatorsInfo(db, height+2, height+2, state.NextValidators) + saveConsensusParamsInfo(db, height+1, height+1, state.ConsensusParams) + return db.SetSync(stateKey, state.Bytes()) +} + //------------------------------------------------------------------------ // ABCIResponses retains the responses diff --git a/statesync/chunks.go b/statesync/chunks.go new file mode 100644 index 000000000..bfbcdb8d8 --- /dev/null +++ b/statesync/chunks.go @@ -0,0 +1,321 @@ +package statesync + +import ( + "errors" + "fmt" + "io/ioutil" + "os" + "path/filepath" + "strconv" + "sync" + "time" + + "github.com/tendermint/tendermint/p2p" +) + +// errDone is returned by chunkQueue.Next() when all chunks have been returned. +var errDone = errors.New("chunk queue has completed") + +// chunk contains data for a chunk. +type chunk struct { + Height uint64 + Format uint32 + Index uint32 + Chunk []byte + Sender p2p.ID +} + +// chunkQueue manages chunks for a state sync process, ordering them if requested. It acts as an +// iterator over all chunks, but callers can request chunks to be retried, optionally after +// refetching. +type chunkQueue struct { + sync.Mutex + snapshot *snapshot // if this is nil, the queue has been closed + dir string // temp dir for on-disk chunk storage + chunkFiles map[uint32]string // path to temporary chunk file + chunkSenders map[uint32]p2p.ID // the peer who sent the given chunk + chunkAllocated map[uint32]bool // chunks that have been allocated via Allocate() + chunkReturned map[uint32]bool // chunks returned via Next() + waiters map[uint32][]chan<- uint32 // signals WaitFor() waiters about chunk arrival +} + +// newChunkQueue creates a new chunk queue for a snapshot, using a temp dir for storage. +// Callers must call Close() when done. +func newChunkQueue(snapshot *snapshot, tempDir string) (*chunkQueue, error) { + dir, err := ioutil.TempDir(tempDir, "tm-statesync") + if err != nil { + return nil, fmt.Errorf("unable to create temp dir for state sync chunks: %w", err) + } + if snapshot.Chunks == 0 { + return nil, errors.New("snapshot has no chunks") + } + return &chunkQueue{ + snapshot: snapshot, + dir: dir, + chunkFiles: make(map[uint32]string, snapshot.Chunks), + chunkSenders: make(map[uint32]p2p.ID, snapshot.Chunks), + chunkAllocated: make(map[uint32]bool, snapshot.Chunks), + chunkReturned: make(map[uint32]bool, snapshot.Chunks), + waiters: make(map[uint32][]chan<- uint32), + }, nil +} + +// Add adds a chunk to the queue. It ignores chunks that already exist, returning false. +func (q *chunkQueue) Add(chunk *chunk) (bool, error) { + if chunk == nil || chunk.Chunk == nil { + return false, errors.New("cannot add nil chunk") + } + q.Lock() + defer q.Unlock() + if q.snapshot == nil { + return false, nil // queue is closed + } + if chunk.Height != q.snapshot.Height { + return false, fmt.Errorf("invalid chunk height %v, expected %v", chunk.Height, q.snapshot.Height) + } + if chunk.Format != q.snapshot.Format { + return false, fmt.Errorf("invalid chunk format %v, expected %v", chunk.Format, q.snapshot.Format) + } + if chunk.Index >= q.snapshot.Chunks { + return false, fmt.Errorf("received unexpected chunk %v", chunk.Index) + } + if q.chunkFiles[chunk.Index] != "" { + return false, nil + } + + path := filepath.Join(q.dir, strconv.FormatUint(uint64(chunk.Index), 10)) + err := ioutil.WriteFile(path, chunk.Chunk, 0644) + if err != nil { + return false, fmt.Errorf("failed to save chunk %v to file %v: %w", chunk.Index, path, err) + } + q.chunkFiles[chunk.Index] = path + q.chunkSenders[chunk.Index] = chunk.Sender + + // Signal any waiters that the chunk has arrived. + for _, waiter := range q.waiters[chunk.Index] { + waiter <- chunk.Index + close(waiter) + } + delete(q.waiters, chunk.Index) + + return true, nil +} + +// Allocate allocates a chunk to the caller, making it responsible for fetching it. Returns +// errDone once no chunks are left or the queue is closed. +func (q *chunkQueue) Allocate() (uint32, error) { + q.Lock() + defer q.Unlock() + if q.snapshot == nil { + return 0, errDone + } + if uint32(len(q.chunkAllocated)) >= q.snapshot.Chunks { + return 0, errDone + } + for i := uint32(0); i < q.snapshot.Chunks; i++ { + if !q.chunkAllocated[i] { + q.chunkAllocated[i] = true + return i, nil + } + } + return 0, errDone +} + +// Close closes the chunk queue, cleaning up all temporary files. +func (q *chunkQueue) Close() error { + q.Lock() + defer q.Unlock() + if q.snapshot == nil { + return nil + } + for _, waiters := range q.waiters { + for _, waiter := range waiters { + close(waiter) + } + } + q.waiters = nil + q.snapshot = nil + err := os.RemoveAll(q.dir) + if err != nil { + return fmt.Errorf("failed to clean up state sync tempdir %v: %w", q.dir, err) + } + return nil +} + +// Discard discards a chunk. It will be removed from the queue, available for allocation, and can +// be added and returned via Next() again. If the chunk is not already in the queue this does +// nothing, to avoid it being allocated to multiple fetchers. +func (q *chunkQueue) Discard(index uint32) error { + q.Lock() + defer q.Unlock() + return q.discard(index) +} + +// discard discards a chunk, scheduling it for refetching. The caller must hold the mutex lock. +func (q *chunkQueue) discard(index uint32) error { + if q.snapshot == nil { + return nil + } + path := q.chunkFiles[index] + if path == "" { + return nil + } + err := os.Remove(path) + if err != nil { + return fmt.Errorf("failed to remove chunk %v: %w", index, err) + } + delete(q.chunkFiles, index) + delete(q.chunkReturned, index) + delete(q.chunkAllocated, index) + return nil +} + +// DiscardSender discards all *unreturned* chunks from a given sender. If the caller wants to +// discard already returned chunks, this can be done via Discard(). +func (q *chunkQueue) DiscardSender(peerID p2p.ID) error { + q.Lock() + defer q.Unlock() + + for index, sender := range q.chunkSenders { + if sender == peerID && !q.chunkReturned[index] { + err := q.discard(index) + if err != nil { + return err + } + delete(q.chunkSenders, index) + } + } + return nil +} + +// GetSender returns the sender of the chunk with the given index, or empty if not found. +func (q *chunkQueue) GetSender(index uint32) p2p.ID { + q.Lock() + defer q.Unlock() + return q.chunkSenders[index] +} + +// Has checks whether a chunk exists in the queue. +func (q *chunkQueue) Has(index uint32) bool { + q.Lock() + defer q.Unlock() + return q.chunkFiles[index] != "" +} + +// load loads a chunk from disk, or nil if the chunk is not in the queue. The caller must hold the +// mutex lock. +func (q *chunkQueue) load(index uint32) (*chunk, error) { + path, ok := q.chunkFiles[index] + if !ok { + return nil, nil + } + body, err := ioutil.ReadFile(path) + if err != nil { + return nil, fmt.Errorf("failed to load chunk %v: %w", index, err) + } + return &chunk{ + Height: q.snapshot.Height, + Format: q.snapshot.Format, + Index: index, + Chunk: body, + Sender: q.chunkSenders[index], + }, nil +} + +// Next returns the next chunk from the queue, or errDone if all chunks have been returned. It +// blocks until the chunk is available. Concurrent Next() calls may return the same chunk. +func (q *chunkQueue) Next() (*chunk, error) { + q.Lock() + var chunk *chunk + index, err := q.nextUp() + if err == nil { + chunk, err = q.load(index) + if err == nil { + q.chunkReturned[index] = true + } + } + q.Unlock() + if chunk != nil || err != nil { + return chunk, err + } + + select { + case _, ok := <-q.WaitFor(index): + if !ok { + return nil, errDone // queue closed + } + case <-time.After(chunkTimeout): + return nil, errTimeout + } + + q.Lock() + defer q.Unlock() + chunk, err = q.load(index) + if err != nil { + return nil, err + } + q.chunkReturned[index] = true + return chunk, nil +} + +// nextUp returns the next chunk to be returned, or errDone if all chunks have been returned. The +// caller must hold the mutex lock. +func (q *chunkQueue) nextUp() (uint32, error) { + if q.snapshot == nil { + return 0, errDone + } + for i := uint32(0); i < q.snapshot.Chunks; i++ { + if !q.chunkReturned[i] { + return i, nil + } + } + return 0, errDone +} + +// Retry schedules a chunk to be retried, without refetching it. +func (q *chunkQueue) Retry(index uint32) { + q.Lock() + defer q.Unlock() + delete(q.chunkReturned, index) +} + +// RetryAll schedules all chunks to be retried, without refetching them. +func (q *chunkQueue) RetryAll() { + q.Lock() + defer q.Unlock() + q.chunkReturned = make(map[uint32]bool) +} + +// Size returns the total number of chunks for the snapshot and queue, or 0 when closed. +func (q *chunkQueue) Size() uint32 { + q.Lock() + defer q.Unlock() + if q.snapshot == nil { + return 0 + } + return q.snapshot.Chunks +} + +// WaitFor returns a channel that receives a chunk index when it arrives in the queue, or +// immediately if it has already arrived. The channel is closed without a value if the queue is +// closed or if the chunk index is not valid. +func (q *chunkQueue) WaitFor(index uint32) <-chan uint32 { + q.Lock() + defer q.Unlock() + ch := make(chan uint32, 1) + switch { + case q.snapshot == nil: + close(ch) + case index >= q.snapshot.Chunks: + close(ch) + case q.chunkFiles[index] != "": + ch <- index + close(ch) + default: + if q.waiters[index] == nil { + q.waiters[index] = make([]chan<- uint32, 0) + } + q.waiters[index] = append(q.waiters[index], ch) + } + return ch +} diff --git a/statesync/chunks_test.go b/statesync/chunks_test.go new file mode 100644 index 000000000..2b9a5d751 --- /dev/null +++ b/statesync/chunks_test.go @@ -0,0 +1,551 @@ +package statesync + +import ( + "io/ioutil" + "os" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/tendermint/tendermint/p2p" +) + +func setupChunkQueue(t *testing.T) (*chunkQueue, func()) { + snapshot := &snapshot{ + Height: 3, + Format: 1, + Chunks: 5, + Hash: []byte{7}, + Metadata: nil, + } + queue, err := newChunkQueue(snapshot, "") + require.NoError(t, err) + teardown := func() { + err := queue.Close() + require.NoError(t, err) + } + return queue, teardown +} + +func TestNewChunkQueue_TempDir(t *testing.T) { + snapshot := &snapshot{ + Height: 3, + Format: 1, + Chunks: 5, + Hash: []byte{7}, + Metadata: nil, + } + dir, err := ioutil.TempDir("", "newchunkqueue") + require.NoError(t, err) + defer os.RemoveAll(dir) + queue, err := newChunkQueue(snapshot, dir) + require.NoError(t, err) + + files, err := ioutil.ReadDir(dir) + require.NoError(t, err) + assert.Len(t, files, 1) + + err = queue.Close() + require.NoError(t, err) + + files, err = ioutil.ReadDir(dir) + require.NoError(t, err) + assert.Len(t, files, 0) +} + +func TestChunkQueue(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + // Adding the first chunk should be fine + added, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}}) + require.NoError(t, err) + assert.True(t, added) + + // Adding the last chunk should also be fine + added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 4, Chunk: []byte{3, 1, 4}}) + require.NoError(t, err) + assert.True(t, added) + + // Adding the first or last chunks again should return false + added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}}) + require.NoError(t, err) + assert.False(t, added) + + added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 4, Chunk: []byte{3, 1, 4}}) + require.NoError(t, err) + assert.False(t, added) + + // Adding the remaining chunks in reverse should be fine + added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 3, Chunk: []byte{3, 1, 3}}) + require.NoError(t, err) + assert.True(t, added) + + added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 2, Chunk: []byte{3, 1, 2}}) + require.NoError(t, err) + assert.True(t, added) + + added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{3, 1, 1}}) + require.NoError(t, err) + assert.True(t, added) + + // At this point, we should be able to retrieve them all via Next + for i := 0; i < 5; i++ { + c, err := queue.Next() + require.NoError(t, err) + assert.Equal(t, &chunk{Height: 3, Format: 1, Index: uint32(i), Chunk: []byte{3, 1, byte(i)}}, c) + } + _, err = queue.Next() + require.Error(t, err) + assert.Equal(t, errDone, err) + + // It should still be possible to try to add chunks (which will be ignored) + added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}}) + require.NoError(t, err) + assert.False(t, added) + + // After closing the queue it will also return false + err = queue.Close() + require.NoError(t, err) + added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}}) + require.NoError(t, err) + assert.False(t, added) + + // Closing the queue again should also be fine + err = queue.Close() + require.NoError(t, err) +} + +func TestChunkQueue_Add_ChunkErrors(t *testing.T) { + testcases := map[string]struct { + chunk *chunk + }{ + "nil chunk": {nil}, + "nil body": {&chunk{Height: 3, Format: 1, Index: 0, Chunk: nil}}, + "wrong height": {&chunk{Height: 9, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}}}, + "wrong format": {&chunk{Height: 3, Format: 9, Index: 0, Chunk: []byte{3, 1, 0}}}, + "invalid index": {&chunk{Height: 3, Format: 1, Index: 5, Chunk: []byte{3, 1, 0}}}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + _, err := queue.Add(tc.chunk) + require.Error(t, err) + }) + } +} + +func TestChunkQueue_Allocate(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + for i := uint32(0); i < queue.Size(); i++ { + index, err := queue.Allocate() + require.NoError(t, err) + assert.EqualValues(t, i, index) + } + + _, err := queue.Allocate() + require.Error(t, err) + assert.Equal(t, errDone, err) + + for i := uint32(0); i < queue.Size(); i++ { + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: i, Chunk: []byte{byte(i)}}) + require.NoError(t, err) + } + + // After all chunks have been allocated and retrieved, discarding a chunk will reallocate it. + err = queue.Discard(2) + require.NoError(t, err) + + index, err := queue.Allocate() + require.NoError(t, err) + assert.EqualValues(t, 2, index) + _, err = queue.Allocate() + require.Error(t, err) + assert.Equal(t, errDone, err) + + // Discarding a chunk the closing the queue will return errDone. + err = queue.Discard(2) + require.NoError(t, err) + err = queue.Close() + require.NoError(t, err) + _, err = queue.Allocate() + require.Error(t, err) + assert.Equal(t, errDone, err) +} + +func TestChunkQueue_Discard(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + // Add a few chunks to the queue and fetch a couple + _, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{byte(0)}}) + require.NoError(t, err) + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{byte(1)}}) + require.NoError(t, err) + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 2, Chunk: []byte{byte(2)}}) + require.NoError(t, err) + + c, err := queue.Next() + require.NoError(t, err) + assert.EqualValues(t, 0, c.Index) + c, err = queue.Next() + require.NoError(t, err) + assert.EqualValues(t, 1, c.Index) + + // Discarding the first chunk and re-adding it should cause it to be returned + // immediately by Next(), before procceeding with chunk 2 + err = queue.Discard(0) + require.NoError(t, err) + added, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{byte(0)}}) + require.NoError(t, err) + assert.True(t, added) + c, err = queue.Next() + require.NoError(t, err) + assert.EqualValues(t, 0, c.Index) + c, err = queue.Next() + require.NoError(t, err) + assert.EqualValues(t, 2, c.Index) + + // Discard then allocate, add and fetch all chunks + for i := uint32(0); i < queue.Size(); i++ { + err := queue.Discard(i) + require.NoError(t, err) + } + for i := uint32(0); i < queue.Size(); i++ { + _, err := queue.Allocate() + require.NoError(t, err) + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: i, Chunk: []byte{byte(i)}}) + require.NoError(t, err) + c, err = queue.Next() + require.NoError(t, err) + assert.EqualValues(t, i, c.Index) + } + + // Discarding a non-existent chunk does nothing. + err = queue.Discard(99) + require.NoError(t, err) + + // When discard a couple of chunks, we should be able to allocate, add, and fetch them again. + err = queue.Discard(3) + require.NoError(t, err) + err = queue.Discard(1) + require.NoError(t, err) + + index, err := queue.Allocate() + require.NoError(t, err) + assert.EqualValues(t, 1, index) + index, err = queue.Allocate() + require.NoError(t, err) + assert.EqualValues(t, 3, index) + + added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 3, Chunk: []byte{3}}) + require.NoError(t, err) + assert.True(t, added) + added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{1}}) + require.NoError(t, err) + assert.True(t, added) + + chunk, err := queue.Next() + require.NoError(t, err) + assert.EqualValues(t, 1, chunk.Index) + + chunk, err = queue.Next() + require.NoError(t, err) + assert.EqualValues(t, 3, chunk.Index) + + _, err = queue.Next() + require.Error(t, err) + assert.Equal(t, errDone, err) + + // After closing the queue, discarding does nothing + err = queue.Close() + require.NoError(t, err) + err = queue.Discard(2) + require.NoError(t, err) +} + +func TestChunkQueue_DiscardSender(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + // Allocate and add all chunks to the queue + senders := []p2p.ID{"a", "b", "c"} + for i := uint32(0); i < queue.Size(); i++ { + _, err := queue.Allocate() + require.NoError(t, err) + _, err = queue.Add(&chunk{ + Height: 3, + Format: 1, + Index: i, + Chunk: []byte{byte(i)}, + Sender: senders[int(i)%len(senders)], + }) + require.NoError(t, err) + } + + // Fetch the first three chunks + for i := uint32(0); i < 3; i++ { + _, err := queue.Next() + require.NoError(t, err) + } + + // Discarding an unknown sender should do nothing + err := queue.DiscardSender("x") + require.NoError(t, err) + _, err = queue.Allocate() + assert.Equal(t, errDone, err) + + // Discarding sender b should discard chunk 4, but not chunk 1 which has already been + // returned. + err = queue.DiscardSender("b") + require.NoError(t, err) + index, err := queue.Allocate() + require.NoError(t, err) + assert.EqualValues(t, 4, index) + _, err = queue.Allocate() + assert.Equal(t, errDone, err) +} + +func TestChunkQueue_GetSender(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + _, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{1}, Sender: p2p.ID("a")}) + require.NoError(t, err) + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{2}, Sender: p2p.ID("b")}) + require.NoError(t, err) + + assert.EqualValues(t, "a", queue.GetSender(0)) + assert.EqualValues(t, "b", queue.GetSender(1)) + assert.EqualValues(t, "", queue.GetSender(2)) + + // After the chunk has been processed, we should still know who the sender was + chunk, err := queue.Next() + require.NoError(t, err) + require.NotNil(t, chunk) + require.EqualValues(t, 0, chunk.Index) + assert.EqualValues(t, "a", queue.GetSender(0)) +} + +func TestChunkQueue_Next(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + // Next should block waiting for the next chunks, even when given out of order. + chNext := make(chan *chunk, 10) + go func() { + for { + c, err := queue.Next() + if err == errDone { + close(chNext) + break + } + require.NoError(t, err) + chNext <- c + } + }() + + assert.Empty(t, chNext) + _, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{3, 1, 1}, Sender: p2p.ID("b")}) + require.NoError(t, err) + select { + case <-chNext: + assert.Fail(t, "channel should be empty") + default: + } + + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}, Sender: p2p.ID("a")}) + require.NoError(t, err) + + assert.Equal(t, + &chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}, Sender: p2p.ID("a")}, + <-chNext) + assert.Equal(t, + &chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{3, 1, 1}, Sender: p2p.ID("b")}, + <-chNext) + + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 4, Chunk: []byte{3, 1, 4}, Sender: p2p.ID("e")}) + require.NoError(t, err) + select { + case <-chNext: + assert.Fail(t, "channel should be empty") + default: + } + + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 2, Chunk: []byte{3, 1, 2}, Sender: p2p.ID("c")}) + require.NoError(t, err) + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 3, Chunk: []byte{3, 1, 3}, Sender: p2p.ID("d")}) + require.NoError(t, err) + + assert.Equal(t, + &chunk{Height: 3, Format: 1, Index: 2, Chunk: []byte{3, 1, 2}, Sender: p2p.ID("c")}, + <-chNext) + assert.Equal(t, + &chunk{Height: 3, Format: 1, Index: 3, Chunk: []byte{3, 1, 3}, Sender: p2p.ID("d")}, + <-chNext) + assert.Equal(t, + &chunk{Height: 3, Format: 1, Index: 4, Chunk: []byte{3, 1, 4}, Sender: p2p.ID("e")}, + <-chNext) + + _, ok := <-chNext + assert.False(t, ok, "channel should be closed") + + // Calling next on a finished queue should return done + _, err = queue.Next() + assert.Equal(t, errDone, err) +} + +func TestChunkQueue_Next_Closed(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + // Calling Next on a closed queue should return done + _, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{3, 1, 1}}) + require.NoError(t, err) + err = queue.Close() + require.NoError(t, err) + + _, err = queue.Next() + assert.Equal(t, errDone, err) +} + +func TestChunkQueue_Retry(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + // Allocate and add all chunks to the queue + for i := uint32(0); i < queue.Size(); i++ { + _, err := queue.Allocate() + require.NoError(t, err) + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: i, Chunk: []byte{byte(i)}}) + require.NoError(t, err) + _, err = queue.Next() + require.NoError(t, err) + } + + // Retrying a couple of chunks makes Next() return them, but they are not allocatable + queue.Retry(3) + queue.Retry(1) + + _, err := queue.Allocate() + assert.Equal(t, errDone, err) + + chunk, err := queue.Next() + require.NoError(t, err) + assert.EqualValues(t, 1, chunk.Index) + + chunk, err = queue.Next() + require.NoError(t, err) + assert.EqualValues(t, 3, chunk.Index) + + _, err = queue.Next() + assert.Equal(t, errDone, err) +} + +func TestChunkQueue_RetryAll(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + // Allocate and add all chunks to the queue + for i := uint32(0); i < queue.Size(); i++ { + _, err := queue.Allocate() + require.NoError(t, err) + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: i, Chunk: []byte{byte(i)}}) + require.NoError(t, err) + _, err = queue.Next() + require.NoError(t, err) + } + + _, err := queue.Next() + assert.Equal(t, errDone, err) + + queue.RetryAll() + + _, err = queue.Allocate() + assert.Equal(t, errDone, err) + + for i := uint32(0); i < queue.Size(); i++ { + chunk, err := queue.Next() + require.NoError(t, err) + assert.EqualValues(t, i, chunk.Index) + } + + _, err = queue.Next() + assert.Equal(t, errDone, err) +} + +func TestChunkQueue_Size(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + assert.EqualValues(t, 5, queue.Size()) + + err := queue.Close() + require.NoError(t, err) + assert.EqualValues(t, 0, queue.Size()) +} + +func TestChunkQueue_WaitFor(t *testing.T) { + queue, teardown := setupChunkQueue(t) + defer teardown() + + waitFor1 := queue.WaitFor(1) + waitFor4 := queue.WaitFor(4) + + // Adding 0 and 2 should not trigger waiters + _, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}}) + require.NoError(t, err) + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 2, Chunk: []byte{3, 1, 2}}) + require.NoError(t, err) + select { + case <-waitFor1: + require.Fail(t, "WaitFor(1) should not trigger on 0 or 2") + case <-waitFor4: + require.Fail(t, "WaitFor(4) should not trigger on 0 or 2") + default: + } + + // Adding 1 should trigger WaitFor(1), but not WaitFor(4). The channel should be closed. + _, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{3, 1, 1}}) + require.NoError(t, err) + assert.EqualValues(t, 1, <-waitFor1) + _, ok := <-waitFor1 + assert.False(t, ok) + select { + case <-waitFor4: + require.Fail(t, "WaitFor(4) should not trigger on 0 or 2") + default: + } + + // Fetch the first chunk. At this point, waiting for either 0 (retrieved from pool) or 1 + // (queued in pool) should immediately return true. + c, err := queue.Next() + require.NoError(t, err) + assert.EqualValues(t, 0, c.Index) + + w := queue.WaitFor(0) + assert.EqualValues(t, 0, <-w) + _, ok = <-w + assert.False(t, ok) + + w = queue.WaitFor(1) + assert.EqualValues(t, 1, <-w) + _, ok = <-w + assert.False(t, ok) + + // Close the queue. This should cause the waiter for 4 to close, and also cause any future + // waiters to get closed channels. + err = queue.Close() + require.NoError(t, err) + _, ok = <-waitFor4 + assert.False(t, ok) + + w = queue.WaitFor(3) + _, ok = <-w + assert.False(t, ok) +} diff --git a/statesync/messages.go b/statesync/messages.go new file mode 100644 index 000000000..83aecd7f1 --- /dev/null +++ b/statesync/messages.go @@ -0,0 +1,129 @@ +package statesync + +import ( + "errors" + "fmt" + + amino "github.com/tendermint/go-amino" + + "github.com/tendermint/tendermint/types" +) + +const ( + // snapshotMsgSize is the maximum size of a snapshotResponseMessage + snapshotMsgSize = int(4e6) + // chunkMsgSize is the maximum size of a chunkResponseMessage + chunkMsgSize = int(16e6) + // maxMsgSize is the maximum size of any message + maxMsgSize = chunkMsgSize +) + +var cdc = amino.NewCodec() + +func init() { + cdc.RegisterInterface((*Message)(nil), nil) + cdc.RegisterConcrete(&snapshotsRequestMessage{}, "tendermint/SnapshotsRequestMessage", nil) + cdc.RegisterConcrete(&snapshotsResponseMessage{}, "tendermint/SnapshotsResponseMessage", nil) + cdc.RegisterConcrete(&chunkRequestMessage{}, "tendermint/ChunkRequestMessage", nil) + cdc.RegisterConcrete(&chunkResponseMessage{}, "tendermint/ChunkResponseMessage", nil) + types.RegisterBlockAmino(cdc) +} + +// decodeMsg decodes a message. +func decodeMsg(bz []byte) (Message, error) { + if len(bz) > maxMsgSize { + return nil, fmt.Errorf("msg exceeds max size (%d > %d)", len(bz), maxMsgSize) + } + var msg Message + err := cdc.UnmarshalBinaryBare(bz, &msg) + if err != nil { + return nil, err + } + return msg, nil +} + +// Message is a message sent and received by the reactor. +type Message interface { + ValidateBasic() error +} + +// snapshotsRequestMessage requests recent snapshots from a peer. +type snapshotsRequestMessage struct{} + +// ValidateBasic implements Message. +func (m *snapshotsRequestMessage) ValidateBasic() error { + if m == nil { + return errors.New("nil message") + } + return nil +} + +// SnapshotResponseMessage contains information about a single snapshot. +type snapshotsResponseMessage struct { + Height uint64 + Format uint32 + Chunks uint32 + Hash []byte + Metadata []byte +} + +// ValidateBasic implements Message. +func (m *snapshotsResponseMessage) ValidateBasic() error { + if m == nil { + return errors.New("nil message") + } + if m.Height == 0 { + return errors.New("height cannot be 0") + } + if len(m.Hash) == 0 { + return errors.New("snapshot has no hash") + } + if m.Chunks == 0 { + return errors.New("snapshot has no chunks") + } + return nil +} + +// chunkRequestMessage requests a single chunk from a peer. +type chunkRequestMessage struct { + Height uint64 + Format uint32 + Index uint32 +} + +// ValidateBasic implements Message. +func (m *chunkRequestMessage) ValidateBasic() error { + if m == nil { + return errors.New("nil message") + } + if m.Height == 0 { + return errors.New("height cannot be 0") + } + return nil +} + +// chunkResponseMessage contains a single chunk from a peer. +type chunkResponseMessage struct { + Height uint64 + Format uint32 + Index uint32 + Chunk []byte + Missing bool +} + +// ValidateBasic implements Message. +func (m *chunkResponseMessage) ValidateBasic() error { + if m == nil { + return errors.New("nil message") + } + if m.Height == 0 { + return errors.New("height cannot be 0") + } + if m.Missing && len(m.Chunk) > 0 { + return errors.New("missing chunk cannot have contents") + } + if !m.Missing && m.Chunk == nil { + return errors.New("chunk cannot be nil") + } + return nil +} diff --git a/statesync/messages_test.go b/statesync/messages_test.go new file mode 100644 index 000000000..68244a5da --- /dev/null +++ b/statesync/messages_test.go @@ -0,0 +1,106 @@ +package statesync + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestSnapshotsRequestMessage_ValidateBasic(t *testing.T) { + testcases := map[string]struct { + msg *snapshotsRequestMessage + valid bool + }{ + "nil": {nil, false}, + "valid": {&snapshotsRequestMessage{}, true}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + err := tc.msg.ValidateBasic() + if tc.valid { + require.NoError(t, err) + } else { + require.Error(t, err) + } + }) + } +} + +func TestSnapshotsResponseMessage_ValidateBasic(t *testing.T) { + testcases := map[string]struct { + msg *snapshotsResponseMessage + valid bool + }{ + "nil": {nil, false}, + "valid": {&snapshotsResponseMessage{Height: 1, Format: 1, Chunks: 2, Hash: []byte{1}}, true}, + "0 height": {&snapshotsResponseMessage{Height: 0, Format: 1, Chunks: 2, Hash: []byte{1}}, false}, + "0 format": {&snapshotsResponseMessage{Height: 1, Format: 0, Chunks: 2, Hash: []byte{1}}, true}, + "0 chunks": {&snapshotsResponseMessage{Height: 1, Format: 1, Hash: []byte{1}}, false}, + "no hash": {&snapshotsResponseMessage{Height: 1, Format: 1, Chunks: 2, Hash: []byte{}}, false}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + err := tc.msg.ValidateBasic() + if tc.valid { + require.NoError(t, err) + } else { + require.Error(t, err) + } + }) + } +} + +func TestChunkRequestMessage_ValidateBasic(t *testing.T) { + testcases := map[string]struct { + msg *chunkRequestMessage + valid bool + }{ + "nil": {nil, false}, + "valid": {&chunkRequestMessage{Height: 1, Format: 1, Index: 1}, true}, + "0 height": {&chunkRequestMessage{Height: 0, Format: 1, Index: 1}, false}, + "0 format": {&chunkRequestMessage{Height: 1, Format: 0, Index: 1}, true}, + "0 chunk": {&chunkRequestMessage{Height: 1, Format: 1, Index: 0}, true}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + err := tc.msg.ValidateBasic() + if tc.valid { + require.NoError(t, err) + } else { + require.Error(t, err) + } + }) + } +} + +func TestChunkResponseMessage_ValidateBasic(t *testing.T) { + testcases := map[string]struct { + msg *chunkResponseMessage + valid bool + }{ + "nil message": {nil, false}, + "valid": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}}, true}, + "0 height": {&chunkResponseMessage{Height: 0, Format: 1, Index: 1, Chunk: []byte{1}}, false}, + "0 format": {&chunkResponseMessage{Height: 1, Format: 0, Index: 1, Chunk: []byte{1}}, true}, + "0 chunk": {&chunkResponseMessage{Height: 1, Format: 1, Index: 0, Chunk: []byte{1}}, true}, + "empty body": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: []byte{}}, true}, + "nil body": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: nil}, false}, + "missing": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true}, true}, + "missing with empty": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true, Chunk: []byte{}}, true}, + "missing with body": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true, Chunk: []byte{1}}, false}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + err := tc.msg.ValidateBasic() + if tc.valid { + require.NoError(t, err) + } else { + require.Error(t, err) + } + }) + } +} diff --git a/statesync/mocks/state_provider.go b/statesync/mocks/state_provider.go new file mode 100644 index 000000000..0a71aae68 --- /dev/null +++ b/statesync/mocks/state_provider.go @@ -0,0 +1,82 @@ +// Code generated by mockery v1.0.0. DO NOT EDIT. + +package mocks + +import ( + mock "github.com/stretchr/testify/mock" + state "github.com/tendermint/tendermint/state" + + types "github.com/tendermint/tendermint/types" +) + +// StateProvider is an autogenerated mock type for the StateProvider type +type StateProvider struct { + mock.Mock +} + +// AppHash provides a mock function with given fields: height +func (_m *StateProvider) AppHash(height uint64) ([]byte, error) { + ret := _m.Called(height) + + var r0 []byte + if rf, ok := ret.Get(0).(func(uint64) []byte); ok { + r0 = rf(height) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]byte) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(height) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// Commit provides a mock function with given fields: height +func (_m *StateProvider) Commit(height uint64) (*types.Commit, error) { + ret := _m.Called(height) + + var r0 *types.Commit + if rf, ok := ret.Get(0).(func(uint64) *types.Commit); ok { + r0 = rf(height) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.Commit) + } + } + + var r1 error + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(height) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// State provides a mock function with given fields: height +func (_m *StateProvider) State(height uint64) (state.State, error) { + ret := _m.Called(height) + + var r0 state.State + if rf, ok := ret.Get(0).(func(uint64) state.State); ok { + r0 = rf(height) + } else { + r0 = ret.Get(0).(state.State) + } + + var r1 error + if rf, ok := ret.Get(1).(func(uint64) error); ok { + r1 = rf(height) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} diff --git a/statesync/reactor.go b/statesync/reactor.go new file mode 100644 index 000000000..b4221bff6 --- /dev/null +++ b/statesync/reactor.go @@ -0,0 +1,261 @@ +package statesync + +import ( + "errors" + "sort" + "sync" + + abci "github.com/tendermint/tendermint/abci/types" + "github.com/tendermint/tendermint/p2p" + "github.com/tendermint/tendermint/proxy" + sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" +) + +const ( + // SnapshotChannel exchanges snapshot metadata + SnapshotChannel = byte(0x60) + // ChunkChannel exchanges chunk contents + ChunkChannel = byte(0x61) + // recentSnapshots is the number of recent snapshots to send and receive per peer. + recentSnapshots = 10 +) + +// Reactor handles state sync, both restoring snapshots for the local node and serving snapshots +// for other nodes. +type Reactor struct { + p2p.BaseReactor + + conn proxy.AppConnSnapshot + connQuery proxy.AppConnQuery + tempDir string + + // This will only be set when a state sync is in progress. It is used to feed received + // snapshots and chunks into the sync. + mtx sync.RWMutex + syncer *syncer +} + +// NewReactor creates a new state sync reactor. +func NewReactor(conn proxy.AppConnSnapshot, connQuery proxy.AppConnQuery, tempDir string) *Reactor { + r := &Reactor{ + conn: conn, + connQuery: connQuery, + } + r.BaseReactor = *p2p.NewBaseReactor("StateSync", r) + return r +} + +// GetChannels implements p2p.Reactor. +func (r *Reactor) GetChannels() []*p2p.ChannelDescriptor { + return []*p2p.ChannelDescriptor{ + { + ID: SnapshotChannel, + Priority: 3, + SendQueueCapacity: 10, + RecvMessageCapacity: snapshotMsgSize, + }, + { + ID: ChunkChannel, + Priority: 1, + SendQueueCapacity: 4, + RecvMessageCapacity: chunkMsgSize, + }, + } +} + +// OnStart implements p2p.Reactor. +func (r *Reactor) OnStart() error { + return nil +} + +// AddPeer implements p2p.Reactor. +func (r *Reactor) AddPeer(peer p2p.Peer) { + r.mtx.RLock() + defer r.mtx.RUnlock() + if r.syncer != nil { + r.syncer.AddPeer(peer) + } +} + +// RemovePeer implements p2p.Reactor. +func (r *Reactor) RemovePeer(peer p2p.Peer, reason interface{}) { + r.mtx.RLock() + defer r.mtx.RUnlock() + if r.syncer != nil { + r.syncer.RemovePeer(peer) + } +} + +// Receive implements p2p.Reactor. +func (r *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { + if !r.IsRunning() { + return + } + + msg, err := decodeMsg(msgBytes) + if err != nil { + r.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes) + r.Switch.StopPeerForError(src, err) + return + } + err = msg.ValidateBasic() + if err != nil { + r.Logger.Error("Invalid message", "peer", src, "msg", msg, "err", err) + r.Switch.StopPeerForError(src, err) + return + } + + switch chID { + case SnapshotChannel: + switch msg := msg.(type) { + case *snapshotsRequestMessage: + snapshots, err := r.recentSnapshots(recentSnapshots) + if err != nil { + r.Logger.Error("Failed to fetch snapshots", "err", err) + return + } + for _, snapshot := range snapshots { + r.Logger.Debug("Advertising snapshot", "height", snapshot.Height, + "format", snapshot.Format, "peer", src.ID()) + src.Send(chID, cdc.MustMarshalBinaryBare(&snapshotsResponseMessage{ + Height: snapshot.Height, + Format: snapshot.Format, + Chunks: snapshot.Chunks, + Hash: snapshot.Hash, + Metadata: snapshot.Metadata, + })) + } + + case *snapshotsResponseMessage: + r.mtx.RLock() + defer r.mtx.RUnlock() + if r.syncer == nil { + r.Logger.Debug("Received unexpected snapshot, no state sync in progress") + return + } + r.Logger.Debug("Received snapshot", "height", msg.Height, "format", msg.Format, "peer", src.ID()) + _, err := r.syncer.AddSnapshot(src, &snapshot{ + Height: msg.Height, + Format: msg.Format, + Chunks: msg.Chunks, + Hash: msg.Hash, + Metadata: msg.Metadata, + }) + if err != nil { + r.Logger.Error("Failed to add snapshot", "height", msg.Height, "format", msg.Format, + "peer", src.ID(), "err", err) + return + } + + default: + r.Logger.Error("Received unknown message %T", msg) + } + + case ChunkChannel: + switch msg := msg.(type) { + case *chunkRequestMessage: + r.Logger.Debug("Received chunk request", "height", msg.Height, "format", msg.Format, + "chunk", msg.Index, "peer", src.ID()) + resp, err := r.conn.LoadSnapshotChunkSync(abci.RequestLoadSnapshotChunk{ + Height: msg.Height, + Format: msg.Format, + Chunk: msg.Index, + }) + if err != nil { + r.Logger.Error("Failed to load chunk", "height", msg.Height, "format", msg.Format, + "chunk", msg.Index, "err", err) + return + } + r.Logger.Debug("Sending chunk", "height", msg.Height, "format", msg.Format, + "chunk", msg.Index, "peer", src.ID()) + src.Send(ChunkChannel, cdc.MustMarshalBinaryBare(&chunkResponseMessage{ + Height: msg.Height, + Format: msg.Format, + Index: msg.Index, + Chunk: resp.Chunk, + Missing: resp.Chunk == nil, + })) + + case *chunkResponseMessage: + r.mtx.RLock() + defer r.mtx.RUnlock() + if r.syncer == nil { + r.Logger.Debug("Received unexpected chunk, no state sync in progress", "peer", src.ID()) + return + } + r.Logger.Debug("Received chunk, adding to sync", "height", msg.Height, "format", msg.Format, + "chunk", msg.Index, "peer", src.ID()) + _, err := r.syncer.AddChunk(&chunk{ + Height: msg.Height, + Format: msg.Format, + Index: msg.Index, + Chunk: msg.Chunk, + Sender: src.ID(), + }) + if err != nil { + r.Logger.Error("Failed to add chunk", "height", msg.Height, "format", msg.Format, + "chunk", msg.Index, "err", err) + return + } + + default: + r.Logger.Error("Received unknown message %T", msg) + } + + default: + r.Logger.Error("Received message on invalid channel %x", chID) + } +} + +// recentSnapshots fetches the n most recent snapshots from the app +func (r *Reactor) recentSnapshots(n uint32) ([]*snapshot, error) { + resp, err := r.conn.ListSnapshotsSync(abci.RequestListSnapshots{}) + if err != nil { + return nil, err + } + sort.Slice(resp.Snapshots, func(i, j int) bool { + a := resp.Snapshots[i] + b := resp.Snapshots[j] + switch { + case a.Height > b.Height: + return true + case a.Height == b.Height && a.Format > b.Format: + return true + default: + return false + } + }) + snapshots := make([]*snapshot, 0, n) + for i, s := range resp.Snapshots { + if i >= recentSnapshots { + break + } + snapshots = append(snapshots, &snapshot{ + Height: s.Height, + Format: s.Format, + Chunks: s.Chunks, + Hash: s.Hash, + Metadata: s.Metadata, + }) + } + return snapshots, nil +} + +// Sync runs a state sync, returning the new state and last commit at the snapshot height. +// The caller must store the state and commit in the state database and block store. +func (r *Reactor) Sync(stateProvider StateProvider) (sm.State, *types.Commit, error) { + r.mtx.Lock() + if r.syncer != nil { + r.mtx.Unlock() + return sm.State{}, nil, errors.New("a state sync is already in progress") + } + r.syncer = newSyncer(r.Logger, r.conn, r.connQuery, stateProvider, r.tempDir) + r.mtx.Unlock() + + state, commit, err := r.syncer.SyncAny(defaultDiscoveryTime) + r.mtx.Lock() + r.syncer = nil + r.mtx.Unlock() + return state, commit, err +} diff --git a/statesync/reactor_test.go b/statesync/reactor_test.go new file mode 100644 index 000000000..402eba6d7 --- /dev/null +++ b/statesync/reactor_test.go @@ -0,0 +1,148 @@ +package statesync + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + + abci "github.com/tendermint/tendermint/abci/types" + "github.com/tendermint/tendermint/p2p" + p2pmocks "github.com/tendermint/tendermint/p2p/mocks" + proxymocks "github.com/tendermint/tendermint/proxy/mocks" +) + +func TestReactor_Receive_ChunkRequestMessage(t *testing.T) { + testcases := map[string]struct { + request *chunkRequestMessage + chunk []byte + expectResponse *chunkResponseMessage + }{ + "chunk is returned": { + &chunkRequestMessage{Height: 1, Format: 1, Index: 1}, + []byte{1, 2, 3}, + &chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: []byte{1, 2, 3}}}, + "empty chunk is returned, as nil": { + &chunkRequestMessage{Height: 1, Format: 1, Index: 1}, + []byte{}, + &chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: nil}}, + "nil (missing) chunk is returned as missing": { + &chunkRequestMessage{Height: 1, Format: 1, Index: 1}, + nil, + &chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true}, + }, + } + + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + // Mock ABCI connection to return local snapshots + conn := &proxymocks.AppConnSnapshot{} + conn.On("LoadSnapshotChunkSync", abci.RequestLoadSnapshotChunk{ + Height: tc.request.Height, + Format: tc.request.Format, + Chunk: tc.request.Index, + }).Return(&abci.ResponseLoadSnapshotChunk{Chunk: tc.chunk}, nil) + + // Mock peer to store response, if found + peer := &p2pmocks.Peer{} + peer.On("ID").Return(p2p.ID("id")) + var response *chunkResponseMessage + if tc.expectResponse != nil { + peer.On("Send", ChunkChannel, mock.Anything).Run(func(args mock.Arguments) { + msg, err := decodeMsg(args[1].([]byte)) + require.NoError(t, err) + response = msg.(*chunkResponseMessage) + }).Return(true) + } + + // Start a reactor and send a chunkRequestMessage, then wait for and check response + r := NewReactor(conn, nil, "") + err := r.Start() + require.NoError(t, err) + defer r.Stop() + + r.Receive(ChunkChannel, peer, cdc.MustMarshalBinaryBare(tc.request)) + time.Sleep(100 * time.Millisecond) + assert.Equal(t, tc.expectResponse, response) + + conn.AssertExpectations(t) + peer.AssertExpectations(t) + }) + } +} + +func TestReactor_Receive_SnapshotRequestMessage(t *testing.T) { + testcases := map[string]struct { + snapshots []*abci.Snapshot + expectResponses []*snapshotsResponseMessage + }{ + "no snapshots": {nil, []*snapshotsResponseMessage{}}, + ">10 unordered snapshots": { + []*abci.Snapshot{ + {Height: 1, Format: 2, Chunks: 7, Hash: []byte{1, 2}, Metadata: []byte{1}}, + {Height: 2, Format: 2, Chunks: 7, Hash: []byte{2, 2}, Metadata: []byte{2}}, + {Height: 3, Format: 2, Chunks: 7, Hash: []byte{3, 2}, Metadata: []byte{3}}, + {Height: 1, Format: 1, Chunks: 7, Hash: []byte{1, 1}, Metadata: []byte{4}}, + {Height: 2, Format: 1, Chunks: 7, Hash: []byte{2, 1}, Metadata: []byte{5}}, + {Height: 3, Format: 1, Chunks: 7, Hash: []byte{3, 1}, Metadata: []byte{6}}, + {Height: 1, Format: 4, Chunks: 7, Hash: []byte{1, 4}, Metadata: []byte{7}}, + {Height: 2, Format: 4, Chunks: 7, Hash: []byte{2, 4}, Metadata: []byte{8}}, + {Height: 3, Format: 4, Chunks: 7, Hash: []byte{3, 4}, Metadata: []byte{9}}, + {Height: 1, Format: 3, Chunks: 7, Hash: []byte{1, 3}, Metadata: []byte{10}}, + {Height: 2, Format: 3, Chunks: 7, Hash: []byte{2, 3}, Metadata: []byte{11}}, + {Height: 3, Format: 3, Chunks: 7, Hash: []byte{3, 3}, Metadata: []byte{12}}, + }, + []*snapshotsResponseMessage{ + {Height: 3, Format: 4, Chunks: 7, Hash: []byte{3, 4}, Metadata: []byte{9}}, + {Height: 3, Format: 3, Chunks: 7, Hash: []byte{3, 3}, Metadata: []byte{12}}, + {Height: 3, Format: 2, Chunks: 7, Hash: []byte{3, 2}, Metadata: []byte{3}}, + {Height: 3, Format: 1, Chunks: 7, Hash: []byte{3, 1}, Metadata: []byte{6}}, + {Height: 2, Format: 4, Chunks: 7, Hash: []byte{2, 4}, Metadata: []byte{8}}, + {Height: 2, Format: 3, Chunks: 7, Hash: []byte{2, 3}, Metadata: []byte{11}}, + {Height: 2, Format: 2, Chunks: 7, Hash: []byte{2, 2}, Metadata: []byte{2}}, + {Height: 2, Format: 1, Chunks: 7, Hash: []byte{2, 1}, Metadata: []byte{5}}, + {Height: 1, Format: 4, Chunks: 7, Hash: []byte{1, 4}, Metadata: []byte{7}}, + {Height: 1, Format: 3, Chunks: 7, Hash: []byte{1, 3}, Metadata: []byte{10}}, + }, + }, + } + + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + // Mock ABCI connection to return local snapshots + conn := &proxymocks.AppConnSnapshot{} + conn.On("ListSnapshotsSync", abci.RequestListSnapshots{}).Return(&abci.ResponseListSnapshots{ + Snapshots: tc.snapshots, + }, nil) + + // Mock peer to catch responses and store them in a slice + responses := []*snapshotsResponseMessage{} + peer := &p2pmocks.Peer{} + if len(tc.expectResponses) > 0 { + peer.On("ID").Return(p2p.ID("id")) + peer.On("Send", SnapshotChannel, mock.Anything).Run(func(args mock.Arguments) { + msg, err := decodeMsg(args[1].([]byte)) + require.NoError(t, err) + responses = append(responses, msg.(*snapshotsResponseMessage)) + }).Return(true) + } + + // Start a reactor and send a SnapshotsRequestMessage, then wait for and check responses + r := NewReactor(conn, nil, "") + err := r.Start() + require.NoError(t, err) + defer r.Stop() + + r.Receive(SnapshotChannel, peer, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{})) + time.Sleep(100 * time.Millisecond) + assert.Equal(t, tc.expectResponses, responses) + + conn.AssertExpectations(t) + peer.AssertExpectations(t) + }) + } +} diff --git a/statesync/snapshots.go b/statesync/snapshots.go new file mode 100644 index 000000000..a2afb4083 --- /dev/null +++ b/statesync/snapshots.go @@ -0,0 +1,263 @@ +package statesync + +import ( + "crypto/sha256" + "fmt" + "math/rand" + "sort" + "sync" + + "github.com/tendermint/tendermint/p2p" +) + +// snapshotKey is a snapshot key used for lookups. +type snapshotKey [sha256.Size]byte + +// snapshot contains data about a snapshot. +type snapshot struct { + Height uint64 + Format uint32 + Chunks uint32 + Hash []byte + Metadata []byte + + trustedAppHash []byte // populated by light client +} + +// Key generates a snapshot key, used for lookups. It takes into account not only the height and +// format, but also the chunks, hash, and metadata in case peers have generated snapshots in a +// non-deterministic manner. All fields must be equal for the snapshot to be considered the same. +func (s *snapshot) Key() snapshotKey { + // Hash.Write() never returns an error. + hasher := sha256.New() + hasher.Write([]byte(fmt.Sprintf("%v:%v:%v", s.Height, s.Format, s.Chunks))) + hasher.Write(s.Hash) + hasher.Write(s.Metadata) + var key snapshotKey + copy(key[:], hasher.Sum(nil)) + return key +} + +// snapshotPool discovers and aggregates snapshots across peers. +type snapshotPool struct { + stateProvider StateProvider + + sync.Mutex + snapshots map[snapshotKey]*snapshot + snapshotPeers map[snapshotKey]map[p2p.ID]p2p.Peer + + // indexes for fast searches + formatIndex map[uint32]map[snapshotKey]bool + heightIndex map[uint64]map[snapshotKey]bool + peerIndex map[p2p.ID]map[snapshotKey]bool + + // blacklists for rejected items + formatBlacklist map[uint32]bool + peerBlacklist map[p2p.ID]bool + snapshotBlacklist map[snapshotKey]bool +} + +// newSnapshotPool creates a new snapshot pool. The state source is used for +func newSnapshotPool(stateProvider StateProvider) *snapshotPool { + return &snapshotPool{ + stateProvider: stateProvider, + snapshots: make(map[snapshotKey]*snapshot), + snapshotPeers: make(map[snapshotKey]map[p2p.ID]p2p.Peer), + formatIndex: make(map[uint32]map[snapshotKey]bool), + heightIndex: make(map[uint64]map[snapshotKey]bool), + peerIndex: make(map[p2p.ID]map[snapshotKey]bool), + formatBlacklist: make(map[uint32]bool), + peerBlacklist: make(map[p2p.ID]bool), + snapshotBlacklist: make(map[snapshotKey]bool), + } +} + +// Add adds a snapshot to the pool, unless the peer has already sent recentSnapshots snapshots. It +// returns true if this was a new, non-blacklisted snapshot. The snapshot height is verified using +// the light client, and the expected app hash is set for the snapshot. +func (p *snapshotPool) Add(peer p2p.Peer, snapshot *snapshot) (bool, error) { + appHash, err := p.stateProvider.AppHash(snapshot.Height) + if err != nil { + return false, err + } + snapshot.trustedAppHash = appHash + key := snapshot.Key() + + p.Lock() + defer p.Unlock() + + switch { + case p.formatBlacklist[snapshot.Format]: + return false, nil + case p.peerBlacklist[peer.ID()]: + return false, nil + case p.snapshotBlacklist[key]: + return false, nil + case len(p.peerIndex[peer.ID()]) >= recentSnapshots: + return false, nil + } + + if p.snapshotPeers[key] == nil { + p.snapshotPeers[key] = make(map[p2p.ID]p2p.Peer) + } + p.snapshotPeers[key][peer.ID()] = peer + + if p.peerIndex[peer.ID()] == nil { + p.peerIndex[peer.ID()] = make(map[snapshotKey]bool) + } + p.peerIndex[peer.ID()][key] = true + + if p.snapshots[key] != nil { + return false, nil + } + p.snapshots[key] = snapshot + + if p.formatIndex[snapshot.Format] == nil { + p.formatIndex[snapshot.Format] = make(map[snapshotKey]bool) + } + p.formatIndex[snapshot.Format][key] = true + + if p.heightIndex[snapshot.Height] == nil { + p.heightIndex[snapshot.Height] = make(map[snapshotKey]bool) + } + p.heightIndex[snapshot.Height][key] = true + + return true, nil +} + +// Best returns the "best" currently known snapshot, if any. +func (p *snapshotPool) Best() *snapshot { + ranked := p.Ranked() + if len(ranked) == 0 { + return nil + } + return ranked[0] +} + +// GetPeer returns a random peer for a snapshot, if any. +func (p *snapshotPool) GetPeer(snapshot *snapshot) p2p.Peer { + peers := p.GetPeers(snapshot) + if len(peers) == 0 { + return nil + } + return peers[rand.Intn(len(peers))] +} + +// GetPeers returns the peers for a snapshot. +func (p *snapshotPool) GetPeers(snapshot *snapshot) []p2p.Peer { + key := snapshot.Key() + p.Lock() + defer p.Unlock() + + peers := make([]p2p.Peer, 0, len(p.snapshotPeers[key])) + for _, peer := range p.snapshotPeers[key] { + peers = append(peers, peer) + } + // sort results, for testability (otherwise order is random, so tests randomly fail) + sort.Slice(peers, func(a int, b int) bool { + return peers[a].ID() < peers[b].ID() + }) + return peers +} + +// Ranked returns a list of snapshots ranked by preference. The current heuristic is very naïve, +// preferring the snapshot with the greatest height, then greatest format, then greatest number of +// peers. This can be improved quite a lot. +func (p *snapshotPool) Ranked() []*snapshot { + p.Lock() + defer p.Unlock() + + candidates := make([]*snapshot, 0, len(p.snapshots)) + for _, snapshot := range p.snapshots { + candidates = append(candidates, snapshot) + } + + sort.Slice(candidates, func(i, j int) bool { + a := candidates[i] + b := candidates[j] + + switch { + case a.Height > b.Height: + return true + case a.Height < b.Height: + return false + case a.Format > b.Format: + return true + case a.Format < b.Format: + return false + case len(p.snapshotPeers[a.Key()]) > len(p.snapshotPeers[b.Key()]): + return true + default: + return false + } + }) + + return candidates +} + +// Reject rejects a snapshot. Rejected snapshots will never be used again. +func (p *snapshotPool) Reject(snapshot *snapshot) { + key := snapshot.Key() + p.Lock() + defer p.Unlock() + + p.snapshotBlacklist[key] = true + p.removeSnapshot(key) +} + +// RejectFormat rejects a snapshot format. It will never be used again. +func (p *snapshotPool) RejectFormat(format uint32) { + p.Lock() + defer p.Unlock() + + p.formatBlacklist[format] = true + for key := range p.formatIndex[format] { + p.removeSnapshot(key) + } +} + +// RejectPeer rejects a peer. It will never be used again. +func (p *snapshotPool) RejectPeer(peerID p2p.ID) { + if peerID == "" { + return + } + p.Lock() + defer p.Unlock() + + p.removePeer(peerID) + p.peerBlacklist[peerID] = true +} + +// RemovePeer removes a peer from the pool, and any snapshots that no longer have peers. +func (p *snapshotPool) RemovePeer(peerID p2p.ID) { + p.Lock() + defer p.Unlock() + p.removePeer(peerID) +} + +// removePeer removes a peer. The caller must hold the mutex lock. +func (p *snapshotPool) removePeer(peerID p2p.ID) { + for key := range p.peerIndex[peerID] { + delete(p.snapshotPeers[key], peerID) + if len(p.snapshotPeers[key]) == 0 { + p.removeSnapshot(key) + } + } + delete(p.peerIndex, peerID) +} + +// removeSnapshot removes a snapshot. The caller must hold the mutex lock. +func (p *snapshotPool) removeSnapshot(key snapshotKey) { + snapshot := p.snapshots[key] + if snapshot == nil { + return + } + + delete(p.snapshots, key) + delete(p.formatIndex[snapshot.Format], key) + delete(p.heightIndex[snapshot.Height], key) + for peerID := range p.snapshotPeers[key] { + delete(p.peerIndex[peerID], key) + } + delete(p.snapshotPeers, key) +} diff --git a/statesync/snapshots_test.go b/statesync/snapshots_test.go new file mode 100644 index 000000000..e512da0b9 --- /dev/null +++ b/statesync/snapshots_test.go @@ -0,0 +1,326 @@ +package statesync + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + + "github.com/tendermint/tendermint/p2p" + p2pmocks "github.com/tendermint/tendermint/p2p/mocks" + "github.com/tendermint/tendermint/statesync/mocks" +) + +func TestSnapshot_Key(t *testing.T) { + testcases := map[string]struct { + modify func(*snapshot) + }{ + "new height": {func(s *snapshot) { s.Height = 9 }}, + "new format": {func(s *snapshot) { s.Format = 9 }}, + "new chunk count": {func(s *snapshot) { s.Chunks = 9 }}, + "new hash": {func(s *snapshot) { s.Hash = []byte{9} }}, + "no metadata": {func(s *snapshot) { s.Metadata = nil }}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + s := snapshot{ + Height: 3, + Format: 1, + Chunks: 7, + Hash: []byte{1, 2, 3}, + Metadata: []byte{255}, + } + before := s.Key() + tc.modify(&s) + after := s.Key() + assert.NotEqual(t, before, after) + }) + } +} + +func TestSnapshotPool_Add(t *testing.T) { + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", uint64(1)).Return([]byte("app_hash"), nil) + + peer := &p2pmocks.Peer{} + peer.On("ID").Return(p2p.ID("id")) + + // Adding to the pool should work + pool := newSnapshotPool(stateProvider) + added, err := pool.Add(peer, &snapshot{ + Height: 1, + Format: 1, + Chunks: 1, + Hash: []byte{1}, + }) + require.NoError(t, err) + assert.True(t, added) + + // Adding again from a different peer should return false + otherPeer := &p2pmocks.Peer{} + otherPeer.On("ID").Return(p2p.ID("other")) + added, err = pool.Add(peer, &snapshot{ + Height: 1, + Format: 1, + Chunks: 1, + Hash: []byte{1}, + }) + require.NoError(t, err) + assert.False(t, added) + + // The pool should have populated the snapshot with the trusted app hash + snapshot := pool.Best() + require.NotNil(t, snapshot) + assert.Equal(t, []byte("app_hash"), snapshot.trustedAppHash) + + stateProvider.AssertExpectations(t) +} + +func TestSnapshotPool_GetPeer(t *testing.T) { + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + pool := newSnapshotPool(stateProvider) + + s := &snapshot{Height: 1, Format: 1, Chunks: 1, Hash: []byte{1}} + peerA := &p2pmocks.Peer{} + peerA.On("ID").Return(p2p.ID("a")) + peerB := &p2pmocks.Peer{} + peerB.On("ID").Return(p2p.ID("b")) + + _, err := pool.Add(peerA, s) + require.NoError(t, err) + _, err = pool.Add(peerB, s) + require.NoError(t, err) + _, err = pool.Add(peerA, &snapshot{Height: 2, Format: 1, Chunks: 1, Hash: []byte{1}}) + require.NoError(t, err) + + // GetPeer currently picks a random peer, so lets run it until we've seen both. + seenA := false + seenB := false + for !seenA || !seenB { + peer := pool.GetPeer(s) + switch peer.ID() { + case p2p.ID("a"): + seenA = true + case p2p.ID("b"): + seenB = true + } + } + + // GetPeer should return nil for an unknown snapshot + peer := pool.GetPeer(&snapshot{Height: 9, Format: 9}) + assert.Nil(t, peer) +} + +func TestSnapshotPool_GetPeers(t *testing.T) { + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + pool := newSnapshotPool(stateProvider) + + s := &snapshot{Height: 1, Format: 1, Chunks: 1, Hash: []byte{1}} + peerA := &p2pmocks.Peer{} + peerA.On("ID").Return(p2p.ID("a")) + peerB := &p2pmocks.Peer{} + peerB.On("ID").Return(p2p.ID("b")) + + _, err := pool.Add(peerA, s) + require.NoError(t, err) + _, err = pool.Add(peerB, s) + require.NoError(t, err) + _, err = pool.Add(peerA, &snapshot{Height: 2, Format: 1, Chunks: 1, Hash: []byte{2}}) + require.NoError(t, err) + + peers := pool.GetPeers(s) + assert.Len(t, peers, 2) + assert.EqualValues(t, "a", peers[0].ID()) + assert.EqualValues(t, "b", peers[1].ID()) +} + +func TestSnapshotPool_Ranked_Best(t *testing.T) { + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + pool := newSnapshotPool(stateProvider) + + // snapshots in expected order (best to worst). Highest height wins, then highest format. + // Snapshots with different chunk hashes are considered different, and the most peers is + // tie-breaker. + expectSnapshots := []struct { + snapshot *snapshot + peers []string + }{ + {&snapshot{Height: 2, Format: 2, Chunks: 4, Hash: []byte{1, 3}}, []string{"a", "b", "c"}}, + {&snapshot{Height: 2, Format: 2, Chunks: 5, Hash: []byte{1, 2}}, []string{"a"}}, + {&snapshot{Height: 2, Format: 1, Chunks: 3, Hash: []byte{1, 2}}, []string{"a", "b"}}, + {&snapshot{Height: 1, Format: 2, Chunks: 5, Hash: []byte{1, 2}}, []string{"a", "b"}}, + {&snapshot{Height: 1, Format: 1, Chunks: 4, Hash: []byte{1, 2}}, []string{"a", "b", "c"}}, + } + + // Add snapshots in reverse order, to make sure the pool enforces some order. + for i := len(expectSnapshots) - 1; i >= 0; i-- { + for _, peerID := range expectSnapshots[i].peers { + peer := &p2pmocks.Peer{} + peer.On("ID").Return(p2p.ID(peerID)) + _, err := pool.Add(peer, expectSnapshots[i].snapshot) + require.NoError(t, err) + } + } + + // Ranked should return the snapshots in the same order + ranked := pool.Ranked() + assert.Len(t, ranked, len(expectSnapshots)) + for i := range ranked { + assert.Equal(t, expectSnapshots[i].snapshot, ranked[i]) + } + + // Check that best snapshots are returned in expected order + for i := range expectSnapshots { + snapshot := expectSnapshots[i].snapshot + require.Equal(t, snapshot, pool.Best()) + pool.Reject(snapshot) + } + assert.Nil(t, pool.Best()) +} + +func TestSnapshotPool_Reject(t *testing.T) { + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + pool := newSnapshotPool(stateProvider) + peer := &p2pmocks.Peer{} + peer.On("ID").Return(p2p.ID("id")) + + snapshots := []*snapshot{ + {Height: 2, Format: 2, Chunks: 1, Hash: []byte{1, 2}}, + {Height: 2, Format: 1, Chunks: 1, Hash: []byte{1, 2}}, + {Height: 1, Format: 2, Chunks: 1, Hash: []byte{1, 2}}, + {Height: 1, Format: 1, Chunks: 1, Hash: []byte{1, 2}}, + } + for _, s := range snapshots { + _, err := pool.Add(peer, s) + require.NoError(t, err) + } + + pool.Reject(snapshots[0]) + assert.Equal(t, snapshots[1:], pool.Ranked()) + + added, err := pool.Add(peer, snapshots[0]) + require.NoError(t, err) + assert.False(t, added) + + added, err = pool.Add(peer, &snapshot{Height: 3, Format: 3, Chunks: 1, Hash: []byte{1}}) + require.NoError(t, err) + assert.True(t, added) +} + +// nolint: dupl +func TestSnapshotPool_RejectFormat(t *testing.T) { + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + pool := newSnapshotPool(stateProvider) + peer := &p2pmocks.Peer{} + peer.On("ID").Return(p2p.ID("id")) + + snapshots := []*snapshot{ + {Height: 2, Format: 2, Chunks: 1, Hash: []byte{1, 2}}, + {Height: 2, Format: 1, Chunks: 1, Hash: []byte{1, 2}}, + {Height: 1, Format: 2, Chunks: 1, Hash: []byte{1, 2}}, + {Height: 1, Format: 1, Chunks: 1, Hash: []byte{1, 2}}, + } + for _, s := range snapshots { + _, err := pool.Add(peer, s) + require.NoError(t, err) + } + + pool.RejectFormat(1) + assert.Equal(t, []*snapshot{snapshots[0], snapshots[2]}, pool.Ranked()) + + added, err := pool.Add(peer, &snapshot{Height: 3, Format: 1, Chunks: 1, Hash: []byte{1}}) + require.NoError(t, err) + assert.False(t, added) + assert.Equal(t, []*snapshot{snapshots[0], snapshots[2]}, pool.Ranked()) + + added, err = pool.Add(peer, &snapshot{Height: 3, Format: 3, Chunks: 1, Hash: []byte{1}}) + require.NoError(t, err) + assert.True(t, added) +} + +func TestSnapshotPool_RejectPeer(t *testing.T) { + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + pool := newSnapshotPool(stateProvider) + + peerA := &p2pmocks.Peer{} + peerA.On("ID").Return(p2p.ID("a")) + peerB := &p2pmocks.Peer{} + peerB.On("ID").Return(p2p.ID("b")) + + s1 := &snapshot{Height: 1, Format: 1, Chunks: 1, Hash: []byte{1}} + s2 := &snapshot{Height: 2, Format: 1, Chunks: 1, Hash: []byte{2}} + s3 := &snapshot{Height: 3, Format: 1, Chunks: 1, Hash: []byte{2}} + + _, err := pool.Add(peerA, s1) + require.NoError(t, err) + _, err = pool.Add(peerA, s2) + require.NoError(t, err) + + _, err = pool.Add(peerB, s2) + require.NoError(t, err) + _, err = pool.Add(peerB, s3) + require.NoError(t, err) + + pool.RejectPeer(peerA.ID()) + + assert.Empty(t, pool.GetPeers(s1)) + + peers2 := pool.GetPeers(s2) + assert.Len(t, peers2, 1) + assert.EqualValues(t, "b", peers2[0].ID()) + + peers3 := pool.GetPeers(s2) + assert.Len(t, peers3, 1) + assert.EqualValues(t, "b", peers3[0].ID()) + + // it should no longer be possible to add the peer back + _, err = pool.Add(peerA, s1) + require.NoError(t, err) + assert.Empty(t, pool.GetPeers(s1)) +} + +func TestSnapshotPool_RemovePeer(t *testing.T) { + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + pool := newSnapshotPool(stateProvider) + + peerA := &p2pmocks.Peer{} + peerA.On("ID").Return(p2p.ID("a")) + peerB := &p2pmocks.Peer{} + peerB.On("ID").Return(p2p.ID("b")) + + s1 := &snapshot{Height: 1, Format: 1, Chunks: 1, Hash: []byte{1}} + s2 := &snapshot{Height: 2, Format: 1, Chunks: 1, Hash: []byte{2}} + + _, err := pool.Add(peerA, s1) + require.NoError(t, err) + _, err = pool.Add(peerA, s2) + require.NoError(t, err) + _, err = pool.Add(peerB, s1) + require.NoError(t, err) + + pool.RemovePeer(peerA.ID()) + + peers1 := pool.GetPeers(s1) + assert.Len(t, peers1, 1) + assert.EqualValues(t, "b", peers1[0].ID()) + + peers2 := pool.GetPeers(s2) + assert.Empty(t, peers2) + + // it should still be possible to add the peer back + _, err = pool.Add(peerA, s1) + require.NoError(t, err) + peers1 = pool.GetPeers(s1) + assert.Len(t, peers1, 2) + assert.EqualValues(t, "a", peers1[0].ID()) + assert.EqualValues(t, "b", peers1[1].ID()) +} diff --git a/statesync/stateprovider.go b/statesync/stateprovider.go new file mode 100644 index 000000000..2726de853 --- /dev/null +++ b/statesync/stateprovider.go @@ -0,0 +1,179 @@ +package statesync + +import ( + "fmt" + "strings" + "sync" + "time" + + dbm "github.com/tendermint/tm-db" + + "github.com/tendermint/tendermint/libs/log" + lite "github.com/tendermint/tendermint/lite2" + liteprovider "github.com/tendermint/tendermint/lite2/provider" + litehttp "github.com/tendermint/tendermint/lite2/provider/http" + literpc "github.com/tendermint/tendermint/lite2/rpc" + litedb "github.com/tendermint/tendermint/lite2/store/db" + rpchttp "github.com/tendermint/tendermint/rpc/client/http" + sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" +) + +//go:generate mockery -case underscore -name StateProvider + +// StateProvider is a provider of trusted state data for bootstrapping a node. This refers +// to the state.State object, not the state machine. +type StateProvider interface { + // AppHash returns the app hash after the given height has been committed. + AppHash(height uint64) ([]byte, error) + // Commit returns the commit at the given height. + Commit(height uint64) (*types.Commit, error) + // State returns a state object at the given height. + State(height uint64) (sm.State, error) +} + +// lightClientStateProvider is a state provider using the light client. +type lightClientStateProvider struct { + sync.Mutex // lite.Client is not concurrency-safe + lc *lite.Client + version sm.Version + providers map[liteprovider.Provider]string +} + +// NewLightClientStateProvider creates a new StateProvider using a light client and RPC clients. +func NewLightClientStateProvider( + chainID string, + version sm.Version, + servers []string, + trustOptions lite.TrustOptions, + logger log.Logger, +) (StateProvider, error) { + if len(servers) < 2 { + return nil, fmt.Errorf("at least 2 RPC servers are required, got %v", len(servers)) + } + + providers := make([]liteprovider.Provider, 0, len(servers)) + providerRemotes := make(map[liteprovider.Provider]string) + for _, server := range servers { + client, err := rpcClient(server) + if err != nil { + return nil, fmt.Errorf("failed to set up RPC client: %w", err) + } + provider := litehttp.NewWithClient(chainID, client) + providers = append(providers, provider) + // We store the RPC addresses keyed by provider, so we can find the address of the primary + // provider used by the light client and use it to fetch consensus parameters. + providerRemotes[provider] = server + } + + lc, err := lite.NewClient(chainID, trustOptions, providers[0], providers[1:], + litedb.New(dbm.NewMemDB(), ""), lite.Logger(logger), lite.MaxRetryAttempts(5)) + if err != nil { + return nil, err + } + return &lightClientStateProvider{ + lc: lc, + version: version, + providers: providerRemotes, + }, nil +} + +// AppHash implements StateProvider. +func (s *lightClientStateProvider) AppHash(height uint64) ([]byte, error) { + s.Lock() + defer s.Unlock() + + // We have to fetch the next height, which contains the app hash for the previous height. + header, err := s.lc.VerifyHeaderAtHeight(int64(height+1), time.Now()) + if err != nil { + return nil, err + } + return header.AppHash, nil +} + +// Commit implements StateProvider. +func (s *lightClientStateProvider) Commit(height uint64) (*types.Commit, error) { + s.Lock() + defer s.Unlock() + header, err := s.lc.VerifyHeaderAtHeight(int64(height), time.Now()) + if err != nil { + return nil, err + } + return header.Commit, nil +} + +// State implements StateProvider. +func (s *lightClientStateProvider) State(height uint64) (sm.State, error) { + s.Lock() + defer s.Unlock() + + state := sm.State{ + ChainID: s.lc.ChainID(), + Version: s.version, + } + + // We need to verify up until h+2, to get the validator set. This also prefetches the headers + // for h and h+1 in the typical case where the trusted header is after the snapshot height. + _, err := s.lc.VerifyHeaderAtHeight(int64(height+2), time.Now()) + if err != nil { + return sm.State{}, err + } + header, err := s.lc.VerifyHeaderAtHeight(int64(height), time.Now()) + if err != nil { + return sm.State{}, err + } + nextHeader, err := s.lc.VerifyHeaderAtHeight(int64(height+1), time.Now()) + if err != nil { + return sm.State{}, err + } + state.LastBlockHeight = header.Height + state.LastBlockTime = header.Time + state.LastBlockID = header.Commit.BlockID + state.AppHash = nextHeader.AppHash + state.LastResultsHash = nextHeader.LastResultsHash + + state.LastValidators, _, err = s.lc.TrustedValidatorSet(int64(height)) + if err != nil { + return sm.State{}, err + } + state.Validators, _, err = s.lc.TrustedValidatorSet(int64(height + 1)) + if err != nil { + return sm.State{}, err + } + state.NextValidators, _, err = s.lc.TrustedValidatorSet(int64(height + 2)) + if err != nil { + return sm.State{}, err + } + state.LastHeightValidatorsChanged = int64(height) + + // We'll also need to fetch consensus params via RPC, using light client verification. + primaryURL, ok := s.providers[s.lc.Primary()] + if !ok || primaryURL == "" { + return sm.State{}, fmt.Errorf("could not find address for primary light client provider") + } + primaryRPC, err := rpcClient(primaryURL) + if err != nil { + return sm.State{}, fmt.Errorf("unable to create RPC client: %w", err) + } + rpcclient := literpc.NewClient(primaryRPC, s.lc) + result, err := rpcclient.ConsensusParams(&nextHeader.Height) + if err != nil { + return sm.State{}, fmt.Errorf("unable to fetch consensus parameters for height %v: %w", + nextHeader.Height, err) + } + state.ConsensusParams = result.ConsensusParams + + return state, nil +} + +// rpcClient sets up a new RPC client +func rpcClient(server string) (*rpchttp.HTTP, error) { + if !strings.Contains(server, "://") { + server = "http://" + server + } + c, err := rpchttp.New(server, "/websocket") + if err != nil { + return nil, err + } + return c, nil +} diff --git a/statesync/syncer.go b/statesync/syncer.go new file mode 100644 index 000000000..b53f452c0 --- /dev/null +++ b/statesync/syncer.go @@ -0,0 +1,442 @@ +package statesync + +import ( + "bytes" + "context" + "errors" + "fmt" + "sync" + "time" + + abci "github.com/tendermint/tendermint/abci/types" + "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/p2p" + "github.com/tendermint/tendermint/proxy" + sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/types" + "github.com/tendermint/tendermint/version" +) + +const ( + // defaultDiscoveryTime is the time to spend discovering snapshots. + defaultDiscoveryTime = 20 * time.Second + // chunkFetchers is the number of concurrent chunk fetchers to run. + chunkFetchers = 4 + // chunkTimeout is the timeout while waiting for the next chunk from the chunk queue. + chunkTimeout = 2 * time.Minute + // requestTimeout is the timeout before rerequesting a chunk, possibly from a different peer. + chunkRequestTimeout = 10 * time.Second +) + +var ( + // errAbort is returned by Sync() when snapshot restoration is aborted. + errAbort = errors.New("state sync aborted") + // errRetrySnapshot is returned by Sync() when the snapshot should be retried. + errRetrySnapshot = errors.New("retry snapshot") + // errRejectSnapshot is returned by Sync() when the snapshot is rejected. + errRejectSnapshot = errors.New("snapshot was rejected") + // errRejectFormat is returned by Sync() when the snapshot format is rejected. + errRejectFormat = errors.New("snapshot format was rejected") + // errRejectSender is returned by Sync() when the snapshot sender is rejected. + errRejectSender = errors.New("snapshot sender was rejected") + // errVerifyFailed is returned by Sync() when app hash or last height verification fails. + errVerifyFailed = errors.New("verification failed") + // errTimeout is returned by Sync() when we've waited too long to receive a chunk. + errTimeout = errors.New("timed out waiting for chunk") + // errNoSnapshots is returned by SyncAny() if no snapshots are found and discovery is disabled. + errNoSnapshots = errors.New("no suitable snapshots found") +) + +// syncer runs a state sync against an ABCI app. Use either SyncAny() to automatically attempt to +// sync all snapshots in the pool (pausing to discover new ones), or Sync() to sync a specific +// snapshot. Snapshots and chunks are fed via AddSnapshot() and AddChunk() as appropriate. +type syncer struct { + logger log.Logger + stateProvider StateProvider + conn proxy.AppConnSnapshot + connQuery proxy.AppConnQuery + snapshots *snapshotPool + tempDir string + + mtx sync.RWMutex + chunks *chunkQueue +} + +// newSyncer creates a new syncer. +func newSyncer(logger log.Logger, conn proxy.AppConnSnapshot, connQuery proxy.AppConnQuery, + stateProvider StateProvider, tempDir string) *syncer { + return &syncer{ + logger: logger, + stateProvider: stateProvider, + conn: conn, + connQuery: connQuery, + snapshots: newSnapshotPool(stateProvider), + tempDir: tempDir, + } +} + +// AddChunk adds a chunk to the chunk queue, if any. It returns false if the chunk has already +// been added to the queue, or an error if there's no sync in progress. +func (s *syncer) AddChunk(chunk *chunk) (bool, error) { + s.mtx.RLock() + defer s.mtx.RUnlock() + if s.chunks == nil { + return false, errors.New("no state sync in progress") + } + added, err := s.chunks.Add(chunk) + if err != nil { + return false, err + } + if added { + s.logger.Debug("Added chunk to queue", "height", chunk.Height, "format", chunk.Format, + "chunk", chunk.Index) + } else { + s.logger.Debug("Ignoring duplicate chunk in queue", "height", chunk.Height, "format", chunk.Format, + "chunk", chunk.Index) + } + return added, nil +} + +// AddSnapshot adds a snapshot to the snapshot pool. It returns true if a new, previously unseen +// snapshot was accepted and added. +func (s *syncer) AddSnapshot(peer p2p.Peer, snapshot *snapshot) (bool, error) { + added, err := s.snapshots.Add(peer, snapshot) + if err != nil { + return false, err + } + if added { + s.logger.Info("Discovered new snapshot", "height", snapshot.Height, "format", snapshot.Format, + "hash", fmt.Sprintf("%X", snapshot.Hash)) + } + return added, nil +} + +// AddPeer adds a peer to the pool. For now we just keep it simple and send a single request +// to discover snapshots, later we may want to do retries and stuff. +func (s *syncer) AddPeer(peer p2p.Peer) { + s.logger.Debug("Requesting snapshots from peer", "peer", peer.ID()) + peer.Send(SnapshotChannel, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{})) +} + +// RemovePeer removes a peer from the pool. +func (s *syncer) RemovePeer(peer p2p.Peer) { + s.logger.Debug("Removing peer from sync", "peer", peer.ID()) + s.snapshots.RemovePeer(peer.ID()) +} + +// SyncAny tries to sync any of the snapshots in the snapshot pool, waiting to discover further +// snapshots if none were found and discoveryTime > 0. It returns the latest state and block commit +// which the caller must use to bootstrap the node. +func (s *syncer) SyncAny(discoveryTime time.Duration) (sm.State, *types.Commit, error) { + if discoveryTime > 0 { + s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime)) + time.Sleep(discoveryTime) + } + + // The app may ask us to retry a snapshot restoration, in which case we need to reuse + // the snapshot and chunk queue from the previous loop iteration. + var ( + snapshot *snapshot + chunks *chunkQueue + err error + ) + for { + // If not nil, we're going to retry restoration of the same snapshot. + if snapshot == nil { + snapshot = s.snapshots.Best() + chunks = nil + } + if snapshot == nil { + if discoveryTime == 0 { + return sm.State{}, nil, errNoSnapshots + } + s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime)) + time.Sleep(discoveryTime) + continue + } + if chunks == nil { + chunks, err = newChunkQueue(snapshot, s.tempDir) + if err != nil { + return sm.State{}, nil, fmt.Errorf("failed to create chunk queue: %w", err) + } + defer chunks.Close() // in case we forget to close it elsewhere + } + + newState, commit, err := s.Sync(snapshot, chunks) + switch { + case err == nil: + return newState, commit, nil + + case errors.Is(err, errAbort): + return sm.State{}, nil, err + + case errors.Is(err, errRetrySnapshot): + chunks.RetryAll() + s.logger.Info("Retrying snapshot", "height", snapshot.Height, "format", snapshot.Format, + "hash", fmt.Sprintf("%X", snapshot.Hash)) + continue + + case errors.Is(err, errTimeout): + s.snapshots.Reject(snapshot) + s.logger.Error("Timed out waiting for snapshot chunks, rejected snapshot", + "height", snapshot.Height, "format", snapshot.Format, "hash", fmt.Sprintf("%X", snapshot.Hash)) + + case errors.Is(err, errRejectSnapshot): + s.snapshots.Reject(snapshot) + s.logger.Info("Snapshot rejected", "height", snapshot.Height, "format", snapshot.Format, + "hash", fmt.Sprintf("%X", snapshot.Hash)) + + case errors.Is(err, errRejectFormat): + s.snapshots.RejectFormat(snapshot.Format) + s.logger.Info("Snapshot format rejected", "format", snapshot.Format) + + case errors.Is(err, errRejectSender): + s.logger.Info("Snapshot senders rejected", "height", snapshot.Height, "format", snapshot.Format, + "hash", fmt.Sprintf("%X", snapshot.Hash)) + for _, peer := range s.snapshots.GetPeers(snapshot) { + s.snapshots.RejectPeer(peer.ID()) + s.logger.Info("Snapshot sender rejected", "peer", peer.ID()) + } + + default: + return sm.State{}, nil, fmt.Errorf("snapshot restoration failed: %w", err) + } + + // Discard snapshot and chunks for next iteration + err = chunks.Close() + if err != nil { + s.logger.Error("Failed to clean up chunk queue", "err", err) + } + snapshot = nil + chunks = nil + } +} + +// Sync executes a sync for a specific snapshot, returning the latest state and block commit which +// the caller must use to bootstrap the node. +func (s *syncer) Sync(snapshot *snapshot, chunks *chunkQueue) (sm.State, *types.Commit, error) { + s.mtx.Lock() + if s.chunks != nil { + s.mtx.Unlock() + return sm.State{}, nil, errors.New("a state sync is already in progress") + } + s.chunks = chunks + s.mtx.Unlock() + defer func() { + s.mtx.Lock() + s.chunks = nil + s.mtx.Unlock() + }() + + // Offer snapshot to ABCI app. + err := s.offerSnapshot(snapshot) + if err != nil { + return sm.State{}, nil, err + } + + // Spawn chunk fetchers. They will terminate when the chunk queue is closed or context cancelled. + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + for i := int32(0); i < chunkFetchers; i++ { + go s.fetchChunks(ctx, snapshot, chunks) + } + + // Optimistically build new state, so we don't discover any light client failures at the end. + state, err := s.stateProvider.State(snapshot.Height) + if err != nil { + return sm.State{}, nil, fmt.Errorf("failed to build new state: %w", err) + } + commit, err := s.stateProvider.Commit(snapshot.Height) + if err != nil { + return sm.State{}, nil, fmt.Errorf("failed to fetch commit: %w", err) + } + + // Restore snapshot + err = s.applyChunks(chunks) + if err != nil { + return sm.State{}, nil, err + } + + // Verify app and update app version + appVersion, err := s.verifyApp(snapshot) + if err != nil { + return sm.State{}, nil, err + } + state.Version.Consensus.App = version.Protocol(appVersion) + + // Done! 🎉 + s.logger.Info("Snapshot restored", "height", snapshot.Height, "format", snapshot.Format, + "hash", fmt.Sprintf("%X", snapshot.Hash)) + + return state, commit, nil +} + +// offerSnapshot offers a snapshot to the app. It returns various errors depending on the app's +// response, or nil if the snapshot was accepted. +func (s *syncer) offerSnapshot(snapshot *snapshot) error { + s.logger.Info("Offering snapshot to ABCI app", "height", snapshot.Height, + "format", snapshot.Format, "hash", fmt.Sprintf("%X", snapshot.Hash)) + resp, err := s.conn.OfferSnapshotSync(abci.RequestOfferSnapshot{ + Snapshot: &abci.Snapshot{ + Height: snapshot.Height, + Format: snapshot.Format, + Chunks: snapshot.Chunks, + Hash: snapshot.Hash, + Metadata: snapshot.Metadata, + }, + AppHash: snapshot.trustedAppHash, + }) + if err != nil { + return fmt.Errorf("failed to offer snapshot: %w", err) + } + switch resp.Result { + case abci.ResponseOfferSnapshot_accept: + s.logger.Info("Snapshot accepted, restoring", "height", snapshot.Height, + "format", snapshot.Format, "hash", fmt.Sprintf("%X", snapshot.Hash)) + return nil + case abci.ResponseOfferSnapshot_abort: + return errAbort + case abci.ResponseOfferSnapshot_reject: + return errRejectSnapshot + case abci.ResponseOfferSnapshot_reject_format: + return errRejectFormat + case abci.ResponseOfferSnapshot_reject_sender: + return errRejectSender + default: + return fmt.Errorf("invalid ResponseOfferSnapshot result %v", resp.Result) + } +} + +// applyChunks applies chunks to the app. It returns various errors depending on the app's +// response, or nil once the snapshot is fully restored. +func (s *syncer) applyChunks(chunks *chunkQueue) error { + for { + chunk, err := chunks.Next() + if err == errDone { + return nil + } else if err != nil { + return fmt.Errorf("failed to fetch chunk: %w", err) + } + + resp, err := s.conn.ApplySnapshotChunkSync(abci.RequestApplySnapshotChunk{ + Index: chunk.Index, + Chunk: chunk.Chunk, + Sender: string(chunk.Sender), + }) + if err != nil { + return fmt.Errorf("failed to apply chunk %v: %w", chunk.Index, err) + } + s.logger.Info("Applied snapshot chunk to ABCI app", "height", chunk.Height, + "format", chunk.Format, "chunk", chunk.Index, "total", chunks.Size()) + + // Discard and refetch any chunks as requested by the app + for _, index := range resp.RefetchChunks { + err := chunks.Discard(index) + if err != nil { + return fmt.Errorf("failed to discard chunk %v: %w", index, err) + } + } + + // Reject any senders as requested by the app + for _, sender := range resp.RejectSenders { + if sender != "" { + s.snapshots.RejectPeer(p2p.ID(sender)) + err := chunks.DiscardSender(p2p.ID(sender)) + if err != nil { + return fmt.Errorf("failed to reject sender: %w", err) + } + } + } + + switch resp.Result { + case abci.ResponseApplySnapshotChunk_accept: + case abci.ResponseApplySnapshotChunk_abort: + return errAbort + case abci.ResponseApplySnapshotChunk_retry: + chunks.Retry(chunk.Index) + case abci.ResponseApplySnapshotChunk_retry_snapshot: + return errRetrySnapshot + case abci.ResponseApplySnapshotChunk_reject_snapshot: + return errRejectSnapshot + default: + return fmt.Errorf("unknown ResponseApplySnapshotChunk result %v", resp.Result) + } + } +} + +// fetchChunks requests chunks from peers, receiving allocations from the chunk queue. Chunks +// will be received from the reactor via syncer.AddChunks() to chunkQueue.Add(). +func (s *syncer) fetchChunks(ctx context.Context, snapshot *snapshot, chunks *chunkQueue) { + for { + index, err := chunks.Allocate() + if err == errDone { + // Keep checking until the context is cancelled (restore is done), in case any + // chunks need to be refetched. + select { + case <-ctx.Done(): + return + default: + } + time.Sleep(2 * time.Second) + continue + } + if err != nil { + s.logger.Error("Failed to allocate chunk from queue", "err", err) + return + } + s.logger.Info("Fetching snapshot chunk", "height", snapshot.Height, + "format", snapshot.Format, "chunk", index, "total", chunks.Size()) + + ticker := time.NewTicker(chunkRequestTimeout) + defer ticker.Stop() + s.requestChunk(snapshot, index) + select { + case <-chunks.WaitFor(index): + case <-ticker.C: + s.requestChunk(snapshot, index) + case <-ctx.Done(): + return + } + ticker.Stop() + } +} + +// requestChunk requests a chunk from a peer. +func (s *syncer) requestChunk(snapshot *snapshot, chunk uint32) { + peer := s.snapshots.GetPeer(snapshot) + if peer == nil { + s.logger.Error("No valid peers found for snapshot", "height", snapshot.Height, + "format", snapshot.Format, "hash", snapshot.Hash) + return + } + s.logger.Debug("Requesting snapshot chunk", "height", snapshot.Height, + "format", snapshot.Format, "chunk", chunk, "peer", peer.ID()) + peer.Send(ChunkChannel, cdc.MustMarshalBinaryBare(&chunkRequestMessage{ + Height: snapshot.Height, + Format: snapshot.Format, + Index: chunk, + })) +} + +// verifyApp verifies the sync, checking the app hash and last block height. It returns the +// app version, which should be returned as part of the initial state. +func (s *syncer) verifyApp(snapshot *snapshot) (uint64, error) { + resp, err := s.connQuery.InfoSync(proxy.RequestInfo) + if err != nil { + return 0, fmt.Errorf("failed to query ABCI app for appHash: %w", err) + } + if !bytes.Equal(snapshot.trustedAppHash, resp.LastBlockAppHash) { + s.logger.Error("appHash verification failed", + "expected", fmt.Sprintf("%X", snapshot.trustedAppHash), + "actual", fmt.Sprintf("%X", resp.LastBlockAppHash)) + return 0, errVerifyFailed + } + if uint64(resp.LastBlockHeight) != snapshot.Height { + s.logger.Error("ABCI app reported unexpected last block height", + "expected", snapshot.Height, "actual", resp.LastBlockHeight) + return 0, errVerifyFailed + } + s.logger.Info("Verified ABCI app", "height", snapshot.Height, + "appHash", fmt.Sprintf("%X", snapshot.trustedAppHash)) + return resp.AppVersion, nil +} diff --git a/statesync/syncer_test.go b/statesync/syncer_test.go new file mode 100644 index 000000000..c56c966a2 --- /dev/null +++ b/statesync/syncer_test.go @@ -0,0 +1,639 @@ +package statesync + +import ( + "errors" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + + abci "github.com/tendermint/tendermint/abci/types" + "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/p2p" + p2pmocks "github.com/tendermint/tendermint/p2p/mocks" + "github.com/tendermint/tendermint/proxy" + proxymocks "github.com/tendermint/tendermint/proxy/mocks" + sm "github.com/tendermint/tendermint/state" + "github.com/tendermint/tendermint/statesync/mocks" + "github.com/tendermint/tendermint/types" + "github.com/tendermint/tendermint/version" +) + +// Sets up a basic syncer that can be used to test OfferSnapshot requests +func setupOfferSyncer(t *testing.T) (*syncer, *proxymocks.AppConnSnapshot) { + connQuery := &proxymocks.AppConnQuery{} + connSnapshot := &proxymocks.AppConnSnapshot{} + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "") + return syncer, connSnapshot +} + +// Sets up a simple peer mock with an ID +func simplePeer(id string) *p2pmocks.Peer { + peer := &p2pmocks.Peer{} + peer.On("ID").Return(p2p.ID(id)) + return peer +} + +func TestSyncer_SyncAny(t *testing.T) { + state := sm.State{ + ChainID: "chain", + Version: sm.Version{ + Consensus: version.Consensus{ + Block: version.BlockProtocol, + App: 0, + }, + + Software: version.TMCoreSemVer, + }, + + LastBlockHeight: 1, + LastBlockID: types.BlockID{Hash: []byte("blockhash")}, + LastBlockTime: time.Now(), + LastResultsHash: []byte("last_results_hash"), + AppHash: []byte("app_hash"), + + LastValidators: &types.ValidatorSet{Proposer: &types.Validator{Address: []byte("val1")}}, + Validators: &types.ValidatorSet{Proposer: &types.Validator{Address: []byte("val2")}}, + NextValidators: &types.ValidatorSet{Proposer: &types.Validator{Address: []byte("val3")}}, + + ConsensusParams: *types.DefaultConsensusParams(), + LastHeightConsensusParamsChanged: 1, + } + commit := &types.Commit{BlockID: types.BlockID{Hash: []byte("blockhash")}} + + chunks := []*chunk{ + {Height: 1, Format: 1, Index: 0, Chunk: []byte{1, 1, 0}}, + {Height: 1, Format: 1, Index: 1, Chunk: []byte{1, 1, 1}}, + {Height: 1, Format: 1, Index: 2, Chunk: []byte{1, 1, 2}}, + } + s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}} + + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", uint64(1)).Return(state.AppHash, nil) + stateProvider.On("AppHash", uint64(2)).Return([]byte("app_hash_2"), nil) + stateProvider.On("Commit", uint64(1)).Return(commit, nil) + stateProvider.On("State", uint64(1)).Return(state, nil) + connSnapshot := &proxymocks.AppConnSnapshot{} + connQuery := &proxymocks.AppConnQuery{} + + syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "") + + // Adding a chunk should error when no sync is in progress + _, err := syncer.AddChunk(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{1}}) + require.Error(t, err) + + // Adding a couple of peers should trigger snapshot discovery messages + peerA := &p2pmocks.Peer{} + peerA.On("ID").Return(p2p.ID("a")) + peerA.On("Send", SnapshotChannel, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{})).Return(true) + syncer.AddPeer(peerA) + peerA.AssertExpectations(t) + + peerB := &p2pmocks.Peer{} + peerB.On("ID").Return(p2p.ID("b")) + peerB.On("Send", SnapshotChannel, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{})).Return(true) + syncer.AddPeer(peerB) + peerB.AssertExpectations(t) + + // Both peers report back with snapshots. One of them also returns a snapshot we don't want, in + // format 2, which will be rejected by the ABCI application. + new, err := syncer.AddSnapshot(peerA, s) + require.NoError(t, err) + assert.True(t, new) + + new, err = syncer.AddSnapshot(peerB, s) + require.NoError(t, err) + assert.False(t, new) + + new, err = syncer.AddSnapshot(peerB, &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1}}) + require.NoError(t, err) + assert.True(t, new) + + // We start a sync, with peers sending back chunks when requested. We first reject the snapshot + // with height 2 format 2, and accept the snapshot at height 1. + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: &abci.Snapshot{ + Height: 2, + Format: 2, + Chunks: 3, + Hash: []byte{1}, + }, + AppHash: []byte("app_hash_2"), + }).Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject_format}, nil) + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: &abci.Snapshot{ + Height: s.Height, + Format: s.Format, + Chunks: s.Chunks, + Hash: s.Hash, + Metadata: s.Metadata, + }, + AppHash: []byte("app_hash"), + }).Times(2).Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_accept}, nil) + + chunkRequests := make(map[uint32]int) + chunkRequestsMtx := sync.Mutex{} + onChunkRequest := func(args mock.Arguments) { + msg := &chunkRequestMessage{} + err := cdc.UnmarshalBinaryBare(args[1].([]byte), &msg) + require.NoError(t, err) + require.EqualValues(t, 1, msg.Height) + require.EqualValues(t, 1, msg.Format) + require.LessOrEqual(t, msg.Index, uint32(len(chunks))) + + added, err := syncer.AddChunk(chunks[msg.Index]) + require.NoError(t, err) + assert.True(t, added) + + chunkRequestsMtx.Lock() + chunkRequests[msg.Index]++ + chunkRequestsMtx.Unlock() + } + peerA.On("Send", ChunkChannel, mock.Anything).Maybe().Run(onChunkRequest).Return(true) + peerB.On("Send", ChunkChannel, mock.Anything).Maybe().Run(onChunkRequest).Return(true) + + // The first time we're applying chunk 2 we tell it to retry the snapshot and discard chunk 1, + // which should cause it to keep the existing chunk 0 and 2, and restart restoration from + // beginning. We also wait for a little while, to exercise the retry logic in fetchChunks(). + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 2, Chunk: []byte{1, 1, 2}, + }).Once().Run(func(args mock.Arguments) { time.Sleep(2 * time.Second) }).Return( + &abci.ResponseApplySnapshotChunk{ + Result: abci.ResponseApplySnapshotChunk_retry_snapshot, + RefetchChunks: []uint32{1}, + }, nil) + + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 0, Chunk: []byte{1, 1, 0}, + }).Times(2).Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil) + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 1, Chunk: []byte{1, 1, 1}, + }).Times(2).Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil) + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 2, Chunk: []byte{1, 1, 2}, + }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil) + connQuery.On("InfoSync", proxy.RequestInfo).Return(&abci.ResponseInfo{ + AppVersion: 9, + LastBlockHeight: 1, + LastBlockAppHash: []byte("app_hash"), + }, nil) + + newState, lastCommit, err := syncer.SyncAny(0) + require.NoError(t, err) + + chunkRequestsMtx.Lock() + assert.Equal(t, map[uint32]int{0: 1, 1: 2, 2: 1}, chunkRequests) + chunkRequestsMtx.Unlock() + + // The syncer should have updated the state app version from the ABCI info response. + expectState := state + expectState.Version.Consensus.App = 9 + + assert.Equal(t, expectState, newState) + assert.Equal(t, commit, lastCommit) + + connSnapshot.AssertExpectations(t) + connQuery.AssertExpectations(t) + peerA.AssertExpectations(t) + peerB.AssertExpectations(t) +} + +func TestSyncer_SyncAny_noSnapshots(t *testing.T) { + syncer, _ := setupOfferSyncer(t) + _, _, err := syncer.SyncAny(0) + assert.Equal(t, errNoSnapshots, err) +} + +func TestSyncer_SyncAny_abort(t *testing.T) { + syncer, connSnapshot := setupOfferSyncer(t) + + s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}} + syncer.AddSnapshot(simplePeer("id"), s) + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: toABCI(s), AppHash: []byte("app_hash"), + }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_abort}, nil) + + _, _, err := syncer.SyncAny(0) + assert.Equal(t, errAbort, err) + connSnapshot.AssertExpectations(t) +} + +func TestSyncer_SyncAny_reject(t *testing.T) { + syncer, connSnapshot := setupOfferSyncer(t) + + // s22 is tried first, then s12, then s11, then errNoSnapshots + s22 := &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}} + s12 := &snapshot{Height: 1, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}} + s11 := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}} + syncer.AddSnapshot(simplePeer("id"), s22) + syncer.AddSnapshot(simplePeer("id"), s12) + syncer.AddSnapshot(simplePeer("id"), s11) + + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: toABCI(s22), AppHash: []byte("app_hash"), + }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject}, nil) + + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: toABCI(s12), AppHash: []byte("app_hash"), + }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject}, nil) + + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: toABCI(s11), AppHash: []byte("app_hash"), + }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject}, nil) + + _, _, err := syncer.SyncAny(0) + assert.Equal(t, errNoSnapshots, err) + connSnapshot.AssertExpectations(t) +} + +func TestSyncer_SyncAny_reject_format(t *testing.T) { + syncer, connSnapshot := setupOfferSyncer(t) + + // s22 is tried first, which reject s22 and s12, then s11 will abort. + s22 := &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}} + s12 := &snapshot{Height: 1, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}} + s11 := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}} + syncer.AddSnapshot(simplePeer("id"), s22) + syncer.AddSnapshot(simplePeer("id"), s12) + syncer.AddSnapshot(simplePeer("id"), s11) + + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: toABCI(s22), AppHash: []byte("app_hash"), + }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject_format}, nil) + + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: toABCI(s11), AppHash: []byte("app_hash"), + }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_abort}, nil) + + _, _, err := syncer.SyncAny(0) + assert.Equal(t, errAbort, err) + connSnapshot.AssertExpectations(t) +} + +func TestSyncer_SyncAny_reject_sender(t *testing.T) { + syncer, connSnapshot := setupOfferSyncer(t) + + peerA := simplePeer("a") + peerB := simplePeer("b") + peerC := simplePeer("c") + + // sbc will be offered first, which will be rejected with reject_sender, causing all snapshots + // submitted by both b and c (i.e. sb, sc, sbc) to be rejected. Finally, sa will reject and + // errNoSnapshots is returned. + sa := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}} + sb := &snapshot{Height: 2, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}} + sc := &snapshot{Height: 3, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}} + sbc := &snapshot{Height: 4, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}} + _, err := syncer.AddSnapshot(peerA, sa) + require.NoError(t, err) + _, err = syncer.AddSnapshot(peerB, sb) + require.NoError(t, err) + _, err = syncer.AddSnapshot(peerC, sc) + require.NoError(t, err) + _, err = syncer.AddSnapshot(peerB, sbc) + require.NoError(t, err) + _, err = syncer.AddSnapshot(peerC, sbc) + require.NoError(t, err) + + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: toABCI(sbc), AppHash: []byte("app_hash"), + }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject_sender}, nil) + + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: toABCI(sa), AppHash: []byte("app_hash"), + }).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject}, nil) + + _, _, err = syncer.SyncAny(0) + assert.Equal(t, errNoSnapshots, err) + connSnapshot.AssertExpectations(t) +} + +func TestSyncer_SyncAny_abciError(t *testing.T) { + syncer, connSnapshot := setupOfferSyncer(t) + + errBoom := errors.New("boom") + s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}} + syncer.AddSnapshot(simplePeer("id"), s) + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: toABCI(s), AppHash: []byte("app_hash"), + }).Once().Return(nil, errBoom) + + _, _, err := syncer.SyncAny(0) + assert.True(t, errors.Is(err, errBoom)) + connSnapshot.AssertExpectations(t) +} + +func TestSyncer_offerSnapshot(t *testing.T) { + unknownErr := errors.New("unknown error") + boom := errors.New("boom") + + testcases := map[string]struct { + result abci.ResponseOfferSnapshot_Result + err error + expectErr error + }{ + "accept": {abci.ResponseOfferSnapshot_accept, nil, nil}, + "abort": {abci.ResponseOfferSnapshot_abort, nil, errAbort}, + "reject": {abci.ResponseOfferSnapshot_reject, nil, errRejectSnapshot}, + "reject_format": {abci.ResponseOfferSnapshot_reject_format, nil, errRejectFormat}, + "reject_sender": {abci.ResponseOfferSnapshot_reject_sender, nil, errRejectSender}, + "error": {0, boom, boom}, + "unknown result": {9, nil, unknownErr}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + syncer, connSnapshot := setupOfferSyncer(t) + s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}, trustedAppHash: []byte("app_hash")} + connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{ + Snapshot: toABCI(s), + AppHash: []byte("app_hash"), + }).Return(&abci.ResponseOfferSnapshot{Result: tc.result}, tc.err) + err := syncer.offerSnapshot(s) + if tc.expectErr == unknownErr { + require.Error(t, err) + } else { + unwrapped := errors.Unwrap(err) + if unwrapped != nil { + err = unwrapped + } + assert.Equal(t, tc.expectErr, err) + } + }) + } +} + +func TestSyncer_applyChunks_Results(t *testing.T) { + unknownErr := errors.New("unknown error") + boom := errors.New("boom") + + testcases := map[string]struct { + result abci.ResponseApplySnapshotChunk_Result + err error + expectErr error + }{ + "accept": {abci.ResponseApplySnapshotChunk_accept, nil, nil}, + "abort": {abci.ResponseApplySnapshotChunk_abort, nil, errAbort}, + "retry": {abci.ResponseApplySnapshotChunk_retry, nil, nil}, + "retry_snapshot": {abci.ResponseApplySnapshotChunk_retry_snapshot, nil, errRetrySnapshot}, + "reject_snapshot": {abci.ResponseApplySnapshotChunk_reject_snapshot, nil, errRejectSnapshot}, + "error": {0, boom, boom}, + "unknown result": {9, nil, unknownErr}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + connQuery := &proxymocks.AppConnQuery{} + connSnapshot := &proxymocks.AppConnSnapshot{} + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "") + + body := []byte{1, 2, 3} + chunks, err := newChunkQueue(&snapshot{Height: 1, Format: 1, Chunks: 1}, "") + chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: body}) + require.NoError(t, err) + + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 0, Chunk: body, + }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: tc.result}, tc.err) + if tc.result == abci.ResponseApplySnapshotChunk_retry { + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 0, Chunk: body, + }).Once().Return(&abci.ResponseApplySnapshotChunk{ + Result: abci.ResponseApplySnapshotChunk_accept}, nil) + } + + err = syncer.applyChunks(chunks) + if tc.expectErr == unknownErr { + require.Error(t, err) + } else { + unwrapped := errors.Unwrap(err) + if unwrapped != nil { + err = unwrapped + } + assert.Equal(t, tc.expectErr, err) + } + connSnapshot.AssertExpectations(t) + }) + } +} + +func TestSyncer_applyChunks_RefetchChunks(t *testing.T) { + // Discarding chunks via refetch_chunks should work the same for all results + testcases := map[string]struct { + result abci.ResponseApplySnapshotChunk_Result + }{ + "accept": {abci.ResponseApplySnapshotChunk_accept}, + "abort": {abci.ResponseApplySnapshotChunk_abort}, + "retry": {abci.ResponseApplySnapshotChunk_retry}, + "retry_snapshot": {abci.ResponseApplySnapshotChunk_retry_snapshot}, + "reject_snapshot": {abci.ResponseApplySnapshotChunk_reject_snapshot}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + connQuery := &proxymocks.AppConnQuery{} + connSnapshot := &proxymocks.AppConnSnapshot{} + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "") + + chunks, err := newChunkQueue(&snapshot{Height: 1, Format: 1, Chunks: 3}, "") + require.NoError(t, err) + added, err := chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{0}}) + require.True(t, added) + require.NoError(t, err) + added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}}) + require.True(t, added) + require.NoError(t, err) + added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 2, Chunk: []byte{2}}) + require.True(t, added) + require.NoError(t, err) + + // The first two chunks are accepted, before the last one asks for 1 to be refetched + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 0, Chunk: []byte{0}, + }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil) + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 1, Chunk: []byte{1}, + }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil) + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 2, Chunk: []byte{2}, + }).Once().Return(&abci.ResponseApplySnapshotChunk{ + Result: tc.result, + RefetchChunks: []uint32{1}, + }, nil) + + // Since removing the chunk will cause Next() to block, we spawn a goroutine, then + // check the queue contents, and finally close the queue to end the goroutine. + // We don't really care about the result of applyChunks, since it has separate test. + go func() { + syncer.applyChunks(chunks) + }() + + time.Sleep(50 * time.Millisecond) + assert.True(t, chunks.Has(0)) + assert.False(t, chunks.Has(1)) + assert.True(t, chunks.Has(2)) + err = chunks.Close() + require.NoError(t, err) + }) + } +} + +func TestSyncer_applyChunks_RejectSenders(t *testing.T) { + // Banning chunks senders via ban_chunk_senders should work the same for all results + testcases := map[string]struct { + result abci.ResponseApplySnapshotChunk_Result + }{ + "accept": {abci.ResponseApplySnapshotChunk_accept}, + "abort": {abci.ResponseApplySnapshotChunk_abort}, + "retry": {abci.ResponseApplySnapshotChunk_retry}, + "retry_snapshot": {abci.ResponseApplySnapshotChunk_retry_snapshot}, + "reject_snapshot": {abci.ResponseApplySnapshotChunk_reject_snapshot}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + connQuery := &proxymocks.AppConnQuery{} + connSnapshot := &proxymocks.AppConnSnapshot{} + stateProvider := &mocks.StateProvider{} + stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil) + syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "") + + // Set up three peers across two snapshots, and ask for one of them to be banned. + // It should be banned from all snapshots. + peerA := simplePeer("a") + peerB := simplePeer("b") + peerC := simplePeer("c") + + s1 := &snapshot{Height: 1, Format: 1, Chunks: 3} + s2 := &snapshot{Height: 2, Format: 1, Chunks: 3} + syncer.AddSnapshot(peerA, s1) + syncer.AddSnapshot(peerA, s2) + syncer.AddSnapshot(peerB, s1) + syncer.AddSnapshot(peerB, s2) + syncer.AddSnapshot(peerC, s1) + syncer.AddSnapshot(peerC, s2) + + chunks, err := newChunkQueue(s1, "") + require.NoError(t, err) + added, err := chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{0}, Sender: peerA.ID()}) + require.True(t, added) + require.NoError(t, err) + added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}, Sender: peerB.ID()}) + require.True(t, added) + require.NoError(t, err) + added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 2, Chunk: []byte{2}, Sender: peerC.ID()}) + require.True(t, added) + require.NoError(t, err) + + // The first two chunks are accepted, before the last one asks for b sender to be rejected + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 0, Chunk: []byte{0}, Sender: "a", + }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil) + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 1, Chunk: []byte{1}, Sender: "b", + }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil) + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 2, Chunk: []byte{2}, Sender: "c", + }).Once().Return(&abci.ResponseApplySnapshotChunk{ + Result: tc.result, + RejectSenders: []string{string(peerB.ID())}, + }, nil) + + // On retry, the last chunk will be tried again, so we just accept it then. + if tc.result == abci.ResponseApplySnapshotChunk_retry { + connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{ + Index: 2, Chunk: []byte{2}, Sender: "c", + }).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil) + } + + // We don't really care about the result of applyChunks, since it has separate test. + // However, it will block on e.g. retry result, so we spawn a goroutine that will + // be shut down when the chunk queue closes. + go func() { + syncer.applyChunks(chunks) + }() + + time.Sleep(50 * time.Millisecond) + + s1peers := syncer.snapshots.GetPeers(s1) + assert.Len(t, s1peers, 2) + assert.EqualValues(t, "a", s1peers[0].ID()) + assert.EqualValues(t, "c", s1peers[1].ID()) + + syncer.snapshots.GetPeers(s1) + assert.Len(t, s1peers, 2) + assert.EqualValues(t, "a", s1peers[0].ID()) + assert.EqualValues(t, "c", s1peers[1].ID()) + + err = chunks.Close() + require.NoError(t, err) + }) + } +} + +func TestSyncer_verifyApp(t *testing.T) { + boom := errors.New("boom") + s := &snapshot{Height: 3, Format: 1, Chunks: 5, Hash: []byte{1, 2, 3}, trustedAppHash: []byte("app_hash")} + + testcases := map[string]struct { + response *abci.ResponseInfo + err error + expectErr error + }{ + "verified": {&abci.ResponseInfo{ + LastBlockHeight: 3, + LastBlockAppHash: []byte("app_hash"), + AppVersion: 9, + }, nil, nil}, + "invalid height": {&abci.ResponseInfo{ + LastBlockHeight: 5, + LastBlockAppHash: []byte("app_hash"), + AppVersion: 9, + }, nil, errVerifyFailed}, + "invalid hash": {&abci.ResponseInfo{ + LastBlockHeight: 3, + LastBlockAppHash: []byte("xxx"), + AppVersion: 9, + }, nil, errVerifyFailed}, + "error": {nil, boom, boom}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + connQuery := &proxymocks.AppConnQuery{} + connSnapshot := &proxymocks.AppConnSnapshot{} + stateProvider := &mocks.StateProvider{} + syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "") + + connQuery.On("InfoSync", proxy.RequestInfo).Return(tc.response, tc.err) + version, err := syncer.verifyApp(s) + unwrapped := errors.Unwrap(err) + if unwrapped != nil { + err = unwrapped + } + assert.Equal(t, tc.expectErr, err) + if err == nil { + assert.Equal(t, tc.response.AppVersion, version) + } + }) + } +} + +func toABCI(s *snapshot) *abci.Snapshot { + return &abci.Snapshot{ + Height: s.Height, + Format: s.Format, + Chunks: s.Chunks, + Hash: s.Hash, + Metadata: s.Metadata, + } +} diff --git a/store/store.go b/store/store.go index c971a9a15..2ef1f3c6e 100644 --- a/store/store.go +++ b/store/store.go @@ -333,6 +333,12 @@ func (bs *BlockStore) saveState() { bsJSON.Save(bs.db) } +// SaveSeenCommit saves a seen commit, used by e.g. the state sync reactor when bootstrapping node. +func (bs *BlockStore) SaveSeenCommit(height int64, seenCommit *types.Commit) error { + seenCommitBytes := cdc.MustMarshalBinaryBare(seenCommit) + return bs.db.Set(calcSeenCommitKey(height), seenCommitBytes) +} + //----------------------------------------------------------------------------- func calcBlockMetaKey(height int64) []byte {