From cbe6dbe7a1fb720a243c6d6234e9e011a052aaa2 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Thu, 16 Feb 2017 17:56:45 -0500 Subject: [PATCH 01/14] handshake replay through consensus using mockApp --- consensus/replay_file.go | 2 +- consensus/state.go | 18 ++++- mempool/mempool.go | 8 +- node/node.go | 2 +- rpc/core/pipe.go | 2 +- state/execution.go | 153 +++++++++++++++++++-------------------- 6 files changed, 95 insertions(+), 90 deletions(-) diff --git a/consensus/replay_file.go b/consensus/replay_file.go index 1c2e9cb05..5d6747859 100644 --- a/consensus/replay_file.go +++ b/consensus/replay_file.go @@ -248,7 +248,7 @@ func newConsensusStateForReplay(config cfg.Config) *ConsensusState { state := sm.MakeGenesisStateFromFile(stateDB, config.GetString("genesis_file")) // Create proxyAppConn connection (consensus, mempool, query) - proxyApp := proxy.NewAppConns(config, proxy.DefaultClientCreator(config), sm.NewHandshaker(config, state, blockStore)) + proxyApp := proxy.NewAppConns(config, proxy.DefaultClientCreator(config), sm.NewHandshaker(config, state, blockStore, ReplayLastBlock)) _, err := proxyApp.Start() if err != nil { Exit(Fmt("Error starting proxy app conns: %v", err)) diff --git a/consensus/state.go b/consensus/state.go index 78e6c0eaa..c872233d2 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -15,7 +15,6 @@ import ( cfg "github.com/tendermint/go-config" "github.com/tendermint/go-wire" bc "github.com/tendermint/tendermint/blockchain" - mempl "github.com/tendermint/tendermint/mempool" "github.com/tendermint/tendermint/proxy" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" @@ -227,7 +226,7 @@ type ConsensusState struct { config cfg.Config proxyAppConn proxy.AppConnConsensus blockStore *bc.BlockStore - mempool *mempl.Mempool + mempool sm.Mempool privValidator PrivValidator // for signing votes @@ -255,7 +254,20 @@ type ConsensusState struct { done chan struct{} } -func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConnConsensus, blockStore *bc.BlockStore, mempool *mempl.Mempool) *ConsensusState { +func ReplayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnConsensus, blockStore sm.BlockStore) { + mempool := sm.MockMempool{} + cs := NewConsensusState(config, state, proxyApp, blockStore.(*bc.BlockStore), mempool) + + evsw := types.NewEventSwitch() + cs.SetEventSwitch(evsw) + newBlockCh := subscribeToEvent(evsw, "consensus-replay", types.EventStringNewBlock(), 0) + + cs.Start() + <-newBlockCh + cs.Stop() +} + +func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConnConsensus, blockStore *bc.BlockStore, mempool sm.Mempool) *ConsensusState { cs := &ConsensusState{ config: config, proxyAppConn: proxyAppConn, diff --git a/mempool/mempool.go b/mempool/mempool.go index 6fe4d7049..e960f520f 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -7,13 +7,13 @@ import ( "sync/atomic" "time" + abci "github.com/tendermint/abci/types" auto "github.com/tendermint/go-autofile" "github.com/tendermint/go-clist" . "github.com/tendermint/go-common" cfg "github.com/tendermint/go-config" "github.com/tendermint/tendermint/proxy" "github.com/tendermint/tendermint/types" - abci "github.com/tendermint/abci/types" ) /* @@ -249,7 +249,7 @@ func (mem *Mempool) resCbRecheck(req *abci.Request, res *abci.Response) { // Get the valid transactions remaining // If maxTxs is -1, there is no cap on returned transactions. -func (mem *Mempool) Reap(maxTxs int) []types.Tx { +func (mem *Mempool) Reap(maxTxs int) types.Txs { mem.proxyMtx.Lock() defer mem.proxyMtx.Unlock() @@ -263,7 +263,7 @@ func (mem *Mempool) Reap(maxTxs int) []types.Tx { } // maxTxs: -1 means uncapped, 0 means none -func (mem *Mempool) collectTxs(maxTxs int) []types.Tx { +func (mem *Mempool) collectTxs(maxTxs int) types.Txs { if maxTxs == 0 { return []types.Tx{} } else if maxTxs < 0 { @@ -281,7 +281,7 @@ func (mem *Mempool) collectTxs(maxTxs int) []types.Tx { // Mempool will discard these txs. // NOTE: this should be called *after* block is committed by consensus. // NOTE: unsafe; Lock/Unlock must be managed by caller -func (mem *Mempool) Update(height int, txs []types.Tx) { +func (mem *Mempool) Update(height int, txs types.Txs) { // TODO: check err ? mem.proxyAppConn.FlushSync() // To flush async resCb calls e.g. from CheckTx diff --git a/node/node.go b/node/node.go index d9eeb1f42..8ccd47c06 100644 --- a/node/node.go +++ b/node/node.go @@ -64,7 +64,7 @@ func NewNode(config cfg.Config, privValidator *types.PrivValidator, clientCreato state := sm.GetState(config, stateDB) // Create the proxyApp, which manages connections (consensus, mempool, query) - proxyApp := proxy.NewAppConns(config, clientCreator, sm.NewHandshaker(config, state, blockStore)) + proxyApp := proxy.NewAppConns(config, clientCreator, sm.NewHandshaker(config, state, blockStore, consensus.ReplayLastBlock)) if _, err := proxyApp.Start(); err != nil { cmn.Exit(cmn.Fmt("Error starting proxy app connections: %v", err)) } diff --git a/rpc/core/pipe.go b/rpc/core/pipe.go index 16bd8f20f..fb06c3ff1 100644 --- a/rpc/core/pipe.go +++ b/rpc/core/pipe.go @@ -31,7 +31,7 @@ type Consensus interface { type Mempool interface { Size() int CheckTx(types.Tx, func(*abci.Response)) error - Reap(int) []types.Tx + Reap(int) types.Txs Flush() } diff --git a/state/execution.go b/state/execution.go index 3dbf5e28c..e4b7cff48 100644 --- a/state/execution.go +++ b/state/execution.go @@ -1,7 +1,6 @@ package state import ( - "bytes" "errors" "github.com/ebuchman/fail-test" @@ -278,15 +277,20 @@ func (s *State) CommitStateUpdateMempool(proxyAppConn proxy.AppConnConsensus, bl type Mempool interface { Lock() Unlock() - Update(height int, txs []types.Tx) + + CheckTx(types.Tx, func(*abci.Response)) error + Reap(int) types.Txs + Update(height int, txs types.Txs) } type MockMempool struct { } -func (m MockMempool) Lock() {} -func (m MockMempool) Unlock() {} -func (m MockMempool) Update(height int, txs []types.Tx) {} +func (m MockMempool) Lock() {} +func (m MockMempool) Unlock() {} +func (m MockMempool) CheckTx(tx types.Tx, cb func(*abci.Response)) error { return nil } +func (m MockMempool) Reap(n int) types.Txs { return types.Txs{} } +func (m MockMempool) Update(height int, txs types.Txs) {} //---------------------------------------------------------------- // Handshake with app to sync to latest state of core by replaying blocks @@ -298,16 +302,19 @@ type BlockStore interface { LoadBlockMeta(height int) *types.BlockMeta } +type blockReplayFunc func(cfg.Config, *State, proxy.AppConnConsensus, BlockStore) + type Handshaker struct { - config cfg.Config - state *State - store BlockStore + config cfg.Config + state *State + store BlockStore + replayLastBlock blockReplayFunc nBlocks int // number of blocks applied to the state } -func NewHandshaker(config cfg.Config, state *State, store BlockStore) *Handshaker { - return &Handshaker{config, state, store, 0} +func NewHandshaker(config cfg.Config, state *State, store BlockStore, f blockReplayFunc) *Handshaker { + return &Handshaker{config, state, store, f, 0} } // TODO: retry the handshake/replay if it fails ? @@ -326,7 +333,7 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error { // TODO: check version // replay blocks up to the latest in the blockstore - err = h.ReplayBlocks(appHash, blockHeight, proxyApp.Consensus()) + err = h.ReplayBlocks(appHash, blockHeight, proxyApp) if err != nil { return errors.New(Fmt("Error on replay: %v", err)) } @@ -340,7 +347,7 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error { } // Replay all blocks after blockHeight and ensure the result matches the current state. -func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, appConnConsensus proxy.AppConnConsensus) error { +func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp proxy.AppConns) error { storeBlockHeight := h.store.Height() stateBlockHeight := h.state.LastBlockHeight @@ -353,85 +360,71 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, appConnCon return ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight} } else if storeBlockHeight == appBlockHeight { - // We ran Commit, but if we crashed before state.Save(), - // load the intermediate state and update the state.AppHash. - // NOTE: If ABCI allowed rollbacks, we could just replay the - // block even though it's been committed - stateAppHash := h.state.AppHash - lastBlockAppHash := h.store.LoadBlock(storeBlockHeight).AppHash - - if bytes.Equal(stateAppHash, appHash) { - // we're all synced up - log.Debug("ABCI RelpayBlocks: Already synced") - } else if bytes.Equal(stateAppHash, lastBlockAppHash) { - // we crashed after commit and before saving state, - // so load the intermediate state and update the hash - h.state.LoadIntermediate() - h.state.AppHash = appHash - log.Debug("ABCI RelpayBlocks: Loaded intermediate state and updated state.AppHash") - } else { - PanicSanity(Fmt("Unexpected state.AppHash: state.AppHash %X; app.AppHash %X, lastBlock.AppHash %X", stateAppHash, appHash, lastBlockAppHash)) + // We already ran Commit, so run through consensus with mock app + mockApp := newMockProxyApp(appHash) - } - return nil + h.replayLastBlock(h.config, h.state, mockApp, h.store) - } else if storeBlockHeight == appBlockHeight+1 && - storeBlockHeight == stateBlockHeight+1 { + } else if storeBlockHeight == appBlockHeight+1 { // We crashed after saving the block // but before Commit (both the state and app are behind), - // so just replay the block - - // check that the lastBlock.AppHash matches the state apphash - block := h.store.LoadBlock(storeBlockHeight) - if !bytes.Equal(block.Header.AppHash, appHash) { - return ErrLastStateMismatch{storeBlockHeight, block.Header.AppHash, appHash} - } - - blockMeta := h.store.LoadBlockMeta(storeBlockHeight) - - h.nBlocks += 1 - var eventCache types.Fireable // nil + // so run through consensus with the real app + h.replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) - // replay the latest block - return h.state.ApplyBlock(eventCache, appConnConsensus, block, blockMeta.BlockID.PartsHeader, MockMempool{}) - } else if storeBlockHeight != stateBlockHeight { - // unless we failed before committing or saving state (previous 2 case), - // the store and state should be at the same height! - PanicSanity(Fmt("Expected storeHeight (%d) and stateHeight (%d) to match.", storeBlockHeight, stateBlockHeight)) } else { // store is more than one ahead, // so app wants to replay many blocks - - // replay all blocks starting with appBlockHeight+1 - var eventCache types.Fireable // nil - - // TODO: use stateBlockHeight instead and let the consensus state - // do the replay - - var appHash []byte - for i := appBlockHeight + 1; i <= storeBlockHeight; i++ { - h.nBlocks += 1 - block := h.store.LoadBlock(i) - _, err := execBlockOnProxyApp(eventCache, appConnConsensus, block) - if err != nil { - log.Warn("Error executing block on proxy app", "height", i, "err", err) - return err + /* + + // replay all blocks starting with appBlockHeight+1 + var eventCache types.Fireable // nil + + // TODO: use stateBlockHeight instead and let the consensus state + // do the replay + + var appHash []byte + for i := appBlockHeight + 1; i <= storeBlockHeight; i++ { + h.nBlocks += 1 + block := h.store.LoadBlock(i) + _, err := execBlockOnProxyApp(eventCache, appConnConsensus, block) + if err != nil { + log.Warn("Error executing block on proxy app", "height", i, "err", err) + return err + } + // Commit block, get hash back + res := appConnConsensus.CommitSync() + if res.IsErr() { + log.Warn("Error in proxyAppConn.CommitSync", "error", res) + return res + } + if res.Log != "" { + log.Info("Commit.Log: " + res.Log) + } + appHash = res.Data } - // Commit block, get hash back - res := appConnConsensus.CommitSync() - if res.IsErr() { - log.Warn("Error in proxyAppConn.CommitSync", "error", res) - return res + if !bytes.Equal(h.state.AppHash, appHash) { + return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) } - if res.Log != "" { - log.Info("Commit.Log: " + res.Log) - } - appHash = res.Data - } - if !bytes.Equal(h.state.AppHash, appHash) { - return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) - } + */ return nil } return nil } + +//-------------------------------------------------------------------------------- + +func newMockProxyApp(appHash []byte) proxy.AppConnConsensus { + clientCreator := proxy.NewLocalClientCreator(&mockProxyApp{appHash: appHash}) + cli, _ := clientCreator.NewABCIClient() + return proxy.NewAppConnConsensus(cli) +} + +type mockProxyApp struct { + abci.BaseApplication + + appHash []byte +} + +func (mock *mockProxyApp) Commit() abci.Result { + return abci.NewResultOK(mock.appHash, "") +} From 6403b2f4681ab08eb1f34d225ed0232dd4dd6df4 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 17 Feb 2017 10:51:05 -0500 Subject: [PATCH 02/14] fixes for handshake replay through consensus --- .../handshake_test.go_ | 63 ++++++++++--------- consensus/state.go | 33 +++------- node/node.go | 10 ++- state/execution.go | 23 ++++--- 4 files changed, 67 insertions(+), 62 deletions(-) rename state/execution_test.go => consensus/handshake_test.go_ (78%) diff --git a/state/execution_test.go b/consensus/handshake_test.go_ similarity index 78% rename from state/execution_test.go rename to consensus/handshake_test.go_ index 452e72e1c..53d052845 100644 --- a/state/execution_test.go +++ b/consensus/handshake_test.go_ @@ -1,4 +1,4 @@ -package state +package consensus import ( "bytes" @@ -13,6 +13,7 @@ import ( "github.com/tendermint/go-crypto" dbm "github.com/tendermint/go-db" "github.com/tendermint/tendermint/proxy" + sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" ) @@ -20,32 +21,25 @@ var ( privKey = crypto.GenPrivKeyEd25519FromSecret([]byte("handshake_test")) chainID = "handshake_chain" nBlocks = 5 - mempool = MockMempool{} + mempool = sm.MockMempool{} testPartSize = 65536 ) -//--------------------------------------- -// Test block execution - -func TestExecBlock(t *testing.T) { - // TODO -} - //--------------------------------------- // Test handshake/replay // Sync from scratch -func TestHandshakeReplayAll(t *testing.T) { +func _TestHandshakeReplayAll(t *testing.T) { testHandshakeReplay(t, 0) } // Sync many, not from scratch -func TestHandshakeReplaySome(t *testing.T) { +func _TestHandshakeReplaySome(t *testing.T) { testHandshakeReplay(t, 1) } // Sync from lagging by one -func TestHandshakeReplayOne(t *testing.T) { +func _TestHandshakeReplayOne(t *testing.T) { testHandshakeReplay(t, nBlocks-1) } @@ -57,16 +51,18 @@ func TestHandshakeReplayNone(t *testing.T) { // Make some blocks. Start a fresh app and apply n blocks. Then restart the app and sync it up with the remaining blocks func testHandshakeReplay(t *testing.T, n int) { config := tendermint_test.ResetConfig("proxy_test_") + config.Set("chain_id", chainID) state, store := stateAndStore(config) clientCreator := proxy.NewLocalClientCreator(dummy.NewPersistentDummyApplication(path.Join(config.GetString("db_dir"), "1"))) clientCreator2 := proxy.NewLocalClientCreator(dummy.NewPersistentDummyApplication(path.Join(config.GetString("db_dir"), "2"))) - proxyApp := proxy.NewAppConns(config, clientCreator, NewHandshaker(config, state, store)) + proxyApp := proxy.NewAppConns(config, clientCreator, sm.NewHandshaker(config, state, store, ReplayLastBlock)) if _, err := proxyApp.Start(); err != nil { t.Fatalf("Error starting proxy app connections: %v", err) } - chain := makeBlockchain(t, proxyApp, state) + chain, commits := makeBlockchain(t, proxyApp, state) store.chain = chain // + store.commits = commits latestAppHash := state.AppHash proxyApp.Stop() @@ -88,7 +84,7 @@ func testHandshakeReplay(t *testing.T, n int) { } // now start it with the handshake - handshaker := NewHandshaker(config, state, store) + handshaker := sm.NewHandshaker(config, state, store, ReplayLastBlock) proxyApp = proxy.NewAppConns(config, clientCreator2, handshaker) if _, err := proxyApp.Start(); err != nil { t.Fatalf("Error starting proxy app connections: %v", err) @@ -105,8 +101,8 @@ func testHandshakeReplay(t *testing.T, n int) { t.Fatalf("Expected app hashes to match after handshake/replay. got %X, expected %X", res.LastBlockAppHash, latestAppHash) } - if handshaker.nBlocks != nBlocks-n { - t.Fatalf("Expected handshake to sync %d blocks, got %d", nBlocks-n, handshaker.nBlocks) + if handshaker.NBlocks() != nBlocks-n { + t.Fatalf("Expected handshake to sync %d blocks, got %d", nBlocks-n, handshaker.NBlocks()) } } @@ -139,7 +135,7 @@ func signCommit(height, round int, hash []byte, header types.PartSetHeader) *typ } // make a blockchain with one validator -func makeBlockchain(t *testing.T, proxyApp proxy.AppConns, state *State) (blockchain []*types.Block) { +func makeBlockchain(t *testing.T, proxyApp proxy.AppConns, state *sm.State) (blockchain []*types.Block, commits []*types.Commit) { prevHash := state.LastBlockID.Hash lastCommit := new(types.Commit) @@ -151,7 +147,6 @@ func makeBlockchain(t *testing.T, proxyApp proxy.AppConns, state *State) (blockc block, parts := types.MakeBlock(i, chainID, txsFunc(i), lastCommit, prevBlockID, valHash, state.AppHash, testPartSize) fmt.Println(i) - fmt.Println(prevBlockID) fmt.Println(block.LastBlockID) err := state.ApplyBlock(nil, proxyApp.Consensus(), block, block.MakePartSet(testPartSize).Header(), mempool) if err != nil { @@ -165,37 +160,40 @@ func makeBlockchain(t *testing.T, proxyApp proxy.AppConns, state *State) (blockc t.Fatal(err) } - blockchain = append(blockchain, block) prevHash = block.Hash() prevParts = parts.Header() lastCommit = voteSet.MakeCommit() prevBlockID = types.BlockID{prevHash, prevParts} + + blockchain = append(blockchain, block) + commits = append(commits, lastCommit) } - return blockchain + return blockchain, commits } // fresh state and mock store -func stateAndStore(config cfg.Config) (*State, *mockBlockStore) { +func stateAndStore(config cfg.Config) (*sm.State, *mockBlockStore) { stateDB := dbm.NewMemDB() - return MakeGenesisState(stateDB, &types.GenesisDoc{ + return sm.MakeGenesisState(stateDB, &types.GenesisDoc{ ChainID: chainID, Validators: []types.GenesisValidator{ types.GenesisValidator{privKey.PubKey(), 10000, "test"}, }, AppHash: nil, - }), NewMockBlockStore(config, nil) + }), NewMockBlockStore(config) } //---------------------------------- // mock block store type mockBlockStore struct { - config cfg.Config - chain []*types.Block + config cfg.Config + chain []*types.Block + commits []*types.Commit } -func NewMockBlockStore(config cfg.Config, chain []*types.Block) *mockBlockStore { - return &mockBlockStore{config, chain} +func NewMockBlockStore(config cfg.Config) *mockBlockStore { + return &mockBlockStore{config, nil, nil} } func (bs *mockBlockStore) Height() int { return len(bs.chain) } @@ -207,3 +205,12 @@ func (bs *mockBlockStore) LoadBlockMeta(height int) *types.BlockMeta { Header: block.Header, } } +func (bs *mockBlockStore) LoadBlockPart(height int, index int) *types.Part { return nil } +func (bs *mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) { +} +func (bs *mockBlockStore) LoadBlockCommit(height int) *types.Commit { + return bs.commits[height-1] +} +func (bs *mockBlockStore) LoadSeenCommit(height int) *types.Commit { + return bs.commits[height-1] +} diff --git a/consensus/state.go b/consensus/state.go index c872233d2..231d74bb3 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -4,7 +4,6 @@ import ( "bytes" "errors" "fmt" - "io" "reflect" "sync" "time" @@ -14,7 +13,6 @@ import ( . "github.com/tendermint/go-common" cfg "github.com/tendermint/go-config" "github.com/tendermint/go-wire" - bc "github.com/tendermint/tendermint/blockchain" "github.com/tendermint/tendermint/proxy" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" @@ -225,7 +223,7 @@ type ConsensusState struct { config cfg.Config proxyAppConn proxy.AppConnConsensus - blockStore *bc.BlockStore + blockStore sm.BlockStore mempool sm.Mempool privValidator PrivValidator // for signing votes @@ -256,18 +254,20 @@ type ConsensusState struct { func ReplayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnConsensus, blockStore sm.BlockStore) { mempool := sm.MockMempool{} - cs := NewConsensusState(config, state, proxyApp, blockStore.(*bc.BlockStore), mempool) + cs := NewConsensusState(config, state, proxyApp, blockStore, mempool) evsw := types.NewEventSwitch() + evsw.Start() cs.SetEventSwitch(evsw) - newBlockCh := subscribeToEvent(evsw, "consensus-replay", types.EventStringNewBlock(), 0) + newBlockCh := subscribeToEvent(evsw, "consensus-replay", types.EventStringNewBlock(), 1) + // run through the WAL, commit new block, stop cs.Start() <-newBlockCh cs.Stop() } -func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConnConsensus, blockStore *bc.BlockStore, mempool sm.Mempool) *ConsensusState { +func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConnConsensus, blockStore sm.BlockStore, mempool sm.Mempool) *ConsensusState { cs := &ConsensusState{ config: config, proxyAppConn: proxyAppConn, @@ -366,23 +366,6 @@ func (cs *ConsensusState) OnStart() error { return err } - // If the latest block was applied in the abci handshake, - // we may not have written the current height to the wal, - // so check here and write it if not found. - // TODO: remove this and run the handhsake/replay - // through the consensus state with a mock app - gr, found, err := cs.wal.group.Search("#HEIGHT: ", makeHeightSearchFunc(cs.Height)) - if (err == io.EOF || !found) && cs.Step == RoundStepNewHeight { - log.Warn("Height not found in wal. Writing new height", "height", cs.Height) - rs := cs.RoundStateEvent() - cs.wal.Save(rs) - } else if err != nil { - return err - } - if gr != nil { - gr.Close() - } - // we need the timeoutRoutine for replay so // we don't block on the tick chan. // NOTE: we will get a build up of garbage go routines @@ -581,7 +564,6 @@ func (cs *ConsensusState) updateToState(state *sm.State) { // Reset fields based on state. validators := state.Validators - height := state.LastBlockHeight + 1 // Next desired block height lastPrecommits := (*types.VoteSet)(nil) if cs.CommitRound > -1 && cs.Votes != nil { if !cs.Votes.Precommits(cs.CommitRound).HasTwoThirdsMajority() { @@ -590,6 +572,9 @@ func (cs *ConsensusState) updateToState(state *sm.State) { lastPrecommits = cs.Votes.Precommits(cs.CommitRound) } + // Next desired block height + height := state.LastBlockHeight + 1 + // RoundState fields cs.updateHeight(height) cs.updateRoundStep(0, RoundStepNewHeight) diff --git a/node/node.go b/node/node.go index 8ccd47c06..755658b7b 100644 --- a/node/node.go +++ b/node/node.go @@ -63,15 +63,19 @@ func NewNode(config cfg.Config, privValidator *types.PrivValidator, clientCreato stateDB := dbm.NewDB("state", config.GetString("db_backend"), config.GetString("db_dir")) state := sm.GetState(config, stateDB) + // add the chainid and number of validators to the global config + config.Set("chain_id", state.ChainID) + config.Set("num_vals", state.Validators.Size()) + // Create the proxyApp, which manages connections (consensus, mempool, query) + // and sync tendermint and the app by replaying any necessary blocks proxyApp := proxy.NewAppConns(config, clientCreator, sm.NewHandshaker(config, state, blockStore, consensus.ReplayLastBlock)) if _, err := proxyApp.Start(); err != nil { cmn.Exit(cmn.Fmt("Error starting proxy app connections: %v", err)) } - // add the chainid and number of validators to the global config - config.Set("chain_id", state.ChainID) - config.Set("num_vals", state.Validators.Size()) + // reload the state (it may have been updated by the handshake) + state = sm.LoadState(stateDB) // Generate node PrivKey privKey := crypto.GenPrivKeyEd25519() diff --git a/state/execution.go b/state/execution.go index e4b7cff48..c767352fe 100644 --- a/state/execution.go +++ b/state/execution.go @@ -298,8 +298,15 @@ func (m MockMempool) Update(height int, txs types.Txs) {} // TODO: Should we move blockchain/store.go to its own package? type BlockStore interface { Height() int - LoadBlock(height int) *types.Block + LoadBlockMeta(height int) *types.BlockMeta + LoadBlock(height int) *types.Block + LoadBlockPart(height int, index int) *types.Part + + SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) + + LoadBlockCommit(height int) *types.Commit + LoadSeenCommit(height int) *types.Commit } type blockReplayFunc func(cfg.Config, *State, proxy.AppConnConsensus, BlockStore) @@ -317,6 +324,10 @@ func NewHandshaker(config cfg.Config, state *State, store BlockStore, f blockRep return &Handshaker{config, state, store, f, 0} } +func (h *Handshaker) NBlocks() int { + return h.nBlocks +} + // TODO: retry the handshake/replay if it fails ? func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error { // handshake is done via info request on the query conn @@ -338,9 +349,6 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error { return errors.New(Fmt("Error on replay: %v", err)) } - // Save the state - h.state.Save() - // TODO: (on restart) replay mempool return nil @@ -359,16 +367,17 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p // if the app is ahead, there's nothing we can do return ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight} - } else if storeBlockHeight == appBlockHeight { - // We already ran Commit, so run through consensus with mock app + } else if storeBlockHeight == appBlockHeight && storeBlockHeight == stateBlockHeight+1 { + // We already ran Commit, but didn't save the state, so run through consensus with mock app mockApp := newMockProxyApp(appHash) - + log.Info("Replay last block using mock app") h.replayLastBlock(h.config, h.state, mockApp, h.store) } else if storeBlockHeight == appBlockHeight+1 { // We crashed after saving the block // but before Commit (both the state and app are behind), // so run through consensus with the real app + log.Info("Replay last block using real app") h.replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) } else { From edc5e272dbacff29493b98860ff470c29f53f09a Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 17 Feb 2017 10:57:09 -0500 Subject: [PATCH 03/14] consensus: nice error msg if ApplyBlock fails --- consensus/state.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/consensus/state.go b/consensus/state.go index 231d74bb3..30ca8e9fa 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -1232,7 +1232,8 @@ func (cs *ConsensusState) finalizeCommit(height int) { seenCommit := precommits.MakeCommit() cs.blockStore.SaveBlock(block, blockParts, seenCommit) } else { - log.Warn("Why are we finalizeCommitting a block height we already have?", "height", block.Height) + // Happens during replay if we already saved the block but didn't commit + log.Notice("Calling finalizeCommit on already stored block", "height", block.Height) } fail.Fail() // XXX @@ -1247,7 +1248,8 @@ func (cs *ConsensusState) finalizeCommit(height int) { // NOTE: the block.AppHash wont reflect these txs until the next block err := stateCopy.ApplyBlock(eventCache, cs.proxyAppConn, block, blockParts.Header(), cs.mempool) if err != nil { - // TODO! + log.Warn("Error on ApplyBlock. Did the application crash? Please restart tendermint", "error", err) + return } fail.Fail() // XXX From 3c5adebcd3f5817304baac652cb80a1f76ede7ac Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 17 Feb 2017 11:32:56 -0500 Subject: [PATCH 04/14] applyBlock to simplify replay of many blocks. still wip --- state/execution.go | 71 +++++++++++++++++++++++++--------------------- 1 file changed, 39 insertions(+), 32 deletions(-) diff --git a/state/execution.go b/state/execution.go index c767352fe..88b80e485 100644 --- a/state/execution.go +++ b/state/execution.go @@ -272,6 +272,27 @@ func (s *State) CommitStateUpdateMempool(proxyAppConn proxy.AppConnConsensus, bl return nil } +// apply a nd commit a block, but with out all the state validation +// returns the application root hash (result of abci.Commit) +func applyBlock(appConnConsensus proxy.AppConnConsensus, block *types.Block) ([]byte, error) { + var eventCache types.Fireable // nil + _, err := execBlockOnProxyApp(eventCache, appConnConsensus, block) + if err != nil { + log.Warn("Error executing block on proxy app", "height", i, "err", err) + return nil, err + } + // Commit block, get hash back + res := appConnConsensus.CommitSync() + if res.IsErr() { + log.Warn("Error in proxyAppConn.CommitSync", "error", res) + return nil, res + } + if res.Log != "" { + log.Info("Commit.Log: " + res.Log) + } + return res.Data, nil +} + // Updates to the mempool need to be synchronized with committing a block // so apps can reset their transient state on Commit type Mempool interface { @@ -382,39 +403,25 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p } else { // store is more than one ahead, - // so app wants to replay many blocks - /* - - // replay all blocks starting with appBlockHeight+1 - var eventCache types.Fireable // nil - - // TODO: use stateBlockHeight instead and let the consensus state - // do the replay - - var appHash []byte - for i := appBlockHeight + 1; i <= storeBlockHeight; i++ { - h.nBlocks += 1 - block := h.store.LoadBlock(i) - _, err := execBlockOnProxyApp(eventCache, appConnConsensus, block) - if err != nil { - log.Warn("Error executing block on proxy app", "height", i, "err", err) - return err - } - // Commit block, get hash back - res := appConnConsensus.CommitSync() - if res.IsErr() { - log.Warn("Error in proxyAppConn.CommitSync", "error", res) - return res - } - if res.Log != "" { - log.Info("Commit.Log: " + res.Log) - } - appHash = res.Data - } - if !bytes.Equal(h.state.AppHash, appHash) { - return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) + // so app wants to replay many blocks. + // replay all blocks from appBlockHeight+1 to storeBlockHeight-1. + // Replay the final block through consensus + + var appHash []byte + var err error + for i := appBlockHeight + 1; i <= storeBlockHeight-1; i++ { + h.nBlocks += 1 + block := h.store.LoadBlock(i) + appHash, err = applyBlock(proxyApp.Consensus(), block) + if err != nil { + return err } - */ + } + + h.replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) + if !bytes.Equal(h.state.AppHash, appHash) { + return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) + } return nil } return nil From 0bec99fbd417df47e0a307ca62fb78477a703e12 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 17 Feb 2017 19:12:05 -0500 Subject: [PATCH 05/14] consensus: handshake replay test using wal --- cmd/tendermint/reset_priv_validator.go | 6 +- config/tendermint/config.go | 2 +- config/tendermint_test/config.go | 2 +- consensus/common_test.go | 8 +- consensus/handshake_test.go_ | 216 -------------- consensus/replay_test.go | 377 +++++++++++++++++++++++-- consensus/state.go | 11 +- consensus/test_data/build.sh | 51 +++- consensus/test_data/many_blocks.cswal | 65 +++++ consensus/wal.go | 4 +- state/execution.go | 14 +- 11 files changed, 495 insertions(+), 261 deletions(-) delete mode 100644 consensus/handshake_test.go_ create mode 100644 consensus/test_data/many_blocks.cswal diff --git a/cmd/tendermint/reset_priv_validator.go b/cmd/tendermint/reset_priv_validator.go index 9ecbaa90b..5bf3ba69b 100644 --- a/cmd/tendermint/reset_priv_validator.go +++ b/cmd/tendermint/reset_priv_validator.go @@ -6,15 +6,15 @@ import ( "github.com/tendermint/tendermint/types" ) -// NOTE: this is totally unsafe. +// XXX: this is totally unsafe. // it's only suitable for testnets. func reset_all() { reset_priv_validator() os.RemoveAll(config.GetString("db_dir")) - os.RemoveAll(config.GetString("cs_wal_dir")) + os.Remove(config.GetString("cs_wal_file")) } -// NOTE: this is totally unsafe. +// XXX: this is totally unsafe. // it's only suitable for testnets. func reset_priv_validator() { // Get PrivValidator diff --git a/config/tendermint/config.go b/config/tendermint/config.go index ab4bf00b1..c210d6e01 100644 --- a/config/tendermint/config.go +++ b/config/tendermint/config.go @@ -72,7 +72,7 @@ func GetConfig(rootDir string) cfg.Config { mapConfig.SetDefault("grpc_laddr", "") mapConfig.SetDefault("prof_laddr", "") mapConfig.SetDefault("revision_file", rootDir+"/revision") - mapConfig.SetDefault("cs_wal_dir", rootDir+"/data/cs.wal") + mapConfig.SetDefault("cs_wal_file", rootDir+"/data/cs.wal/wal") mapConfig.SetDefault("cs_wal_light", false) mapConfig.SetDefault("filter_peers", false) diff --git a/config/tendermint_test/config.go b/config/tendermint_test/config.go index 930cf27e5..55e3adb4e 100644 --- a/config/tendermint_test/config.go +++ b/config/tendermint_test/config.go @@ -86,7 +86,7 @@ func ResetConfig(localPath string) cfg.Config { mapConfig.SetDefault("grpc_laddr", "tcp://0.0.0.0:36658") mapConfig.SetDefault("prof_laddr", "") mapConfig.SetDefault("revision_file", rootDir+"/revision") - mapConfig.SetDefault("cs_wal_dir", rootDir+"/data/cs.wal") + mapConfig.SetDefault("cs_wal_file", rootDir+"/data/cs.wal/wal") mapConfig.SetDefault("cs_wal_light", false) mapConfig.SetDefault("filter_peers", false) diff --git a/consensus/common_test.go b/consensus/common_test.go index c345fe663..334c66dc6 100644 --- a/consensus/common_test.go +++ b/consensus/common_test.go @@ -11,6 +11,8 @@ import ( "testing" "time" + abcicli "github.com/tendermint/abci/client" + abci "github.com/tendermint/abci/types" . "github.com/tendermint/go-common" cfg "github.com/tendermint/go-config" dbm "github.com/tendermint/go-db" @@ -20,8 +22,6 @@ import ( mempl "github.com/tendermint/tendermint/mempool" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" - abcicli "github.com/tendermint/abci/client" - abci "github.com/tendermint/abci/types" "github.com/tendermint/abci/example/counter" "github.com/tendermint/abci/example/dummy" @@ -320,7 +320,7 @@ func randConsensusNet(nValidators int, testName string, tickerFunc func() Timeou state := sm.MakeGenesisState(db, genDoc) state.Save() thisConfig := tendermint_test.ResetConfig(Fmt("%s_%d", testName, i)) - ensureDir(thisConfig.GetString("cs_wal_dir"), 0700) // dir for wal + ensureDir(path.Dir(thisConfig.GetString("cs_wal_file")), 0700) // dir for wal css[i] = newConsensusStateWithConfig(thisConfig, state, privVals[i], appFunc()) css[i].SetTimeoutTicker(tickerFunc()) } @@ -336,7 +336,7 @@ func randConsensusNetWithPeers(nValidators, nPeers int, testName string, tickerF state := sm.MakeGenesisState(db, genDoc) state.Save() thisConfig := tendermint_test.ResetConfig(Fmt("%s_%d", testName, i)) - ensureDir(thisConfig.GetString("cs_wal_dir"), 0700) // dir for wal + ensureDir(path.Dir(thisConfig.GetString("cs_wal_file")), 0700) // dir for wal var privVal *types.PrivValidator if i < nValidators { privVal = privVals[i] diff --git a/consensus/handshake_test.go_ b/consensus/handshake_test.go_ deleted file mode 100644 index 53d052845..000000000 --- a/consensus/handshake_test.go_ +++ /dev/null @@ -1,216 +0,0 @@ -package consensus - -import ( - "bytes" - "fmt" - "path" - "testing" - - "github.com/tendermint/tendermint/config/tendermint_test" - // . "github.com/tendermint/go-common" - "github.com/tendermint/abci/example/dummy" - cfg "github.com/tendermint/go-config" - "github.com/tendermint/go-crypto" - dbm "github.com/tendermint/go-db" - "github.com/tendermint/tendermint/proxy" - sm "github.com/tendermint/tendermint/state" - "github.com/tendermint/tendermint/types" -) - -var ( - privKey = crypto.GenPrivKeyEd25519FromSecret([]byte("handshake_test")) - chainID = "handshake_chain" - nBlocks = 5 - mempool = sm.MockMempool{} - testPartSize = 65536 -) - -//--------------------------------------- -// Test handshake/replay - -// Sync from scratch -func _TestHandshakeReplayAll(t *testing.T) { - testHandshakeReplay(t, 0) -} - -// Sync many, not from scratch -func _TestHandshakeReplaySome(t *testing.T) { - testHandshakeReplay(t, 1) -} - -// Sync from lagging by one -func _TestHandshakeReplayOne(t *testing.T) { - testHandshakeReplay(t, nBlocks-1) -} - -// Sync from caught up -func TestHandshakeReplayNone(t *testing.T) { - testHandshakeReplay(t, nBlocks) -} - -// Make some blocks. Start a fresh app and apply n blocks. Then restart the app and sync it up with the remaining blocks -func testHandshakeReplay(t *testing.T, n int) { - config := tendermint_test.ResetConfig("proxy_test_") - config.Set("chain_id", chainID) - - state, store := stateAndStore(config) - clientCreator := proxy.NewLocalClientCreator(dummy.NewPersistentDummyApplication(path.Join(config.GetString("db_dir"), "1"))) - clientCreator2 := proxy.NewLocalClientCreator(dummy.NewPersistentDummyApplication(path.Join(config.GetString("db_dir"), "2"))) - proxyApp := proxy.NewAppConns(config, clientCreator, sm.NewHandshaker(config, state, store, ReplayLastBlock)) - if _, err := proxyApp.Start(); err != nil { - t.Fatalf("Error starting proxy app connections: %v", err) - } - chain, commits := makeBlockchain(t, proxyApp, state) - store.chain = chain // - store.commits = commits - latestAppHash := state.AppHash - proxyApp.Stop() - - if n > 0 { - // start a new app without handshake, play n blocks - proxyApp = proxy.NewAppConns(config, clientCreator2, nil) - if _, err := proxyApp.Start(); err != nil { - t.Fatalf("Error starting proxy app connections: %v", err) - } - state2, _ := stateAndStore(config) - for i := 0; i < n; i++ { - block := chain[i] - err := state2.ApplyBlock(nil, proxyApp.Consensus(), block, block.MakePartSet(testPartSize).Header(), mempool) - if err != nil { - t.Fatal(err) - } - } - proxyApp.Stop() - } - - // now start it with the handshake - handshaker := sm.NewHandshaker(config, state, store, ReplayLastBlock) - proxyApp = proxy.NewAppConns(config, clientCreator2, handshaker) - if _, err := proxyApp.Start(); err != nil { - t.Fatalf("Error starting proxy app connections: %v", err) - } - - // get the latest app hash from the app - res, err := proxyApp.Query().InfoSync() - if err != nil { - t.Fatal(err) - } - - // the app hash should be synced up - if !bytes.Equal(latestAppHash, res.LastBlockAppHash) { - t.Fatalf("Expected app hashes to match after handshake/replay. got %X, expected %X", res.LastBlockAppHash, latestAppHash) - } - - if handshaker.NBlocks() != nBlocks-n { - t.Fatalf("Expected handshake to sync %d blocks, got %d", nBlocks-n, handshaker.NBlocks()) - } - -} - -//-------------------------- -// utils for making blocks - -// make some bogus txs -func txsFunc(blockNum int) (txs []types.Tx) { - for i := 0; i < 10; i++ { - txs = append(txs, types.Tx([]byte{byte(blockNum), byte(i)})) - } - return txs -} - -// sign a commit vote -func signCommit(height, round int, hash []byte, header types.PartSetHeader) *types.Vote { - vote := &types.Vote{ - ValidatorIndex: 0, - ValidatorAddress: privKey.PubKey().Address(), - Height: height, - Round: round, - Type: types.VoteTypePrecommit, - BlockID: types.BlockID{hash, header}, - } - - sig := privKey.Sign(types.SignBytes(chainID, vote)) - vote.Signature = sig - return vote -} - -// make a blockchain with one validator -func makeBlockchain(t *testing.T, proxyApp proxy.AppConns, state *sm.State) (blockchain []*types.Block, commits []*types.Commit) { - - prevHash := state.LastBlockID.Hash - lastCommit := new(types.Commit) - prevParts := types.PartSetHeader{} - valHash := state.Validators.Hash() - prevBlockID := types.BlockID{prevHash, prevParts} - - for i := 1; i < nBlocks+1; i++ { - block, parts := types.MakeBlock(i, chainID, txsFunc(i), lastCommit, - prevBlockID, valHash, state.AppHash, testPartSize) - fmt.Println(i) - fmt.Println(block.LastBlockID) - err := state.ApplyBlock(nil, proxyApp.Consensus(), block, block.MakePartSet(testPartSize).Header(), mempool) - if err != nil { - t.Fatal(i, err) - } - - voteSet := types.NewVoteSet(chainID, i, 0, types.VoteTypePrecommit, state.Validators) - vote := signCommit(i, 0, block.Hash(), parts.Header()) - _, err = voteSet.AddVote(vote) - if err != nil { - t.Fatal(err) - } - - prevHash = block.Hash() - prevParts = parts.Header() - lastCommit = voteSet.MakeCommit() - prevBlockID = types.BlockID{prevHash, prevParts} - - blockchain = append(blockchain, block) - commits = append(commits, lastCommit) - } - return blockchain, commits -} - -// fresh state and mock store -func stateAndStore(config cfg.Config) (*sm.State, *mockBlockStore) { - stateDB := dbm.NewMemDB() - return sm.MakeGenesisState(stateDB, &types.GenesisDoc{ - ChainID: chainID, - Validators: []types.GenesisValidator{ - types.GenesisValidator{privKey.PubKey(), 10000, "test"}, - }, - AppHash: nil, - }), NewMockBlockStore(config) -} - -//---------------------------------- -// mock block store - -type mockBlockStore struct { - config cfg.Config - chain []*types.Block - commits []*types.Commit -} - -func NewMockBlockStore(config cfg.Config) *mockBlockStore { - return &mockBlockStore{config, nil, nil} -} - -func (bs *mockBlockStore) Height() int { return len(bs.chain) } -func (bs *mockBlockStore) LoadBlock(height int) *types.Block { return bs.chain[height-1] } -func (bs *mockBlockStore) LoadBlockMeta(height int) *types.BlockMeta { - block := bs.chain[height-1] - return &types.BlockMeta{ - BlockID: types.BlockID{block.Hash(), block.MakePartSet(bs.config.GetInt("block_part_size")).Header()}, - Header: block.Header, - } -} -func (bs *mockBlockStore) LoadBlockPart(height int, index int) *types.Part { return nil } -func (bs *mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) { -} -func (bs *mockBlockStore) LoadBlockCommit(height int) *types.Commit { - return bs.commits[height-1] -} -func (bs *mockBlockStore) LoadSeenCommit(height int) *types.Commit { - return bs.commits[height-1] -} diff --git a/consensus/replay_test.go b/consensus/replay_test.go index d60fa9f2a..a9123a8e4 100644 --- a/consensus/replay_test.go +++ b/consensus/replay_test.go @@ -1,7 +1,10 @@ package consensus import ( + "bytes" + "errors" "fmt" + "io" "io/ioutil" "os" "path" @@ -11,8 +14,14 @@ import ( "github.com/tendermint/tendermint/config/tendermint_test" + "github.com/tendermint/abci/example/dummy" . "github.com/tendermint/go-common" + cfg "github.com/tendermint/go-config" + "github.com/tendermint/go-crypto" + dbm "github.com/tendermint/go-db" "github.com/tendermint/go-wire" + "github.com/tendermint/tendermint/proxy" + sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" ) @@ -20,14 +29,23 @@ func init() { config = tendermint_test.ResetConfig("consensus_replay_test") } -// TODO: these tests ensure we can always recover from any state of the wal, -// assuming it comes with a correct related state for the priv_validator.json. -// It would be better to verify explicitly which states we can recover from without the wal -// and which ones we need the wal for - then we'd also be able to only flush the -// wal writer when we need to, instead of with every message. +// These tests ensure we can always recover from failure at any part of the consensus process. +// There are two general failure scenarios: failure during consensus, and failure while applying the block. +// Only the latter interacts with the app and store, +// but the former has to deal with restrictions on re-use of priv_validator keys. +// The `WAL Tests` are for failures during the consensus; +// the `Handshake Tests` are for failures in applying the block. +// With the help of the WAL, we can recover from it all! var data_dir = path.Join(GoPath, "src/github.com/tendermint/tendermint/consensus", "test_data") +//------------------------------------------------------------------------------------------ +// WAL Tests + +// TODO: It would be better to verify explicitly which states we can recover from without the wal +// and which ones we need the wal for - then we'd also be able to only flush the +// wal writer when we need to, instead of with every message. + // the priv validator changes step at these lines for a block with 1 val and 1 part var baseStepChanges = []int{3, 6, 8} @@ -85,18 +103,19 @@ func readWAL(p string) string { func writeWAL(walMsgs string) string { tempDir := os.TempDir() - walDir := tempDir + "/wal" + RandStr(12) + walDir := path.Join(tempDir, "/wal"+RandStr(12)) + walFile := path.Join(walDir, "wal") // Create WAL directory err := EnsureDir(walDir, 0700) if err != nil { panic(err) } // Write the needed WAL to file - err = WriteFile(walDir+"/wal", []byte(walMsgs), 0600) + err = WriteFile(walFile, []byte(walMsgs), 0600) if err != nil { panic(err) } - return walDir + return walFile } func waitForBlock(newBlockCh chan interface{}, thisCase *testCase, i int) { @@ -108,10 +127,10 @@ func waitForBlock(newBlockCh chan interface{}, thisCase *testCase, i int) { } } -func runReplayTest(t *testing.T, cs *ConsensusState, walDir string, newBlockCh chan interface{}, +func runReplayTest(t *testing.T, cs *ConsensusState, walFile string, newBlockCh chan interface{}, thisCase *testCase, i int) { - cs.config.Set("cs_wal_dir", walDir) + cs.config.Set("cs_wal_file", walFile) cs.Start() // Wait to make a new block. // This is just a signal that we haven't halted; its not something contained in the WAL itself. @@ -148,7 +167,7 @@ func setupReplayTest(thisCase *testCase, nLines int, crashAfter bool) (*Consensu lastMsg := split[nLines] // we write those lines up to (not including) one with the signature - walDir := writeWAL(strings.Join(split[:nLines], "\n") + "\n") + walFile := writeWAL(strings.Join(split[:nLines], "\n") + "\n") cs := fixedConsensusStateDummy() @@ -160,7 +179,7 @@ func setupReplayTest(thisCase *testCase, nLines int, crashAfter bool) (*Consensu newBlockCh := subscribeToEvent(cs.evsw, "tester", types.EventStringNewBlock(), 1) - return cs, newBlockCh, lastMsg, walDir + return cs, newBlockCh, lastMsg, walFile } func readTimedWALMessage(t *testing.T, walMsg string) TimedWALMessage { @@ -177,12 +196,12 @@ func readTimedWALMessage(t *testing.T, walMsg string) TimedWALMessage { // Test the log at every iteration, and set the privVal last step // as if the log was written after signing, before the crash -func TestReplayCrashAfterWrite(t *testing.T) { +func TestWALCrashAfterWrite(t *testing.T) { for _, thisCase := range testCases { split := strings.Split(thisCase.log, "\n") for i := 0; i < len(split)-1; i++ { - cs, newBlockCh, _, walDir := setupReplayTest(thisCase, i+1, true) - runReplayTest(t, cs, walDir, newBlockCh, thisCase, i+1) + cs, newBlockCh, _, walFile := setupReplayTest(thisCase, i+1, true) + runReplayTest(t, cs, walFile, newBlockCh, thisCase, i+1) } } } @@ -191,27 +210,27 @@ func TestReplayCrashAfterWrite(t *testing.T) { // Test the log as if we crashed after signing but before writing. // This relies on privValidator.LastSignature being set -func TestReplayCrashBeforeWritePropose(t *testing.T) { +func TestWALCrashBeforeWritePropose(t *testing.T) { for _, thisCase := range testCases { lineNum := thisCase.proposeLine // setup replay test where last message is a proposal - cs, newBlockCh, proposalMsg, walDir := setupReplayTest(thisCase, lineNum, false) + cs, newBlockCh, proposalMsg, walFile := setupReplayTest(thisCase, lineNum, false) msg := readTimedWALMessage(t, proposalMsg) proposal := msg.Msg.(msgInfo).Msg.(*ProposalMessage) // Set LastSig toPV(cs.privValidator).LastSignBytes = types.SignBytes(cs.state.ChainID, proposal.Proposal) toPV(cs.privValidator).LastSignature = proposal.Proposal.Signature - runReplayTest(t, cs, walDir, newBlockCh, thisCase, lineNum) + runReplayTest(t, cs, walFile, newBlockCh, thisCase, lineNum) } } -func TestReplayCrashBeforeWritePrevote(t *testing.T) { +func TestWALCrashBeforeWritePrevote(t *testing.T) { for _, thisCase := range testCases { testReplayCrashBeforeWriteVote(t, thisCase, thisCase.prevoteLine, types.EventStringCompleteProposal()) } } -func TestReplayCrashBeforeWritePrecommit(t *testing.T) { +func TestWALCrashBeforeWritePrecommit(t *testing.T) { for _, thisCase := range testCases { testReplayCrashBeforeWriteVote(t, thisCase, thisCase.precommitLine, types.EventStringPolka()) } @@ -219,7 +238,7 @@ func TestReplayCrashBeforeWritePrecommit(t *testing.T) { func testReplayCrashBeforeWriteVote(t *testing.T, thisCase *testCase, lineNum int, eventString string) { // setup replay test where last message is a vote - cs, newBlockCh, voteMsg, walDir := setupReplayTest(thisCase, lineNum, false) + cs, newBlockCh, voteMsg, walFile := setupReplayTest(thisCase, lineNum, false) types.AddListenerForEvent(cs.evsw, "tester", eventString, func(data types.TMEventData) { msg := readTimedWALMessage(t, voteMsg) vote := msg.Msg.(msgInfo).Msg.(*VoteMessage) @@ -227,5 +246,319 @@ func testReplayCrashBeforeWriteVote(t *testing.T, thisCase *testCase, lineNum in toPV(cs.privValidator).LastSignBytes = types.SignBytes(cs.state.ChainID, vote.Vote) toPV(cs.privValidator).LastSignature = vote.Vote.Signature }) - runReplayTest(t, cs, walDir, newBlockCh, thisCase, lineNum) + runReplayTest(t, cs, walFile, newBlockCh, thisCase, lineNum) +} + +//------------------------------------------------------------------------------------------ +// Handshake Tests + +var ( + NUM_BLOCKS = 6 // number of blocks in the test_data/many_blocks.cswal + mempool = sm.MockMempool{} + + testPartSize int +) + +//--------------------------------------- +// Test handshake/replay + +// Sync from scratch +func TestHandshakeReplayAll(t *testing.T) { + testHandshakeReplay(t, 0) +} + +// Sync many, not from scratch +func TestHandshakeReplaySome(t *testing.T) { + testHandshakeReplay(t, 1) +} + +// Sync from lagging by one +func TestHandshakeReplayOne(t *testing.T) { + testHandshakeReplay(t, NUM_BLOCKS-1) +} + +// Sync from caught up +func TestHandshakeReplayNone(t *testing.T) { + testHandshakeReplay(t, NUM_BLOCKS) +} + +// Make some blocks. Start a fresh app and apply nBlocks blocks. Then restart the app and sync it up with the remaining blocks +func testHandshakeReplay(t *testing.T, nBlocks int) { + config := tendermint_test.ResetConfig("proxy_test_") + walFile := path.Join(data_dir, "many_blocks.cswal") + config.Set("cs_wal_file", walFile) + privVal := types.LoadPrivValidator(config.GetString("priv_validator_file")) + testPartSize = config.GetInt("block_part_size") + + wal, err := NewWAL(walFile, false) + if err != nil { + t.Fatal(err) + } + chain, commits, err := makeBlockchainFromWAL(wal) + if err != nil { + t.Fatalf(err.Error()) + } + + state, store := stateAndStore(config, privVal.PubKey) + store.chain = chain + store.commits = commits + + // run the whole chain against this client to build up the tendermint state + clientCreator := proxy.NewLocalClientCreator(dummy.NewPersistentDummyApplication(path.Join(config.GetString("db_dir"), "1"))) + proxyApp := proxy.NewAppConns(config, clientCreator, nil) // sm.NewHandshaker(config, state, store, ReplayLastBlock)) + if _, err := proxyApp.Start(); err != nil { + t.Fatalf("Error starting proxy app connections: %v", err) + } + for _, block := range chain { + err := state.ApplyBlock(nil, proxyApp.Consensus(), block, block.MakePartSet(testPartSize).Header(), mempool) + if err != nil { + t.Fatal(err) + } + } + proxyApp.Stop() + latestAppHash := state.AppHash + + // run nBlocks against a new client to build up the app state. + // use a throwaway tendermint state + clientCreator2 := proxy.NewLocalClientCreator(dummy.NewPersistentDummyApplication(path.Join(config.GetString("db_dir"), "2"))) + if nBlocks > 0 { + // start a new app without handshake, play nBlocks blocks + proxyApp := proxy.NewAppConns(config, clientCreator2, nil) + if _, err := proxyApp.Start(); err != nil { + t.Fatalf("Error starting proxy app connections: %v", err) + } + state2, _ := stateAndStore(config, privVal.PubKey) + for i := 0; i < nBlocks; i++ { + block := chain[i] + err := state2.ApplyBlock(nil, proxyApp.Consensus(), block, block.MakePartSet(testPartSize).Header(), mempool) + if err != nil { + t.Fatal(err) + } + } + proxyApp.Stop() + } + + // now start the app using the handshake - it should sync + handshaker := sm.NewHandshaker(config, state, store, ReplayLastBlock) + proxyApp = proxy.NewAppConns(config, clientCreator2, handshaker) + if _, err := proxyApp.Start(); err != nil { + t.Fatalf("Error starting proxy app connections: %v", err) + } + + // get the latest app hash from the app + res, err := proxyApp.Query().InfoSync() + if err != nil { + t.Fatal(err) + } + + // the app hash should be synced up + if !bytes.Equal(latestAppHash, res.LastBlockAppHash) { + t.Fatalf("Expected app hashes to match after handshake/replay. got %X, expected %X", res.LastBlockAppHash, latestAppHash) + } + + if handshaker.NBlocks() != NUM_BLOCKS-nBlocks { + t.Fatalf("Expected handshake to sync %d blocks, got %d", NUM_BLOCKS-nBlocks, handshaker.NBlocks()) + } +} + +//-------------------------- +// utils for making blocks + +func makeBlockchainFromWAL(wal *WAL) ([]*types.Block, []*types.Commit, error) { + // Search for height marker + gr, found, err := wal.group.Search("#HEIGHT: ", makeHeightSearchFunc(1)) + if err != nil { + return nil, nil, err + } + if !found { + return nil, nil, errors.New(Fmt("WAL does not contain height %d.", 1)) + } + defer gr.Close() + + log.Notice("Build a blockchain by reading from the WAL") + + var blockParts *types.PartSet + var blocks []*types.Block + var commits []*types.Commit + for { + line, err := gr.ReadLine() + if err != nil { + if err == io.EOF { + break + } else { + return nil, nil, err + } + } + + piece, err := readPieceFromWAL([]byte(line)) + if err != nil { + return nil, nil, err + } + if piece == nil { + continue + } + + switch p := piece.(type) { + case *types.PartSetHeader: + // if its not the first one, we have a full block + if blockParts != nil { + var n int + block := wire.ReadBinary(&types.Block{}, blockParts.GetReader(), types.MaxBlockSize, &n, &err).(*types.Block) + blocks = append(blocks, block) + } + blockParts = types.NewPartSetFromHeader(*p) + case *types.Part: + _, err := blockParts.AddPart(p, false) + if err != nil { + return nil, nil, err + } + case *types.Vote: + if p.Type == types.VoteTypePrecommit { + commit := &types.Commit{ + BlockID: p.BlockID, + Precommits: []*types.Vote{p}, + } + commits = append(commits, commit) + } + } + } + // grab the last block too + var n int + block := wire.ReadBinary(&types.Block{}, blockParts.GetReader(), types.MaxBlockSize, &n, &err).(*types.Block) + blocks = append(blocks, block) + return blocks, commits, nil +} + +func readPieceFromWAL(msgBytes []byte) (interface{}, error) { + // Skip over empty and meta lines + if len(msgBytes) == 0 || msgBytes[0] == '#' { + return nil, nil + } + var err error + var msg TimedWALMessage + wire.ReadJSON(&msg, msgBytes, &err) + if err != nil { + fmt.Println("MsgBytes:", msgBytes, string(msgBytes)) + return nil, fmt.Errorf("Error reading json data: %v", err) + } + + // for logging + switch m := msg.Msg.(type) { + case msgInfo: + switch msg := m.Msg.(type) { + case *ProposalMessage: + return &msg.Proposal.BlockPartsHeader, nil + case *BlockPartMessage: + return msg.Part, nil + case *VoteMessage: + return msg.Vote, nil + } + } + return nil, nil +} + +// make some bogus txs +func txsFunc(blockNum int) (txs []types.Tx) { + for i := 0; i < 10; i++ { + txs = append(txs, types.Tx([]byte{byte(blockNum), byte(i)})) + } + return txs +} + +// sign a commit vote +func signCommit(chainID string, privVal *types.PrivValidator, height, round int, hash []byte, header types.PartSetHeader) *types.Vote { + vote := &types.Vote{ + ValidatorIndex: 0, + ValidatorAddress: privVal.Address, + Height: height, + Round: round, + Type: types.VoteTypePrecommit, + BlockID: types.BlockID{hash, header}, + } + + sig := privVal.Sign(types.SignBytes(chainID, vote)) + vote.Signature = sig + return vote +} + +// make a blockchain with one validator +func makeBlockchain(t *testing.T, chainID string, nBlocks int, privVal *types.PrivValidator, proxyApp proxy.AppConns, state *sm.State) (blockchain []*types.Block, commits []*types.Commit) { + + prevHash := state.LastBlockID.Hash + lastCommit := new(types.Commit) + prevParts := types.PartSetHeader{} + valHash := state.Validators.Hash() + prevBlockID := types.BlockID{prevHash, prevParts} + + for i := 1; i < nBlocks+1; i++ { + block, parts := types.MakeBlock(i, chainID, txsFunc(i), lastCommit, + prevBlockID, valHash, state.AppHash, testPartSize) + fmt.Println(i) + fmt.Println(block.LastBlockID) + err := state.ApplyBlock(nil, proxyApp.Consensus(), block, block.MakePartSet(testPartSize).Header(), mempool) + if err != nil { + t.Fatal(i, err) + } + + voteSet := types.NewVoteSet(chainID, i, 0, types.VoteTypePrecommit, state.Validators) + vote := signCommit(chainID, privVal, i, 0, block.Hash(), parts.Header()) + _, err = voteSet.AddVote(vote) + if err != nil { + t.Fatal(err) + } + + prevHash = block.Hash() + prevParts = parts.Header() + lastCommit = voteSet.MakeCommit() + prevBlockID = types.BlockID{prevHash, prevParts} + + blockchain = append(blockchain, block) + commits = append(commits, lastCommit) + } + return blockchain, commits +} + +// fresh state and mock store +func stateAndStore(config cfg.Config, pubKey crypto.PubKey) (*sm.State, *mockBlockStore) { + stateDB := dbm.NewMemDB() + return sm.MakeGenesisState(stateDB, &types.GenesisDoc{ + ChainID: config.GetString("chain_id"), + Validators: []types.GenesisValidator{ + types.GenesisValidator{pubKey, 10000, "test"}, + }, + AppHash: nil, + }), NewMockBlockStore(config) +} + +//---------------------------------- +// mock block store + +type mockBlockStore struct { + config cfg.Config + chain []*types.Block + commits []*types.Commit +} + +// TODO: NewBlockStore(db.NewMemDB) ... +func NewMockBlockStore(config cfg.Config) *mockBlockStore { + return &mockBlockStore{config, nil, nil} +} + +func (bs *mockBlockStore) Height() int { return len(bs.chain) } +func (bs *mockBlockStore) LoadBlock(height int) *types.Block { return bs.chain[height-1] } +func (bs *mockBlockStore) LoadBlockMeta(height int) *types.BlockMeta { + block := bs.chain[height-1] + return &types.BlockMeta{ + BlockID: types.BlockID{block.Hash(), block.MakePartSet(bs.config.GetInt("block_part_size")).Header()}, + Header: block.Header, + } +} +func (bs *mockBlockStore) LoadBlockPart(height int, index int) *types.Part { return nil } +func (bs *mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) { +} +func (bs *mockBlockStore) LoadBlockCommit(height int) *types.Commit { + return bs.commits[height-1] +} +func (bs *mockBlockStore) LoadSeenCommit(height int) *types.Commit { + return bs.commits[height-1] } diff --git a/consensus/state.go b/consensus/state.go index 30ca8e9fa..a985fd2c8 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -4,6 +4,7 @@ import ( "bytes" "errors" "fmt" + "path" "reflect" "sync" "time" @@ -354,13 +355,13 @@ func (cs *ConsensusState) LoadCommit(height int) *types.Commit { func (cs *ConsensusState) OnStart() error { cs.BaseService.OnStart() - walDir := cs.config.GetString("cs_wal_dir") - err := EnsureDir(walDir, 0700) + walFile := cs.config.GetString("cs_wal_file") + err := EnsureDir(path.Dir(walFile), 0700) if err != nil { log.Error("Error ensuring ConsensusState wal dir", "error", err.Error()) return err } - err = cs.OpenWAL(walDir) + err = cs.OpenWAL(walFile) if err != nil { log.Error("Error loading ConsensusState wal", "error", err.Error()) return err @@ -415,10 +416,10 @@ func (cs *ConsensusState) Wait() { } // Open file to log all consensus messages and timeouts for deterministic accountability -func (cs *ConsensusState) OpenWAL(walDir string) (err error) { +func (cs *ConsensusState) OpenWAL(walFile string) (err error) { cs.mtx.Lock() defer cs.mtx.Unlock() - wal, err := NewWAL(walDir, cs.config.GetBool("cs_wal_light")) + wal, err := NewWAL(walFile, cs.config.GetBool("cs_wal_light")) if err != nil { return err } diff --git a/consensus/test_data/build.sh b/consensus/test_data/build.sh index 970eb7835..ea0c9604a 100644 --- a/consensus/test_data/build.sh +++ b/consensus/test_data/build.sh @@ -1,12 +1,13 @@ #! /bin/bash +# XXX: removes tendermint dir + cd $GOPATH/src/github.com/tendermint/tendermint # specify a dir to copy -# NOTE: eventually we should replace with `tendermint init --test` +# TODO: eventually we should replace with `tendermint init --test` DIR=$HOME/.tendermint_test/consensus_state_test -# XXX: remove tendermint dir rm -rf $HOME/.tendermint cp -r $DIR $HOME/.tendermint @@ -18,6 +19,7 @@ function reset(){ reset # empty block +function empty_block(){ tendermint node --proxy_app=dummy &> /dev/null & sleep 5 killall tendermint @@ -28,21 +30,40 @@ killall tendermint sed '/HEIGHT: 2/Q' ~/.tendermint/data/cs.wal/wal > consensus/test_data/empty_block.cswal reset +} + +# many blocks +function many_blocks(){ +bash scripts/txs/random.sh 1000 36657 &> /dev/null & +PID=$! +tendermint node --proxy_app=dummy &> /dev/null & +sleep 7 +killall tendermint +kill -9 $PID + +sed '/HEIGHT: 7/Q' ~/.tendermint/data/cs.wal/wal > consensus/test_data/many_blocks.cswal + +reset +} + # small block 1 +function small_block1(){ bash scripts/txs/random.sh 1000 36657 &> /dev/null & PID=$! tendermint node --proxy_app=dummy &> /dev/null & -sleep 5 +sleep 10 killall tendermint kill -9 $PID sed '/HEIGHT: 2/Q' ~/.tendermint/data/cs.wal/wal > consensus/test_data/small_block1.cswal reset +} # small block 2 (part size = 512) +function small_block2(){ echo "" >> ~/.tendermint/config.toml echo "block_part_size = 512" >> ~/.tendermint/config.toml bash scripts/txs/random.sh 1000 36657 &> /dev/null & @@ -55,4 +76,28 @@ kill -9 $PID sed '/HEIGHT: 2/Q' ~/.tendermint/data/cs.wal/wal > consensus/test_data/small_block2.cswal reset +} + + + +case "$1" in + "small_block1") + small_block1 + ;; + "small_block2") + small_block2 + ;; + "empty_block") + empty_block + ;; + "many_blocks") + many_blocks + ;; + *) + small_block1 + small_block2 + empty_block + many_blocks +esac + diff --git a/consensus/test_data/many_blocks.cswal b/consensus/test_data/many_blocks.cswal new file mode 100644 index 000000000..9ef06c32c --- /dev/null +++ b/consensus/test_data/many_blocks.cswal @@ -0,0 +1,65 @@ +#HEIGHT: 1 +{"time":"2017-02-17T23:54:19.013Z","msg":[3,{"duration":969121813,"height":1,"round":0,"step":1}]} +{"time":"2017-02-17T23:54:19.014Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPropose"}]} +{"time":"2017-02-17T23:54:19.014Z","msg":[2,{"msg":[17,{"Proposal":{"height":1,"round":0,"block_parts_header":{"total":1,"hash":"2E32C8D500E936D27A47FCE3FF4BE7C1AFB3FAE1"},"pol_round":-1,"pol_block_id":{"hash":"","parts":{"total":0,"hash":""}},"signature":[1,"105A5A834E9AE2FA2191CAB5CB20D63594BA7859BD3EB92F055C8A35476D71F0D89F9FD5B0FF030D021533C71A81BF6E8F026BF4A37FC637CF38CA35291A9D00"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:19.015Z","msg":[2,{"msg":[19,{"Height":1,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F74657374010114A438480B084B40017600000000011477134726D7D54ABC03516888951EBC652413B20B0114354594CBFC1A7BCA1AD0050ED6AA010023EADA3900010176011A3631363236333634333133313344363436333632363133313331011A3631363236333634333133323344363436333632363133313332011A3631363236333634333133333344363436333632363133313333011A3631363236333634333133343344363436333632363133313334011A3631363236333634333133353344363436333632363133313335011A3631363236333634333133363344363436333632363133313336011A3631363236333634333133373344363436333632363133313337011A3631363236333634333133383344363436333632363133313338011A3631363236333634333133393344363436333632363133313339011A3631363236333634333233303344363436333632363133323330011A3631363236333634333233313344363436333632363133323331011A3631363236333634333233323344363436333632363133323332011A3631363236333634333233333344363436333632363133323333011A3631363236333634333233343344363436333632363133323334011A3631363236333634333233353344363436333632363133323335011A3631363236333634333233363344363436333632363133323336011A3631363236333634333233373344363436333632363133323337011A3631363236333634333233383344363436333632363133323338011A3631363236333634333233393344363436333632363133323339011A3631363236333634333333303344363436333632363133333330011A3631363236333634333333313344363436333632363133333331011A3631363236333634333333323344363436333632363133333332011A3631363236333634333333333344363436333632363133333333011A3631363236333634333333343344363436333632363133333334011A3631363236333634333333353344363436333632363133333335011A3631363236333634333333363344363436333632363133333336011A3631363236333634333333373344363436333632363133333337011A3631363236333634333333383344363436333632363133333338011A3631363236333634333333393344363436333632363133333339011A3631363236333634333433303344363436333632363133343330011A3631363236333634333433313344363436333632363133343331011A3631363236333634333433323344363436333632363133343332011A3631363236333634333433333344363436333632363133343333011A3631363236333634333433343344363436333632363133343334011A3631363236333634333433353344363436333632363133343335011A3631363236333634333433363344363436333632363133343336011A3631363236333634333433373344363436333632363133343337011A3631363236333634333433383344363436333632363133343338011A3631363236333634333433393344363436333632363133343339011A3631363236333634333533303344363436333632363133353330011A3631363236333634333533313344363436333632363133353331011A3631363236333634333533323344363436333632363133353332011A3631363236333634333533333344363436333632363133353333011A3631363236333634333533343344363436333632363133353334011A3631363236333634333533353344363436333632363133353335011A3631363236333634333533363344363436333632363133353336011A3631363236333634333533373344363436333632363133353337011A3631363236333634333533383344363436333632363133353338011A3631363236333634333533393344363436333632363133353339011A3631363236333634333633303344363436333632363133363330011A3631363236333634333633313344363436333632363133363331011A3631363236333634333633323344363436333632363133363332011A3631363236333634333633333344363436333632363133363333011A3631363236333634333633343344363436333632363133363334011A3631363236333634333633353344363436333632363133363335011A3631363236333634333633363344363436333632363133363336011A3631363236333634333633373344363436333632363133363337011A3631363236333634333633383344363436333632363133363338011A3631363236333634333633393344363436333632363133363339011A3631363236333634333733303344363436333632363133373330011A3631363236333634333733313344363436333632363133373331011A3631363236333634333733323344363436333632363133373332011A3631363236333634333733333344363436333632363133373333011A3631363236333634333733343344363436333632363133373334011A3631363236333634333733353344363436333632363133373335011A3631363236333634333733363344363436333632363133373336011A3631363236333634333733373344363436333632363133373337011A3631363236333634333733383344363436333632363133373338011A3631363236333634333733393344363436333632363133373339011A3631363236333634333833303344363436333632363133383330011A3631363236333634333833313344363436333632363133383331011A3631363236333634333833323344363436333632363133383332011A3631363236333634333833333344363436333632363133383333011A3631363236333634333833343344363436333632363133383334011A3631363236333634333833353344363436333632363133383335011A3631363236333634333833363344363436333632363133383336011A3631363236333634333833373344363436333632363133383337011A3631363236333634333833383344363436333632363133383338011A3631363236333634333833393344363436333632363133383339011A3631363236333634333933303344363436333632363133393330011A3631363236333634333933313344363436333632363133393331011A3631363236333634333933323344363436333632363133393332011A3631363236333634333933333344363436333632363133393333011A3631363236333634333933343344363436333632363133393334011A3631363236333634333933353344363436333632363133393335011A3631363236333634333933363344363436333632363133393336011A3631363236333634333933373344363436333632363133393337011A3631363236333634333933383344363436333632363133393338011A3631363236333634333933393344363436333632363133393339011E363136323633363433313330333033443634363336323631333133303330011E363136323633363433313330333133443634363336323631333133303331011E363136323633363433313330333233443634363336323631333133303332011E363136323633363433313330333333443634363336323631333133303333011E363136323633363433313330333433443634363336323631333133303334011E363136323633363433313330333533443634363336323631333133303335011E363136323633363433313330333633443634363336323631333133303336011E363136323633363433313330333733443634363336323631333133303337011E363136323633363433313330333833443634363336323631333133303338011E363136323633363433313330333933443634363336323631333133303339011E363136323633363433313331333033443634363336323631333133313330011E363136323633363433313331333133443634363336323631333133313331011E363136323633363433313331333233443634363336323631333133313332011E363136323633363433313331333333443634363336323631333133313333011E363136323633363433313331333433443634363336323631333133313334011E363136323633363433313331333533443634363336323631333133313335011E363136323633363433313331333633443634363336323631333133313336011E363136323633363433313331333733443634363336323631333133313337011E363136323633363433313331333833443634363336323631333133313338011E363136323633363433313331333933443634363336323631333133313339011E363136323633363433313332333033443634363336323631333133323330011E363136323633363433313332333133443634363336323631333133323331011E363136323633363433313332333233443634363336323631333133323332011E363136323633363433313332333333443634363336323631333133323333011E363136323633363433313332333433443634363336323631333133323334011E363136323633363433313332333533443634363336323631333133323335011E363136323633363433313332333633443634363336323631333133323336011E363136323633363433313332333733443634363336323631333133323337011E3631363236333634333133323338334436343633363236313331333233380100000000","proof":{"aunts":[]}}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:19.016Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrevote"}]} +{"time":"2017-02-17T23:54:19.016Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":1,"round":0,"type":1,"block_id":{"hash":"3F32EE37F9EA674A2173CAD651836A8EE628B5C7","parts":{"total":1,"hash":"2E32C8D500E936D27A47FCE3FF4BE7C1AFB3FAE1"}},"signature":[1,"31851DA0008AFF4223245EDFCCF1AD7BE96F8D66F8BD02D87F06B2F800A9405413861877D08798F0F6297D29936F5380B352C82212D2EC6F0E194A8C22A1EB0E"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:19.016Z","msg":[1,{"height":1,"round":0,"step":"RoundStepPrecommit"}]} +{"time":"2017-02-17T23:54:19.016Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":1,"round":0,"type":2,"block_id":{"hash":"3F32EE37F9EA674A2173CAD651836A8EE628B5C7","parts":{"total":1,"hash":"2E32C8D500E936D27A47FCE3FF4BE7C1AFB3FAE1"}},"signature":[1,"2B1070A5AB9305612A3AE74A8036D82B5E49E0DBBFBC7D723DB985CC8A8E72A52FF8E34D85273FEB8B901945CA541FA5142C3C4D43A04E9205ACECF53FD19B01"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:19.017Z","msg":[1,{"height":1,"round":0,"step":"RoundStepCommit"}]} +#HEIGHT: 2 +{"time":"2017-02-17T23:54:19.019Z","msg":[1,{"height":2,"round":0,"step":"RoundStepNewHeight"}]} +{"time":"2017-02-17T23:54:20.017Z","msg":[3,{"duration":998073370,"height":2,"round":0,"step":1}]} +{"time":"2017-02-17T23:54:20.018Z","msg":[1,{"height":2,"round":0,"step":"RoundStepPropose"}]} +{"time":"2017-02-17T23:54:20.018Z","msg":[2,{"msg":[17,{"Proposal":{"height":2,"round":0,"block_parts_header":{"total":1,"hash":"D008E9014CDDEA8EC95E1E99E21333241BD52DFC"},"pol_round":-1,"pol_block_id":{"hash":"","parts":{"total":0,"hash":""}},"signature":[1,"03E06975CD5A83E2B6AADC82F0C5965BE13CCB589912B7CBEF847BDBED6E8EAEE0901C02FAE8BC96B269C4750E5BA5C351C587537E3C063358A7D769007D8509"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:20.019Z","msg":[2,{"msg":[19,{"Height":2,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F74657374010214A4384846E01E40018101143F32EE37F9EA674A2173CAD651836A8EE628B5C7010101142E32C8D500E936D27A47FCE3FF4BE7C1AFB3FAE101142F7319866C6844639F566F5A201DE2C339EAF8670114A0DC0254E5A6831C03FB4E1BE09CB26AEAC5C73D0114354594CBFC1A7BCA1AD0050ED6AA010023EADA39011402B7F5E97ED0892197CF5779CA6B904B10C87E10010181011E363136323633363433313332333933443634363336323631333133323339011E363136323633363433313333333033443634363336323631333133333330011E363136323633363433313333333133443634363336323631333133333331011E363136323633363433313333333233443634363336323631333133333332011E363136323633363433313333333333443634363336323631333133333333011E363136323633363433313333333433443634363336323631333133333334011E363136323633363433313333333533443634363336323631333133333335011E363136323633363433313333333633443634363336323631333133333336011E363136323633363433313333333733443634363336323631333133333337011E363136323633363433313333333833443634363336323631333133333338011E363136323633363433313333333933443634363336323631333133333339011E363136323633363433313334333033443634363336323631333133343330011E363136323633363433313334333133443634363336323631333133343331011E363136323633363433313334333233443634363336323631333133343332011E363136323633363433313334333333443634363336323631333133343333011E363136323633363433313334333433443634363336323631333133343334011E363136323633363433313334333533443634363336323631333133343335011E363136323633363433313334333633443634363336323631333133343336011E363136323633363433313334333733443634363336323631333133343337011E363136323633363433313334333833443634363336323631333133343338011E363136323633363433313334333933443634363336323631333133343339011E363136323633363433313335333033443634363336323631333133353330011E363136323633363433313335333133443634363336323631333133353331011E363136323633363433313335333233443634363336323631333133353332011E363136323633363433313335333333443634363336323631333133353333011E363136323633363433313335333433443634363336323631333133353334011E363136323633363433313335333533443634363336323631333133353335011E363136323633363433313335333633443634363336323631333133353336011E363136323633363433313335333733443634363336323631333133353337011E363136323633363433313335333833443634363336323631333133353338011E363136323633363433313335333933443634363336323631333133353339011E363136323633363433313336333033443634363336323631333133363330011E363136323633363433313336333133443634363336323631333133363331011E363136323633363433313336333233443634363336323631333133363332011E363136323633363433313336333333443634363336323631333133363333011E363136323633363433313336333433443634363336323631333133363334011E363136323633363433313336333533443634363336323631333133363335011E363136323633363433313336333633443634363336323631333133363336011E363136323633363433313336333733443634363336323631333133363337011E363136323633363433313336333833443634363336323631333133363338011E363136323633363433313336333933443634363336323631333133363339011E363136323633363433313337333033443634363336323631333133373330011E363136323633363433313337333133443634363336323631333133373331011E363136323633363433313337333233443634363336323631333133373332011E363136323633363433313337333333443634363336323631333133373333011E363136323633363433313337333433443634363336323631333133373334011E363136323633363433313337333533443634363336323631333133373335011E363136323633363433313337333633443634363336323631333133373336011E363136323633363433313337333733443634363336323631333133373337011E363136323633363433313337333833443634363336323631333133373338011E363136323633363433313337333933443634363336323631333133373339011E363136323633363433313338333033443634363336323631333133383330011E363136323633363433313338333133443634363336323631333133383331011E363136323633363433313338333233443634363336323631333133383332011E363136323633363433313338333333443634363336323631333133383333011E363136323633363433313338333433443634363336323631333133383334011E363136323633363433313338333533443634363336323631333133383335011E363136323633363433313338333633443634363336323631333133383336011E363136323633363433313338333733443634363336323631333133383337011E363136323633363433313338333833443634363336323631333133383338011E363136323633363433313338333933443634363336323631333133383339011E363136323633363433313339333033443634363336323631333133393330011E363136323633363433313339333133443634363336323631333133393331011E363136323633363433313339333233443634363336323631333133393332011E363136323633363433313339333333443634363336323631333133393333011E363136323633363433313339333433443634363336323631333133393334011E363136323633363433313339333533443634363336323631333133393335011E363136323633363433313339333633443634363336323631333133393336011E363136323633363433313339333733443634363336323631333133393337011E363136323633363433313339333833443634363336323631333133393338011E363136323633363433313339333933443634363336323631333133393339011E363136323633363433323330333033443634363336323631333233303330011E363136323633363433323330333133443634363336323631333233303331011E363136323633363433323330333233443634363336323631333233303332011E363136323633363433323330333333443634363336323631333233303333011E363136323633363433323330333433443634363336323631333233303334011E363136323633363433323330333533443634363336323631333233303335011E363136323633363433323330333633443634363336323631333233303336011E363136323633363433323330333733443634363336323631333233303337011E363136323633363433323330333833443634363336323631333233303338011E363136323633363433323330333933443634363336323631333233303339011E363136323633363433323331333033443634363336323631333233313330011E363136323633363433323331333133443634363336323631333233313331011E363136323633363433323331333233443634363336323631333233313332011E363136323633363433323331333333443634363336323631333233313333011E363136323633363433323331333433443634363336323631333233313334011E363136323633363433323331333533443634363336323631333233313335011E363136323633363433323331333633443634363336323631333233313336011E363136323633363433323331333733443634363336323631333233313337011E363136323633363433323331333833443634363336323631333233313338011E363136323633363433323331333933443634363336323631333233313339011E363136323633363433323332333033443634363336323631333233323330011E363136323633363433323332333133443634363336323631333233323331011E363136323633363433323332333233443634363336323631333233323332011E363136323633363433323332333333443634363336323631333233323333011E363136323633363433323332333433443634363336323631333233323334011E363136323633363433323332333533443634363336323631333233323335011E363136323633363433323332333633443634363336323631333233323336011E363136323633363433323332333733443634363336323631333233323337011E363136323633363433323332333833443634363336323631333233323338011E363136323633363433323332333933443634363336323631333233323339011E363136323633363433323333333033443634363336323631333233333330011E363136323633363433323333333133443634363336323631333233333331011E363136323633363433323333333233443634363336323631333233333332011E363136323633363433323333333333443634363336323631333233333333011E363136323633363433323333333433443634363336323631333233333334011E363136323633363433323333333533443634363336323631333233333335011E363136323633363433323333333633443634363336323631333233333336011E363136323633363433323333333733443634363336323631333233333337011E363136323633363433323333333833443634363336323631333233333338011E363136323633363433323333333933443634363336323631333233333339011E363136323633363433323334333033443634363336323631333233343330011E363136323633363433323334333133443634363336323631333233343331011E363136323633363433323334333233443634363336323631333233343332011E363136323633363433323334333333443634363336323631333233343333011E363136323633363433323334333433443634363336323631333233343334011E363136323633363433323334333533443634363336323631333233343335011E363136323633363433323334333633443634363336323631333233343336011E363136323633363433323334333733443634363336323631333233343337011E363136323633363433323334333833443634363336323631333233343338011E363136323633363433323334333933443634363336323631333233343339011E363136323633363433323335333033443634363336323631333233353330011E363136323633363433323335333133443634363336323631333233353331011E363136323633363433323335333233443634363336323631333233353332011E363136323633363433323335333333443634363336323631333233353333011E363136323633363433323335333433443634363336323631333233353334011E363136323633363433323335333533443634363336323631333233353335011E363136323633363433323335333633443634363336323631333233353336011E3631363236333634333233353337334436343633363236313332333533370101143F32EE37F9EA674A2173CAD651836A8EE628B5C7010101142E32C8D500E936D27A47FCE3FF4BE7C1AFB3FAE10101010114D028C9981F7A87F3093672BF0D5B0E2A1B3ED456000101000201143F32EE37F9EA674A2173CAD651836A8EE628B5C7010101142E32C8D500E936D27A47FCE3FF4BE7C1AFB3FAE1012B1070A5AB9305612A3AE74A8036D82B5E49E0DBBFBC7D723DB985CC8A8E72A52FF8E34D85273FEB8B901945CA541FA5142C3C4D43A04E9205ACECF53FD19B01","proof":{"aunts":[]}}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:20.020Z","msg":[1,{"height":2,"round":0,"step":"RoundStepPrevote"}]} +{"time":"2017-02-17T23:54:20.020Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":2,"round":0,"type":1,"block_id":{"hash":"32310D174A99844713693C9815D2CA660364E028","parts":{"total":1,"hash":"D008E9014CDDEA8EC95E1E99E21333241BD52DFC"}},"signature":[1,"E0289DE621820D9236632B4862BB4D1518A4B194C5AE8194192F375C9A52775A54A7F172A5D7A2014E404A1C3AFA386923E7A20329AFDDFA14655881C04A1A02"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:20.021Z","msg":[1,{"height":2,"round":0,"step":"RoundStepPrecommit"}]} +{"time":"2017-02-17T23:54:20.021Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":2,"round":0,"type":2,"block_id":{"hash":"32310D174A99844713693C9815D2CA660364E028","parts":{"total":1,"hash":"D008E9014CDDEA8EC95E1E99E21333241BD52DFC"}},"signature":[1,"AA9F03D0707752301D7CBFCF4F0BCDBD666A46C1CAED3910BD64A3C5C2874AAF328172646C951C5E2FD962359C382A3CBBA2C73EC9B533668C6386995B83EC08"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:20.022Z","msg":[1,{"height":2,"round":0,"step":"RoundStepCommit"}]} +#HEIGHT: 3 +{"time":"2017-02-17T23:54:20.025Z","msg":[1,{"height":3,"round":0,"step":"RoundStepNewHeight"}]} +{"time":"2017-02-17T23:54:21.022Z","msg":[3,{"duration":997103974,"height":3,"round":0,"step":1}]} +{"time":"2017-02-17T23:54:21.024Z","msg":[1,{"height":3,"round":0,"step":"RoundStepPropose"}]} +{"time":"2017-02-17T23:54:21.024Z","msg":[2,{"msg":[17,{"Proposal":{"height":3,"round":0,"block_parts_header":{"total":1,"hash":"2E5DE5777A5AD899CD2531304F42A470509DE989"},"pol_round":-1,"pol_block_id":{"hash":"","parts":{"total":0,"hash":""}},"signature":[1,"5F6A6A8097BD6A1780568C7E064D932BC1F941E1D5AC408DE970C4EEDCCD939C0F163466D20F0E98A7599792341441422980C09D23E03009BD9CE565673C9704"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:21.024Z","msg":[2,{"msg":[19,{"Height":3,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F74657374010314A4384882C73380017C011432310D174A99844713693C9815D2CA660364E02801010114D008E9014CDDEA8EC95E1E99E21333241BD52DFC0114ABAB9E28967792BDA02172D3CB99DA99A696738E0114513B0891921FB0A5F6471950AFE2598445153CFF0114354594CBFC1A7BCA1AD0050ED6AA010023EADA3901141FBD44B8259B2A6632F08F88BE5EC8C203075CD001017C011E363136323633363433323335333833443634363336323631333233353338011E363136323633363433323335333933443634363336323631333233353339011E363136323633363433323336333033443634363336323631333233363330011E363136323633363433323336333133443634363336323631333233363331011E363136323633363433323336333233443634363336323631333233363332011E363136323633363433323336333333443634363336323631333233363333011E363136323633363433323336333433443634363336323631333233363334011E363136323633363433323336333533443634363336323631333233363335011E363136323633363433323336333633443634363336323631333233363336011E363136323633363433323336333733443634363336323631333233363337011E363136323633363433323336333833443634363336323631333233363338011E363136323633363433323336333933443634363336323631333233363339011E363136323633363433323337333033443634363336323631333233373330011E363136323633363433323337333133443634363336323631333233373331011E363136323633363433323337333233443634363336323631333233373332011E363136323633363433323337333333443634363336323631333233373333011E363136323633363433323337333433443634363336323631333233373334011E363136323633363433323337333533443634363336323631333233373335011E363136323633363433323337333633443634363336323631333233373336011E363136323633363433323337333733443634363336323631333233373337011E363136323633363433323337333833443634363336323631333233373338011E363136323633363433323337333933443634363336323631333233373339011E363136323633363433323338333033443634363336323631333233383330011E363136323633363433323338333133443634363336323631333233383331011E363136323633363433323338333233443634363336323631333233383332011E363136323633363433323338333333443634363336323631333233383333011E363136323633363433323338333433443634363336323631333233383334011E363136323633363433323338333533443634363336323631333233383335011E363136323633363433323338333633443634363336323631333233383336011E363136323633363433323338333733443634363336323631333233383337011E363136323633363433323338333833443634363336323631333233383338011E363136323633363433323338333933443634363336323631333233383339011E363136323633363433323339333033443634363336323631333233393330011E363136323633363433323339333133443634363336323631333233393331011E363136323633363433323339333233443634363336323631333233393332011E363136323633363433323339333333443634363336323631333233393333011E363136323633363433323339333433443634363336323631333233393334011E363136323633363433323339333533443634363336323631333233393335011E363136323633363433323339333633443634363336323631333233393336011E363136323633363433323339333733443634363336323631333233393337011E363136323633363433323339333833443634363336323631333233393338011E363136323633363433323339333933443634363336323631333233393339011E363136323633363433333330333033443634363336323631333333303330011E363136323633363433333330333133443634363336323631333333303331011E363136323633363433333330333233443634363336323631333333303332011E363136323633363433333330333333443634363336323631333333303333011E363136323633363433333330333433443634363336323631333333303334011E363136323633363433333330333533443634363336323631333333303335011E363136323633363433333330333633443634363336323631333333303336011E363136323633363433333330333733443634363336323631333333303337011E363136323633363433333330333833443634363336323631333333303338011E363136323633363433333330333933443634363336323631333333303339011E363136323633363433333331333033443634363336323631333333313330011E363136323633363433333331333133443634363336323631333333313331011E363136323633363433333331333233443634363336323631333333313332011E363136323633363433333331333333443634363336323631333333313333011E363136323633363433333331333433443634363336323631333333313334011E363136323633363433333331333533443634363336323631333333313335011E363136323633363433333331333633443634363336323631333333313336011E363136323633363433333331333733443634363336323631333333313337011E363136323633363433333331333833443634363336323631333333313338011E363136323633363433333331333933443634363336323631333333313339011E363136323633363433333332333033443634363336323631333333323330011E363136323633363433333332333133443634363336323631333333323331011E363136323633363433333332333233443634363336323631333333323332011E363136323633363433333332333333443634363336323631333333323333011E363136323633363433333332333433443634363336323631333333323334011E363136323633363433333332333533443634363336323631333333323335011E363136323633363433333332333633443634363336323631333333323336011E363136323633363433333332333733443634363336323631333333323337011E363136323633363433333332333833443634363336323631333333323338011E363136323633363433333332333933443634363336323631333333323339011E363136323633363433333333333033443634363336323631333333333330011E363136323633363433333333333133443634363336323631333333333331011E363136323633363433333333333233443634363336323631333333333332011E363136323633363433333333333333443634363336323631333333333333011E363136323633363433333333333433443634363336323631333333333334011E363136323633363433333333333533443634363336323631333333333335011E363136323633363433333333333633443634363336323631333333333336011E363136323633363433333333333733443634363336323631333333333337011E363136323633363433333333333833443634363336323631333333333338011E363136323633363433333333333933443634363336323631333333333339011E363136323633363433333334333033443634363336323631333333343330011E363136323633363433333334333133443634363336323631333333343331011E363136323633363433333334333233443634363336323631333333343332011E363136323633363433333334333333443634363336323631333333343333011E363136323633363433333334333433443634363336323631333333343334011E363136323633363433333334333533443634363336323631333333343335011E363136323633363433333334333633443634363336323631333333343336011E363136323633363433333334333733443634363336323631333333343337011E363136323633363433333334333833443634363336323631333333343338011E363136323633363433333334333933443634363336323631333333343339011E363136323633363433333335333033443634363336323631333333353330011E363136323633363433333335333133443634363336323631333333353331011E363136323633363433333335333233443634363336323631333333353332011E363136323633363433333335333333443634363336323631333333353333011E363136323633363433333335333433443634363336323631333333353334011E363136323633363433333335333533443634363336323631333333353335011E363136323633363433333335333633443634363336323631333333353336011E363136323633363433333335333733443634363336323631333333353337011E363136323633363433333335333833443634363336323631333333353338011E363136323633363433333335333933443634363336323631333333353339011E363136323633363433333336333033443634363336323631333333363330011E363136323633363433333336333133443634363336323631333333363331011E363136323633363433333336333233443634363336323631333333363332011E363136323633363433333336333333443634363336323631333333363333011E363136323633363433333336333433443634363336323631333333363334011E363136323633363433333336333533443634363336323631333333363335011E363136323633363433333336333633443634363336323631333333363336011E363136323633363433333336333733443634363336323631333333363337011E363136323633363433333336333833443634363336323631333333363338011E363136323633363433333336333933443634363336323631333333363339011E363136323633363433333337333033443634363336323631333333373330011E363136323633363433333337333133443634363336323631333333373331011E363136323633363433333337333233443634363336323631333333373332011E363136323633363433333337333333443634363336323631333333373333011E363136323633363433333337333433443634363336323631333333373334011E363136323633363433333337333533443634363336323631333333373335011E363136323633363433333337333633443634363336323631333333373336011E363136323633363433333337333733443634363336323631333333373337011E363136323633363433333337333833443634363336323631333333373338011E363136323633363433333337333933443634363336323631333333373339011E363136323633363433333338333033443634363336323631333333383330011E36313632363336343333333833313344363436333632363133333338333101011432310D174A99844713693C9815D2CA660364E02801010114D008E9014CDDEA8EC95E1E99E21333241BD52DFC0101010114D028C9981F7A87F3093672BF0D5B0E2A1B3ED4560001020002011432310D174A99844713693C9815D2CA660364E02801010114D008E9014CDDEA8EC95E1E99E21333241BD52DFC01AA9F03D0707752301D7CBFCF4F0BCDBD666A46C1CAED3910BD64A3C5C2874AAF328172646C951C5E2FD962359C382A3CBBA2C73EC9B533668C6386995B83EC08","proof":{"aunts":[]}}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:21.026Z","msg":[1,{"height":3,"round":0,"step":"RoundStepPrevote"}]} +{"time":"2017-02-17T23:54:21.026Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":3,"round":0,"type":1,"block_id":{"hash":"37AF6866DA8C3167CFC280FAE47B6ED441B00D5B","parts":{"total":1,"hash":"2E5DE5777A5AD899CD2531304F42A470509DE989"}},"signature":[1,"F0AAB604A8CE724453A378BBC66142C418464C3C0EC3EB2E15A1CB7524A92B9F36BE8A191238A4D317F542D999DF698B5C2A28D754240524FF8CCADA0947DE00"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:21.028Z","msg":[1,{"height":3,"round":0,"step":"RoundStepPrecommit"}]} +{"time":"2017-02-17T23:54:21.028Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":3,"round":0,"type":2,"block_id":{"hash":"37AF6866DA8C3167CFC280FAE47B6ED441B00D5B","parts":{"total":1,"hash":"2E5DE5777A5AD899CD2531304F42A470509DE989"}},"signature":[1,"C900519E305EC03392E7D197D5FAB535DB240C9C0BA5375A1679C75BAAA07C7410C0EF43CF97D98F2C08A1D739667D5ACFF6233A1FAE75D3DA275AEA422EFD0F"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:21.028Z","msg":[1,{"height":3,"round":0,"step":"RoundStepCommit"}]} +#HEIGHT: 4 +{"time":"2017-02-17T23:54:21.032Z","msg":[1,{"height":4,"round":0,"step":"RoundStepNewHeight"}]} +{"time":"2017-02-17T23:54:22.028Z","msg":[3,{"duration":996302067,"height":4,"round":0,"step":1}]} +{"time":"2017-02-17T23:54:22.030Z","msg":[1,{"height":4,"round":0,"step":"RoundStepPropose"}]} +{"time":"2017-02-17T23:54:22.030Z","msg":[2,{"msg":[17,{"Proposal":{"height":4,"round":0,"block_parts_header":{"total":1,"hash":"24CEBCBEB833F56D47AD14354071B3B7A243068A"},"pol_round":-1,"pol_block_id":{"hash":"","parts":{"total":0,"hash":""}},"signature":[1,"CAECE2342987295CCB562C9B6AB0E296D0ECDBE0B40CDB5260B32DF07E07E7F30C4E815B76BC04B8E830143409E598F7BA24699F5B5A01A6237221C948A0920C"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:22.030Z","msg":[2,{"msg":[19,{"Height":4,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F74657374010414A43848BECCCD400169011437AF6866DA8C3167CFC280FAE47B6ED441B00D5B010101142E5DE5777A5AD899CD2531304F42A470509DE9890114EA4CCD80AE261EF694AF4292F8AF5659FB66317301144FFC4AAC3EDFBB81DA9E9DDEC9D6A205AD49049E0114354594CBFC1A7BCA1AD0050ED6AA010023EADA390114E7563C252781F893D7A191A31566713888CCA3B2010169011E363136323633363433333338333233443634363336323631333333383332011E363136323633363433333338333333443634363336323631333333383333011E363136323633363433333338333433443634363336323631333333383334011E363136323633363433333338333533443634363336323631333333383335011E363136323633363433333338333633443634363336323631333333383336011E363136323633363433333338333733443634363336323631333333383337011E363136323633363433333338333833443634363336323631333333383338011E363136323633363433333338333933443634363336323631333333383339011E363136323633363433333339333033443634363336323631333333393330011E363136323633363433333339333133443634363336323631333333393331011E363136323633363433333339333233443634363336323631333333393332011E363136323633363433333339333333443634363336323631333333393333011E363136323633363433333339333433443634363336323631333333393334011E363136323633363433333339333533443634363336323631333333393335011E363136323633363433333339333633443634363336323631333333393336011E363136323633363433333339333733443634363336323631333333393337011E363136323633363433333339333833443634363336323631333333393338011E363136323633363433333339333933443634363336323631333333393339011E363136323633363433343330333033443634363336323631333433303330011E363136323633363433343330333133443634363336323631333433303331011E363136323633363433343330333233443634363336323631333433303332011E363136323633363433343330333333443634363336323631333433303333011E363136323633363433343330333433443634363336323631333433303334011E363136323633363433343330333533443634363336323631333433303335011E363136323633363433343330333633443634363336323631333433303336011E363136323633363433343330333733443634363336323631333433303337011E363136323633363433343330333833443634363336323631333433303338011E363136323633363433343330333933443634363336323631333433303339011E363136323633363433343331333033443634363336323631333433313330011E363136323633363433343331333133443634363336323631333433313331011E363136323633363433343331333233443634363336323631333433313332011E363136323633363433343331333333443634363336323631333433313333011E363136323633363433343331333433443634363336323631333433313334011E363136323633363433343331333533443634363336323631333433313335011E363136323633363433343331333633443634363336323631333433313336011E363136323633363433343331333733443634363336323631333433313337011E363136323633363433343331333833443634363336323631333433313338011E363136323633363433343331333933443634363336323631333433313339011E363136323633363433343332333033443634363336323631333433323330011E363136323633363433343332333133443634363336323631333433323331011E363136323633363433343332333233443634363336323631333433323332011E363136323633363433343332333333443634363336323631333433323333011E363136323633363433343332333433443634363336323631333433323334011E363136323633363433343332333533443634363336323631333433323335011E363136323633363433343332333633443634363336323631333433323336011E363136323633363433343332333733443634363336323631333433323337011E363136323633363433343332333833443634363336323631333433323338011E363136323633363433343332333933443634363336323631333433323339011E363136323633363433343333333033443634363336323631333433333330011E363136323633363433343333333133443634363336323631333433333331011E363136323633363433343333333233443634363336323631333433333332011E363136323633363433343333333333443634363336323631333433333333011E363136323633363433343333333433443634363336323631333433333334011E363136323633363433343333333533443634363336323631333433333335011E363136323633363433343333333633443634363336323631333433333336011E363136323633363433343333333733443634363336323631333433333337011E363136323633363433343333333833443634363336323631333433333338011E363136323633363433343333333933443634363336323631333433333339011E363136323633363433343334333033443634363336323631333433343330011E363136323633363433343334333133443634363336323631333433343331011E363136323633363433343334333233443634363336323631333433343332011E363136323633363433343334333333443634363336323631333433343333011E363136323633363433343334333433443634363336323631333433343334011E363136323633363433343334333533443634363336323631333433343335011E363136323633363433343334333633443634363336323631333433343336011E363136323633363433343334333733443634363336323631333433343337011E363136323633363433343334333833443634363336323631333433343338011E363136323633363433343334333933443634363336323631333433343339011E363136323633363433343335333033443634363336323631333433353330011E363136323633363433343335333133443634363336323631333433353331011E363136323633363433343335333233443634363336323631333433353332011E363136323633363433343335333333443634363336323631333433353333011E363136323633363433343335333433443634363336323631333433353334011E363136323633363433343335333533443634363336323631333433353335011E363136323633363433343335333633443634363336323631333433353336011E363136323633363433343335333733443634363336323631333433353337011E363136323633363433343335333833443634363336323631333433353338011E363136323633363433343335333933443634363336323631333433353339011E363136323633363433343336333033443634363336323631333433363330011E363136323633363433343336333133443634363336323631333433363331011E363136323633363433343336333233443634363336323631333433363332011E363136323633363433343336333333443634363336323631333433363333011E363136323633363433343336333433443634363336323631333433363334011E363136323633363433343336333533443634363336323631333433363335011E363136323633363433343336333633443634363336323631333433363336011E363136323633363433343336333733443634363336323631333433363337011E363136323633363433343336333833443634363336323631333433363338011E363136323633363433343336333933443634363336323631333433363339011E363136323633363433343337333033443634363336323631333433373330011E363136323633363433343337333133443634363336323631333433373331011E363136323633363433343337333233443634363336323631333433373332011E363136323633363433343337333333443634363336323631333433373333011E363136323633363433343337333433443634363336323631333433373334011E363136323633363433343337333533443634363336323631333433373335011E363136323633363433343337333633443634363336323631333433373336011E363136323633363433343337333733443634363336323631333433373337011E363136323633363433343337333833443634363336323631333433373338011E363136323633363433343337333933443634363336323631333433373339011E363136323633363433343338333033443634363336323631333433383330011E363136323633363433343338333133443634363336323631333433383331011E363136323633363433343338333233443634363336323631333433383332011E363136323633363433343338333333443634363336323631333433383333011E363136323633363433343338333433443634363336323631333433383334011E363136323633363433343338333533443634363336323631333433383335011E36313632363336343334333833363344363436333632363133343338333601011437AF6866DA8C3167CFC280FAE47B6ED441B00D5B010101142E5DE5777A5AD899CD2531304F42A470509DE9890101010114D028C9981F7A87F3093672BF0D5B0E2A1B3ED4560001030002011437AF6866DA8C3167CFC280FAE47B6ED441B00D5B010101142E5DE5777A5AD899CD2531304F42A470509DE98901C900519E305EC03392E7D197D5FAB535DB240C9C0BA5375A1679C75BAAA07C7410C0EF43CF97D98F2C08A1D739667D5ACFF6233A1FAE75D3DA275AEA422EFD0F","proof":{"aunts":[]}}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:22.032Z","msg":[1,{"height":4,"round":0,"step":"RoundStepPrevote"}]} +{"time":"2017-02-17T23:54:22.032Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":4,"round":0,"type":1,"block_id":{"hash":"04715E223BF4327FFA9B0D5AD849B74A099D5DEC","parts":{"total":1,"hash":"24CEBCBEB833F56D47AD14354071B3B7A243068A"}},"signature":[1,"B1BFF3641FE1931C78A792540384B9D4CFC3D9008FD4988B24FAD872326C2A380A34F37610C6E076FA5B4DB9E4B3166B703B0429AF0BF1ABCCDB7B2EDB3C8F08"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:22.033Z","msg":[1,{"height":4,"round":0,"step":"RoundStepPrecommit"}]} +{"time":"2017-02-17T23:54:22.033Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":4,"round":0,"type":2,"block_id":{"hash":"04715E223BF4327FFA9B0D5AD849B74A099D5DEC","parts":{"total":1,"hash":"24CEBCBEB833F56D47AD14354071B3B7A243068A"}},"signature":[1,"F544743F17479A61F94B0F68C63D254BD60493D78E818D48A5859133619AEE5E92C47CAD89C654DF64E0911C3152091E047555D5F14655D95B9681AE9B336505"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:22.034Z","msg":[1,{"height":4,"round":0,"step":"RoundStepCommit"}]} +#HEIGHT: 5 +{"time":"2017-02-17T23:54:22.036Z","msg":[1,{"height":5,"round":0,"step":"RoundStepNewHeight"}]} +{"time":"2017-02-17T23:54:23.034Z","msg":[3,{"duration":997096276,"height":5,"round":0,"step":1}]} +{"time":"2017-02-17T23:54:23.035Z","msg":[1,{"height":5,"round":0,"step":"RoundStepPropose"}]} +{"time":"2017-02-17T23:54:23.035Z","msg":[2,{"msg":[17,{"Proposal":{"height":5,"round":0,"block_parts_header":{"total":1,"hash":"A52BAA9C2E52E633A1605F4B930205613E3E7A2F"},"pol_round":-1,"pol_block_id":{"hash":"","parts":{"total":0,"hash":""}},"signature":[1,"32A96AA44440B6FDB28B590A029649CE37B0F1091B9E648658E910207BB2F96E4936102C63F3908942F1A45F52C01231680593FB3E53B8B29BF588A613116A0B"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:23.035Z","msg":[2,{"msg":[19,{"Height":5,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F74657374010514A43848FAB3E280017A011404715E223BF4327FFA9B0D5AD849B74A099D5DEC0101011424CEBCBEB833F56D47AD14354071B3B7A243068A01144C1EF483E19426AC412E12F33E1D814FD019279801146440978BB85314393E824F6BBAFEE59FA1A5E30A0114354594CBFC1A7BCA1AD0050ED6AA010023EADA390114845C15D3FE4AA16021B85CD0F18E0188A673E73E01017A011E363136323633363433343338333733443634363336323631333433383337011E363136323633363433343338333833443634363336323631333433383338011E363136323633363433343338333933443634363336323631333433383339011E363136323633363433343339333033443634363336323631333433393330011E363136323633363433343339333133443634363336323631333433393331011E363136323633363433343339333233443634363336323631333433393332011E363136323633363433343339333333443634363336323631333433393333011E363136323633363433343339333433443634363336323631333433393334011E363136323633363433343339333533443634363336323631333433393335011E363136323633363433343339333633443634363336323631333433393336011E363136323633363433343339333733443634363336323631333433393337011E363136323633363433343339333833443634363336323631333433393338011E363136323633363433343339333933443634363336323631333433393339011E363136323633363433353330333033443634363336323631333533303330011E363136323633363433353330333133443634363336323631333533303331011E363136323633363433353330333233443634363336323631333533303332011E363136323633363433353330333333443634363336323631333533303333011E363136323633363433353330333433443634363336323631333533303334011E363136323633363433353330333533443634363336323631333533303335011E363136323633363433353330333633443634363336323631333533303336011E363136323633363433353330333733443634363336323631333533303337011E363136323633363433353330333833443634363336323631333533303338011E363136323633363433353330333933443634363336323631333533303339011E363136323633363433353331333033443634363336323631333533313330011E363136323633363433353331333133443634363336323631333533313331011E363136323633363433353331333233443634363336323631333533313332011E363136323633363433353331333333443634363336323631333533313333011E363136323633363433353331333433443634363336323631333533313334011E363136323633363433353331333533443634363336323631333533313335011E363136323633363433353331333633443634363336323631333533313336011E363136323633363433353331333733443634363336323631333533313337011E363136323633363433353331333833443634363336323631333533313338011E363136323633363433353331333933443634363336323631333533313339011E363136323633363433353332333033443634363336323631333533323330011E363136323633363433353332333133443634363336323631333533323331011E363136323633363433353332333233443634363336323631333533323332011E363136323633363433353332333333443634363336323631333533323333011E363136323633363433353332333433443634363336323631333533323334011E363136323633363433353332333533443634363336323631333533323335011E363136323633363433353332333633443634363336323631333533323336011E363136323633363433353332333733443634363336323631333533323337011E363136323633363433353332333833443634363336323631333533323338011E363136323633363433353332333933443634363336323631333533323339011E363136323633363433353333333033443634363336323631333533333330011E363136323633363433353333333133443634363336323631333533333331011E363136323633363433353333333233443634363336323631333533333332011E363136323633363433353333333333443634363336323631333533333333011E363136323633363433353333333433443634363336323631333533333334011E363136323633363433353333333533443634363336323631333533333335011E363136323633363433353333333633443634363336323631333533333336011E363136323633363433353333333733443634363336323631333533333337011E363136323633363433353333333833443634363336323631333533333338011E363136323633363433353333333933443634363336323631333533333339011E363136323633363433353334333033443634363336323631333533343330011E363136323633363433353334333133443634363336323631333533343331011E363136323633363433353334333233443634363336323631333533343332011E363136323633363433353334333333443634363336323631333533343333011E363136323633363433353334333433443634363336323631333533343334011E363136323633363433353334333533443634363336323631333533343335011E363136323633363433353334333633443634363336323631333533343336011E363136323633363433353334333733443634363336323631333533343337011E363136323633363433353334333833443634363336323631333533343338011E363136323633363433353334333933443634363336323631333533343339011E363136323633363433353335333033443634363336323631333533353330011E363136323633363433353335333133443634363336323631333533353331011E363136323633363433353335333233443634363336323631333533353332011E363136323633363433353335333333443634363336323631333533353333011E363136323633363433353335333433443634363336323631333533353334011E363136323633363433353335333533443634363336323631333533353335011E363136323633363433353335333633443634363336323631333533353336011E363136323633363433353335333733443634363336323631333533353337011E363136323633363433353335333833443634363336323631333533353338011E363136323633363433353335333933443634363336323631333533353339011E363136323633363433353336333033443634363336323631333533363330011E363136323633363433353336333133443634363336323631333533363331011E363136323633363433353336333233443634363336323631333533363332011E363136323633363433353336333333443634363336323631333533363333011E363136323633363433353336333433443634363336323631333533363334011E363136323633363433353336333533443634363336323631333533363335011E363136323633363433353336333633443634363336323631333533363336011E363136323633363433353336333733443634363336323631333533363337011E363136323633363433353336333833443634363336323631333533363338011E363136323633363433353336333933443634363336323631333533363339011E363136323633363433353337333033443634363336323631333533373330011E363136323633363433353337333133443634363336323631333533373331011E363136323633363433353337333233443634363336323631333533373332011E363136323633363433353337333333443634363336323631333533373333011E363136323633363433353337333433443634363336323631333533373334011E363136323633363433353337333533443634363336323631333533373335011E363136323633363433353337333633443634363336323631333533373336011E363136323633363433353337333733443634363336323631333533373337011E363136323633363433353337333833443634363336323631333533373338011E363136323633363433353337333933443634363336323631333533373339011E363136323633363433353338333033443634363336323631333533383330011E363136323633363433353338333133443634363336323631333533383331011E363136323633363433353338333233443634363336323631333533383332011E363136323633363433353338333333443634363336323631333533383333011E363136323633363433353338333433443634363336323631333533383334011E363136323633363433353338333533443634363336323631333533383335011E363136323633363433353338333633443634363336323631333533383336011E363136323633363433353338333733443634363336323631333533383337011E363136323633363433353338333833443634363336323631333533383338011E363136323633363433353338333933443634363336323631333533383339011E363136323633363433353339333033443634363336323631333533393330011E363136323633363433353339333133443634363336323631333533393331011E363136323633363433353339333233443634363336323631333533393332011E363136323633363433353339333333443634363336323631333533393333011E363136323633363433353339333433443634363336323631333533393334011E363136323633363433353339333533443634363336323631333533393335011E363136323633363433353339333633443634363336323631333533393336011E363136323633363433353339333733443634363336323631333533393337011E363136323633363433353339333833443634363336323631333533393338011E363136323633363433353339333933443634363336323631333533393339011E363136323633363433363330333033443634363336323631333633303330011E363136323633363433363330333133443634363336323631333633303331011E363136323633363433363330333233443634363336323631333633303332011E363136323633363433363330333333443634363336323631333633303333011E363136323633363433363330333433443634363336323631333633303334011E363136323633363433363330333533443634363336323631333633303335011E363136323633363433363330333633443634363336323631333633303336011E363136323633363433363330333733443634363336323631333633303337011E36313632363336343336333033383344363436333632363133363330333801011404715E223BF4327FFA9B0D5AD849B74A099D5DEC0101011424CEBCBEB833F56D47AD14354071B3B7A243068A0101010114D028C9981F7A87F3093672BF0D5B0E2A1B3ED4560001040002011404715E223BF4327FFA9B0D5AD849B74A099D5DEC0101011424CEBCBEB833F56D47AD14354071B3B7A243068A01F544743F17479A61F94B0F68C63D254BD60493D78E818D48A5859133619AEE5E92C47CAD89C654DF64E0911C3152091E047555D5F14655D95B9681AE9B336505","proof":{"aunts":[]}}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:23.037Z","msg":[1,{"height":5,"round":0,"step":"RoundStepPrevote"}]} +{"time":"2017-02-17T23:54:23.037Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":5,"round":0,"type":1,"block_id":{"hash":"FDC6D837995BEBBBFCBF3E7D7CF44F8FDA448543","parts":{"total":1,"hash":"A52BAA9C2E52E633A1605F4B930205613E3E7A2F"}},"signature":[1,"684AB4918389E06ADD5DCC4EFCCD0464EAE2BC4212344D88694706837A4D47D484747C7B5906537181E0FBD35EF78EDF673E8492C6E875BB33934456A8254B03"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:23.038Z","msg":[1,{"height":5,"round":0,"step":"RoundStepPrecommit"}]} +{"time":"2017-02-17T23:54:23.038Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":5,"round":0,"type":2,"block_id":{"hash":"FDC6D837995BEBBBFCBF3E7D7CF44F8FDA448543","parts":{"total":1,"hash":"A52BAA9C2E52E633A1605F4B930205613E3E7A2F"}},"signature":[1,"DF51D23D5D2C57598F67791D953A6C2D9FC5865A3048ADA4469B37500D2996B95732E0DC6F99EAEAEA12B4818CE355C7B701D16857D2AC767D740C2E30E9260C"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:23.038Z","msg":[1,{"height":5,"round":0,"step":"RoundStepCommit"}]} +#HEIGHT: 6 +{"time":"2017-02-17T23:54:23.041Z","msg":[1,{"height":6,"round":0,"step":"RoundStepNewHeight"}]} +{"time":"2017-02-17T23:54:24.038Z","msg":[3,{"duration":997341910,"height":6,"round":0,"step":1}]} +{"time":"2017-02-17T23:54:24.040Z","msg":[1,{"height":6,"round":0,"step":"RoundStepPropose"}]} +{"time":"2017-02-17T23:54:24.040Z","msg":[2,{"msg":[17,{"Proposal":{"height":6,"round":0,"block_parts_header":{"total":1,"hash":"EA1E4111198195006BF7C23322B1051BE6C11582"},"pol_round":-1,"pol_block_id":{"hash":"","parts":{"total":0,"hash":""}},"signature":[1,"571309E5959472CF453B83BB00F75BFAE9ACA8981279CBCBF19FD1A104BAD544D43A4F67FC54C17C9D51CEE821E4F514A1742FA5220EFF432C334D81B03B4C08"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:24.040Z","msg":[2,{"msg":[19,{"Height":6,"Round":0,"Part":{"index":0,"bytes":"0101010F74656E6465726D696E745F74657374010614A43849369AF7C0018C0114FDC6D837995BEBBBFCBF3E7D7CF44F8FDA44854301010114A52BAA9C2E52E633A1605F4B930205613E3E7A2F0114AD3EFC80659CF2DCB932CAAA832249B1100C7D0901149D3F259328643A0DA0B83CF23A1AC918965E393D0114354594CBFC1A7BCA1AD0050ED6AA010023EADA390114BFB5455B51A6694370771F5072E0204CE75E6C6901018C011E363136323633363433363330333933443634363336323631333633303339011E363136323633363433363331333033443634363336323631333633313330011E363136323633363433363331333133443634363336323631333633313331011E363136323633363433363331333233443634363336323631333633313332011E363136323633363433363331333333443634363336323631333633313333011E363136323633363433363331333433443634363336323631333633313334011E363136323633363433363331333533443634363336323631333633313335011E363136323633363433363331333633443634363336323631333633313336011E363136323633363433363331333733443634363336323631333633313337011E363136323633363433363331333833443634363336323631333633313338011E363136323633363433363331333933443634363336323631333633313339011E363136323633363433363332333033443634363336323631333633323330011E363136323633363433363332333133443634363336323631333633323331011E363136323633363433363332333233443634363336323631333633323332011E363136323633363433363332333333443634363336323631333633323333011E363136323633363433363332333433443634363336323631333633323334011E363136323633363433363332333533443634363336323631333633323335011E363136323633363433363332333633443634363336323631333633323336011E363136323633363433363332333733443634363336323631333633323337011E363136323633363433363332333833443634363336323631333633323338011E363136323633363433363332333933443634363336323631333633323339011E363136323633363433363333333033443634363336323631333633333330011E363136323633363433363333333133443634363336323631333633333331011E363136323633363433363333333233443634363336323631333633333332011E363136323633363433363333333333443634363336323631333633333333011E363136323633363433363333333433443634363336323631333633333334011E363136323633363433363333333533443634363336323631333633333335011E363136323633363433363333333633443634363336323631333633333336011E363136323633363433363333333733443634363336323631333633333337011E363136323633363433363333333833443634363336323631333633333338011E363136323633363433363333333933443634363336323631333633333339011E363136323633363433363334333033443634363336323631333633343330011E363136323633363433363334333133443634363336323631333633343331011E363136323633363433363334333233443634363336323631333633343332011E363136323633363433363334333333443634363336323631333633343333011E363136323633363433363334333433443634363336323631333633343334011E363136323633363433363334333533443634363336323631333633343335011E363136323633363433363334333633443634363336323631333633343336011E363136323633363433363334333733443634363336323631333633343337011E363136323633363433363334333833443634363336323631333633343338011E363136323633363433363334333933443634363336323631333633343339011E363136323633363433363335333033443634363336323631333633353330011E363136323633363433363335333133443634363336323631333633353331011E363136323633363433363335333233443634363336323631333633353332011E363136323633363433363335333333443634363336323631333633353333011E363136323633363433363335333433443634363336323631333633353334011E363136323633363433363335333533443634363336323631333633353335011E363136323633363433363335333633443634363336323631333633353336011E363136323633363433363335333733443634363336323631333633353337011E363136323633363433363335333833443634363336323631333633353338011E363136323633363433363335333933443634363336323631333633353339011E363136323633363433363336333033443634363336323631333633363330011E363136323633363433363336333133443634363336323631333633363331011E363136323633363433363336333233443634363336323631333633363332011E363136323633363433363336333333443634363336323631333633363333011E363136323633363433363336333433443634363336323631333633363334011E363136323633363433363336333533443634363336323631333633363335011E363136323633363433363336333633443634363336323631333633363336011E363136323633363433363336333733443634363336323631333633363337011E363136323633363433363336333833443634363336323631333633363338011E363136323633363433363336333933443634363336323631333633363339011E363136323633363433363337333033443634363336323631333633373330011E363136323633363433363337333133443634363336323631333633373331011E363136323633363433363337333233443634363336323631333633373332011E363136323633363433363337333333443634363336323631333633373333011E363136323633363433363337333433443634363336323631333633373334011E363136323633363433363337333533443634363336323631333633373335011E363136323633363433363337333633443634363336323631333633373336011E363136323633363433363337333733443634363336323631333633373337011E363136323633363433363337333833443634363336323631333633373338011E363136323633363433363337333933443634363336323631333633373339011E363136323633363433363338333033443634363336323631333633383330011E363136323633363433363338333133443634363336323631333633383331011E363136323633363433363338333233443634363336323631333633383332011E363136323633363433363338333333443634363336323631333633383333011E363136323633363433363338333433443634363336323631333633383334011E363136323633363433363338333533443634363336323631333633383335011E363136323633363433363338333633443634363336323631333633383336011E363136323633363433363338333733443634363336323631333633383337011E363136323633363433363338333833443634363336323631333633383338011E363136323633363433363338333933443634363336323631333633383339011E363136323633363433363339333033443634363336323631333633393330011E363136323633363433363339333133443634363336323631333633393331011E363136323633363433363339333233443634363336323631333633393332011E363136323633363433363339333333443634363336323631333633393333011E363136323633363433363339333433443634363336323631333633393334011E363136323633363433363339333533443634363336323631333633393335011E363136323633363433363339333633443634363336323631333633393336011E363136323633363433363339333733443634363336323631333633393337011E363136323633363433363339333833443634363336323631333633393338011E363136323633363433363339333933443634363336323631333633393339011E363136323633363433373330333033443634363336323631333733303330011E363136323633363433373330333133443634363336323631333733303331011E363136323633363433373330333233443634363336323631333733303332011E363136323633363433373330333333443634363336323631333733303333011E363136323633363433373330333433443634363336323631333733303334011E363136323633363433373330333533443634363336323631333733303335011E363136323633363433373330333633443634363336323631333733303336011E363136323633363433373330333733443634363336323631333733303337011E363136323633363433373330333833443634363336323631333733303338011E363136323633363433373330333933443634363336323631333733303339011E363136323633363433373331333033443634363336323631333733313330011E363136323633363433373331333133443634363336323631333733313331011E363136323633363433373331333233443634363336323631333733313332011E363136323633363433373331333333443634363336323631333733313333011E363136323633363433373331333433443634363336323631333733313334011E363136323633363433373331333533443634363336323631333733313335011E363136323633363433373331333633443634363336323631333733313336011E363136323633363433373331333733443634363336323631333733313337011E363136323633363433373331333833443634363336323631333733313338011E363136323633363433373331333933443634363336323631333733313339011E363136323633363433373332333033443634363336323631333733323330011E363136323633363433373332333133443634363336323631333733323331011E363136323633363433373332333233443634363336323631333733323332011E363136323633363433373332333333443634363336323631333733323333011E363136323633363433373332333433443634363336323631333733323334011E363136323633363433373332333533443634363336323631333733323335011E363136323633363433373332333633443634363336323631333733323336011E363136323633363433373332333733443634363336323631333733323337011E363136323633363433373332333833443634363336323631333733323338011E363136323633363433373332333933443634363336323631333733323339011E363136323633363433373333333033443634363336323631333733333330011E363136323633363433373333333133443634363336323631333733333331011E363136323633363433373333333233443634363336323631333733333332011E363136323633363433373333333333443634363336323631333733333333011E363136323633363433373333333433443634363336323631333733333334011E363136323633363433373333333533443634363336323631333733333335011E363136323633363433373333333633443634363336323631333733333336011E363136323633363433373333333733443634363336323631333733333337011E363136323633363433373333333833443634363336323631333733333338011E363136323633363433373333333933443634363336323631333733333339011E363136323633363433373334333033443634363336323631333733343330011E363136323633363433373334333133443634363336323631333733343331011E363136323633363433373334333233443634363336323631333733343332011E363136323633363433373334333333443634363336323631333733343333011E363136323633363433373334333433443634363336323631333733343334011E363136323633363433373334333533443634363336323631333733343335011E363136323633363433373334333633443634363336323631333733343336011E363136323633363433373334333733443634363336323631333733343337011E363136323633363433373334333833443634363336323631333733343338010114FDC6D837995BEBBBFCBF3E7D7CF44F8FDA44854301010114A52BAA9C2E52E633A1605F4B930205613E3E7A2F0101010114D028C9981F7A87F3093672BF0D5B0E2A1B3ED45600010500020114FDC6D837995BEBBBFCBF3E7D7CF44F8FDA44854301010114A52BAA9C2E52E633A1605F4B930205613E3E7A2F01DF51D23D5D2C57598F67791D953A6C2D9FC5865A3048ADA4469B37500D2996B95732E0DC6F99EAEAEA12B4818CE355C7B701D16857D2AC767D740C2E30E9260C","proof":{"aunts":[]}}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:24.041Z","msg":[1,{"height":6,"round":0,"step":"RoundStepPrevote"}]} +{"time":"2017-02-17T23:54:24.041Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":6,"round":0,"type":1,"block_id":{"hash":"1F7C249FF99B67AC57C4EEC94C42E0B95C9AFB6B","parts":{"total":1,"hash":"EA1E4111198195006BF7C23322B1051BE6C11582"}},"signature":[1,"1F79910354E1F4ACA11FC16DBA1ED6F75063A15BF8093C4AAEF87F69B3990F65E51FFC8B35A409838ECD0FF3C26E87637B068D0DC7E5863D5F1CF97826222300"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:24.042Z","msg":[1,{"height":6,"round":0,"step":"RoundStepPrecommit"}]} +{"time":"2017-02-17T23:54:24.042Z","msg":[2,{"msg":[20,{"Vote":{"validator_address":"D028C9981F7A87F3093672BF0D5B0E2A1B3ED456","validator_index":0,"height":6,"round":0,"type":2,"block_id":{"hash":"1F7C249FF99B67AC57C4EEC94C42E0B95C9AFB6B","parts":{"total":1,"hash":"EA1E4111198195006BF7C23322B1051BE6C11582"}},"signature":[1,"E7838F403E4D5E651317D8563355CB2E140409B1B471B4AC12EBF7085989228CFA062029DF78A405CF977925777177D876804D78D80DF2312977E6D804394A0E"]}}],"peer_key":""}]} +{"time":"2017-02-17T23:54:24.042Z","msg":[1,{"height":6,"round":0,"step":"RoundStepCommit"}]} diff --git a/consensus/wal.go b/consensus/wal.go index 099e3c1aa..99035ee2e 100644 --- a/consensus/wal.go +++ b/consensus/wal.go @@ -40,8 +40,8 @@ type WAL struct { light bool // ignore block parts } -func NewWAL(walDir string, light bool) (*WAL, error) { - group, err := auto.OpenGroup(walDir + "/wal") +func NewWAL(walFile string, light bool) (*WAL, error) { + group, err := auto.OpenGroup(walFile) if err != nil { return nil, err } diff --git a/state/execution.go b/state/execution.go index 88b80e485..3ae0a9d33 100644 --- a/state/execution.go +++ b/state/execution.go @@ -1,6 +1,7 @@ package state import ( + "bytes" "errors" "github.com/ebuchman/fail-test" @@ -278,7 +279,7 @@ func applyBlock(appConnConsensus proxy.AppConnConsensus, block *types.Block) ([] var eventCache types.Fireable // nil _, err := execBlockOnProxyApp(eventCache, appConnConsensus, block) if err != nil { - log.Warn("Error executing block on proxy app", "height", i, "err", err) + log.Warn("Error executing block on proxy app", "height", block.Height, "err", err) return nil, err } // Commit block, get hash back @@ -388,13 +389,17 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p // if the app is ahead, there's nothing we can do return ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight} + } else if storeBlockHeight == appBlockHeight && storeBlockHeight == stateBlockHeight { + // all good! + return nil + } else if storeBlockHeight == appBlockHeight && storeBlockHeight == stateBlockHeight+1 { // We already ran Commit, but didn't save the state, so run through consensus with mock app mockApp := newMockProxyApp(appHash) log.Info("Replay last block using mock app") h.replayLastBlock(h.config, h.state, mockApp, h.store) - } else if storeBlockHeight == appBlockHeight+1 { + } else if storeBlockHeight == appBlockHeight+1 && storeBlockHeight == stateBlockHeight+1 { // We crashed after saving the block // but before Commit (both the state and app are behind), // so run through consensus with the real app @@ -409,7 +414,8 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p var appHash []byte var err error - for i := appBlockHeight + 1; i <= storeBlockHeight-1; i++ { + for i := appBlockHeight + 1; i <= storeBlockHeight; i++ { + log.Info("Applying block", "height", i) h.nBlocks += 1 block := h.store.LoadBlock(i) appHash, err = applyBlock(proxyApp.Consensus(), block) @@ -418,7 +424,7 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p } } - h.replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) + // h.replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) if !bytes.Equal(h.state.AppHash, appHash) { return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) } From 7228b11e3f63d27046b76cb95d89110c2c3e0ef2 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 17 Feb 2017 19:13:35 -0500 Subject: [PATCH 06/14] state: remove StateIntermediate --- state/execution.go | 4 ---- state/state.go | 32 +------------------------------- 2 files changed, 1 insertion(+), 35 deletions(-) diff --git a/state/execution.go b/state/execution.go index 3ae0a9d33..e03001da2 100644 --- a/state/execution.go +++ b/state/execution.go @@ -54,10 +54,6 @@ func (s *State) ExecBlock(eventCache types.Fireable, proxyAppConn proxy.AppConnC nextValSet.IncrementAccum(1) s.SetBlockAndValidators(block.Header, blockPartsHeader, valSet, nextValSet) - // save state with updated height/blockhash/validators - // but stale apphash, in case we fail between Commit and Save - s.SaveIntermediate() - fail.Fail() // XXX return nil diff --git a/state/state.go b/state/state.go index f4af8ee01..c4c6d7489 100644 --- a/state/state.go +++ b/state/state.go @@ -14,8 +14,7 @@ import ( ) var ( - stateKey = []byte("stateKey") - stateIntermediateKey = []byte("stateIntermediateKey") + stateKey = []byte("stateKey") ) //----------------------------------------------------------------------------- @@ -82,35 +81,6 @@ func (s *State) Save() { s.db.SetSync(stateKey, s.Bytes()) } -func (s *State) SaveIntermediate() { - s.mtx.Lock() - defer s.mtx.Unlock() - s.db.SetSync(stateIntermediateKey, s.Bytes()) -} - -// Load the intermediate state into the current state -// and do some sanity checks -func (s *State) LoadIntermediate() { - s2 := loadState(s.db, stateIntermediateKey) - if s.ChainID != s2.ChainID { - PanicSanity(Fmt("State mismatch for ChainID. Got %v, Expected %v", s2.ChainID, s.ChainID)) - } - - if s.LastBlockHeight+1 != s2.LastBlockHeight { - PanicSanity(Fmt("State mismatch for LastBlockHeight. Got %v, Expected %v", s2.LastBlockHeight, s.LastBlockHeight+1)) - } - - if !bytes.Equal(s.Validators.Hash(), s2.LastValidators.Hash()) { - PanicSanity(Fmt("State mismatch for LastValidators. Got %X, Expected %X", s2.LastValidators.Hash(), s.Validators.Hash())) - } - - if !bytes.Equal(s.AppHash, s2.AppHash) { - PanicSanity(Fmt("State mismatch for AppHash. Got %X, Expected %X", s2.AppHash, s.AppHash)) - } - - s.setBlockAndValidators(s2.LastBlockHeight, s2.LastBlockID, s2.LastBlockTime, s2.Validators.Copy(), s2.LastValidators.Copy()) -} - func (s *State) Equals(s2 *State) bool { return bytes.Equal(s.Bytes(), s2.Bytes()) } From 44d472ddd3e5c7048deeb5022533abc28bc2c799 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Sat, 18 Feb 2017 22:15:59 -0500 Subject: [PATCH 07/14] comments from review --- consensus/state.go | 1 + state/execution.go | 6 ++++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/consensus/state.go b/consensus/state.go index a985fd2c8..1c864c6d1 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -259,6 +259,7 @@ func ReplayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnC evsw := types.NewEventSwitch() evsw.Start() + defer evsw.Stop() cs.SetEventSwitch(evsw) newBlockCh := subscribeToEvent(evsw, "consensus-replay", types.EventStringNewBlock(), 1) diff --git a/state/execution.go b/state/execution.go index e03001da2..71c2dd759 100644 --- a/state/execution.go +++ b/state/execution.go @@ -269,8 +269,8 @@ func (s *State) CommitStateUpdateMempool(proxyAppConn proxy.AppConnConsensus, bl return nil } -// apply a nd commit a block, but with out all the state validation -// returns the application root hash (result of abci.Commit) +// Apply and commit a block, but without all the state validation. +// Returns the application root hash (result of abci.Commit) func applyBlock(appConnConsensus proxy.AppConnConsensus, block *types.Block) ([]byte, error) { var eventCache types.Fireable // nil _, err := execBlockOnProxyApp(eventCache, appConnConsensus, block) @@ -420,7 +420,9 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p } } + // TODO: should we be playing the final block through the consensus, instead of using applyBlock? // h.replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) + if !bytes.Equal(h.state.AppHash, appHash) { return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) } From bc678596729376465f56ac0a339a6799eef38e0a Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 20 Feb 2017 16:24:35 -0500 Subject: [PATCH 08/14] make ReplayBlocks logic exhaustive --- consensus/replay_test.go | 27 +++++--- consensus/state.go | 11 ++-- state/execution.go | 139 ++++++++++++++++++++++++++------------- 3 files changed, 118 insertions(+), 59 deletions(-) diff --git a/consensus/replay_test.go b/consensus/replay_test.go index a9123a8e4..e8aa9e903 100644 --- a/consensus/replay_test.go +++ b/consensus/replay_test.go @@ -15,7 +15,7 @@ import ( "github.com/tendermint/tendermint/config/tendermint_test" "github.com/tendermint/abci/example/dummy" - . "github.com/tendermint/go-common" + cmn "github.com/tendermint/go-common" cfg "github.com/tendermint/go-config" "github.com/tendermint/go-crypto" dbm "github.com/tendermint/go-db" @@ -37,7 +37,7 @@ func init() { // the `Handshake Tests` are for failures in applying the block. // With the help of the WAL, we can recover from it all! -var data_dir = path.Join(GoPath, "src/github.com/tendermint/tendermint/consensus", "test_data") +var data_dir = path.Join(cmn.GoPath, "src/github.com/tendermint/tendermint/consensus", "test_data") //------------------------------------------------------------------------------------------ // WAL Tests @@ -68,7 +68,7 @@ type testCase struct { func newTestCase(name string, stepChanges []int) *testCase { if len(stepChanges) != 3 { - panic(Fmt("a full wal has 3 step changes! Got array %v", stepChanges)) + panic(cmn.Fmt("a full wal has 3 step changes! Got array %v", stepChanges)) } return &testCase{ name: name, @@ -103,15 +103,15 @@ func readWAL(p string) string { func writeWAL(walMsgs string) string { tempDir := os.TempDir() - walDir := path.Join(tempDir, "/wal"+RandStr(12)) + walDir := path.Join(tempDir, "/wal"+cmn.RandStr(12)) walFile := path.Join(walDir, "wal") // Create WAL directory - err := EnsureDir(walDir, 0700) + err := cmn.EnsureDir(walDir, 0700) if err != nil { panic(err) } // Write the needed WAL to file - err = WriteFile(walFile, []byte(walMsgs), 0600) + err = cmn.WriteFile(walFile, []byte(walMsgs), 0600) if err != nil { panic(err) } @@ -123,7 +123,7 @@ func waitForBlock(newBlockCh chan interface{}, thisCase *testCase, i int) { select { case <-newBlockCh: case <-after: - panic(Fmt("Timed out waiting for new block for case '%s' line %d", thisCase.name, i)) + panic(cmn.Fmt("Timed out waiting for new block for case '%s' line %d", thisCase.name, i)) } } @@ -156,7 +156,7 @@ func toPV(pv PrivValidator) *types.PrivValidator { func setupReplayTest(thisCase *testCase, nLines int, crashAfter bool) (*ConsensusState, chan interface{}, string, string) { fmt.Println("-------------------------------------") - log.Notice(Fmt("Starting replay test %v (of %d lines of WAL). Crash after = %v", thisCase.name, nLines, crashAfter)) + log.Notice(cmn.Fmt("Starting replay test %v (of %d lines of WAL). Crash after = %v", thisCase.name, nLines, crashAfter)) lineStep := nLines if crashAfter { @@ -285,8 +285,15 @@ func TestHandshakeReplayNone(t *testing.T) { // Make some blocks. Start a fresh app and apply nBlocks blocks. Then restart the app and sync it up with the remaining blocks func testHandshakeReplay(t *testing.T, nBlocks int) { config := tendermint_test.ResetConfig("proxy_test_") - walFile := path.Join(data_dir, "many_blocks.cswal") + + // copy the many_blocks file + walBody, err := cmn.ReadFile(path.Join(data_dir, "many_blocks.cswal")) + if err != nil { + t.Fatal(err) + } + walFile := writeWAL(string(walBody)) config.Set("cs_wal_file", walFile) + privVal := types.LoadPrivValidator(config.GetString("priv_validator_file")) testPartSize = config.GetInt("block_part_size") @@ -371,7 +378,7 @@ func makeBlockchainFromWAL(wal *WAL) ([]*types.Block, []*types.Commit, error) { return nil, nil, err } if !found { - return nil, nil, errors.New(Fmt("WAL does not contain height %d.", 1)) + return nil, nil, errors.New(cmn.Fmt("WAL does not contain height %d.", 1)) } defer gr.Close() diff --git a/consensus/state.go b/consensus/state.go index 1c864c6d1..c077c412c 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -253,7 +253,8 @@ type ConsensusState struct { done chan struct{} } -func ReplayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnConsensus, blockStore sm.BlockStore) { +// Replay the last block through the consensus and return the AppHash after commit. +func ReplayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnConsensus, blockStore sm.BlockStore) ([]byte, error) { mempool := sm.MockMempool{} cs := NewConsensusState(config, state, proxyApp, blockStore, mempool) @@ -265,8 +266,10 @@ func ReplayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnC // run through the WAL, commit new block, stop cs.Start() - <-newBlockCh + <-newBlockCh // TODO: use a timeout and return err? cs.Stop() + + return cs.state.AppHash, nil } func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConnConsensus, blockStore sm.BlockStore, mempool sm.Mempool) *ConsensusState { @@ -1235,7 +1238,7 @@ func (cs *ConsensusState) finalizeCommit(height int) { cs.blockStore.SaveBlock(block, blockParts, seenCommit) } else { // Happens during replay if we already saved the block but didn't commit - log.Notice("Calling finalizeCommit on already stored block", "height", block.Height) + log.Info("Calling finalizeCommit on already stored block", "height", block.Height) } fail.Fail() // XXX @@ -1250,7 +1253,7 @@ func (cs *ConsensusState) finalizeCommit(height int) { // NOTE: the block.AppHash wont reflect these txs until the next block err := stateCopy.ApplyBlock(eventCache, cs.proxyAppConn, block, blockParts.Header(), cs.mempool) if err != nil { - log.Warn("Error on ApplyBlock. Did the application crash? Please restart tendermint", "error", err) + log.Error("Error on ApplyBlock. Did the application crash? Please restart tendermint", "error", err) return } diff --git a/state/execution.go b/state/execution.go index 71c2dd759..7677a672a 100644 --- a/state/execution.go +++ b/state/execution.go @@ -260,6 +260,7 @@ func (s *State) CommitStateUpdateMempool(proxyAppConn proxy.AppConnConsensus, bl log.Debug("Commit.Log: " + res.Log) } + log.Info("Committed state", "hash", res.Data) // Set the state's new AppHash s.AppHash = res.Data @@ -327,7 +328,8 @@ type BlockStore interface { LoadSeenCommit(height int) *types.Commit } -type blockReplayFunc func(cfg.Config, *State, proxy.AppConnConsensus, BlockStore) +// returns the apphash from Commit +type blockReplayFunc func(cfg.Config, *State, proxy.AppConnConsensus, BlockStore) ([]byte, error) type Handshaker struct { config cfg.Config @@ -362,73 +364,120 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error { // TODO: check version // replay blocks up to the latest in the blockstore - err = h.ReplayBlocks(appHash, blockHeight, proxyApp) + appHash, err = h.ReplayBlocks(appHash, blockHeight, proxyApp) if err != nil { return errors.New(Fmt("Error on replay: %v", err)) } + // NOTE: the h.state may now be behind the cs state + // TODO: (on restart) replay mempool return nil } // Replay all blocks after blockHeight and ensure the result matches the current state. -func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp proxy.AppConns) error { +// Returns the final AppHash or an error +func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp proxy.AppConns) ([]byte, error) { storeBlockHeight := h.store.Height() stateBlockHeight := h.state.LastBlockHeight log.Notice("ABCI Replay Blocks", "appHeight", appBlockHeight, "storeHeight", storeBlockHeight, "stateHeight", stateBlockHeight) + // First handle edge cases and constraints on the storeBlockHeight if storeBlockHeight == 0 { - return nil + return nil, nil + } else if storeBlockHeight < appBlockHeight { - // if the app is ahead, there's nothing we can do - return ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight} - - } else if storeBlockHeight == appBlockHeight && storeBlockHeight == stateBlockHeight { - // all good! - return nil - - } else if storeBlockHeight == appBlockHeight && storeBlockHeight == stateBlockHeight+1 { - // We already ran Commit, but didn't save the state, so run through consensus with mock app - mockApp := newMockProxyApp(appHash) - log.Info("Replay last block using mock app") - h.replayLastBlock(h.config, h.state, mockApp, h.store) - - } else if storeBlockHeight == appBlockHeight+1 && storeBlockHeight == stateBlockHeight+1 { - // We crashed after saving the block - // but before Commit (both the state and app are behind), - // so run through consensus with the real app - log.Info("Replay last block using real app") - h.replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) + // the app should never be ahead of the store (but this is under app's control) + return nil, ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight} - } else { - // store is more than one ahead, - // so app wants to replay many blocks. - // replay all blocks from appBlockHeight+1 to storeBlockHeight-1. - // Replay the final block through consensus - - var appHash []byte - var err error - for i := appBlockHeight + 1; i <= storeBlockHeight; i++ { - log.Info("Applying block", "height", i) - h.nBlocks += 1 - block := h.store.LoadBlock(i) - appHash, err = applyBlock(proxyApp.Consensus(), block) - if err != nil { - return err - } + } else if storeBlockHeight < stateBlockHeight { + // the state should never be ahead of the store (this is under tendermint's control) + PanicSanity(Fmt("StateBlockHeight (%d) > StoreBlockHeight (%d)", stateBlockHeight, storeBlockHeight)) + + } else if storeBlockHeight > stateBlockHeight+1 { + // store should be at most one ahead of the state (this is under tendermint's control) + PanicSanity(Fmt("StoreBlockHeight (%d) > StateBlockHeight + 1 (%d)", storeBlockHeight, stateBlockHeight+1)) + } + + // Now either store is equal to state, or one ahead. + // For each, consider all cases of where the app could be, given app <= store + if storeBlockHeight == stateBlockHeight { + // Tendermint ran Commit and saved the state. + // Either the app is asking for replay, or we're all synced up. + if appBlockHeight < storeBlockHeight { + // the app is behind, so replay blocks, but no need to go through WAL (state is already synced to store) + return h.replayBlocks(proxyApp, appBlockHeight, storeBlockHeight, false) + + } else if appBlockHeight == storeBlockHeight { + // we're good! + return appHash, nil + } + + } else if storeBlockHeight == stateBlockHeight+1 { + // We saved the block in the store but haven't updated the state, + // so we'll need to replay a block using the WAL. + if appBlockHeight < stateBlockHeight { + // the app is further behind than it should be, so replay blocks + // but leave the last block to go through the WAL + return h.replayBlocks(proxyApp, appBlockHeight, storeBlockHeight, true) + + } else if appBlockHeight == stateBlockHeight { + // We haven't run Commit (both the state and app are one block behind), + // so run through consensus with the real app + log.Info("Replay last block using real app") + return h.replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) + + } else if appBlockHeight == storeBlockHeight { + // We ran Commit, but didn't save the state, so run through consensus with mock app + mockApp := newMockProxyApp(appHash) + log.Info("Replay last block using mock app") + return h.replayLastBlock(h.config, h.state, mockApp, h.store) } - // TODO: should we be playing the final block through the consensus, instead of using applyBlock? - // h.replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) + } + + PanicSanity("Should never happen") + return nil, nil +} - if !bytes.Equal(h.state.AppHash, appHash) { - return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) +func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, storeBlockHeight int, useReplayFunc bool) ([]byte, error) { + // App is further behind than it should be, so we need to replay blocks. + // We replay all blocks from appBlockHeight+1 to storeBlockHeight-1, + // and let the final block be replayed through ReplayBlocks. + // Note that we don't have an old version of the state, + // so we by-pass state validation using applyBlock here. + + var appHash []byte + var err error + finalBlock := storeBlockHeight + if useReplayFunc { + finalBlock -= 1 + } + for i := appBlockHeight + 1; i <= finalBlock; i++ { + log.Info("Applying block", "height", i) + h.nBlocks += 1 + block := h.store.LoadBlock(i) + appHash, err = applyBlock(proxyApp.Consensus(), block) + if err != nil { + return nil, err } - return nil } - return nil + + if useReplayFunc { + // sync the final block + appHash, err = h.ReplayBlocks(appHash, finalBlock, proxyApp) + if err != nil { + return appHash, err + } + } + + if !bytes.Equal(h.state.AppHash, appHash) { + return nil, errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) + } + + return appHash, nil } //-------------------------------------------------------------------------------- From 756213c5f509425e9a253d707873cbe8504b7a28 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 20 Feb 2017 17:08:38 -0500 Subject: [PATCH 09/14] check appHash --- consensus/state.go | 2 +- state/execution.go | 21 +++++++++++---------- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/consensus/state.go b/consensus/state.go index c077c412c..8e721ed3e 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -253,7 +253,7 @@ type ConsensusState struct { done chan struct{} } -// Replay the last block through the consensus and return the AppHash after commit. +// Replay the last block through the consensus and return the AppHash from after Commit. func ReplayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnConsensus, blockStore sm.BlockStore) ([]byte, error) { mempool := sm.MockMempool{} cs := NewConsensusState(config, state, proxyApp, blockStore, mempool) diff --git a/state/execution.go b/state/execution.go index 7677a672a..c1471bf86 100644 --- a/state/execution.go +++ b/state/execution.go @@ -364,19 +364,17 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error { // TODO: check version // replay blocks up to the latest in the blockstore - appHash, err = h.ReplayBlocks(appHash, blockHeight, proxyApp) + _, err = h.ReplayBlocks(appHash, blockHeight, proxyApp) if err != nil { return errors.New(Fmt("Error on replay: %v", err)) } - // NOTE: the h.state may now be behind the cs state - // TODO: (on restart) replay mempool return nil } -// Replay all blocks after blockHeight and ensure the result matches the current state. +// Replay all blocks since appBlockHeight and ensure the result matches the current state. // Returns the final AppHash or an error func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp proxy.AppConns) ([]byte, error) { @@ -386,11 +384,11 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p // First handle edge cases and constraints on the storeBlockHeight if storeBlockHeight == 0 { - return nil, nil + return appHash, h.checkAppHash(appHash) } else if storeBlockHeight < appBlockHeight { // the app should never be ahead of the store (but this is under app's control) - return nil, ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight} + return appHash, ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight} } else if storeBlockHeight < stateBlockHeight { // the state should never be ahead of the store (this is under tendermint's control) @@ -412,7 +410,7 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p } else if appBlockHeight == storeBlockHeight { // we're good! - return appHash, nil + return appHash, h.checkAppHash(appHash) } } else if storeBlockHeight == stateBlockHeight+1 { @@ -473,11 +471,14 @@ func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, store } } + return appHash, h.checkAppHash(appHash) +} + +func (h *Handshaker) checkAppHash(appHash []byte) error { if !bytes.Equal(h.state.AppHash, appHash) { - return nil, errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) + return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) } - - return appHash, nil + return nil } //-------------------------------------------------------------------------------- From 2b1b8da58d9a07defc5d97baff6741d61bba397b Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 20 Feb 2017 18:41:29 -0500 Subject: [PATCH 10/14] test/persist: dont use log files on circle --- test/persist/test_failure_indices.sh | 22 +++++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/test/persist/test_failure_indices.sh b/test/persist/test_failure_indices.sh index d828afe3f..b24412352 100644 --- a/test/persist/test_failure_indices.sh +++ b/test/persist/test_failure_indices.sh @@ -6,19 +6,35 @@ export TMROOT=$HOME/.tendermint_persist rm -rf $TMROOT tendermint init +TM_CMD="tendermint node --log_level=debug" # &> tendermint_${name}.log" +DUMMY_CMD="dummy --persist $TMROOT/dummy" # &> dummy_${name}.log" + + function start_procs(){ name=$1 indexToFail=$2 echo "Starting persistent dummy and tendermint" - dummy --persist $TMROOT/dummy &> "dummy_${name}.log" & + if [[ "$CIRCLECI" == true ]]; then + $DUMMY_CMD & + else + $DUMMY_CMD &> "dummy_${name}.log" & + fi PID_DUMMY=$! if [[ "$indexToFail" == "" ]]; then # run in background, dont fail - tendermint node --log_level=debug &> tendermint_${name}.log & + if [[ "$CIRCLECI" == true ]]; then + $TM_CMD & + else + $TM_CMD &> "tendermint_${name}.log" & + fi PID_TENDERMINT=$! else # run in foreground, fail - FAIL_TEST_INDEX=$indexToFail tendermint node --log_level=debug &> tendermint_${name}.log + if [[ "$CIRCLECI" == true ]]; then + FAIL_TEST_INDEX=$indexToFail $TM_CMD + else + FAIL_TEST_INDEX=$indexToFail $TM_CMD &> "tendermint_${name}.log" + fi PID_TENDERMINT=$! fi } From 1fa6e7f3b100760d788f0bbb15e178ca33f45130 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 20 Feb 2017 18:51:00 -0500 Subject: [PATCH 11/14] test: shellcheck --- test/persist/test.sh | 2 +- test/persist/test_failure_indices.sh | 38 ++++++++++++++-------------- test/run_test.sh | 4 +-- test/test_cover.sh | 2 +- test/test_libs.sh | 12 ++++----- 5 files changed, 29 insertions(+), 29 deletions(-) diff --git a/test/persist/test.sh b/test/persist/test.sh index b27394c51..e24e81deb 100644 --- a/test/persist/test.sh +++ b/test/persist/test.sh @@ -1,5 +1,5 @@ #! /bin/bash -cd $GOPATH/src/github.com/tendermint/tendermint +cd "$GOPATH/src/github.com/tendermint/tendermint" bash ./test/persist/test_failure_indices.sh diff --git a/test/persist/test_failure_indices.sh b/test/persist/test_failure_indices.sh index b24412352..8feb0005c 100644 --- a/test/persist/test_failure_indices.sh +++ b/test/persist/test_failure_indices.sh @@ -3,7 +3,7 @@ export TMROOT=$HOME/.tendermint_persist -rm -rf $TMROOT +rm -rf "$TMROOT" tendermint init TM_CMD="tendermint node --log_level=debug" # &> tendermint_${name}.log" @@ -40,9 +40,9 @@ function start_procs(){ } function kill_procs(){ - kill -9 $PID_DUMMY $PID_TENDERMINT - wait $PID_DUMMY - wait $PID_TENDERMINT + kill -9 "$PID_DUMMY" "$PID_TENDERMINT" + wait "$PID_DUMMY" + wait "$PID_TENDERMINT" } @@ -59,10 +59,10 @@ function send_txs(){ # send a bunch of txs over a few blocks echo "Node is up, sending txs" - for i in `seq 1 5`; do - for j in `seq 1 100`; do - tx=`head -c 8 /dev/urandom | hexdump -ve '1/1 "%.2X"'` - curl -s $addr/broadcast_tx_async?tx=0x$tx &> /dev/null + for i in $(seq 1 5); do + for _ in $(seq 1 100); do + tx=$(head -c 8 /dev/urandom | hexdump -ve '1/1 "%.2X"') + curl -s "$addr/broadcast_tx_async?tx=0x$tx" &> /dev/null done sleep 1 done @@ -70,33 +70,33 @@ function send_txs(){ failsStart=0 -fails=`grep -r "fail.Fail" --include \*.go . | wc -l` -failsEnd=$(($fails-1)) +fails=$(grep -r "fail.Fail" --include \*.go . | wc -l) +failsEnd=$((fails-1)) -for failIndex in `seq $failsStart $failsEnd`; do +for failIndex in $(seq $failsStart $failsEnd); do echo "" echo "* Test FailIndex $failIndex" # test failure at failIndex send_txs & - start_procs 1 $failIndex + start_procs 1 "$failIndex" # tendermint should fail when it hits the fail index - kill -9 $PID_DUMMY - wait $PID_DUMMY + kill -9 "$PID_DUMMY" + wait "$PID_DUMMY" start_procs 2 # wait for node to handshake and make a new block addr="localhost:46657" - curl -s $addr/status > /dev/null + curl -s "$addr/status" > /dev/null ERR=$? i=0 while [ "$ERR" != 0 ]; do sleep 1 - curl -s $addr/status > /dev/null + curl -s "$addr/status" > /dev/null ERR=$? - i=$(($i + 1)) + i=$((i + 1)) if [[ $i == 10 ]]; then echo "Timed out waiting for tendermint to start" exit 1 @@ -104,11 +104,11 @@ for failIndex in `seq $failsStart $failsEnd`; do done # wait for a new block - h1=`curl -s $addr/status | jq .result[1].latest_block_height` + h1=$(curl -s $addr/status | jq .result[1].latest_block_height) h2=$h1 while [ "$h2" == "$h1" ]; do sleep 1 - h2=`curl -s $addr/status | jq .result[1].latest_block_height` + h2=$(curl -s $addr/status | jq .result[1].latest_block_height) done kill_procs diff --git a/test/run_test.sh b/test/run_test.sh index ba4e1b0e4..fcc82d984 100644 --- a/test/run_test.sh +++ b/test/run_test.sh @@ -1,9 +1,9 @@ #! /bin/bash set -e -echo `pwd` +pwd -BRANCH=`git rev-parse --abbrev-ref HEAD` +BRANCH=$(git rev-parse --abbrev-ref HEAD) echo "Current branch: $BRANCH" bash test/test_cover.sh diff --git a/test/test_cover.sh b/test/test_cover.sh index 60c84284d..b1b754ba4 100644 --- a/test/test_cover.sh +++ b/test/test_cover.sh @@ -5,7 +5,7 @@ PKGS=$(go list github.com/tendermint/tendermint/... | grep -v /vendor/) set -e echo "mode: atomic" > coverage.txt for pkg in ${PKGS[@]}; do - go test -v -timeout 30m -race -coverprofile=profile.out -covermode=atomic $pkg + go test -v -timeout 30m -race -coverprofile=profile.out -covermode=atomic "$pkg" if [ -f profile.out ]; then tail -n +2 profile.out >> coverage.txt; rm profile.out diff --git a/test/test_libs.sh b/test/test_libs.sh index 28ec07980..4a531bf35 100644 --- a/test/test_libs.sh +++ b/test/test_libs.sh @@ -18,30 +18,30 @@ LIBS_MAKE_TEST=(go-rpc go-wire abci) for lib in "${LIBS_GO_TEST[@]}"; do # checkout vendored version of lib - bash scripts/glide/checkout.sh $GLIDE $lib + bash scripts/glide/checkout.sh "$GLIDE" "$lib" echo "Testing $lib ..." - go test -v --race github.com/tendermint/$lib/... + go test -v --race "github.com/tendermint/$lib/..." if [[ "$?" != 0 ]]; then echo "FAIL" exit 1 fi done -DIR=`pwd` +DIR=$(pwd) for lib in "${LIBS_MAKE_TEST[@]}"; do # checkout vendored version of lib - bash scripts/glide/checkout.sh $GLIDE $lib + bash scripts/glide/checkout.sh "$GLIDE" "$lib" echo "Testing $lib ..." - cd $GOPATH/src/github.com/tendermint/$lib + cd "$GOPATH/src/github.com/tendermint/$lib" make test if [[ "$?" != 0 ]]; then echo "FAIL" exit 1 fi - cd $DIR + cd "$DIR" done echo "" From 07656137780412a2341ec21ed1466647b656944a Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 20 Feb 2017 19:52:36 -0500 Subject: [PATCH 12/14] move handshake to consensus package --- consensus/replay.go | 205 +++++++++++++++++++++++++++++++++++++++ consensus/replay_file.go | 2 +- consensus/replay_test.go | 2 +- consensus/state.go | 29 ------ node/node.go | 2 +- state/errors.go | 32 +++--- state/execution.go | 181 +--------------------------------- 7 files changed, 228 insertions(+), 225 deletions(-) diff --git a/consensus/replay.go b/consensus/replay.go index d534827be..2ab84dc20 100644 --- a/consensus/replay.go +++ b/consensus/replay.go @@ -1,6 +1,7 @@ package consensus import ( + "bytes" "errors" "fmt" "io" @@ -9,13 +10,26 @@ import ( "strings" "time" + abci "github.com/tendermint/abci/types" auto "github.com/tendermint/go-autofile" . "github.com/tendermint/go-common" + cfg "github.com/tendermint/go-config" "github.com/tendermint/go-wire" + "github.com/tendermint/tendermint/proxy" + sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" ) +// Functionality to replay blocks and messages on recovery from a crash. +// There are two general failure scenarios: failure during consensus, and failure while applying the block. +// The former is handled by the WAL, the latter by the proxyApp Handshake on restart, +// which ultimately hands off the work to the WAL. + +//----------------------------------------- +// recover from failure during consensus +// by replaying messages from the WAL + // Unmarshal and apply a single message to the consensus state // as if it were received in receiveRoutine // Lines that start with "#" are ignored. @@ -154,3 +168,194 @@ func makeHeightSearchFunc(height int) auto.SearchFunc { } } } + +//---------------------------------------------- +// Recover from failure during block processing +// by handshaking with the app to figure out where +// we were last and using the WAL to recover there + +// Replay the last block through the consensus and return the AppHash from after Commit. +func replayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnConsensus, blockStore sm.BlockStore) ([]byte, error) { + mempool := sm.MockMempool{} + cs := NewConsensusState(config, state, proxyApp, blockStore, mempool) + + evsw := types.NewEventSwitch() + evsw.Start() + defer evsw.Stop() + cs.SetEventSwitch(evsw) + newBlockCh := subscribeToEvent(evsw, "consensus-replay", types.EventStringNewBlock(), 1) + + // run through the WAL, commit new block, stop + cs.Start() + <-newBlockCh // TODO: use a timeout and return err? + cs.Stop() + + return cs.state.AppHash, nil +} + +type Handshaker struct { + config cfg.Config + state *sm.State + store sm.BlockStore + + nBlocks int // number of blocks applied to the state +} + +func NewHandshaker(config cfg.Config, state *sm.State, store sm.BlockStore) *Handshaker { + return &Handshaker{config, state, store, 0} +} + +func (h *Handshaker) NBlocks() int { + return h.nBlocks +} + +// TODO: retry the handshake/replay if it fails ? +func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error { + // handshake is done via info request on the query conn + res, err := proxyApp.Query().InfoSync() + if err != nil { + return errors.New(Fmt("Error calling Info: %v", err)) + } + + blockHeight := int(res.LastBlockHeight) // XXX: beware overflow + appHash := res.LastBlockAppHash + + log.Notice("ABCI Handshake", "appHeight", blockHeight, "appHash", appHash) + + // TODO: check version + + // replay blocks up to the latest in the blockstore + _, err = h.ReplayBlocks(appHash, blockHeight, proxyApp) + if err != nil { + return errors.New(Fmt("Error on replay: %v", err)) + } + + // TODO: (on restart) replay mempool + + return nil +} + +// Replay all blocks since appBlockHeight and ensure the result matches the current state. +// Returns the final AppHash or an error +func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp proxy.AppConns) ([]byte, error) { + + storeBlockHeight := h.store.Height() + stateBlockHeight := h.state.LastBlockHeight + log.Notice("ABCI Replay Blocks", "appHeight", appBlockHeight, "storeHeight", storeBlockHeight, "stateHeight", stateBlockHeight) + + // First handle edge cases and constraints on the storeBlockHeight + if storeBlockHeight == 0 { + return appHash, h.checkAppHash(appHash) + + } else if storeBlockHeight < appBlockHeight { + // the app should never be ahead of the store (but this is under app's control) + return appHash, sm.ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight} + + } else if storeBlockHeight < stateBlockHeight { + // the state should never be ahead of the store (this is under tendermint's control) + PanicSanity(Fmt("StateBlockHeight (%d) > StoreBlockHeight (%d)", stateBlockHeight, storeBlockHeight)) + + } else if storeBlockHeight > stateBlockHeight+1 { + // store should be at most one ahead of the state (this is under tendermint's control) + PanicSanity(Fmt("StoreBlockHeight (%d) > StateBlockHeight + 1 (%d)", storeBlockHeight, stateBlockHeight+1)) + } + + // Now either store is equal to state, or one ahead. + // For each, consider all cases of where the app could be, given app <= store + if storeBlockHeight == stateBlockHeight { + // Tendermint ran Commit and saved the state. + // Either the app is asking for replay, or we're all synced up. + if appBlockHeight < storeBlockHeight { + // the app is behind, so replay blocks, but no need to go through WAL (state is already synced to store) + return h.replayBlocks(proxyApp, appBlockHeight, storeBlockHeight, false) + + } else if appBlockHeight == storeBlockHeight { + // we're good! + return appHash, h.checkAppHash(appHash) + } + + } else if storeBlockHeight == stateBlockHeight+1 { + // We saved the block in the store but haven't updated the state, + // so we'll need to replay a block using the WAL. + if appBlockHeight < stateBlockHeight { + // the app is further behind than it should be, so replay blocks + // but leave the last block to go through the WAL + return h.replayBlocks(proxyApp, appBlockHeight, storeBlockHeight, true) + + } else if appBlockHeight == stateBlockHeight { + // We haven't run Commit (both the state and app are one block behind), + // so run through consensus with the real app + log.Info("Replay last block using real app") + return replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) + + } else if appBlockHeight == storeBlockHeight { + // We ran Commit, but didn't save the state, so run through consensus with mock app + mockApp := newMockProxyApp(appHash) + log.Info("Replay last block using mock app") + return replayLastBlock(h.config, h.state, mockApp, h.store) + } + + } + + PanicSanity("Should never happen") + return nil, nil +} + +func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, storeBlockHeight int, useReplayFunc bool) ([]byte, error) { + // App is further behind than it should be, so we need to replay blocks. + // We replay all blocks from appBlockHeight+1 to storeBlockHeight-1, + // and let the final block be replayed through ReplayBlocks. + // Note that we don't have an old version of the state, + // so we by-pass state validation using applyBlock here. + + var appHash []byte + var err error + finalBlock := storeBlockHeight + if useReplayFunc { + finalBlock -= 1 + } + for i := appBlockHeight + 1; i <= finalBlock; i++ { + log.Info("Applying block", "height", i) + h.nBlocks += 1 + block := h.store.LoadBlock(i) + appHash, err = sm.ApplyBlock(proxyApp.Consensus(), block) + if err != nil { + return nil, err + } + } + + if useReplayFunc { + // sync the final block + appHash, err = h.ReplayBlocks(appHash, finalBlock, proxyApp) + if err != nil { + return appHash, err + } + } + + return appHash, h.checkAppHash(appHash) +} + +func (h *Handshaker) checkAppHash(appHash []byte) error { + if !bytes.Equal(h.state.AppHash, appHash) { + return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) + } + return nil +} + +//-------------------------------------------------------------------------------- + +func newMockProxyApp(appHash []byte) proxy.AppConnConsensus { + clientCreator := proxy.NewLocalClientCreator(&mockProxyApp{appHash: appHash}) + cli, _ := clientCreator.NewABCIClient() + return proxy.NewAppConnConsensus(cli) +} + +type mockProxyApp struct { + abci.BaseApplication + + appHash []byte +} + +func (mock *mockProxyApp) Commit() abci.Result { + return abci.NewResultOK(mock.appHash, "") +} diff --git a/consensus/replay_file.go b/consensus/replay_file.go index 5d6747859..6ff380880 100644 --- a/consensus/replay_file.go +++ b/consensus/replay_file.go @@ -248,7 +248,7 @@ func newConsensusStateForReplay(config cfg.Config) *ConsensusState { state := sm.MakeGenesisStateFromFile(stateDB, config.GetString("genesis_file")) // Create proxyAppConn connection (consensus, mempool, query) - proxyApp := proxy.NewAppConns(config, proxy.DefaultClientCreator(config), sm.NewHandshaker(config, state, blockStore, ReplayLastBlock)) + proxyApp := proxy.NewAppConns(config, proxy.DefaultClientCreator(config), NewHandshaker(config, state, blockStore)) _, err := proxyApp.Start() if err != nil { Exit(Fmt("Error starting proxy app conns: %v", err)) diff --git a/consensus/replay_test.go b/consensus/replay_test.go index e8aa9e903..3016f1048 100644 --- a/consensus/replay_test.go +++ b/consensus/replay_test.go @@ -346,7 +346,7 @@ func testHandshakeReplay(t *testing.T, nBlocks int) { } // now start the app using the handshake - it should sync - handshaker := sm.NewHandshaker(config, state, store, ReplayLastBlock) + handshaker := NewHandshaker(config, state, store) proxyApp = proxy.NewAppConns(config, clientCreator2, handshaker) if _, err := proxyApp.Start(); err != nil { t.Fatalf("Error starting proxy app connections: %v", err) diff --git a/consensus/state.go b/consensus/state.go index 8e721ed3e..dc1324e36 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -253,25 +253,6 @@ type ConsensusState struct { done chan struct{} } -// Replay the last block through the consensus and return the AppHash from after Commit. -func ReplayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnConsensus, blockStore sm.BlockStore) ([]byte, error) { - mempool := sm.MockMempool{} - cs := NewConsensusState(config, state, proxyApp, blockStore, mempool) - - evsw := types.NewEventSwitch() - evsw.Start() - defer evsw.Stop() - cs.SetEventSwitch(evsw) - newBlockCh := subscribeToEvent(evsw, "consensus-replay", types.EventStringNewBlock(), 1) - - // run through the WAL, commit new block, stop - cs.Start() - <-newBlockCh // TODO: use a timeout and return err? - cs.Stop() - - return cs.state.AppHash, nil -} - func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConnConsensus, blockStore sm.BlockStore, mempool sm.Mempool) *ConsensusState { cs := &ConsensusState{ config: config, @@ -624,11 +605,6 @@ func (cs *ConsensusState) newStep() { //----------------------------------------- // the main go routines -// a nice idea but probably more trouble than its worth -func (cs *ConsensusState) stopTimer() { - cs.timeoutTicker.Stop() -} - // receiveRoutine handles messages which may cause state transitions. // it's argument (n) is the number of messages to process before exiting - use 0 to run forever // It keeps the RoundState and is the only thing that updates it. @@ -767,7 +743,6 @@ func (cs *ConsensusState) enterNewRound(height int, round int) { if now := time.Now(); cs.StartTime.After(now) { log.Warn("Need to set a buffer and log.Warn() here for sanity.", "startTime", cs.StartTime, "now", now) } - // cs.stopTimer() log.Notice(Fmt("enterNewRound(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step)) @@ -947,8 +922,6 @@ func (cs *ConsensusState) enterPrevote(height int, round int) { // TODO: catchup event? } - // cs.stopTimer() - log.Info(Fmt("enterPrevote(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step)) // Sign and broadcast vote as necessary @@ -1022,8 +995,6 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) { return } - // cs.stopTimer() - log.Info(Fmt("enterPrecommit(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step)) defer func() { diff --git a/node/node.go b/node/node.go index 755658b7b..deaa9e163 100644 --- a/node/node.go +++ b/node/node.go @@ -69,7 +69,7 @@ func NewNode(config cfg.Config, privValidator *types.PrivValidator, clientCreato // Create the proxyApp, which manages connections (consensus, mempool, query) // and sync tendermint and the app by replaying any necessary blocks - proxyApp := proxy.NewAppConns(config, clientCreator, sm.NewHandshaker(config, state, blockStore, consensus.ReplayLastBlock)) + proxyApp := proxy.NewAppConns(config, clientCreator, consensus.NewHandshaker(config, state, blockStore)) if _, err := proxyApp.Start(); err != nil { cmn.Exit(cmn.Fmt("Error starting proxy app connections: %v", err)) } diff --git a/state/errors.go b/state/errors.go index 0d0eae14c..32a9351ce 100644 --- a/state/errors.go +++ b/state/errors.go @@ -9,47 +9,47 @@ type ( ErrProxyAppConn error ErrUnknownBlock struct { - height int + Height int } ErrBlockHashMismatch struct { - coreHash []byte - appHash []byte - height int + CoreHash []byte + AppHash []byte + Height int } ErrAppBlockHeightTooHigh struct { - coreHeight int - appHeight int + CoreHeight int + AppHeight int } ErrLastStateMismatch struct { - height int - core []byte - app []byte + Height int + Core []byte + App []byte } ErrStateMismatch struct { - got *State - expected *State + Got *State + Expected *State } ) func (e ErrUnknownBlock) Error() string { - return Fmt("Could not find block #%d", e.height) + return Fmt("Could not find block #%d", e.Height) } func (e ErrBlockHashMismatch) Error() string { - return Fmt("App block hash (%X) does not match core block hash (%X) for height %d", e.appHash, e.coreHash, e.height) + return Fmt("App block hash (%X) does not match core block hash (%X) for height %d", e.AppHash, e.CoreHash, e.Height) } func (e ErrAppBlockHeightTooHigh) Error() string { - return Fmt("App block height (%d) is higher than core (%d)", e.appHeight, e.coreHeight) + return Fmt("App block height (%d) is higher than core (%d)", e.AppHeight, e.CoreHeight) } func (e ErrLastStateMismatch) Error() string { - return Fmt("Latest tendermint block (%d) LastAppHash (%X) does not match app's AppHash (%X)", e.height, e.core, e.app) + return Fmt("Latest tendermint block (%d) LastAppHash (%X) does not match app's AppHash (%X)", e.Height, e.Core, e.App) } func (e ErrStateMismatch) Error() string { - return Fmt("State after replay does not match saved state. Got ----\n%v\nExpected ----\n%v\n", e.got, e.expected) + return Fmt("State after replay does not match saved state. Got ----\n%v\nExpected ----\n%v\n", e.Got, e.Expected) } diff --git a/state/execution.go b/state/execution.go index c1471bf86..3e7ad91bd 100644 --- a/state/execution.go +++ b/state/execution.go @@ -1,14 +1,12 @@ package state import ( - "bytes" "errors" "github.com/ebuchman/fail-test" abci "github.com/tendermint/abci/types" . "github.com/tendermint/go-common" - cfg "github.com/tendermint/go-config" "github.com/tendermint/go-crypto" "github.com/tendermint/tendermint/proxy" "github.com/tendermint/tendermint/types" @@ -272,7 +270,7 @@ func (s *State) CommitStateUpdateMempool(proxyAppConn proxy.AppConnConsensus, bl // Apply and commit a block, but without all the state validation. // Returns the application root hash (result of abci.Commit) -func applyBlock(appConnConsensus proxy.AppConnConsensus, block *types.Block) ([]byte, error) { +func ApplyBlock(appConnConsensus proxy.AppConnConsensus, block *types.Block) ([]byte, error) { var eventCache types.Fireable // nil _, err := execBlockOnProxyApp(eventCache, appConnConsensus, block) if err != nil { @@ -291,6 +289,9 @@ func applyBlock(appConnConsensus proxy.AppConnConsensus, block *types.Block) ([] return res.Data, nil } +//------------------------------------------------------ +// blockchain services types + // Updates to the mempool need to be synchronized with committing a block // so apps can reset their transient state on Commit type Mempool interface { @@ -311,9 +312,6 @@ func (m MockMempool) CheckTx(tx types.Tx, cb func(*abci.Response)) error { retur func (m MockMempool) Reap(n int) types.Txs { return types.Txs{} } func (m MockMempool) Update(height int, txs types.Txs) {} -//---------------------------------------------------------------- -// Handshake with app to sync to latest state of core by replaying blocks - // TODO: Should we move blockchain/store.go to its own package? type BlockStore interface { Height() int @@ -327,174 +325,3 @@ type BlockStore interface { LoadBlockCommit(height int) *types.Commit LoadSeenCommit(height int) *types.Commit } - -// returns the apphash from Commit -type blockReplayFunc func(cfg.Config, *State, proxy.AppConnConsensus, BlockStore) ([]byte, error) - -type Handshaker struct { - config cfg.Config - state *State - store BlockStore - replayLastBlock blockReplayFunc - - nBlocks int // number of blocks applied to the state -} - -func NewHandshaker(config cfg.Config, state *State, store BlockStore, f blockReplayFunc) *Handshaker { - return &Handshaker{config, state, store, f, 0} -} - -func (h *Handshaker) NBlocks() int { - return h.nBlocks -} - -// TODO: retry the handshake/replay if it fails ? -func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error { - // handshake is done via info request on the query conn - res, err := proxyApp.Query().InfoSync() - if err != nil { - return errors.New(Fmt("Error calling Info: %v", err)) - } - - blockHeight := int(res.LastBlockHeight) // XXX: beware overflow - appHash := res.LastBlockAppHash - - log.Notice("ABCI Handshake", "appHeight", blockHeight, "appHash", appHash) - - // TODO: check version - - // replay blocks up to the latest in the blockstore - _, err = h.ReplayBlocks(appHash, blockHeight, proxyApp) - if err != nil { - return errors.New(Fmt("Error on replay: %v", err)) - } - - // TODO: (on restart) replay mempool - - return nil -} - -// Replay all blocks since appBlockHeight and ensure the result matches the current state. -// Returns the final AppHash or an error -func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp proxy.AppConns) ([]byte, error) { - - storeBlockHeight := h.store.Height() - stateBlockHeight := h.state.LastBlockHeight - log.Notice("ABCI Replay Blocks", "appHeight", appBlockHeight, "storeHeight", storeBlockHeight, "stateHeight", stateBlockHeight) - - // First handle edge cases and constraints on the storeBlockHeight - if storeBlockHeight == 0 { - return appHash, h.checkAppHash(appHash) - - } else if storeBlockHeight < appBlockHeight { - // the app should never be ahead of the store (but this is under app's control) - return appHash, ErrAppBlockHeightTooHigh{storeBlockHeight, appBlockHeight} - - } else if storeBlockHeight < stateBlockHeight { - // the state should never be ahead of the store (this is under tendermint's control) - PanicSanity(Fmt("StateBlockHeight (%d) > StoreBlockHeight (%d)", stateBlockHeight, storeBlockHeight)) - - } else if storeBlockHeight > stateBlockHeight+1 { - // store should be at most one ahead of the state (this is under tendermint's control) - PanicSanity(Fmt("StoreBlockHeight (%d) > StateBlockHeight + 1 (%d)", storeBlockHeight, stateBlockHeight+1)) - } - - // Now either store is equal to state, or one ahead. - // For each, consider all cases of where the app could be, given app <= store - if storeBlockHeight == stateBlockHeight { - // Tendermint ran Commit and saved the state. - // Either the app is asking for replay, or we're all synced up. - if appBlockHeight < storeBlockHeight { - // the app is behind, so replay blocks, but no need to go through WAL (state is already synced to store) - return h.replayBlocks(proxyApp, appBlockHeight, storeBlockHeight, false) - - } else if appBlockHeight == storeBlockHeight { - // we're good! - return appHash, h.checkAppHash(appHash) - } - - } else if storeBlockHeight == stateBlockHeight+1 { - // We saved the block in the store but haven't updated the state, - // so we'll need to replay a block using the WAL. - if appBlockHeight < stateBlockHeight { - // the app is further behind than it should be, so replay blocks - // but leave the last block to go through the WAL - return h.replayBlocks(proxyApp, appBlockHeight, storeBlockHeight, true) - - } else if appBlockHeight == stateBlockHeight { - // We haven't run Commit (both the state and app are one block behind), - // so run through consensus with the real app - log.Info("Replay last block using real app") - return h.replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) - - } else if appBlockHeight == storeBlockHeight { - // We ran Commit, but didn't save the state, so run through consensus with mock app - mockApp := newMockProxyApp(appHash) - log.Info("Replay last block using mock app") - return h.replayLastBlock(h.config, h.state, mockApp, h.store) - } - - } - - PanicSanity("Should never happen") - return nil, nil -} - -func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, storeBlockHeight int, useReplayFunc bool) ([]byte, error) { - // App is further behind than it should be, so we need to replay blocks. - // We replay all blocks from appBlockHeight+1 to storeBlockHeight-1, - // and let the final block be replayed through ReplayBlocks. - // Note that we don't have an old version of the state, - // so we by-pass state validation using applyBlock here. - - var appHash []byte - var err error - finalBlock := storeBlockHeight - if useReplayFunc { - finalBlock -= 1 - } - for i := appBlockHeight + 1; i <= finalBlock; i++ { - log.Info("Applying block", "height", i) - h.nBlocks += 1 - block := h.store.LoadBlock(i) - appHash, err = applyBlock(proxyApp.Consensus(), block) - if err != nil { - return nil, err - } - } - - if useReplayFunc { - // sync the final block - appHash, err = h.ReplayBlocks(appHash, finalBlock, proxyApp) - if err != nil { - return appHash, err - } - } - - return appHash, h.checkAppHash(appHash) -} - -func (h *Handshaker) checkAppHash(appHash []byte) error { - if !bytes.Equal(h.state.AppHash, appHash) { - return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) - } - return nil -} - -//-------------------------------------------------------------------------------- - -func newMockProxyApp(appHash []byte) proxy.AppConnConsensus { - clientCreator := proxy.NewLocalClientCreator(&mockProxyApp{appHash: appHash}) - cli, _ := clientCreator.NewABCIClient() - return proxy.NewAppConnConsensus(cli) -} - -type mockProxyApp struct { - abci.BaseApplication - - appHash []byte -} - -func (mock *mockProxyApp) Commit() abci.Result { - return abci.NewResultOK(mock.appHash, "") -} From f9df4294f34eccbdd1bea8c524f1b48f8b7d7cb4 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 20 Feb 2017 20:09:15 -0500 Subject: [PATCH 13/14] move some interfaces to types/services.go --- blockchain/reactor.go | 2 +- consensus/replay.go | 8 +++--- consensus/replay_test.go | 2 +- consensus/state.go | 6 ++--- rpc/core/pipe.go | 39 +++++++++------------------- state/execution.go | 41 ++--------------------------- types/services.go | 56 ++++++++++++++++++++++++++++++++++++++++ 7 files changed, 79 insertions(+), 75 deletions(-) create mode 100644 types/services.go diff --git a/blockchain/reactor.go b/blockchain/reactor.go index bfa671d02..4a7f21d00 100644 --- a/blockchain/reactor.go +++ b/blockchain/reactor.go @@ -242,7 +242,7 @@ FOR_LOOP: // NOTE: we could improve performance if we // didn't make the app commit to disk every block // ... but we would need a way to get the hash without it persisting - err := bcR.state.ApplyBlock(bcR.evsw, bcR.proxyAppConn, first, firstPartsHeader, sm.MockMempool{}) + err := bcR.state.ApplyBlock(bcR.evsw, bcR.proxyAppConn, first, firstPartsHeader, types.MockMempool{}) if err != nil { // TODO This is bad, are we zombie? PanicQ(Fmt("Failed to process committed block (%d:%X): %v", first.Height, first.Hash(), err)) diff --git a/consensus/replay.go b/consensus/replay.go index 2ab84dc20..07308bf17 100644 --- a/consensus/replay.go +++ b/consensus/replay.go @@ -175,8 +175,8 @@ func makeHeightSearchFunc(height int) auto.SearchFunc { // we were last and using the WAL to recover there // Replay the last block through the consensus and return the AppHash from after Commit. -func replayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnConsensus, blockStore sm.BlockStore) ([]byte, error) { - mempool := sm.MockMempool{} +func replayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnConsensus, blockStore types.BlockStore) ([]byte, error) { + mempool := types.MockMempool{} cs := NewConsensusState(config, state, proxyApp, blockStore, mempool) evsw := types.NewEventSwitch() @@ -196,12 +196,12 @@ func replayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnC type Handshaker struct { config cfg.Config state *sm.State - store sm.BlockStore + store types.BlockStore nBlocks int // number of blocks applied to the state } -func NewHandshaker(config cfg.Config, state *sm.State, store sm.BlockStore) *Handshaker { +func NewHandshaker(config cfg.Config, state *sm.State, store types.BlockStore) *Handshaker { return &Handshaker{config, state, store, 0} } diff --git a/consensus/replay_test.go b/consensus/replay_test.go index 3016f1048..2d812de91 100644 --- a/consensus/replay_test.go +++ b/consensus/replay_test.go @@ -254,7 +254,7 @@ func testReplayCrashBeforeWriteVote(t *testing.T, thisCase *testCase, lineNum in var ( NUM_BLOCKS = 6 // number of blocks in the test_data/many_blocks.cswal - mempool = sm.MockMempool{} + mempool = types.MockMempool{} testPartSize int ) diff --git a/consensus/state.go b/consensus/state.go index dc1324e36..882d5c87b 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -224,8 +224,8 @@ type ConsensusState struct { config cfg.Config proxyAppConn proxy.AppConnConsensus - blockStore sm.BlockStore - mempool sm.Mempool + blockStore types.BlockStore + mempool types.Mempool privValidator PrivValidator // for signing votes @@ -253,7 +253,7 @@ type ConsensusState struct { done chan struct{} } -func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConnConsensus, blockStore sm.BlockStore, mempool sm.Mempool) *ConsensusState { +func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConnConsensus, blockStore types.BlockStore, mempool types.Mempool) *ConsensusState { cs := &ConsensusState{ config: config, proxyAppConn: proxyAppConn, diff --git a/rpc/core/pipe.go b/rpc/core/pipe.go index fb06c3ff1..40ef70817 100644 --- a/rpc/core/pipe.go +++ b/rpc/core/pipe.go @@ -5,36 +5,19 @@ import ( "github.com/tendermint/go-crypto" "github.com/tendermint/go-p2p" - abci "github.com/tendermint/abci/types" "github.com/tendermint/tendermint/consensus" "github.com/tendermint/tendermint/proxy" "github.com/tendermint/tendermint/types" ) -//----------------------------------------------------- -// Interfaces for use by RPC -// NOTE: these methods must be thread safe! - -type BlockStore interface { - Height() int - LoadBlockMeta(height int) *types.BlockMeta - LoadBlock(height int) *types.Block - LoadSeenCommit(height int) *types.Commit - LoadBlockCommit(height int) *types.Commit -} +//---------------------------------------------- +// These interfaces are used by RPC and must be thread safe type Consensus interface { GetValidators() (int, []*types.Validator) GetRoundState() *consensus.RoundState } -type Mempool interface { - Size() int - CheckTx(types.Tx, func(*abci.Response)) error - Reap(int) types.Txs - Flush() -} - type P2P interface { Listeners() []p2p.Listener Peers() p2p.IPeerSet @@ -44,16 +27,18 @@ type P2P interface { DialSeeds([]string) } +//---------------------------------------------- + var ( // external, thread safe interfaces eventSwitch types.EventSwitch proxyAppQuery proxy.AppConnQuery config cfg.Config - // interfaces defined above - blockStore BlockStore + // interfaces defined in types and above + blockStore types.BlockStore + mempool types.Mempool consensusState Consensus - mempool Mempool p2pSwitch P2P // objects @@ -69,16 +54,16 @@ func SetEventSwitch(evsw types.EventSwitch) { eventSwitch = evsw } -func SetBlockStore(bs BlockStore) { +func SetBlockStore(bs types.BlockStore) { blockStore = bs } -func SetConsensusState(cs Consensus) { - consensusState = cs +func SetMempool(mem types.Mempool) { + mempool = mem } -func SetMempool(mem Mempool) { - mempool = mem +func SetConsensusState(cs Consensus) { + consensusState = cs } func SetSwitch(sw P2P) { diff --git a/state/execution.go b/state/execution.go index 3e7ad91bd..aa9113011 100644 --- a/state/execution.go +++ b/state/execution.go @@ -223,7 +223,7 @@ func (s *State) validateBlock(block *types.Block) error { // Execute and commit block against app, save block and state func (s *State) ApplyBlock(eventCache types.Fireable, proxyAppConn proxy.AppConnConsensus, - block *types.Block, partsHeader types.PartSetHeader, mempool Mempool) error { + block *types.Block, partsHeader types.PartSetHeader, mempool types.Mempool) error { // Run the block on the State: // + update validator sets @@ -244,7 +244,7 @@ func (s *State) ApplyBlock(eventCache types.Fireable, proxyAppConn proxy.AppConn // mempool must be locked during commit and update // because state is typically reset on Commit and old txs must be replayed // against committed state before new txs are run in the mempool, lest they be invalid -func (s *State) CommitStateUpdateMempool(proxyAppConn proxy.AppConnConsensus, block *types.Block, mempool Mempool) error { +func (s *State) CommitStateUpdateMempool(proxyAppConn proxy.AppConnConsensus, block *types.Block, mempool types.Mempool) error { mempool.Lock() defer mempool.Unlock() @@ -288,40 +288,3 @@ func ApplyBlock(appConnConsensus proxy.AppConnConsensus, block *types.Block) ([] } return res.Data, nil } - -//------------------------------------------------------ -// blockchain services types - -// Updates to the mempool need to be synchronized with committing a block -// so apps can reset their transient state on Commit -type Mempool interface { - Lock() - Unlock() - - CheckTx(types.Tx, func(*abci.Response)) error - Reap(int) types.Txs - Update(height int, txs types.Txs) -} - -type MockMempool struct { -} - -func (m MockMempool) Lock() {} -func (m MockMempool) Unlock() {} -func (m MockMempool) CheckTx(tx types.Tx, cb func(*abci.Response)) error { return nil } -func (m MockMempool) Reap(n int) types.Txs { return types.Txs{} } -func (m MockMempool) Update(height int, txs types.Txs) {} - -// TODO: Should we move blockchain/store.go to its own package? -type BlockStore interface { - Height() int - - LoadBlockMeta(height int) *types.BlockMeta - LoadBlock(height int) *types.Block - LoadBlockPart(height int, index int) *types.Part - - SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) - - LoadBlockCommit(height int) *types.Commit - LoadSeenCommit(height int) *types.Commit -} diff --git a/types/services.go b/types/services.go new file mode 100644 index 000000000..ee20487e2 --- /dev/null +++ b/types/services.go @@ -0,0 +1,56 @@ +package types + +import ( + abci "github.com/tendermint/abci/types" +) + +//------------------------------------------------------ +// blockchain services types +// NOTE: Interfaces used by RPC must be thread safe! +//------------------------------------------------------ + +//------------------------------------------------------ +// mempool + +// Updates to the mempool need to be synchronized with committing a block +// so apps can reset their transient state on Commit +type Mempool interface { + Lock() + Unlock() + + Size() int + CheckTx(Tx, func(*abci.Response)) error + Reap(int) Txs + Update(height int, txs Txs) + Flush() +} + +type MockMempool struct { +} + +func (m MockMempool) Lock() {} +func (m MockMempool) Unlock() {} +func (m MockMempool) Size() int { return 0 } +func (m MockMempool) CheckTx(tx Tx, cb func(*abci.Response)) error { return nil } +func (m MockMempool) Reap(n int) Txs { return Txs{} } +func (m MockMempool) Update(height int, txs Txs) {} +func (m MockMempool) Flush() {} + +//------------------------------------------------------ +// blockstore + +type BlockStoreRPC interface { + Height() int + + LoadBlockMeta(height int) *BlockMeta + LoadBlock(height int) *Block + LoadBlockPart(height int, index int) *Part + + LoadBlockCommit(height int) *Commit + LoadSeenCommit(height int) *Commit +} + +type BlockStore interface { + BlockStoreRPC + SaveBlock(block *Block, blockParts *PartSet, seenCommit *Commit) +} From 0c4b6cd0716c3385cf71ae686e5196a0ee63a27c Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 20 Feb 2017 21:45:53 -0500 Subject: [PATCH 14/14] consensus: more handshake replay tests --- consensus/replay.go | 55 +++++++-------- consensus/replay_test.go | 146 +++++++++++++++++++++++++++++---------- 2 files changed, 136 insertions(+), 65 deletions(-) diff --git a/consensus/replay.go b/consensus/replay.go index 07308bf17..f117e66d4 100644 --- a/consensus/replay.go +++ b/consensus/replay.go @@ -174,25 +174,6 @@ func makeHeightSearchFunc(height int) auto.SearchFunc { // by handshaking with the app to figure out where // we were last and using the WAL to recover there -// Replay the last block through the consensus and return the AppHash from after Commit. -func replayLastBlock(config cfg.Config, state *sm.State, proxyApp proxy.AppConnConsensus, blockStore types.BlockStore) ([]byte, error) { - mempool := types.MockMempool{} - cs := NewConsensusState(config, state, proxyApp, blockStore, mempool) - - evsw := types.NewEventSwitch() - evsw.Start() - defer evsw.Stop() - cs.SetEventSwitch(evsw) - newBlockCh := subscribeToEvent(evsw, "consensus-replay", types.EventStringNewBlock(), 1) - - // run through the WAL, commit new block, stop - cs.Start() - <-newBlockCh // TODO: use a timeout and return err? - cs.Stop() - - return cs.state.AppHash, nil -} - type Handshaker struct { config cfg.Config state *sm.State @@ -286,13 +267,13 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p // We haven't run Commit (both the state and app are one block behind), // so run through consensus with the real app log.Info("Replay last block using real app") - return replayLastBlock(h.config, h.state, proxyApp.Consensus(), h.store) + return h.replayLastBlock(proxyApp.Consensus()) } else if appBlockHeight == storeBlockHeight { // We ran Commit, but didn't save the state, so run through consensus with mock app mockApp := newMockProxyApp(appHash) log.Info("Replay last block using mock app") - return replayLastBlock(h.config, h.state, mockApp, h.store) + return h.replayLastBlock(mockApp) } } @@ -316,28 +297,48 @@ func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, store } for i := appBlockHeight + 1; i <= finalBlock; i++ { log.Info("Applying block", "height", i) - h.nBlocks += 1 block := h.store.LoadBlock(i) appHash, err = sm.ApplyBlock(proxyApp.Consensus(), block) if err != nil { return nil, err } + + h.nBlocks += 1 } if useReplayFunc { // sync the final block - appHash, err = h.ReplayBlocks(appHash, finalBlock, proxyApp) - if err != nil { - return appHash, err - } + return h.ReplayBlocks(appHash, finalBlock, proxyApp) } return appHash, h.checkAppHash(appHash) } +// Replay the last block through the consensus and return the AppHash from after Commit. +func (h *Handshaker) replayLastBlock(proxyApp proxy.AppConnConsensus) ([]byte, error) { + mempool := types.MockMempool{} + cs := NewConsensusState(h.config, h.state, proxyApp, h.store, mempool) + + evsw := types.NewEventSwitch() + evsw.Start() + defer evsw.Stop() + cs.SetEventSwitch(evsw) + newBlockCh := subscribeToEvent(evsw, "consensus-replay", types.EventStringNewBlock(), 1) + + // run through the WAL, commit new block, stop + cs.Start() + <-newBlockCh // TODO: use a timeout and return err? + cs.Stop() + + h.nBlocks += 1 + + return cs.state.AppHash, nil +} + func (h *Handshaker) checkAppHash(appHash []byte) error { if !bytes.Equal(h.state.AppHash, appHash) { - return errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)) + panic(errors.New(Fmt("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, h.state.AppHash)).Error()) + return nil } return nil } diff --git a/consensus/replay_test.go b/consensus/replay_test.go index 2d812de91..c70b60fa0 100644 --- a/consensus/replay_test.go +++ b/consensus/replay_test.go @@ -262,28 +262,41 @@ var ( //--------------------------------------- // Test handshake/replay +// 0 - all synced up +// 1 - saved block but app and state are behind +// 2 - save block and committed but state is behind +var modes = []uint{0, 1, 2} + // Sync from scratch func TestHandshakeReplayAll(t *testing.T) { - testHandshakeReplay(t, 0) + for _, m := range modes { + testHandshakeReplay(t, 0, m) + } } // Sync many, not from scratch func TestHandshakeReplaySome(t *testing.T) { - testHandshakeReplay(t, 1) + for _, m := range modes { + testHandshakeReplay(t, 1, m) + } } // Sync from lagging by one func TestHandshakeReplayOne(t *testing.T) { - testHandshakeReplay(t, NUM_BLOCKS-1) + for _, m := range modes { + testHandshakeReplay(t, NUM_BLOCKS-1, m) + } } // Sync from caught up func TestHandshakeReplayNone(t *testing.T) { - testHandshakeReplay(t, NUM_BLOCKS) + for _, m := range modes { + testHandshakeReplay(t, NUM_BLOCKS, m) + } } // Make some blocks. Start a fresh app and apply nBlocks blocks. Then restart the app and sync it up with the remaining blocks -func testHandshakeReplay(t *testing.T, nBlocks int) { +func testHandshakeReplay(t *testing.T, nBlocks int, mode uint) { config := tendermint_test.ResetConfig("proxy_test_") // copy the many_blocks file @@ -310,44 +323,23 @@ func testHandshakeReplay(t *testing.T, nBlocks int) { store.chain = chain store.commits = commits - // run the whole chain against this client to build up the tendermint state - clientCreator := proxy.NewLocalClientCreator(dummy.NewPersistentDummyApplication(path.Join(config.GetString("db_dir"), "1"))) - proxyApp := proxy.NewAppConns(config, clientCreator, nil) // sm.NewHandshaker(config, state, store, ReplayLastBlock)) - if _, err := proxyApp.Start(); err != nil { - t.Fatalf("Error starting proxy app connections: %v", err) - } - for _, block := range chain { - err := state.ApplyBlock(nil, proxyApp.Consensus(), block, block.MakePartSet(testPartSize).Header(), mempool) - if err != nil { - t.Fatal(err) - } - } - proxyApp.Stop() - latestAppHash := state.AppHash + // run the chain through state.ApplyBlock to build up the tendermint state + latestAppHash := buildTMStateFromChain(config, state, chain, mode) - // run nBlocks against a new client to build up the app state. - // use a throwaway tendermint state - clientCreator2 := proxy.NewLocalClientCreator(dummy.NewPersistentDummyApplication(path.Join(config.GetString("db_dir"), "2"))) + // make a new client creator + dummyApp := dummy.NewPersistentDummyApplication(path.Join(config.GetString("db_dir"), "2")) + clientCreator2 := proxy.NewLocalClientCreator(dummyApp) if nBlocks > 0 { - // start a new app without handshake, play nBlocks blocks + // run nBlocks against a new client to build up the app state. + // use a throwaway tendermint state proxyApp := proxy.NewAppConns(config, clientCreator2, nil) - if _, err := proxyApp.Start(); err != nil { - t.Fatalf("Error starting proxy app connections: %v", err) - } - state2, _ := stateAndStore(config, privVal.PubKey) - for i := 0; i < nBlocks; i++ { - block := chain[i] - err := state2.ApplyBlock(nil, proxyApp.Consensus(), block, block.MakePartSet(testPartSize).Header(), mempool) - if err != nil { - t.Fatal(err) - } - } - proxyApp.Stop() + state, _ := stateAndStore(config, privVal.PubKey) + buildAppStateFromChain(proxyApp, state, chain, nBlocks, mode) } // now start the app using the handshake - it should sync handshaker := NewHandshaker(config, state, store) - proxyApp = proxy.NewAppConns(config, clientCreator2, handshaker) + proxyApp := proxy.NewAppConns(config, clientCreator2, handshaker) if _, err := proxyApp.Start(); err != nil { t.Fatalf("Error starting proxy app connections: %v", err) } @@ -363,9 +355,87 @@ func testHandshakeReplay(t *testing.T, nBlocks int) { t.Fatalf("Expected app hashes to match after handshake/replay. got %X, expected %X", res.LastBlockAppHash, latestAppHash) } - if handshaker.NBlocks() != NUM_BLOCKS-nBlocks { - t.Fatalf("Expected handshake to sync %d blocks, got %d", NUM_BLOCKS-nBlocks, handshaker.NBlocks()) + expectedBlocksToSync := NUM_BLOCKS - nBlocks + if nBlocks == NUM_BLOCKS && mode > 0 { + expectedBlocksToSync += 1 + } else if nBlocks > 0 && mode == 1 { + expectedBlocksToSync += 1 } + + if handshaker.NBlocks() != expectedBlocksToSync { + t.Fatalf("Expected handshake to sync %d blocks, got %d", expectedBlocksToSync, handshaker.NBlocks()) + } +} + +func applyBlock(st *sm.State, blk *types.Block, proxyApp proxy.AppConns) { + err := st.ApplyBlock(nil, proxyApp.Consensus(), blk, blk.MakePartSet(testPartSize).Header(), mempool) + if err != nil { + panic(err) + } +} + +func buildAppStateFromChain(proxyApp proxy.AppConns, + state *sm.State, chain []*types.Block, nBlocks int, mode uint) { + // start a new app without handshake, play nBlocks blocks + if _, err := proxyApp.Start(); err != nil { + panic(err) + } + defer proxyApp.Stop() + switch mode { + case 0: + for i := 0; i < nBlocks; i++ { + block := chain[i] + applyBlock(state, block, proxyApp) + } + case 1, 2: + for i := 0; i < nBlocks-1; i++ { + block := chain[i] + applyBlock(state, block, proxyApp) + } + + if mode == 2 { + // update the dummy height and apphash + // as if we ran commit but not + applyBlock(state, chain[nBlocks-1], proxyApp) + } + } + +} + +func buildTMStateFromChain(config cfg.Config, state *sm.State, chain []*types.Block, mode uint) []byte { + // run the whole chain against this client to build up the tendermint state + clientCreator := proxy.NewLocalClientCreator(dummy.NewPersistentDummyApplication(path.Join(config.GetString("db_dir"), "1"))) + proxyApp := proxy.NewAppConns(config, clientCreator, nil) // sm.NewHandshaker(config, state, store, ReplayLastBlock)) + if _, err := proxyApp.Start(); err != nil { + panic(err) + } + defer proxyApp.Stop() + + var latestAppHash []byte + + switch mode { + case 0: + // sync right up + for _, block := range chain { + applyBlock(state, block, proxyApp) + } + + latestAppHash = state.AppHash + case 1, 2: + // sync up to the penultimate as if we stored the block. + // whether we commit or not depends on the appHash + for _, block := range chain[:len(chain)-1] { + applyBlock(state, block, proxyApp) + } + + // apply the final block to a state copy so we can + // get the right next appHash but keep the state back + stateCopy := state.Copy() + applyBlock(stateCopy, chain[len(chain)-1], proxyApp) + latestAppHash = stateCopy.AppHash + } + + return latestAppHash } //--------------------------