mirror of
https://github.com/fluencelabs/tendermint
synced 2025-04-25 14:52:17 +00:00
comment out failing consensus tests for now rewrite rpc httpclient to use new pubsub package import pubsub as tmpubsub, query as tmquery make event IDs constants EventKey -> EventTypeKey rename EventsPubsub to PubSub mempool does not use pubsub rename eventsSub to pubsub new subscribe API fix channel size issues and consensus tests bugs refactor rpc client add missing discardFromChan method add mutex rename pubsub to eventBus remove IsRunning from WSRPCConnection interface (not needed) add a comment in broadcastNewRoundStepsAndVotes rename registerEventCallbacks to broadcastNewRoundStepsAndVotes See https://dave.cheney.net/2014/03/19/channel-axioms stop eventBuses after reactor tests remove unnecessary Unsubscribe return subscribe helper function move discardFromChan to where it is used subscribe now returns an err this gives us ability to refuse to subscribe if pubsub is at its max capacity. use context for control overflow cache queries handle err when subscribing in replay_test rename testClientID to testSubscriber extract var set channel buffer capacity to 1 in replay_file fix byzantine_test unsubscribe from single event, not all events refactor httpclient to return events to appropriate channels return failing testReplayCrashBeforeWriteVote test fix TestValidatorSetChanges refactor code a bit fix testReplayCrashBeforeWriteVote add comment fix TestValidatorSetChanges fixes from Bucky's review update comment [ci skip] test TxEventBuffer update changelog fix TestValidatorSetChanges (2nd attempt) only do wg.Done when no errors benchmark event bus create pubsub server inside NewEventBus only expose config params (later if needed) set buffer capacity to 0 so we are not testing cache new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ} This should allow to subscribe to all transactions! or a specific one using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'" use TimeoutCommit instead of afterPublishEventNewBlockTimeout TimeoutCommit is the time a node waits after committing a block, before it goes into the next height. So it will finish everything from the last block, but then wait a bit. The idea is this gives it time to hear more votes from other validators, to strengthen the commit it includes in the next block. But it also gives it time to hear about new transactions. waitForBlockWithUpdatedVals rewrite WAL crash tests Task: test that we can recover from any WAL crash. Solution: the old tests were relying on event hub being run in the same thread (we were injecting the private validator's last signature). when considering a rewrite, we considered two possible solutions: write a "fuzzy" testing system where WAL is crashing upon receiving a new message, or inject failures and trigger them in tests using something like https://github.com/coreos/gofail. remove sleep no cs.Lock around wal.Save test different cases (empty block, non-empty block, ...) comments add comments test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks fixes as per Bucky's last review reset subscriptions on UnsubscribeAll use a simple counter to track message for which we panicked also, set a smaller part size for all test cases
607 lines
18 KiB
Go
607 lines
18 KiB
Go
package consensus
|
|
|
|
import (
|
|
"bytes"
|
|
"context"
|
|
"errors"
|
|
"fmt"
|
|
"io"
|
|
"io/ioutil"
|
|
"os"
|
|
"path"
|
|
"runtime"
|
|
"testing"
|
|
"time"
|
|
|
|
"github.com/stretchr/testify/require"
|
|
|
|
"github.com/tendermint/abci/example/dummy"
|
|
abci "github.com/tendermint/abci/types"
|
|
crypto "github.com/tendermint/go-crypto"
|
|
wire "github.com/tendermint/go-wire"
|
|
auto "github.com/tendermint/tmlibs/autofile"
|
|
cmn "github.com/tendermint/tmlibs/common"
|
|
dbm "github.com/tendermint/tmlibs/db"
|
|
|
|
cfg "github.com/tendermint/tendermint/config"
|
|
"github.com/tendermint/tendermint/proxy"
|
|
sm "github.com/tendermint/tendermint/state"
|
|
"github.com/tendermint/tendermint/types"
|
|
"github.com/tendermint/tmlibs/log"
|
|
)
|
|
|
|
var consensusReplayConfig *cfg.Config
|
|
|
|
func init() {
|
|
consensusReplayConfig = ResetConfig("consensus_replay_test")
|
|
}
|
|
|
|
// 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!
|
|
|
|
// NOTE: Files in this dir are generated by running the `build.sh` therein.
|
|
// It's a simple way to generate wals for a single block, or multiple blocks, with random transactions,
|
|
// and different part sizes. The output is not deterministic.
|
|
// It should only have to be re-run if there is some breaking change to the consensus data structures (eg. blocks, votes)
|
|
// or to the behaviour of the app (eg. computes app hash differently)
|
|
var data_dir = path.Join(cmn.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.
|
|
|
|
func startNewConsensusStateAndWaitForBlock(t *testing.T, lastBlockHeight int, blockDB dbm.DB, stateDB dbm.DB) {
|
|
logger := log.TestingLogger()
|
|
state, _ := sm.GetState(stateDB, consensusReplayConfig.GenesisFile())
|
|
state.SetLogger(logger.With("module", "state"))
|
|
privValidator := loadPrivValidator(consensusReplayConfig)
|
|
cs := newConsensusStateWithConfigAndBlockStore(consensusReplayConfig, state, privValidator, dummy.NewDummyApplication(), blockDB)
|
|
cs.SetLogger(logger)
|
|
|
|
bytes, _ := ioutil.ReadFile(cs.config.WalFile())
|
|
// fmt.Printf("====== WAL: \n\r%s\n", bytes)
|
|
t.Logf("====== WAL: \n\r%s\n", bytes)
|
|
|
|
_, err := cs.Start()
|
|
require.NoError(t, err)
|
|
defer func() {
|
|
cs.Stop()
|
|
}()
|
|
|
|
// This is just a signal that we haven't halted; its not something contained
|
|
// in the WAL itself. Assuming the consensus state is running, replay of any
|
|
// WAL, including the empty one, should eventually be followed by a new
|
|
// block, or else something is wrong.
|
|
newBlockCh := make(chan interface{}, 1)
|
|
err = cs.eventBus.Subscribe(context.Background(), testSubscriber, types.EventQueryNewBlock, newBlockCh)
|
|
require.NoError(t, err)
|
|
select {
|
|
case <-newBlockCh:
|
|
case <-time.After(10 * time.Second):
|
|
t.Fatalf("Timed out waiting for new block (see trace above)")
|
|
}
|
|
}
|
|
|
|
func sendTxs(cs *ConsensusState, ctx context.Context) {
|
|
i := 0
|
|
for {
|
|
select {
|
|
case <-ctx.Done():
|
|
return
|
|
default:
|
|
cs.mempool.CheckTx([]byte{byte(i)}, nil)
|
|
i++
|
|
}
|
|
}
|
|
}
|
|
|
|
// TestWALCrash uses crashing WAL to test we can recover from any WAL failure.
|
|
func TestWALCrash(t *testing.T) {
|
|
testCases := []struct {
|
|
name string
|
|
initFn func(*ConsensusState, context.Context)
|
|
heightToStop uint64
|
|
}{
|
|
{"empty block",
|
|
func(cs *ConsensusState, ctx context.Context) {},
|
|
1},
|
|
{"block with a smaller part size",
|
|
func(cs *ConsensusState, ctx context.Context) {
|
|
// XXX: is there a better way to change BlockPartSizeBytes?
|
|
params := cs.state.Params
|
|
params.BlockPartSizeBytes = 512
|
|
cs.state.Params = params
|
|
sendTxs(cs, ctx)
|
|
},
|
|
1},
|
|
{"many non-empty blocks",
|
|
sendTxs,
|
|
3},
|
|
}
|
|
|
|
for _, tc := range testCases {
|
|
t.Run(tc.name, func(t *testing.T) {
|
|
crashWALandCheckLiveness(t, tc.initFn, tc.heightToStop)
|
|
})
|
|
}
|
|
}
|
|
|
|
func crashWALandCheckLiveness(t *testing.T, initFn func(*ConsensusState, context.Context), heightToStop uint64) {
|
|
walPaniced := make(chan error)
|
|
crashingWal := &crashingWAL{panicCh: walPaniced, heightToStop: heightToStop}
|
|
|
|
i := 1
|
|
LOOP:
|
|
for {
|
|
// fmt.Printf("====== LOOP %d\n", i)
|
|
t.Logf("====== LOOP %d\n", i)
|
|
|
|
// create consensus state from a clean slate
|
|
logger := log.NewNopLogger()
|
|
stateDB := dbm.NewMemDB()
|
|
state, _ := sm.MakeGenesisStateFromFile(stateDB, consensusReplayConfig.GenesisFile())
|
|
state.SetLogger(logger.With("module", "state"))
|
|
privValidator := loadPrivValidator(consensusReplayConfig)
|
|
blockDB := dbm.NewMemDB()
|
|
cs := newConsensusStateWithConfigAndBlockStore(consensusReplayConfig, state, privValidator, dummy.NewDummyApplication(), blockDB)
|
|
cs.SetLogger(logger)
|
|
|
|
// start sending transactions
|
|
ctx, cancel := context.WithCancel(context.Background())
|
|
go initFn(cs, ctx)
|
|
|
|
// clean up WAL file from the previous iteration
|
|
walFile := cs.config.WalFile()
|
|
os.Remove(walFile)
|
|
|
|
// set crashing WAL
|
|
csWal, err := cs.OpenWAL(walFile)
|
|
require.NoError(t, err)
|
|
crashingWal.next = csWal
|
|
// reset the message counter
|
|
crashingWal.msgIndex = 1
|
|
cs.wal = crashingWal
|
|
|
|
// start consensus state
|
|
_, err = cs.Start()
|
|
require.NoError(t, err)
|
|
|
|
i++
|
|
|
|
select {
|
|
case err := <-walPaniced:
|
|
t.Logf("WAL paniced: %v", err)
|
|
|
|
// make sure we can make blocks after a crash
|
|
startNewConsensusStateAndWaitForBlock(t, cs.Height, blockDB, stateDB)
|
|
|
|
// stop consensus state and transactions sender (initFn)
|
|
cs.Stop()
|
|
cancel()
|
|
|
|
// if we reached the required height, exit
|
|
if _, ok := err.(ReachedHeightToStopError); ok {
|
|
break LOOP
|
|
}
|
|
case <-time.After(10 * time.Second):
|
|
t.Fatal("WAL did not panic for 10 seconds (check the log)")
|
|
}
|
|
}
|
|
}
|
|
|
|
// crashingWAL is a WAL which crashes or rather simulates a crash during Save
|
|
// (before and after). It remembers a message for which we last panicked
|
|
// (lastPanicedForMsgIndex), so we don't panic for it in subsequent iterations.
|
|
type crashingWAL struct {
|
|
next WAL
|
|
panicCh chan error
|
|
heightToStop uint64
|
|
|
|
msgIndex int // current message index
|
|
lastPanicedForMsgIndex int // last message for which we panicked
|
|
}
|
|
|
|
// WALWriteError indicates a WAL crash.
|
|
type WALWriteError struct {
|
|
msg string
|
|
}
|
|
|
|
func (e WALWriteError) Error() string {
|
|
return e.msg
|
|
}
|
|
|
|
// ReachedHeightToStopError indicates we've reached the required consensus
|
|
// height and may exit.
|
|
type ReachedHeightToStopError struct {
|
|
height uint64
|
|
}
|
|
|
|
func (e ReachedHeightToStopError) Error() string {
|
|
return fmt.Sprintf("reached height to stop %d", e.height)
|
|
}
|
|
|
|
// Save simulate WAL's crashing by sending an error to the panicCh and then
|
|
// exiting the cs.receiveRoutine.
|
|
func (w *crashingWAL) Save(m WALMessage) {
|
|
if endMsg, ok := m.(EndHeightMessage); ok {
|
|
if endMsg.Height == w.heightToStop {
|
|
w.panicCh <- ReachedHeightToStopError{endMsg.Height}
|
|
runtime.Goexit()
|
|
} else {
|
|
w.next.Save(m)
|
|
}
|
|
return
|
|
}
|
|
|
|
if w.msgIndex > w.lastPanicedForMsgIndex {
|
|
w.lastPanicedForMsgIndex = w.msgIndex
|
|
_, file, line, _ := runtime.Caller(1)
|
|
w.panicCh <- WALWriteError{fmt.Sprintf("failed to write %T to WAL (fileline: %s:%d)", m, file, line)}
|
|
runtime.Goexit()
|
|
} else {
|
|
w.msgIndex++
|
|
w.next.Save(m)
|
|
}
|
|
}
|
|
|
|
func (w *crashingWAL) Group() *auto.Group { return w.next.Group() }
|
|
func (w *crashingWAL) SearchForEndHeight(height uint64) (gr *auto.GroupReader, found bool, err error) {
|
|
return w.next.SearchForEndHeight(height)
|
|
}
|
|
|
|
func (w *crashingWAL) Start() (bool, error) { return w.next.Start() }
|
|
func (w *crashingWAL) Stop() bool { return w.next.Stop() }
|
|
func (w *crashingWAL) Wait() { w.next.Wait() }
|
|
|
|
//------------------------------------------------------------------------------------------
|
|
// Handshake Tests
|
|
|
|
var (
|
|
NUM_BLOCKS = 6 // number of blocks in the test_data/many_blocks.cswal
|
|
mempool = types.MockMempool{}
|
|
)
|
|
|
|
//---------------------------------------
|
|
// 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) {
|
|
for _, m := range modes {
|
|
testHandshakeReplay(t, 0, m)
|
|
}
|
|
}
|
|
|
|
// Sync many, not from scratch
|
|
func TestHandshakeReplaySome(t *testing.T) {
|
|
for _, m := range modes {
|
|
testHandshakeReplay(t, 1, m)
|
|
}
|
|
}
|
|
|
|
// Sync from lagging by one
|
|
func TestHandshakeReplayOne(t *testing.T) {
|
|
for _, m := range modes {
|
|
testHandshakeReplay(t, NUM_BLOCKS-1, m)
|
|
}
|
|
}
|
|
|
|
// Sync from caught up
|
|
func TestHandshakeReplayNone(t *testing.T) {
|
|
for _, m := range modes {
|
|
testHandshakeReplay(t, NUM_BLOCKS, m)
|
|
}
|
|
}
|
|
|
|
func writeWAL(walMsgs []byte) string {
|
|
walFile, err := ioutil.TempFile("", "wal")
|
|
if err != nil {
|
|
panic(fmt.Errorf("failed to create temp WAL file: %v", err))
|
|
}
|
|
_, err = walFile.Write(walMsgs)
|
|
if err != nil {
|
|
panic(fmt.Errorf("failed to write to temp WAL file: %v", err))
|
|
}
|
|
if err := walFile.Close(); err != nil {
|
|
panic(fmt.Errorf("failed to close temp WAL file: %v", err))
|
|
}
|
|
return walFile.Name()
|
|
}
|
|
|
|
// 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, mode uint) {
|
|
config := ResetConfig("proxy_test_")
|
|
|
|
// copy the many_blocks file
|
|
walBody, err := cmn.ReadFile(path.Join(data_dir, "many_blocks.cswal"))
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
walFile := writeWAL(walBody)
|
|
config.Consensus.SetWalFile(walFile)
|
|
|
|
privVal := types.LoadPrivValidatorFS(config.PrivValidatorFile())
|
|
|
|
wal, err := NewWAL(walFile, false)
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
wal.SetLogger(log.TestingLogger())
|
|
if _, err := wal.Start(); err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
chain, commits, err := makeBlockchainFromWAL(wal)
|
|
if err != nil {
|
|
t.Fatalf(err.Error())
|
|
}
|
|
|
|
state, store := stateAndStore(config, privVal.GetPubKey())
|
|
store.chain = chain
|
|
store.commits = commits
|
|
|
|
// run the chain through state.ApplyBlock to build up the tendermint state
|
|
latestAppHash := buildTMStateFromChain(config, state, chain, mode)
|
|
|
|
// make a new client creator
|
|
dummyApp := dummy.NewPersistentDummyApplication(path.Join(config.DBDir(), "2"))
|
|
clientCreator2 := proxy.NewLocalClientCreator(dummyApp)
|
|
if nBlocks > 0 {
|
|
// run nBlocks against a new client to build up the app state.
|
|
// use a throwaway tendermint state
|
|
proxyApp := proxy.NewAppConns(clientCreator2, nil)
|
|
state, _ := stateAndStore(config, privVal.GetPubKey())
|
|
buildAppStateFromChain(proxyApp, state, chain, nBlocks, mode)
|
|
}
|
|
|
|
// now start the app using the handshake - it should sync
|
|
handshaker := NewHandshaker(state, store)
|
|
proxyApp := proxy.NewAppConns(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(abci.RequestInfo{""})
|
|
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)
|
|
}
|
|
|
|
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) {
|
|
testPartSize := st.Params.BlockPartSizeBytes
|
|
err := st.ApplyBlock(types.NopEventBus{}, 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)
|
|
}
|
|
|
|
validators := types.TM2PB.Validators(state.Validators)
|
|
proxyApp.Consensus().InitChainSync(abci.RequestInitChain{validators})
|
|
|
|
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.DBDir(), "1")))
|
|
proxyApp := proxy.NewAppConns(clientCreator, nil) // sm.NewHandshaker(config, state, store, ReplayLastBlock))
|
|
if _, err := proxyApp.Start(); err != nil {
|
|
panic(err)
|
|
}
|
|
defer proxyApp.Stop()
|
|
|
|
validators := types.TM2PB.Validators(state.Validators)
|
|
proxyApp.Consensus().InitChainSync(abci.RequestInitChain{validators})
|
|
|
|
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
|
|
}
|
|
|
|
//--------------------------
|
|
// utils for making blocks
|
|
|
|
func makeBlockchainFromWAL(wal WAL) ([]*types.Block, []*types.Commit, error) {
|
|
// Search for height marker
|
|
gr, found, err := wal.SearchForEndHeight(0)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
if !found {
|
|
return nil, nil, errors.New(cmn.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
|
|
|
|
dec := NewWALDecoder(gr)
|
|
for {
|
|
msg, err := dec.Decode()
|
|
if err == io.EOF {
|
|
break
|
|
} else if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
piece := readPieceFromWAL(msg)
|
|
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(), 0, &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(), 0, &n, &err).(*types.Block)
|
|
blocks = append(blocks, block)
|
|
return blocks, commits, nil
|
|
}
|
|
|
|
func readPieceFromWAL(msg *TimedWALMessage) interface{} {
|
|
// skip meta messages
|
|
if _, ok := msg.Msg.(EndHeightMessage); ok {
|
|
return nil
|
|
}
|
|
|
|
// for logging
|
|
switch m := msg.Msg.(type) {
|
|
case msgInfo:
|
|
switch msg := m.Msg.(type) {
|
|
case *ProposalMessage:
|
|
return &msg.Proposal.BlockPartsHeader
|
|
case *BlockPartMessage:
|
|
return msg.Part
|
|
case *VoteMessage:
|
|
return msg.Vote
|
|
}
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
// fresh state and mock store
|
|
func stateAndStore(config *cfg.Config, pubKey crypto.PubKey) (*sm.State, *mockBlockStore) {
|
|
stateDB := dbm.NewMemDB()
|
|
state, _ := sm.MakeGenesisStateFromFile(stateDB, config.GenesisFile())
|
|
state.SetLogger(log.TestingLogger().With("module", "state"))
|
|
|
|
store := NewMockBlockStore(config, state.Params)
|
|
return state, store
|
|
}
|
|
|
|
//----------------------------------
|
|
// mock block store
|
|
|
|
type mockBlockStore struct {
|
|
config *cfg.Config
|
|
params types.ConsensusParams
|
|
chain []*types.Block
|
|
commits []*types.Commit
|
|
}
|
|
|
|
// TODO: NewBlockStore(db.NewMemDB) ...
|
|
func NewMockBlockStore(config *cfg.Config, params types.ConsensusParams) *mockBlockStore {
|
|
return &mockBlockStore{config, params, 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.params.BlockPartSizeBytes).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]
|
|
}
|