Browse Source

cs/replay: check appHash for each block (#3579)

* check every block appHash

Fixes #3460

Not really needed, but it would detect if the app changed in a way it
shouldn't have.

* add a changelog entry

* no need to return an error if we panic

* rename methods

* rename methods once again

* add a test

* correct error msg

plus fix a few go-lint warnings

* better panic msg
pull/3594/head
Anton Kaliaev 5 years ago
committed by GitHub
parent
commit
ebf815ee57
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 223 additions and 55 deletions
  1. +1
    -0
      CHANGELOG_PENDING.md
  2. +42
    -16
      consensus/replay.go
  3. +180
    -39
      consensus/replay_test.go

+ 1
- 0
CHANGELOG_PENDING.md View File

@ -18,6 +18,7 @@
### IMPROVEMENTS:
- [rpc] [\#3534](https://github.com/tendermint/tendermint/pull/3534) Add support for batched requests/responses in JSON RPC
- [cs/replay] \#3460 check appHash for each block
### BUG FIXES:
- [state] [\#3537](https://github.com/tendermint/tendermint/pull/3537#issuecomment-482711833) LoadValidators: do not return an empty validator set


+ 42
- 16
consensus/replay.go View File

@ -13,7 +13,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
//auto "github.com/tendermint/tendermint/libs/autofile"
cmn "github.com/tendermint/tendermint/libs/common"
dbm "github.com/tendermint/tendermint/libs/db"
"github.com/tendermint/tendermint/libs/log"
@ -230,6 +230,7 @@ func (h *Handshaker) SetEventBus(eventBus types.BlockEventPublisher) {
h.eventBus = eventBus
}
// NBlocks returns the number of blocks applied to the state.
func (h *Handshaker) NBlocks() int {
return h.nBlocks
}
@ -263,7 +264,7 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error {
// Replay blocks up to the latest in the blockstore.
_, err = h.ReplayBlocks(h.initialState, appHash, blockHeight, proxyApp)
if err != nil {
return fmt.Errorf("Error on replay: %v", err)
return fmt.Errorf("error on replay: %v", err)
}
h.logger.Info("Completed ABCI Handshake - Tendermint and App are synced",
@ -274,7 +275,8 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error {
return nil
}
// Replay all blocks since appBlockHeight and ensure the result matches the current state.
// ReplayBlocks replays all blocks since appBlockHeight and ensures the result
// matches the current state.
// Returns the final AppHash or an error.
func (h *Handshaker) ReplayBlocks(
state sm.State,
@ -319,7 +321,7 @@ func (h *Handshaker) ReplayBlocks(
} else {
// If validator set is not set in genesis and still empty after InitChain, exit.
if len(h.genDoc.Validators) == 0 {
return nil, fmt.Errorf("Validator set is nil in genesis and still empty after InitChain")
return nil, fmt.Errorf("validator set is nil in genesis and still empty after InitChain")
}
}
@ -332,7 +334,8 @@ func (h *Handshaker) ReplayBlocks(
// First handle edge cases and constraints on the storeBlockHeight.
if storeBlockHeight == 0 {
return appHash, checkAppHash(state, appHash)
assertAppHashEqualsOneFromState(appHash, state)
return appHash, nil
} else if storeBlockHeight < appBlockHeight {
// the app should never be ahead of the store (but this is under app's control)
@ -340,11 +343,11 @@ func (h *Handshaker) ReplayBlocks(
} else if storeBlockHeight < stateBlockHeight {
// the state should never be ahead of the store (this is under tendermint's control)
cmn.PanicSanity(fmt.Sprintf("StateBlockHeight (%d) > StoreBlockHeight (%d)", stateBlockHeight, storeBlockHeight))
panic(fmt.Sprintf("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)
cmn.PanicSanity(fmt.Sprintf("StoreBlockHeight (%d) > StateBlockHeight + 1 (%d)", storeBlockHeight, stateBlockHeight+1))
panic(fmt.Sprintf("StoreBlockHeight (%d) > StateBlockHeight + 1 (%d)", storeBlockHeight, stateBlockHeight+1))
}
var err error
@ -359,7 +362,8 @@ func (h *Handshaker) ReplayBlocks(
} else if appBlockHeight == storeBlockHeight {
// We're good!
return appHash, checkAppHash(state, appHash)
assertAppHashEqualsOneFromState(appHash, state)
return appHash, nil
}
} else if storeBlockHeight == stateBlockHeight+1 {
@ -380,7 +384,7 @@ func (h *Handshaker) ReplayBlocks(
return state.AppHash, err
} else if appBlockHeight == storeBlockHeight {
// We ran Commit, but didn't save the state, so replayBlock with mock app
// We ran Commit, but didn't save the state, so replayBlock with mock app.
abciResponses, err := sm.LoadABCIResponses(h.stateDB, storeBlockHeight)
if err != nil {
return nil, err
@ -393,8 +397,8 @@ func (h *Handshaker) ReplayBlocks(
}
cmn.PanicSanity("Should never happen")
return nil, nil
panic(fmt.Sprintf("uncovered case! appHeight: %d, storeHeight: %d, stateHeight: %d",
appBlockHeight, storeBlockHeight, stateBlockHeight))
}
func (h *Handshaker) replayBlocks(state sm.State, proxyApp proxy.AppConns, appBlockHeight, storeBlockHeight int64, mutateState bool) ([]byte, error) {
@ -417,6 +421,12 @@ func (h *Handshaker) replayBlocks(state sm.State, proxyApp proxy.AppConns, appBl
for i := appBlockHeight + 1; i <= finalBlock; i++ {
h.logger.Info("Applying block", "height", i)
block := h.store.LoadBlock(i)
// Extra check to ensure the app was not changed in a way it shouldn't have.
if len(appHash) > 0 {
assertAppHashEqualsOneFromBlock(appHash, block)
}
appHash, err = sm.ExecCommitBlock(proxyApp.Consensus(), block, h.logger, state.LastValidators, h.stateDB)
if err != nil {
return nil, err
@ -434,7 +444,8 @@ func (h *Handshaker) replayBlocks(state sm.State, proxyApp proxy.AppConns, appBl
appHash = state.AppHash
}
return appHash, checkAppHash(state, appHash)
assertAppHashEqualsOneFromState(appHash, state)
return appHash, nil
}
// ApplyBlock on the proxyApp with the last block.
@ -456,11 +467,26 @@ func (h *Handshaker) replayBlock(state sm.State, height int64, proxyApp proxy.Ap
return state, nil
}
func checkAppHash(state sm.State, appHash []byte) error {
if !bytes.Equal(state.AppHash, appHash) {
panic(fmt.Errorf("Tendermint state.AppHash does not match AppHash after replay. Got %X, expected %X", appHash, state.AppHash).Error())
func assertAppHashEqualsOneFromBlock(appHash []byte, block *types.Block) {
if !bytes.Equal(appHash, block.AppHash) {
panic(fmt.Sprintf(`block.AppHash does not match AppHash after replay. Got %X, expected %X.
Block: %v
`,
appHash, block.AppHash, block))
}
}
func assertAppHashEqualsOneFromState(appHash []byte, state sm.State) {
if !bytes.Equal(appHash, state.AppHash) {
panic(fmt.Sprintf(`state.AppHash does not match AppHash after replay. Got
%X, expected %X.
State: %v
Did you reset Tendermint without resetting your application's data?`,
appHash, state.AppHash, state))
}
return nil
}
//--------------------------------------------------------------------------------


+ 180
- 39
consensus/replay_test.go View File

@ -7,7 +7,7 @@ import (
"io"
"io/ioutil"
"os"
"path"
"path/filepath"
"runtime"
"testing"
"time"
@ -19,12 +19,14 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto"
cmn "github.com/tendermint/tendermint/libs/common"
dbm "github.com/tendermint/tendermint/libs/db"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/privval"
"github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
tmtime "github.com/tendermint/tendermint/types/time"
"github.com/tendermint/tendermint/version"
)
@ -88,7 +90,7 @@ func startNewConsensusStateAndWaitForBlock(t *testing.T, consensusReplayConfig *
}
}
func sendTxs(cs *ConsensusState, ctx context.Context) {
func sendTxs(ctx context.Context, cs *ConsensusState) {
for i := 0; i < 256; i++ {
select {
case <-ctx.Done():
@ -113,7 +115,7 @@ func TestWALCrash(t *testing.T) {
1},
{"many non-empty blocks",
func(stateDB dbm.DB, cs *ConsensusState, ctx context.Context) {
go sendTxs(cs, ctx)
go sendTxs(ctx, cs)
},
3},
}
@ -263,7 +265,7 @@ func (w *crashingWAL) Wait() { w.next.Wait() }
// Handshake Tests
const (
NUM_BLOCKS = 6
numBlocks = 6
)
var (
@ -284,6 +286,7 @@ func TestHandshakeReplayAll(t *testing.T) {
for i, m := range modes {
config := ResetConfig(fmt.Sprintf("%s_%v", t.Name(), i))
defer os.RemoveAll(config.RootDir)
testHandshakeReplay(t, config, 0, m)
}
}
@ -293,6 +296,7 @@ func TestHandshakeReplaySome(t *testing.T) {
for i, m := range modes {
config := ResetConfig(fmt.Sprintf("%s_%v", t.Name(), i))
defer os.RemoveAll(config.RootDir)
testHandshakeReplay(t, config, 1, m)
}
}
@ -302,7 +306,8 @@ func TestHandshakeReplayOne(t *testing.T) {
for i, m := range modes {
config := ResetConfig(fmt.Sprintf("%s_%v", t.Name(), i))
defer os.RemoveAll(config.RootDir)
testHandshakeReplay(t, config, NUM_BLOCKS-1, m)
testHandshakeReplay(t, config, numBlocks-1, m)
}
}
@ -311,28 +316,30 @@ func TestHandshakeReplayNone(t *testing.T) {
for i, m := range modes {
config := ResetConfig(fmt.Sprintf("%s_%v", t.Name(), i))
defer os.RemoveAll(config.RootDir)
testHandshakeReplay(t, config, NUM_BLOCKS, m)
testHandshakeReplay(t, config, numBlocks, m)
}
}
func tempWALWithData(data []byte) string {
walFile, err := ioutil.TempFile("", "wal")
if err != nil {
panic(fmt.Errorf("failed to create temp WAL file: %v", err))
panic(fmt.Sprintf("failed to create temp WAL file: %v", err))
}
_, err = walFile.Write(data)
if err != nil {
panic(fmt.Errorf("failed to write to temp WAL file: %v", err))
panic(fmt.Sprintf("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))
panic(fmt.Sprintf("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
// 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, config *cfg.Config, nBlocks int, mode uint) {
walBody, err := WALWithNBlocks(t, NUM_BLOCKS)
walBody, err := WALWithNBlocks(t, numBlocks)
require.NoError(t, err)
walFile := tempWALWithData(walBody)
config.Consensus.SetWalFile(walFile)
@ -354,11 +361,16 @@ func testHandshakeReplay(t *testing.T, config *cfg.Config, nBlocks int, mode uin
store.commits = commits
// run the chain through state.ApplyBlock to build up the tendermint state
state = buildTMStateFromChain(config, stateDB, state, chain, mode)
clientCreator := proxy.NewLocalClientCreator(kvstore.NewPersistentKVStoreApplication(filepath.Join(config.DBDir(), "1")))
proxyApp := proxy.NewAppConns(clientCreator)
err = proxyApp.Start()
require.NoError(t, err)
state = buildTMStateFromChain(config, stateDB, state, chain, proxyApp, mode)
proxyApp.Stop()
latestAppHash := state.AppHash
// make a new client creator
kvstoreApp := kvstore.NewPersistentKVStoreApplication(path.Join(config.DBDir(), "2"))
kvstoreApp := kvstore.NewPersistentKVStoreApplication(filepath.Join(config.DBDir(), "2"))
clientCreator2 := proxy.NewLocalClientCreator(kvstoreApp)
if nBlocks > 0 {
// run nBlocks against a new client to build up the app state.
@ -371,7 +383,7 @@ func testHandshakeReplay(t *testing.T, config *cfg.Config, nBlocks int, mode uin
// now start the app using the handshake - it should sync
genDoc, _ := sm.MakeGenesisDocFromFile(config.GenesisFile())
handshaker := NewHandshaker(stateDB, state, store, genDoc)
proxyApp := proxy.NewAppConns(clientCreator2)
proxyApp = proxy.NewAppConns(clientCreator2)
if err := proxyApp.Start(); err != nil {
t.Fatalf("Error starting proxy app connections: %v", err)
}
@ -391,8 +403,8 @@ func testHandshakeReplay(t *testing.T, config *cfg.Config, nBlocks int, mode uin
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 := numBlocks - nBlocks
if nBlocks == numBlocks && mode > 0 {
expectedBlocksToSync++
} else if nBlocks > 0 && mode == 1 {
expectedBlocksToSync++
@ -407,7 +419,7 @@ func applyBlock(stateDB dbm.DB, st sm.State, blk *types.Block, proxyApp proxy.Ap
testPartSize := types.BlockPartSizeBytes
blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), proxyApp.Consensus(), mempool, evpool)
blkID := types.BlockID{blk.Hash(), blk.MakePartSet(testPartSize).Header()}
blkID := types.BlockID{Hash: blk.Hash(), PartsHeader: blk.MakePartSet(testPartSize).Header()}
newState, err := blockExec.ApplyBlock(st, blkID, blk)
if err != nil {
panic(err)
@ -451,14 +463,8 @@ func buildAppStateFromChain(proxyApp proxy.AppConns, stateDB dbm.DB,
}
func buildTMStateFromChain(config *cfg.Config, stateDB dbm.DB, state sm.State, chain []*types.Block, mode uint) sm.State {
// run the whole chain against this client to build up the tendermint state
clientCreator := proxy.NewLocalClientCreator(kvstore.NewPersistentKVStoreApplication(path.Join(config.DBDir(), "1")))
proxyApp := proxy.NewAppConns(clientCreator)
if err := proxyApp.Start(); err != nil {
panic(err)
}
defer proxyApp.Stop()
func buildTMStateFromChain(config *cfg.Config, stateDB dbm.DB, state sm.State,
chain []*types.Block, proxyApp proxy.AppConns, mode uint) sm.State {
validators := types.TM2PB.ValidatorUpdates(state.Validators)
if _, err := proxyApp.Consensus().InitChainSync(abci.RequestInitChain{
@ -489,28 +495,162 @@ func buildTMStateFromChain(config *cfg.Config, stateDB dbm.DB, state sm.State, c
return state
}
func TestHandshakePanicsIfAppReturnsWrongAppHash(t *testing.T) {
// 1. Initialize tendermint and commit 3 blocks with the following app hashes:
// - 0x01
// - 0x02
// - 0x03
config := ResetConfig("handshake_test_")
defer os.RemoveAll(config.RootDir)
privVal := privval.LoadFilePV(config.PrivValidatorKeyFile(), config.PrivValidatorStateFile())
const appVersion = 0x0
stateDB, state, store := stateAndStore(config, privVal.GetPubKey(), appVersion)
genDoc, _ := sm.MakeGenesisDocFromFile(config.GenesisFile())
state.LastValidators = state.Validators.Copy()
// mode = 0 for committing all the blocks
blocks := makeBlocks(3, &state, privVal)
store.chain = blocks
// 2. Tendermint must panic if app returns wrong hash for the first block
// - RANDOM HASH
// - 0x02
// - 0x03
{
app := &badApp{numBlocks: 3, allHashesAreWrong: true}
clientCreator := proxy.NewLocalClientCreator(app)
proxyApp := proxy.NewAppConns(clientCreator)
err := proxyApp.Start()
require.NoError(t, err)
defer proxyApp.Stop()
assert.Panics(t, func() {
h := NewHandshaker(stateDB, state, store, genDoc)
h.Handshake(proxyApp)
})
}
// 3. Tendermint must panic if app returns wrong hash for the last block
// - 0x01
// - 0x02
// - RANDOM HASH
{
app := &badApp{numBlocks: 3, onlyLastHashIsWrong: true}
clientCreator := proxy.NewLocalClientCreator(app)
proxyApp := proxy.NewAppConns(clientCreator)
err := proxyApp.Start()
require.NoError(t, err)
defer proxyApp.Stop()
assert.Panics(t, func() {
h := NewHandshaker(stateDB, state, store, genDoc)
h.Handshake(proxyApp)
})
}
}
func makeBlocks(n int, state *sm.State, privVal types.PrivValidator) []*types.Block {
blocks := make([]*types.Block, 0)
var (
prevBlock *types.Block
prevBlockMeta *types.BlockMeta
)
appHeight := byte(0x01)
for i := 0; i < n; i++ {
height := int64(i + 1)
block, parts := makeBlock(*state, prevBlock, prevBlockMeta, privVal, height)
blocks = append(blocks, block)
prevBlock = block
prevBlockMeta = types.NewBlockMeta(block, parts)
// update state
state.AppHash = []byte{appHeight}
appHeight++
state.LastBlockHeight = height
}
return blocks
}
func makeVote(header *types.Header, blockID types.BlockID, valset *types.ValidatorSet, privVal types.PrivValidator) *types.Vote {
addr := privVal.GetPubKey().Address()
idx, _ := valset.GetByAddress(addr)
vote := &types.Vote{
ValidatorAddress: addr,
ValidatorIndex: idx,
Height: header.Height,
Round: 1,
Timestamp: tmtime.Now(),
Type: types.PrecommitType,
BlockID: blockID,
}
privVal.SignVote(header.ChainID, vote)
return vote
}
func makeBlock(state sm.State, lastBlock *types.Block, lastBlockMeta *types.BlockMeta,
privVal types.PrivValidator, height int64) (*types.Block, *types.PartSet) {
lastCommit := types.NewCommit(types.BlockID{}, nil)
if height > 1 {
vote := makeVote(&lastBlock.Header, lastBlockMeta.BlockID, state.Validators, privVal).CommitSig()
lastCommit = types.NewCommit(lastBlockMeta.BlockID, []*types.CommitSig{vote})
}
return state.MakeBlock(height, []types.Tx{}, lastCommit, nil, state.Validators.GetProposer().Address)
}
type badApp struct {
abci.BaseApplication
numBlocks byte
height byte
allHashesAreWrong bool
onlyLastHashIsWrong bool
}
func (app *badApp) Commit() abci.ResponseCommit {
app.height++
if app.onlyLastHashIsWrong {
if app.height == app.numBlocks {
return abci.ResponseCommit{Data: cmn.RandBytes(8)}
}
return abci.ResponseCommit{Data: []byte{app.height}}
} else if app.allHashesAreWrong {
return abci.ResponseCommit{Data: cmn.RandBytes(8)}
}
panic("either allHashesAreWrong or onlyLastHashIsWrong must be set")
}
//--------------------------
// utils for making blocks
func makeBlockchainFromWAL(wal WAL) ([]*types.Block, []*types.Commit, error) {
var height int64
// Search for height marker
gr, found, err := wal.SearchForEndHeight(0, &WALSearchOptions{})
gr, found, err := wal.SearchForEndHeight(height, &WALSearchOptions{})
if err != nil {
return nil, nil, err
}
if !found {
return nil, nil, fmt.Errorf("WAL does not contain height %d.", 1)
return nil, nil, fmt.Errorf("WAL does not contain height %d", height)
}
defer gr.Close() // nolint: errcheck
// log.Notice("Build a blockchain by reading from the WAL")
var blocks []*types.Block
var commits []*types.Commit
var thisBlockParts *types.PartSet
var thisBlockCommit *types.Commit
var height int64
var (
blocks []*types.Block
commits []*types.Commit
thisBlockParts *types.PartSet
thisBlockCommit *types.Commit
)
dec := NewWALDecoder(gr)
for {
@ -602,7 +742,7 @@ func stateAndStore(config *cfg.Config, pubKey crypto.PubKey, appVersion version.
stateDB := dbm.NewMemDB()
state, _ := sm.MakeGenesisStateFromFile(config.GenesisFile())
state.Version.Consensus.App = appVersion
store := NewMockBlockStore(config, state.ConsensusParams)
store := newMockBlockStore(config, state.ConsensusParams)
return stateDB, state, store
}
@ -617,7 +757,7 @@ type mockBlockStore struct {
}
// TODO: NewBlockStore(db.NewMemDB) ...
func NewMockBlockStore(config *cfg.Config, params types.ConsensusParams) *mockBlockStore {
func newMockBlockStore(config *cfg.Config, params types.ConsensusParams) *mockBlockStore {
return &mockBlockStore{config, params, nil, nil}
}
@ -626,7 +766,7 @@ func (bs *mockBlockStore) LoadBlock(height int64) *types.Block { return bs.chain
func (bs *mockBlockStore) LoadBlockMeta(height int64) *types.BlockMeta {
block := bs.chain[height-1]
return &types.BlockMeta{
BlockID: types.BlockID{block.Hash(), block.MakePartSet(types.BlockPartSizeBytes).Header()},
BlockID: types.BlockID{Hash: block.Hash(), PartsHeader: block.MakePartSet(types.BlockPartSizeBytes).Header()},
Header: block.Header,
}
}
@ -640,15 +780,16 @@ func (bs *mockBlockStore) LoadSeenCommit(height int64) *types.Commit {
return bs.commits[height-1]
}
//----------------------------------------
//---------------------------------------
// Test handshake/init chain
func TestInitChainUpdateValidators(t *testing.T) {
func TestHandshakeUpdatesValidators(t *testing.T) {
val, _ := types.RandValidator(true, 10)
vals := types.NewValidatorSet([]*types.Validator{val})
app := &initChainApp{vals: types.TM2PB.ValidatorUpdates(vals)}
clientCreator := proxy.NewLocalClientCreator(app)
config := ResetConfig("proxy_test_")
config := ResetConfig("handshake_test_")
defer os.RemoveAll(config.RootDir)
privVal := privval.LoadFilePV(config.PrivValidatorKeyFile(), config.PrivValidatorStateFile())
stateDB, state, store := stateAndStore(config, privVal.GetPubKey(), 0x0)


Loading…
Cancel
Save