Browse Source

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.
pull/4758/head
Erik Grinaker 4 years ago
committed by GitHub
parent
commit
511ab6717c
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 4145 additions and 106 deletions
  1. +1
    -0
      CHANGELOG_PENDING.md
  2. +18
    -4
      blockchain/v0/reactor.go
  3. +17
    -3
      blockchain/v1/reactor.go
  4. +1
    -1
      blockchain/v1/reactor_test.go
  5. +4
    -4
      blockchain/v2/io.go
  6. +1
    -1
      blockchain/v2/reactor.go
  7. +1
    -1
      blockchain/v2/reactor_test.go
  8. +81
    -12
      config/config.go
  9. +6
    -0
      config/config_test.go
  10. +24
    -0
      config/toml.go
  11. +4
    -4
      consensus/byzantine_test.go
  12. +9
    -0
      consensus/metrics.go
  13. +25
    -36
      consensus/reactor.go
  14. +1
    -1
      consensus/reactor_test.go
  15. +154
    -37
      node/node.go
  16. +331
    -0
      p2p/mocks/peer.go
  17. +2
    -0
      p2p/peer.go
  18. +2
    -2
      rpc/core/status.go
  19. +10
    -0
      state/store.go
  20. +321
    -0
      statesync/chunks.go
  21. +551
    -0
      statesync/chunks_test.go
  22. +129
    -0
      statesync/messages.go
  23. +106
    -0
      statesync/messages_test.go
  24. +82
    -0
      statesync/mocks/state_provider.go
  25. +261
    -0
      statesync/reactor.go
  26. +148
    -0
      statesync/reactor_test.go
  27. +263
    -0
      statesync/snapshots.go
  28. +326
    -0
      statesync/snapshots_test.go
  29. +179
    -0
      statesync/stateprovider.go
  30. +442
    -0
      statesync/syncer.go
  31. +639
    -0
      statesync/syncer_test.go
  32. +6
    -0
      store/store.go

+ 1
- 0
CHANGELOG_PENDING.md View File

@ -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)


+ 18
- 4
blockchain/v0/reactor.go View File

@ -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


+ 17
- 3
blockchain/v1/reactor.go View File

@ -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 {


+ 1
- 1
blockchain/v1/reactor_test.go View File

@ -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


+ 4
- 4
blockchain/v2/io.go View File

@ -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)
}
}


+ 1
- 1
blockchain/v2/reactor.go View File

@ -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()
}


+ 1
- 1
blockchain/v2/reactor_test.go View File

@ -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


+ 81
- 12
config/config.go View File

@ -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)
}
if err := cfg.Instrumentation.ValidateBasic(); err != nil {
return fmt.Errorf("error in [instrumentation] section: %w", err)
}
return errors.Wrap(
cfg.Instrumentation.ValidateBasic(),
"Error in [instrumentation] section",
)
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


+ 6
- 0
config/config_test.go View File

@ -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())


+ 24
- 0
config/toml.go View File

@ -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]


+ 4
- 4
consensus/byzantine_test.go View File

@ -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)
}
}


+ 9
- 0
consensus/metrics.go View File

@ -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(),
}
}

+ 25
- 36
consensus/reactor.go View File

@ -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 }
}
//-----------------------------------------------------------------------------


+ 1
- 1
consensus/reactor_test.go View File

@ -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
}


+ 154
- 37
node/node.go View File

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


+ 331
- 0
p2p/mocks/peer.go View File

@ -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
}

+ 2
- 0
p2p/peer.go View File

@ -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.


+ 2
- 2
rpc/core/status.go View File

@ -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(),


+ 10
- 0
state/store.go View File

@ -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


+ 321
- 0
statesync/chunks.go View File

@ -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
}

+ 551
- 0
statesync/chunks_test.go View File

@ -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)
}

+ 129
- 0
statesync/messages.go View File

@ -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
}

+ 106
- 0
statesync/messages_test.go View File

@ -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)
}
})
}
}

+ 82
- 0
statesync/mocks/state_provider.go View File

@ -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
}

+ 261
- 0
statesync/reactor.go View File

@ -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
}

+ 148
- 0
statesync/reactor_test.go View File

@ -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)
})
}
}

+ 263
- 0
statesync/snapshots.go View File

@ -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)
}

+ 326
- 0
statesync/snapshots_test.go View File

@ -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())
}

+ 179
- 0
statesync/stateprovider.go View File

@ -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
}

+ 442
- 0
statesync/syncer.go View File

@ -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
}

+ 639
- 0
statesync/syncer_test.go View File

@ -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,
}
}

+ 6
- 0
store/store.go View File

@ -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 {


Loading…
Cancel
Save