cli: add --hard flag to rollback command to remove block as well (backport #9261) (#9465)

* cli: add --hard flag to rollback command to remove block as well  (#9261)

Co-authored-by: Levi Aul <levi@leviaul.com>
(cherry picked from commit e84d43ec93)

* Fix lint

Signed-off-by: Thane Thomson <connect@thanethomson.com>

Signed-off-by: Thane Thomson <connect@thanethomson.com>
Co-authored-by: Callum Waters <cmwaters19@gmail.com>
Co-authored-by: Thane Thomson <connect@thanethomson.com>
This commit is contained in:
mergify[bot]
2022-12-21 17:56:06 -05:00
committed by GitHub
parent 37cb51c2dc
commit bd9ed68a04
10 changed files with 232 additions and 14 deletions

View File

@@ -43,6 +43,9 @@ Friendly reminder, we have a [bug bounty program](https://hackerone.com/tendermi
### IMPROVEMENTS ### IMPROVEMENTS
- [crypto] \#9250 Update to use btcec v2 and the latest btcutil. (@wcsiu) - [crypto] \#9250 Update to use btcec v2 and the latest btcutil. (@wcsiu)
- [cli] \#9171 add `--hard` flag to rollback command (and a boolean to the `RollbackState` method). This will rollback
state and remove the last block. This command can be triggered multiple times. The application must also rollback
state to the same height. (@tsutsu, @cmwaters)
- [proto] \#9356 Migrate from `gogo/protobuf` to `cosmos/gogoproto` (@julienrbrt) - [proto] \#9356 Migrate from `gogo/protobuf` to `cosmos/gogoproto` (@julienrbrt)
- [rpc] \#9276 Added `header` and `header_by_hash` queries to the RPC client (@samricotta) - [rpc] \#9276 Added `header` and `header_by_hash` queries to the RPC client (@samricotta)
- [abci] \#5706 Added `AbciVersion` to `RequestInfo` allowing applications to check ABCI version when connecting to Tendermint. (@marbar3778) - [abci] \#5706 Added `AbciVersion` to `RequestInfo` allowing applications to check ABCI version when connecting to Tendermint. (@marbar3778)

View File

@@ -14,6 +14,12 @@ import (
"github.com/tendermint/tendermint/store" "github.com/tendermint/tendermint/store"
) )
var removeBlock = false
func init() {
RollbackStateCmd.Flags().BoolVar(&removeBlock, "hard", false, "remove last block as well as state")
}
var RollbackStateCmd = &cobra.Command{ var RollbackStateCmd = &cobra.Command{
Use: "rollback", Use: "rollback",
Short: "rollback tendermint state by one height", Short: "rollback tendermint state by one height",
@@ -21,17 +27,23 @@ var RollbackStateCmd = &cobra.Command{
A state rollback is performed to recover from an incorrect application state transition, A state rollback is performed to recover from an incorrect application state transition,
when Tendermint has persisted an incorrect app hash and is thus unable to make when Tendermint 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. 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 The application should also roll back to height n - 1. If the --hard flag is not used,
restarting Tendermint the transactions in block n will be re-executed against the no blocks will be removed so upon restarting Tendermint the transactions in block n will be
application. 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 { RunE: func(cmd *cobra.Command, args []string) error {
height, hash, err := RollbackState(config) height, hash, err := RollbackState(config, removeBlock)
if err != nil { if err != nil {
return fmt.Errorf("failed to rollback state: %w", err) 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 return nil
}, },
} }
@@ -39,7 +51,7 @@ application.
// RollbackState takes the state at the current height n and overwrites it with the state // RollbackState takes the state at the current height n and overwrites it with the state
// at height n - 1. Note state here refers to tendermint state not application state. // at height n - 1. Note state here refers to tendermint state not application state.
// Returns the latest state height and app hash alongside an error if there was one. // 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 // use the parsed config to load the block and state store
blockStore, stateStore, err := loadStateAndBlockStore(config) blockStore, stateStore, err := loadStateAndBlockStore(config)
if err != nil { if err != nil {
@@ -51,7 +63,7 @@ func RollbackState(config *cfg.Config) (int64, []byte, error) {
}() }()
// rollback the last state // 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) { func loadStateAndBlockStore(config *cfg.Config) (*store.BlockStore, state.Store, error) {

View File

@@ -1195,6 +1195,8 @@ func (bs *mockBlockStore) PruneBlocks(height int64) (uint64, error) {
return pruned, nil return pruned, nil
} }
func (bs *mockBlockStore) DeleteLatestBlock() error { return nil }
//--------------------------------------- //---------------------------------------
// Test handshake/init chain // Test handshake/init chain

View File

@@ -458,6 +458,7 @@ func (_m *Client) GenesisChunked(_a0 context.Context, _a1 uint) (*coretypes.Resu
return r0, r1 return r0, r1
} }
// Header provides a mock function with given fields: ctx, height // Header provides a mock function with given fields: ctx, height
func (_m *Client) Header(ctx context.Context, height *int64) (*coretypes.ResultHeader, error) { func (_m *Client) Header(ctx context.Context, height *int64) (*coretypes.ResultHeader, error) {
ret := _m.Called(ctx, height) ret := _m.Called(ctx, height)

View File

@@ -27,6 +27,20 @@ func (_m *BlockStore) Base() int64 {
return r0 return r0
} }
// DeleteLatestBlock provides a mock function with given fields:
func (_m *BlockStore) DeleteLatestBlock() error {
ret := _m.Called()
var r0 error
if rf, ok := ret.Get(0).(func() error); ok {
r0 = rf()
} else {
r0 = ret.Error(0)
}
return r0
}
// Height provides a mock function with given fields: // Height provides a mock function with given fields:
func (_m *BlockStore) Height() int64 { func (_m *BlockStore) Height() int64 {
ret := _m.Called() ret := _m.Called()

View File

@@ -12,7 +12,7 @@ import (
// Rollback overwrites the current Tendermint state (height n) with the most // Rollback overwrites the current Tendermint state (height n) with the most
// recent previous state (height n - 1). // recent previous state (height n - 1).
// Note that this function does not affect application state. // 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() invalidState, err := ss.Load()
if err != nil { if err != nil {
return -1, nil, err return -1, nil, err
@@ -24,9 +24,14 @@ func Rollback(bs BlockStore, ss Store) (int64, []byte, error) {
height := bs.Height() height := bs.Height()
// NOTE: persistence of state and blocks don't happen atomically. Therefore it is possible that // 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 // when the user stopped the node the state wasn't updated but the blockstore was. Discard the
// we don't need to rollback any state and can just return early // pending block before continuing.
if height == invalidState.LastBlockHeight+1 { 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 return invalidState.LastBlockHeight, invalidState.AppHash, nil
} }
@@ -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) 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 return rolledBackState.LastBlockHeight, rolledBackState.AppHash, nil
} }

View File

@@ -3,6 +3,7 @@ package state_test
import ( import (
"crypto/rand" "crypto/rand"
"testing" "testing"
"time"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
dbm "github.com/tendermint/tm-db" dbm "github.com/tendermint/tm-db"
@@ -13,6 +14,7 @@ import (
tmversion "github.com/tendermint/tendermint/proto/tendermint/version" tmversion "github.com/tendermint/tendermint/proto/tendermint/version"
"github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/state/mocks" "github.com/tendermint/tendermint/state/mocks"
"github.com/tendermint/tendermint/store"
"github.com/tendermint/tendermint/types" "github.com/tendermint/tendermint/types"
"github.com/tendermint/tendermint/version" "github.com/tendermint/tendermint/version"
) )
@@ -50,6 +52,7 @@ func TestRollback(t *testing.T) {
BlockID: initialState.LastBlockID, BlockID: initialState.LastBlockID,
Header: types.Header{ Header: types.Header{
Height: initialState.LastBlockHeight, Height: initialState.LastBlockHeight,
Time: initialState.LastBlockTime,
AppHash: crypto.CRandBytes(tmhash.Size), AppHash: crypto.CRandBytes(tmhash.Size),
LastBlockID: makeBlockIDRandom(), LastBlockID: makeBlockIDRandom(),
LastResultsHash: initialState.LastResultsHash, LastResultsHash: initialState.LastResultsHash,
@@ -61,6 +64,7 @@ func TestRollback(t *testing.T) {
Height: nextState.LastBlockHeight, Height: nextState.LastBlockHeight,
AppHash: initialState.AppHash, AppHash: initialState.AppHash,
LastBlockID: block.BlockID, LastBlockID: block.BlockID,
Time: nextState.LastBlockTime,
LastResultsHash: nextState.LastResultsHash, LastResultsHash: nextState.LastResultsHash,
}, },
} }
@@ -69,7 +73,7 @@ func TestRollback(t *testing.T) {
blockStore.On("Height").Return(nextHeight) blockStore.On("Height").Return(nextHeight)
// rollback the state // rollback the state
rollbackHeight, rollbackHash, err := state.Rollback(blockStore, stateStore) rollbackHeight, rollbackHash, err := state.Rollback(blockStore, stateStore, false)
require.NoError(t, err) require.NoError(t, err)
require.EqualValues(t, height, rollbackHeight) require.EqualValues(t, height, rollbackHeight)
require.EqualValues(t, initialState.AppHash, rollbackHash) require.EqualValues(t, initialState.AppHash, rollbackHash)
@@ -81,6 +85,122 @@ func TestRollback(t *testing.T) {
require.EqualValues(t, initialState, loadedState) 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: tmversion.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, err := block.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
blockStore.SaveBlock(block, partSet, &types.Commit{Height: block.Height})
currState := state.State{
Version: tmstate.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: tmversion.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, err := nextBlock.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
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: tmstate.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) { func TestRollbackNoState(t *testing.T) {
stateStore := state.NewStore(dbm.NewMemDB(), stateStore := state.NewStore(dbm.NewMemDB(),
state.StoreOptions{ state.StoreOptions{
@@ -88,7 +208,7 @@ func TestRollbackNoState(t *testing.T) {
}) })
blockStore := &mocks.BlockStore{} blockStore := &mocks.BlockStore{}
_, _, err := state.Rollback(blockStore, stateStore) _, _, err := state.Rollback(blockStore, stateStore, false)
require.Error(t, err) require.Error(t, err)
require.Contains(t, err.Error(), "no state found") require.Contains(t, err.Error(), "no state found")
} }
@@ -101,7 +221,7 @@ func TestRollbackNoBlocks(t *testing.T) {
blockStore.On("LoadBlockMeta", height).Return(nil) blockStore.On("LoadBlockMeta", height).Return(nil)
blockStore.On("LoadBlockMeta", height-1).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.Error(t, err)
require.Contains(t, err.Error(), "block at height 99 not found") require.Contains(t, err.Error(), "block at height 99 not found")
} }
@@ -112,7 +232,7 @@ func TestRollbackDifferentStateHeight(t *testing.T) {
blockStore := &mocks.BlockStore{} blockStore := &mocks.BlockStore{}
blockStore.On("Height").Return(height + 2) blockStore.On("Height").Return(height + 2)
_, _, err := state.Rollback(blockStore, stateStore) _, _, err := state.Rollback(blockStore, stateStore, false)
require.Error(t, err) require.Error(t, err)
require.Equal(t, err.Error(), "statestore height (100) is not one below or equal to blockstore height (102)") require.Equal(t, err.Error(), "statestore height (100) is not one below or equal to blockstore height (102)")
} }
@@ -138,6 +258,7 @@ func setupStateStore(t *testing.T, height int64) state.Store {
AppHash: tmhash.Sum([]byte("app_hash")), AppHash: tmhash.Sum([]byte("app_hash")),
LastResultsHash: tmhash.Sum([]byte("last_results_hash")), LastResultsHash: tmhash.Sum([]byte("last_results_hash")),
LastBlockHeight: height, LastBlockHeight: height,
LastBlockTime: time.Now(),
LastValidators: valSet, LastValidators: valSet,
Validators: valSet.CopyIncrementProposerPriority(1), Validators: valSet.CopyIncrementProposerPriority(1),
NextValidators: valSet.CopyIncrementProposerPriority(2), NextValidators: valSet.CopyIncrementProposerPriority(2),

View File

@@ -34,6 +34,8 @@ type BlockStore interface {
LoadBlockCommit(height int64) *types.Commit LoadBlockCommit(height int64) *types.Commit
LoadSeenCommit(height int64) *types.Commit LoadSeenCommit(height int64) *types.Commit
DeleteLatestBlock() error
} }
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------

View File

@@ -521,3 +521,50 @@ func mustEncode(pb proto.Message) []byte {
} }
return bz 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
}

View File

@@ -388,6 +388,9 @@ func TestLoadBaseMeta(t *testing.T) {
baseBlock := bs.LoadBaseMeta() baseBlock := bs.LoadBaseMeta()
assert.EqualValues(t, 4, baseBlock.Header.Height) assert.EqualValues(t, 4, baseBlock.Header.Height)
assert.EqualValues(t, 4, bs.Base()) assert.EqualValues(t, 4, bs.Base())
require.NoError(t, bs.DeleteLatestBlock())
require.EqualValues(t, 9, bs.Height())
} }
func TestLoadBlockPart(t *testing.T) { func TestLoadBlockPart(t *testing.T) {