Skip to content

Commit

Permalink
chore: add --hard flag to rollback command to remove block as well (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
cmwaters committed Sep 20, 2023
1 parent 4132340 commit 86e67b7
Show file tree
Hide file tree
Showing 9 changed files with 227 additions and 14 deletions.
26 changes: 19 additions & 7 deletions cmd/cometbft/commands/rollback.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,32 +14,44 @@ import (
"github.com/cometbft/cometbft/store"
)

var removeBlock = false

func init() {
RollbackStateCmd.Flags().BoolVar(&removeBlock, "hard", false, "remove last block as well as state")
}

var RollbackStateCmd = &cobra.Command{
Use: "rollback",
Short: "rollback CometBFT state by one height",
Long: `
A state rollback is performed to recover from an incorrect application state transition,
when CometBFT has persisted an incorrect app hash and is thus unable to make
progress. Rollback overwrites a state at height n with the state at height n - 1.
The application should also roll back to height n - 1. No blocks are removed, so upon
restarting CometBFT the transactions in block n will be re-executed against the
application.
The application should also roll back to height n - 1. If the --hard flag is not used,
no blocks will be removed so upon restarting Tendermint the transactions in block n will be
re-executed against the application. Using --hard will also remove block n. This can
be done multiple times.
`,
RunE: func(cmd *cobra.Command, args []string) error {
height, hash, err := RollbackState(config)
height, hash, err := RollbackState(config, removeBlock)
if err != nil {
return fmt.Errorf("failed to rollback state: %w", err)
}

fmt.Printf("Rolled back state to height %d and hash %v", height, hash)
if removeBlock {
fmt.Printf("Rolled back both state and block to height %d and hash %X\n", height, hash)
} else {
fmt.Printf("Rolled back state to height %d and hash %X\n", height, hash)
}

return nil
},
}

// RollbackState takes the state at the current height n and overwrites it with the state
// at height n - 1. Note state here refers to CometBFT state not application state.
// Returns the latest state height and app hash alongside an error if there was one.
func RollbackState(config *cfg.Config) (int64, []byte, error) {
func RollbackState(config *cfg.Config, removeBlock bool) (int64, []byte, error) {
// use the parsed config to load the block and state store
blockStore, stateStore, err := loadStateAndBlockStore(config)
if err != nil {
Expand All @@ -51,7 +63,7 @@ func RollbackState(config *cfg.Config) (int64, []byte, error) {
}()

// rollback the last state
return state.Rollback(blockStore, stateStore)
return state.Rollback(blockStore, stateStore, removeBlock)
}

func loadStateAndBlockStore(config *cfg.Config) (*store.BlockStore, state.Store, error) {
Expand Down
2 changes: 2 additions & 0 deletions consensus/replay_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1241,6 +1241,8 @@ func (bs *mockBlockStore) PruneBlocks(height int64) (uint64, error) {
return pruned, nil
}

func (bs *mockBlockStore) DeleteLatestBlock() error { return nil }

//---------------------------------------
// Test handshake/init chain

Expand Down
1 change: 1 addition & 0 deletions rpc/core/blocks_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -298,6 +298,7 @@ func (mockBlockStore) LoadSeenCommit(height int64) *types.Commit { retur
func (mockBlockStore) PruneBlocks(height int64) (uint64, error) { return 0, nil }
func (mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) {
}
func (mockBlockStore) DeleteLatestBlock() error { return nil }

func (store mockBlockStore) LoadBlockMeta(height int64) *types.BlockMeta {
if height > store.height {
Expand Down
14 changes: 14 additions & 0 deletions state/mocks/block_store.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

19 changes: 16 additions & 3 deletions state/rollback.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@ import (
// Rollback overwrites the current CometBFT state (height n) with the most
// recent previous state (height n - 1).
// Note that this function does not affect application state.
func Rollback(bs BlockStore, ss Store) (int64, []byte, error) {
func Rollback(bs BlockStore, ss Store, removeBlock bool) (int64, []byte, error) {
invalidState, err := ss.Load()
if err != nil {
return -1, nil, err
Expand All @@ -24,9 +24,14 @@ func Rollback(bs BlockStore, ss Store) (int64, []byte, error) {
height := bs.Height()

// NOTE: persistence of state and blocks don't happen atomically. Therefore it is possible that
// when the user stopped the node the state wasn't updated but the blockstore was. In this situation
// we don't need to rollback any state and can just return early
// when the user stopped the node the state wasn't updated but the blockstore was. Discard the
// pending block before continuing.
if height == invalidState.LastBlockHeight+1 {
if removeBlock {
if err := bs.DeleteLatestBlock(); err != nil {
return -1, nil, fmt.Errorf("failed to remove final block from blockstore: %w", err)
}
}
return invalidState.LastBlockHeight, invalidState.AppHash, nil
}

Expand Down Expand Up @@ -108,5 +113,13 @@ func Rollback(bs BlockStore, ss Store) (int64, []byte, error) {
return -1, nil, fmt.Errorf("failed to save rolled back state: %w", err)
}

// If removeBlock is true then also remove the block associated with the previous state.
// This will mean both the last state and last block height is equal to n - 1
if removeBlock {
if err := bs.DeleteLatestBlock(); err != nil {
return -1, nil, fmt.Errorf("failed to remove final block from blockstore: %w", err)
}
}

return rolledBackState.LastBlockHeight, rolledBackState.AppHash, nil
}
127 changes: 123 additions & 4 deletions state/rollback_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ package state_test
import (
"crypto/rand"
"testing"
"time"

dbm "github.com/cometbft/cometbft-db"
"github.com/stretchr/testify/require"
Expand All @@ -13,6 +14,7 @@ import (
cmtversion "github.com/cometbft/cometbft/proto/tendermint/version"
"github.com/cometbft/cometbft/state"
"github.com/cometbft/cometbft/state/mocks"
"github.com/cometbft/cometbft/store"
"github.com/cometbft/cometbft/types"
"github.com/cometbft/cometbft/version"
)
Expand Down Expand Up @@ -50,6 +52,7 @@ func TestRollback(t *testing.T) {
BlockID: initialState.LastBlockID,
Header: types.Header{
Height: initialState.LastBlockHeight,
Time: initialState.LastBlockTime,
AppHash: crypto.CRandBytes(tmhash.Size),
LastBlockID: makeBlockIDRandom(),
LastResultsHash: initialState.LastResultsHash,
Expand All @@ -61,6 +64,7 @@ func TestRollback(t *testing.T) {
Height: nextState.LastBlockHeight,
AppHash: initialState.AppHash,
LastBlockID: block.BlockID,
Time: nextState.LastBlockTime,
LastResultsHash: nextState.LastResultsHash,
},
}
Expand All @@ -69,7 +73,7 @@ func TestRollback(t *testing.T) {
blockStore.On("Height").Return(nextHeight)

// rollback the state
rollbackHeight, rollbackHash, err := state.Rollback(blockStore, stateStore)
rollbackHeight, rollbackHash, err := state.Rollback(blockStore, stateStore, false)
require.NoError(t, err)
require.EqualValues(t, height, rollbackHeight)
require.EqualValues(t, initialState.AppHash, rollbackHash)
Expand All @@ -81,14 +85,128 @@ func TestRollback(t *testing.T) {
require.EqualValues(t, initialState, loadedState)
}

func TestRollbackHard(t *testing.T) {
const height int64 = 100
blockStore := store.NewBlockStore(dbm.NewMemDB())
stateStore := state.NewStore(dbm.NewMemDB(), state.StoreOptions{DiscardABCIResponses: false})

valSet, _ := types.RandValidatorSet(5, 10)

params := types.DefaultConsensusParams()
params.Version.App = 10
now := time.Date(2020, 1, 1, 0, 0, 0, 0, time.UTC)

block := &types.Block{
Header: types.Header{
Version: cmtversion.Consensus{Block: version.BlockProtocol, App: 1},
ChainID: "test-chain",
Time: now,
Height: height,
AppHash: crypto.CRandBytes(tmhash.Size),
LastBlockID: makeBlockIDRandom(),
LastCommitHash: crypto.CRandBytes(tmhash.Size),
DataHash: crypto.CRandBytes(tmhash.Size),
ValidatorsHash: valSet.Hash(),
NextValidatorsHash: valSet.CopyIncrementProposerPriority(1).Hash(),
ConsensusHash: params.Hash(),
LastResultsHash: crypto.CRandBytes(tmhash.Size),
EvidenceHash: crypto.CRandBytes(tmhash.Size),
ProposerAddress: crypto.CRandBytes(crypto.AddressSize),
},
LastCommit: &types.Commit{Height: height - 1},
}

partSet := block.MakePartSet(types.BlockPartSizeBytes)
blockStore.SaveBlock(block, partSet, &types.Commit{Height: block.Height})

currState := state.State{
Version: cmtstate.Version{
Consensus: block.Header.Version,
Software: version.TMCoreSemVer,
},
LastBlockHeight: block.Height,
LastBlockTime: block.Time,
AppHash: crypto.CRandBytes(tmhash.Size),
LastValidators: valSet,
Validators: valSet.CopyIncrementProposerPriority(1),
NextValidators: valSet.CopyIncrementProposerPriority(2),
ConsensusParams: *params,
LastHeightConsensusParamsChanged: height + 1,
LastHeightValidatorsChanged: height + 1,
LastResultsHash: crypto.CRandBytes(tmhash.Size),
}
require.NoError(t, stateStore.Bootstrap(currState))

nextBlock := &types.Block{
Header: types.Header{
Version: cmtversion.Consensus{Block: version.BlockProtocol, App: 1},
ChainID: block.ChainID,
Time: block.Time,
Height: currState.LastBlockHeight + 1,
AppHash: currState.AppHash,
LastBlockID: types.BlockID{Hash: block.Hash(), PartSetHeader: partSet.Header()},
LastCommitHash: crypto.CRandBytes(tmhash.Size),
DataHash: crypto.CRandBytes(tmhash.Size),
ValidatorsHash: valSet.CopyIncrementProposerPriority(1).Hash(),
NextValidatorsHash: valSet.CopyIncrementProposerPriority(2).Hash(),
ConsensusHash: params.Hash(),
LastResultsHash: currState.LastResultsHash,
EvidenceHash: crypto.CRandBytes(tmhash.Size),
ProposerAddress: crypto.CRandBytes(crypto.AddressSize),
},
LastCommit: &types.Commit{Height: currState.LastBlockHeight},
}

nextPartSet := nextBlock.MakePartSet(types.BlockPartSizeBytes)
blockStore.SaveBlock(nextBlock, nextPartSet, &types.Commit{Height: nextBlock.Height})

rollbackHeight, rollbackHash, err := state.Rollback(blockStore, stateStore, true)
require.NoError(t, err)
require.Equal(t, rollbackHeight, currState.LastBlockHeight)
require.Equal(t, rollbackHash, currState.AppHash)

// state should not have been changed
loadedState, err := stateStore.Load()
require.NoError(t, err)
require.Equal(t, currState, loadedState)

// resave the same block
blockStore.SaveBlock(nextBlock, nextPartSet, &types.Commit{Height: nextBlock.Height})

params.Version.App = 11

nextState := state.State{
Version: cmtstate.Version{
Consensus: block.Header.Version,
Software: version.TMCoreSemVer,
},
LastBlockHeight: nextBlock.Height,
LastBlockTime: nextBlock.Time,
AppHash: crypto.CRandBytes(tmhash.Size),
LastValidators: valSet.CopyIncrementProposerPriority(1),
Validators: valSet.CopyIncrementProposerPriority(2),
NextValidators: valSet.CopyIncrementProposerPriority(3),
ConsensusParams: *params,
LastHeightConsensusParamsChanged: nextBlock.Height + 1,
LastHeightValidatorsChanged: nextBlock.Height + 1,
LastResultsHash: crypto.CRandBytes(tmhash.Size),
}
require.NoError(t, stateStore.Save(nextState))

rollbackHeight, rollbackHash, err = state.Rollback(blockStore, stateStore, true)
require.NoError(t, err)
require.Equal(t, rollbackHeight, currState.LastBlockHeight)
require.Equal(t, rollbackHash, currState.AppHash)
}

func TestRollbackNoState(t *testing.T) {
stateStore := state.NewStore(dbm.NewMemDB(),
state.StoreOptions{
DiscardABCIResponses: false,
})
blockStore := &mocks.BlockStore{}

_, _, err := state.Rollback(blockStore, stateStore)
_, _, err := state.Rollback(blockStore, stateStore, false)
require.Error(t, err)
require.Contains(t, err.Error(), "no state found")
}
Expand All @@ -101,7 +219,7 @@ func TestRollbackNoBlocks(t *testing.T) {
blockStore.On("LoadBlockMeta", height).Return(nil)
blockStore.On("LoadBlockMeta", height-1).Return(nil)

_, _, err := state.Rollback(blockStore, stateStore)
_, _, err := state.Rollback(blockStore, stateStore, false)
require.Error(t, err)
require.Contains(t, err.Error(), "block at height 99 not found")
}
Expand All @@ -112,7 +230,7 @@ func TestRollbackDifferentStateHeight(t *testing.T) {
blockStore := &mocks.BlockStore{}
blockStore.On("Height").Return(height + 2)

_, _, err := state.Rollback(blockStore, stateStore)
_, _, err := state.Rollback(blockStore, stateStore, false)
require.Error(t, err)
require.Equal(t, err.Error(), "statestore height (100) is not one below or equal to blockstore height (102)")
}
Expand All @@ -138,6 +256,7 @@ func setupStateStore(t *testing.T, height int64) state.Store {
AppHash: tmhash.Sum([]byte("app_hash")),
LastResultsHash: tmhash.Sum([]byte("last_results_hash")),
LastBlockHeight: height,
LastBlockTime: time.Now(),
LastValidators: valSet,
Validators: valSet.CopyIncrementProposerPriority(1),
NextValidators: valSet.CopyIncrementProposerPriority(2),
Expand Down
2 changes: 2 additions & 0 deletions state/services.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,8 @@ type BlockStore interface {

LoadBlockCommit(height int64) *types.Commit
LoadSeenCommit(height int64) *types.Commit

DeleteLatestBlock() error
}

//-----------------------------------------------------------------------------
Expand Down
47 changes: 47 additions & 0 deletions store/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -521,3 +521,50 @@ func mustEncode(pb proto.Message) []byte {
}
return bz
}

//-----------------------------------------------------------------------------

// DeleteLatestBlock removes the block pointed to by height,
// lowering height by one.
func (bs *BlockStore) DeleteLatestBlock() error {
bs.mtx.RLock()
targetHeight := bs.height
bs.mtx.RUnlock()

batch := bs.db.NewBatch()
defer batch.Close()

// delete what we can, skipping what's already missing, to ensure partial
// blocks get deleted fully.
if meta := bs.LoadBlockMeta(targetHeight); meta != nil {
if err := batch.Delete(calcBlockHashKey(meta.BlockID.Hash)); err != nil {
return err
}
for p := 0; p < int(meta.BlockID.PartSetHeader.Total); p++ {
if err := batch.Delete(calcBlockPartKey(targetHeight, p)); err != nil {
return err
}
}
}
if err := batch.Delete(calcBlockCommitKey(targetHeight)); err != nil {
return err
}
if err := batch.Delete(calcSeenCommitKey(targetHeight)); err != nil {
return err
}
// delete last, so as to not leave keys built on meta.BlockID dangling
if err := batch.Delete(calcBlockMetaKey(targetHeight)); err != nil {
return err
}

bs.mtx.Lock()
bs.height = targetHeight - 1
bs.mtx.Unlock()
bs.saveState()

err := batch.WriteSync()
if err != nil {
return fmt.Errorf("failed to delete height %v: %w", targetHeight, err)
}
return nil
}
Loading

0 comments on commit 86e67b7

Please sign in to comment.