refactor!: abstractions for snapshot and pruning; snapshot intervals eventually pruned; unit tests (#11496)

This commit is contained in:
Roman 2022-04-21 15:30:36 -04:00 committed by GitHub
parent d4dd44469f
commit 42f8d45b68
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
57 changed files with 2643 additions and 638 deletions

View File

@ -85,6 +85,7 @@ Ref: https://keepachangelog.com/en/1.0.0/
### API Breaking Changes
* [\#11496](https://github.com/cosmos/cosmos-sdk/pull/11496) Refactor abstractions for snapshot and pruning; snapshot intervals eventually pruned; unit tests.
* (types) [\#11689](https://github.com/cosmos/cosmos-sdk/pull/11689) Make `Coins#Sub` and `Coins#SafeSub` consistent with `Coins#Add`.
* (store)[\#11152](https://github.com/cosmos/cosmos-sdk/pull/11152) Remove `keep-every` from pruning options.
* [\#10950](https://github.com/cosmos/cosmos-sdk/pull/10950) Add `envPrefix` parameter to `cmd.Execute`.

View File

@ -338,9 +338,7 @@ func (app *BaseApp) Commit() (res abci.ResponseCommit) {
app.halt()
}
if app.snapshotInterval > 0 && uint64(header.Height)%app.snapshotInterval == 0 {
go app.snapshot(header.Height)
}
go app.snapshotManager.SnapshotIfApplicable(header.Height)
return abci.ResponseCommit{
Data: commitID.Hash,
@ -370,36 +368,6 @@ func (app *BaseApp) halt() {
os.Exit(0)
}
// snapshot takes a snapshot of the current state and prunes any old snapshottypes.
func (app *BaseApp) snapshot(height int64) {
if app.snapshotManager == nil {
app.logger.Info("snapshot manager not configured")
return
}
app.logger.Info("creating state snapshot", "height", height)
snapshot, err := app.snapshotManager.Create(uint64(height))
if err != nil {
app.logger.Error("failed to create state snapshot", "height", height, "err", err)
return
}
app.logger.Info("completed state snapshot", "height", height, "format", snapshot.Format)
if app.snapshotKeepRecent > 0 {
app.logger.Debug("pruning state snapshots")
pruned, err := app.snapshotManager.Prune(app.snapshotKeepRecent)
if err != nil {
app.logger.Error("Failed to prune state snapshots", "err", err)
return
}
app.logger.Debug("pruned state snapshots", "pruned", pruned)
}
}
// Query implements the ABCI interface. It delegates to CommitMultiStore if it
// implements Queryable.
func (app *BaseApp) Query(req abci.RequestQuery) (res abci.ResponseQuery) {
@ -718,9 +686,11 @@ func (app *BaseApp) GetBlockRetentionHeight(commitHeight int64) int64 {
retentionHeight = commitHeight - cp.Evidence.MaxAgeNumBlocks
}
if app.snapshotInterval > 0 && app.snapshotKeepRecent > 0 {
v := commitHeight - int64((app.snapshotInterval * uint64(app.snapshotKeepRecent)))
retentionHeight = minNonZero(retentionHeight, v)
if app.snapshotManager != nil {
snapshotRetentionHeights := app.snapshotManager.GetSnapshotBlockRetentionHeights()
if snapshotRetentionHeights > 0 {
retentionHeight = minNonZero(retentionHeight, commitHeight-snapshotRetentionHeights)
}
}
v := commitHeight - int64(app.minRetainBlocks)

View File

@ -5,11 +5,14 @@ import (
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
tmprototypes "github.com/tendermint/tendermint/proto/tendermint/types"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/baseapp"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types"
"github.com/cosmos/cosmos-sdk/snapshots"
"github.com/cosmos/cosmos-sdk/testutil"
)
func TestGetBlockRentionHeight(t *testing.T) {
@ -17,6 +20,9 @@ func TestGetBlockRentionHeight(t *testing.T) {
db := dbm.NewMemDB()
name := t.Name()
snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), testutil.GetTempDir(t))
require.NoError(t, err)
testCases := map[string]struct {
bapp *baseapp.BaseApp
maxAgeBlocks int64
@ -38,17 +44,18 @@ func TestGetBlockRentionHeight(t *testing.T) {
"pruning iavl snapshot only": {
bapp: baseapp.NewBaseApp(
name, logger, db,
baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)),
baseapp.SetMinRetainBlocks(1),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(10000, 1)),
),
maxAgeBlocks: 0,
commitHeight: 499000,
expected: 498999,
expected: 489000,
},
"pruning state sync snapshot only": {
bapp: baseapp.NewBaseApp(
name, logger, db,
baseapp.SetSnapshotInterval(50000),
baseapp.SetSnapshotKeepRecent(3),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)),
baseapp.SetMinRetainBlocks(1),
),
maxAgeBlocks: 0,
@ -67,8 +74,9 @@ func TestGetBlockRentionHeight(t *testing.T) {
"pruning all conditions": {
bapp: baseapp.NewBaseApp(
name, logger, db,
baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(0, 0)),
baseapp.SetMinRetainBlocks(400000),
baseapp.SetSnapshotInterval(50000), baseapp.SetSnapshotKeepRecent(3),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)),
),
maxAgeBlocks: 362880,
commitHeight: 499000,
@ -77,8 +85,9 @@ func TestGetBlockRentionHeight(t *testing.T) {
"no pruning due to no persisted state": {
bapp: baseapp.NewBaseApp(
name, logger, db,
baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(0, 0)),
baseapp.SetMinRetainBlocks(400000),
baseapp.SetSnapshotInterval(50000), baseapp.SetSnapshotKeepRecent(3),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)),
),
maxAgeBlocks: 362880,
commitHeight: 10000,
@ -87,8 +96,9 @@ func TestGetBlockRentionHeight(t *testing.T) {
"disable pruning": {
bapp: baseapp.NewBaseApp(
name, logger, db,
baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(0, 0)),
baseapp.SetMinRetainBlocks(0),
baseapp.SetSnapshotInterval(50000), baseapp.SetSnapshotKeepRecent(3),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)),
),
maxAgeBlocks: 362880,
commitHeight: 499000,
@ -126,10 +136,10 @@ func TestBaseAppCreateQueryContext(t *testing.T) {
name := t.Name()
app := baseapp.NewBaseApp(name, logger, db)
app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: 1}})
app.BeginBlock(abci.RequestBeginBlock{Header: tmprototypes.Header{Height: 1}})
app.Commit()
app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: 2}})
app.BeginBlock(abci.RequestBeginBlock{Header: tmprototypes.Header{Height: 2}})
app.Commit()
testCases := []struct {

View File

@ -2,7 +2,6 @@ package baseapp
import (
"context"
"errors"
"fmt"
abci "github.com/tendermint/tendermint/abci/types"
@ -62,9 +61,7 @@ type BaseApp struct { // nolint: maligned
fauxMerkleMode bool // if true, IAVL MountStores uses MountStoresDB for simulation speed.
// manages snapshots, i.e. dumps of app state at certain intervals
snapshotManager *snapshots.Manager
snapshotInterval uint64 // block interval between state sync snapshots
snapshotKeepRecent uint32 // recent state sync snapshots to keep
snapshotManager *snapshots.Manager
// volatile states:
//
@ -252,7 +249,7 @@ func (app *BaseApp) LoadLatestVersion() error {
return fmt.Errorf("failed to load latest version: %w", err)
}
return app.init()
return app.Init()
}
// DefaultStoreLoader will be used by default and loads the latest version
@ -284,7 +281,7 @@ func (app *BaseApp) LoadVersion(version int64) error {
return fmt.Errorf("failed to load version %d: %w", version, err)
}
return app.init()
return app.Init()
}
// LastCommitID returns the last CommitID of the multistore.
@ -297,7 +294,11 @@ func (app *BaseApp) LastBlockHeight() int64 {
return app.cms.LastCommitID().Version
}
func (app *BaseApp) init() error {
// Init initializes the app. It seals the app, preventing any
// further modifications. In addition, it validates the app against
// the earlier provided settings. Returns an error if validation fails.
// nil otherwise. Panics if the app is already sealed.
func (app *BaseApp) Init() error {
if app.sealed {
panic("cannot call initFromMainStore: baseapp already sealed")
}
@ -306,14 +307,11 @@ func (app *BaseApp) init() error {
app.setCheckState(tmproto.Header{})
app.Seal()
// make sure the snapshot interval is a multiple of the pruning KeepEvery interval
if app.snapshotManager != nil && app.snapshotInterval > 0 {
if _, ok := app.cms.(*rootmulti.Store); !ok {
return errors.New("state sync snapshots require a rootmulti store")
}
rms, ok := app.cms.(*rootmulti.Store)
if !ok {
return fmt.Errorf("invalid commit multi-store; expected %T, got: %T", &rootmulti.Store{}, app.cms)
}
return nil
return rms.GetPruning().Validate()
}
func (app *BaseApp) setMinGasPrices(gasPrices sdk.DecCoins) {

View File

@ -8,16 +8,11 @@ import (
"fmt"
"math"
"math/rand"
"os"
"strings"
"sync"
"testing"
"time"
"google.golang.org/protobuf/proto"
"github.com/cosmos/cosmos-sdk/testutil/testdata_pulsar"
"github.com/gogo/protobuf/jsonpb"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
@ -25,17 +20,21 @@ import (
"github.com/tendermint/tendermint/libs/log"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
dbm "github.com/tendermint/tm-db"
"google.golang.org/protobuf/proto"
"github.com/cosmos/cosmos-sdk/baseapp"
"github.com/cosmos/cosmos-sdk/codec"
"github.com/cosmos/cosmos-sdk/codec/legacy"
codectypes "github.com/cosmos/cosmos-sdk/codec/types"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/simapp"
"github.com/cosmos/cosmos-sdk/snapshots"
snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types"
"github.com/cosmos/cosmos-sdk/store/rootmulti"
storetypes "github.com/cosmos/cosmos-sdk/store/types"
"github.com/cosmos/cosmos-sdk/testutil"
"github.com/cosmos/cosmos-sdk/testutil/testdata"
"github.com/cosmos/cosmos-sdk/testutil/testdata_pulsar"
sdk "github.com/cosmos/cosmos-sdk/types"
sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
"github.com/cosmos/cosmos-sdk/types/tx"
@ -58,6 +57,14 @@ type paramStore struct {
db *dbm.MemDB
}
type setupConfig struct {
blocks uint64
blockTxs int
snapshotInterval uint64
snapshotKeepRecent uint32
pruningOpts pruningtypes.PruningOptions
}
func (ps *paramStore) Set(_ sdk.Context, key []byte, value interface{}) {
bz, err := json.Marshal(value)
if err != nil {
@ -121,7 +128,7 @@ func aminoTxEncoder(cdc *codec.LegacyAmino) sdk.TxEncoder {
}
// simple one store baseapp
func setupBaseApp(t *testing.T, options ...func(*baseapp.BaseApp)) *baseapp.BaseApp {
func setupBaseApp(t *testing.T, options ...func(*baseapp.BaseApp)) (*baseapp.BaseApp, error) {
app := newBaseApp(t.Name(), options...)
require.Equal(t, t.Name(), app.Name())
@ -130,8 +137,7 @@ func setupBaseApp(t *testing.T, options ...func(*baseapp.BaseApp)) *baseapp.Base
// stores are mounted
err := app.LoadLatestVersion()
require.Nil(t, err)
return app
return app, err
}
// testTxHandler is a tx.Handler used for the mock app, it does not
@ -150,7 +156,7 @@ func testTxHandler(options middleware.TxHandlerOptions, customTxHandlerMiddlewar
}
// simple one store baseapp with data and snapshots. Each tx is 1 MB in size (uncompressed).
func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options ...func(*baseapp.BaseApp)) (*baseapp.BaseApp, func()) {
func setupBaseAppWithSnapshots(t *testing.T, config *setupConfig) (*baseapp.BaseApp, error) {
codec := codec.NewLegacyAmino()
registerTestCodec(codec)
routerOpt := func(bapp *baseapp.BaseApp) {
@ -178,28 +184,22 @@ func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options
bapp.SetTxHandler(txHandler)
}
snapshotInterval := uint64(2)
snapshotTimeout := 1 * time.Minute
snapshotDir, err := os.MkdirTemp("", "baseapp")
snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), testutil.GetTempDir(t))
require.NoError(t, err)
snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), snapshotDir)
require.NoError(t, err)
teardown := func() {
_ = os.RemoveAll(snapshotDir)
}
app := setupBaseApp(t, append(options,
baseapp.SetSnapshotStore(snapshotStore),
baseapp.SetSnapshotInterval(snapshotInterval),
routerOpt)...)
app, err := setupBaseApp(t, routerOpt, baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(config.snapshotInterval, uint32(config.snapshotKeepRecent))), baseapp.SetPruning(config.pruningOpts))
if err != nil {
return nil, err
}
app.InitChain(abci.RequestInitChain{})
r := rand.New(rand.NewSource(3920758213583))
keyCounter := 0
for height := int64(1); height <= int64(blocks); height++ {
for height := int64(1); height <= int64(config.blocks); height++ {
app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: height}})
for txNum := 0; txNum < blockTxs; txNum++ {
for txNum := 0; txNum < config.blockTxs; txNum++ {
tx := txTest{Msgs: []sdk.Msg{}}
for msgNum := 0; msgNum < 100; msgNum++ {
key := []byte(fmt.Sprintf("%v", keyCounter))
@ -218,7 +218,7 @@ func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options
app.Commit()
// Wait for snapshot to be taken, since it happens asynchronously.
if uint64(height)%snapshotInterval == 0 {
if config.snapshotInterval > 0 && uint64(height)%config.snapshotInterval == 0 {
start := time.Now()
for {
if time.Since(start) > snapshotTimeout {
@ -234,11 +234,12 @@ func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options
}
}
return app, teardown
return app, nil
}
func TestMountStores(t *testing.T) {
app := setupBaseApp(t)
app, err := setupBaseApp(t)
require.NoError(t, err)
// check both stores
store1 := app.CMS().GetCommitKVStore(capKey1)
@ -270,7 +271,7 @@ func (th MockTxHandler) SimulateTx(goCtx context.Context, req tx.Request) (tx.Re
}
func TestConsensusParamsNotNil(t *testing.T) {
app := setupBaseApp(t, func(app *baseapp.BaseApp) {
app, err := setupBaseApp(t, func(app *baseapp.BaseApp) {
app.SetBeginBlocker(func(ctx sdk.Context, req abci.RequestBeginBlock) abci.ResponseBeginBlock {
require.NotNil(t, ctx.ConsensusParams())
return abci.ResponseBeginBlock{}
@ -283,13 +284,14 @@ func TestConsensusParamsNotNil(t *testing.T) {
}, func(app *baseapp.BaseApp) {
app.SetTxHandler(MockTxHandler{T: t})
})
require.NoError(t, err)
header := tmproto.Header{Height: 1}
app.BeginBlock(abci.RequestBeginBlock{Header: header})
app.EndBlock(abci.RequestEndBlock{Height: header.Height})
app.CheckTx(abci.RequestCheckTx{})
app.DeliverTx(abci.RequestDeliverTx{})
_, _, err := app.Simulate([]byte{})
_, _, err = app.Simulate([]byte{})
require.NoError(t, err)
}
@ -297,7 +299,7 @@ func TestConsensusParamsNotNil(t *testing.T) {
// Test that LoadLatestVersion actually does.
func TestLoadVersion(t *testing.T) {
logger := defaultLogger()
pruningOpt := baseapp.SetPruning(storetypes.PruneNothing)
pruningOpt := baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
db := dbm.NewMemDB()
name := t.Name()
app := baseapp.NewBaseApp(name, logger, db, pruningOpt)
@ -349,8 +351,8 @@ func useDefaultLoader(app *baseapp.BaseApp) {
}
func initStore(t *testing.T, db dbm.DB, storeKey string, k, v []byte) {
rs := rootmulti.NewStore(db)
rs.SetPruning(storetypes.PruneNothing)
rs := rootmulti.NewStore(db, log.NewNopLogger())
rs.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
key := sdk.NewKVStoreKey(storeKey)
rs.MountStoreWithDB(key, storetypes.StoreTypeIAVL, nil)
err := rs.LoadLatestVersion()
@ -366,8 +368,8 @@ func initStore(t *testing.T, db dbm.DB, storeKey string, k, v []byte) {
}
func checkStore(t *testing.T, db dbm.DB, ver int64, storeKey string, k, v []byte) {
rs := rootmulti.NewStore(db)
rs.SetPruning(storetypes.PruneDefault)
rs := rootmulti.NewStore(db, log.NewNopLogger())
rs.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningDefault))
key := sdk.NewKVStoreKey(storeKey)
rs.MountStoreWithDB(key, storetypes.StoreTypeIAVL, nil)
err := rs.LoadLatestVersion()
@ -410,7 +412,7 @@ func TestSetLoader(t *testing.T) {
initStore(t, db, tc.origStoreKey, k, v)
// load the app with the existing db
opts := []func(*baseapp.BaseApp){baseapp.SetPruning(storetypes.PruneNothing)}
opts := []func(*baseapp.BaseApp){baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))}
if tc.setLoader != nil {
opts = append(opts, tc.setLoader)
}
@ -433,7 +435,7 @@ func TestSetLoader(t *testing.T) {
func TestVersionSetterGetter(t *testing.T) {
logger := defaultLogger()
pruningOpt := baseapp.SetPruning(storetypes.PruneDefault)
pruningOpt := baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningDefault))
db := dbm.NewMemDB()
name := t.Name()
app := baseapp.NewBaseApp(name, logger, db, pruningOpt)
@ -453,7 +455,7 @@ func TestVersionSetterGetter(t *testing.T) {
func TestLoadVersionInvalid(t *testing.T) {
logger := log.NewNopLogger()
pruningOpt := baseapp.SetPruning(storetypes.PruneNothing)
pruningOpt := baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
db := dbm.NewMemDB()
name := t.Name()
app := baseapp.NewBaseApp(name, logger, db, pruningOpt)
@ -485,20 +487,22 @@ func TestLoadVersionInvalid(t *testing.T) {
func TestLoadVersionPruning(t *testing.T) {
logger := log.NewNopLogger()
pruningOptions := storetypes.PruningOptions{
KeepRecent: 2,
Interval: 1,
}
pruningOptions := pruningtypes.NewCustomPruningOptions(10, 15)
pruningOpt := baseapp.SetPruning(pruningOptions)
db := dbm.NewMemDB()
name := t.Name()
app := baseapp.NewBaseApp(name, logger, db, pruningOpt)
snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), testutil.GetTempDir(t))
require.NoError(t, err)
snapshotOpt := baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(3, 1))
app := baseapp.NewBaseApp(name, logger, db, pruningOpt, snapshotOpt)
// make a cap key and mount the store
capKey := sdk.NewKVStoreKey("key1")
app.MountStores(capKey)
err := app.LoadLatestVersion() // needed to make stores non-nil
err = app.LoadLatestVersion() // needed to make stores non-nil
require.Nil(t, err)
emptyCommitID := storetypes.CommitID{}
@ -511,31 +515,31 @@ func TestLoadVersionPruning(t *testing.T) {
var lastCommitID storetypes.CommitID
// Commit seven blocks, of which 7 (latest) is kept in addition to 6, 5
// (keep recent) and 3 (keep every).
for i := int64(1); i <= 7; i++ {
// Commit 15 blocks, of which 15 (latest) is kept in addition to 5-14 inclusive
// (keep recent) and 3 (snapshot-interval).
for i := int64(1); i <= 15; i++ {
app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: i}})
res := app.Commit()
lastCommitID = storetypes.CommitID{Version: i, Hash: res.Data}
}
for _, v := range []int64{1, 2, 4} {
for _, v := range []int64{1, 2, 3, 4} {
_, err = app.CMS().CacheMultiStoreWithVersion(v)
require.NoError(t, err)
}
for _, v := range []int64{3, 5, 6, 7} {
for _, v := range []int64{3, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14} {
_, err = app.CMS().CacheMultiStoreWithVersion(v)
require.NoError(t, err)
}
// reload with LoadLatestVersion, check it loads last version
app = baseapp.NewBaseApp(name, logger, db, pruningOpt)
app = baseapp.NewBaseApp(name, logger, db, pruningOpt, snapshotOpt)
app.MountStores(capKey)
err = app.LoadLatestVersion()
require.Nil(t, err)
testLoadVersionHelper(t, app, int64(7), lastCommitID)
testLoadVersionHelper(t, app, int64(15), lastCommitID)
}
func testLoadVersionHelper(t *testing.T, app *baseapp.BaseApp, expectedHeight int64, expectedID storetypes.CommitID) {
@ -577,7 +581,8 @@ func TestInfo(t *testing.T) {
}
func TestBaseAppOptionSeal(t *testing.T) {
app := setupBaseApp(t)
app, err := setupBaseApp(t)
require.NoError(t, err)
require.Panics(t, func() {
app.SetName("")
@ -1018,7 +1023,8 @@ func TestCheckTx(t *testing.T) {
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
nTxs := int64(5)
app.InitChain(abci.RequestInitChain{})
@ -1075,7 +1081,9 @@ func TestDeliverTx(t *testing.T) {
)
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
app.InitChain(abci.RequestInitChain{})
nBlocks := 3
@ -1135,7 +1143,9 @@ func TestMultiMsgDeliverTx(t *testing.T) {
)
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
// run a multi-msg tx
// with all msgs the same route
@ -1220,7 +1230,8 @@ func TestSimulateTx(t *testing.T) {
)
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
app.InitChain(abci.RequestInitChain{})
@ -1294,7 +1305,8 @@ func TestRunInvalidTransaction(t *testing.T) {
)
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
header := tmproto.Header{Height: 1}
app.BeginBlock(abci.RequestBeginBlock{Header: header})
@ -1420,7 +1432,8 @@ func TestTxGasLimits(t *testing.T) {
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
header := tmproto.Header{Height: 1}
app.BeginBlock(abci.RequestBeginBlock{Header: header})
@ -1507,8 +1520,8 @@ func TestMaxBlockGasLimits(t *testing.T) {
)
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
app.InitChain(abci.RequestInitChain{
ConsensusParams: &tmproto.ConsensusParams{
Block: &tmproto.BlockParams{
@ -1593,7 +1606,8 @@ func TestBaseAppMiddleware(t *testing.T) {
)
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
app.InitChain(abci.RequestInitChain{})
@ -1684,8 +1698,8 @@ func TestGasConsumptionBadTx(t *testing.T) {
)
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
app.InitChain(abci.RequestInitChain{
ConsensusParams: &tmproto.ConsensusParams{
Block: &tmproto.BlockParams{
@ -1750,7 +1764,8 @@ func TestQuery(t *testing.T) {
)
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
app.InitChain(abci.RequestInitChain{})
@ -1798,7 +1813,8 @@ func TestGRPCQuery(t *testing.T) {
)
}
app := setupBaseApp(t, grpcQueryOpt)
app, err := setupBaseApp(t, grpcQueryOpt)
require.NoError(t, err)
app.GRPCQueryRouter().SetInterfaceRegistry(codectypes.NewInterfaceRegistry())
app.InitChain(abci.RequestInitChain{})
@ -1833,7 +1849,8 @@ func TestGRPCQueryPulsar(t *testing.T) {
)
}
app := setupBaseApp(t, grpcQueryOpt)
app, err := setupBaseApp(t, grpcQueryOpt)
require.NoError(t, err)
app.GRPCQueryRouter().SetInterfaceRegistry(codectypes.NewInterfaceRegistry())
app.InitChain(abci.RequestInitChain{})
@ -1876,7 +1893,8 @@ func TestP2PQuery(t *testing.T) {
})
}
app := setupBaseApp(t, addrPeerFilterOpt, idPeerFilterOpt)
app, err := setupBaseApp(t, addrPeerFilterOpt, idPeerFilterOpt)
require.NoError(t, err)
addrQuery := abci.RequestQuery{
Path: "/p2p/filter/addr/1.1.1.1:8000",
@ -1892,7 +1910,8 @@ func TestP2PQuery(t *testing.T) {
}
func TestGetMaximumBlockGas(t *testing.T) {
app := setupBaseApp(t)
app, err := setupBaseApp(t)
require.NoError(t, err)
app.InitChain(abci.RequestInitChain{})
ctx := app.NewContext(true, tmproto.Header{})
@ -1910,8 +1929,16 @@ func TestGetMaximumBlockGas(t *testing.T) {
}
func TestListSnapshots(t *testing.T) {
app, teardown := setupBaseAppWithSnapshots(t, 5, 4)
defer teardown()
setupConfig := &setupConfig{
blocks: 5,
blockTxs: 4,
snapshotInterval: 2,
snapshotKeepRecent: 2,
pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
}
app, err := setupBaseAppWithSnapshots(t, setupConfig)
require.NoError(t, err)
resp := app.ListSnapshots(abci.RequestListSnapshots{})
for _, s := range resp.Snapshots {
@ -1926,9 +1953,148 @@ func TestListSnapshots(t *testing.T) {
}}, resp)
}
func TestSnapshotWithPruning(t *testing.T) {
testcases := map[string]struct {
config *setupConfig
expectedSnapshots []*abci.Snapshot
expectedErr error
}{
"prune nothing with snapshot": {
config: &setupConfig{
blocks: 20,
blockTxs: 2,
snapshotInterval: 5,
snapshotKeepRecent: 1,
pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
},
expectedSnapshots: []*abci.Snapshot{
{Height: 20, Format: 2, Chunks: 5},
},
},
"prune everything with snapshot": {
config: &setupConfig{
blocks: 20,
blockTxs: 2,
snapshotInterval: 5,
snapshotKeepRecent: 1,
pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningEverything),
},
expectedSnapshots: []*abci.Snapshot{
{Height: 20, Format: 2, Chunks: 5},
},
},
"default pruning with snapshot": {
config: &setupConfig{
blocks: 20,
blockTxs: 2,
snapshotInterval: 5,
snapshotKeepRecent: 1,
pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningDefault),
},
expectedSnapshots: []*abci.Snapshot{
{Height: 20, Format: 2, Chunks: 5},
},
},
"custom": {
config: &setupConfig{
blocks: 25,
blockTxs: 2,
snapshotInterval: 5,
snapshotKeepRecent: 2,
pruningOpts: pruningtypes.NewCustomPruningOptions(12, 12),
},
expectedSnapshots: []*abci.Snapshot{
{Height: 25, Format: 2, Chunks: 6},
{Height: 20, Format: 2, Chunks: 5},
},
},
"no snapshots": {
config: &setupConfig{
blocks: 10,
blockTxs: 2,
snapshotInterval: 0, // 0 implies disable snapshots
pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
},
expectedSnapshots: []*abci.Snapshot{},
},
"keep all snapshots": {
config: &setupConfig{
blocks: 10,
blockTxs: 2,
snapshotInterval: 3,
snapshotKeepRecent: 0, // 0 implies keep all snapshots
pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
},
expectedSnapshots: []*abci.Snapshot{
{Height: 9, Format: 2, Chunks: 2},
{Height: 6, Format: 2, Chunks: 2},
{Height: 3, Format: 2, Chunks: 1},
},
},
}
for name, tc := range testcases {
t.Run(name, func(t *testing.T) {
app, err := setupBaseAppWithSnapshots(t, tc.config)
if tc.expectedErr != nil {
require.Error(t, err)
require.Equal(t, tc.expectedErr.Error(), err.Error())
return
}
require.NoError(t, err)
resp := app.ListSnapshots(abci.RequestListSnapshots{})
for _, s := range resp.Snapshots {
assert.NotEmpty(t, s.Hash)
assert.NotEmpty(t, s.Metadata)
s.Hash = nil
s.Metadata = nil
}
fmt.Println(resp)
assert.Equal(t, abci.ResponseListSnapshots{Snapshots: tc.expectedSnapshots}, resp)
// Validate that heights were pruned correctly by querying the state at the last height that should be present relative to latest
// and the first height that should be pruned.
//
// Exceptions:
// * Prune nothing: should be able to query all heights (we only test first and latest)
// * Prune default: should be able to query all heights (we only test first and latest)
// * The reason for default behaving this way is that we only commit 20 heights but default has 100_000 keep-recent
var lastExistingHeight int64
if tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningNothing || tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningDefault {
lastExistingHeight = 1
} else {
// Integer division rounds down so by multiplying back we get the last height at which we pruned
lastExistingHeight = int64((tc.config.blocks/tc.config.pruningOpts.Interval)*tc.config.pruningOpts.Interval - tc.config.pruningOpts.KeepRecent)
}
// Query 1
res := app.Query(abci.RequestQuery{Path: fmt.Sprintf("/store/%s/key", capKey2.Name()), Data: []byte("0"), Height: lastExistingHeight})
require.NotNil(t, res, "height: %d", lastExistingHeight)
require.NotNil(t, res.Value, "height: %d", lastExistingHeight)
// Query 2
res = app.Query(abci.RequestQuery{Path: fmt.Sprintf("/store/%s/key", capKey2.Name()), Data: []byte("0"), Height: lastExistingHeight - 1})
require.NotNil(t, res, "height: %d", lastExistingHeight-1)
if tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningNothing || tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningDefault {
// With prune nothing or default, we query height 0 which translates to the latest height.
require.NotNil(t, res.Value, "height: %d", lastExistingHeight-1)
}
})
}
}
func TestLoadSnapshotChunk(t *testing.T) {
app, teardown := setupBaseAppWithSnapshots(t, 2, 5)
defer teardown()
setupConfig := &setupConfig{
blocks: 2,
blockTxs: 5,
snapshotInterval: 2,
snapshotKeepRecent: 2,
pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
}
app, err := setupBaseAppWithSnapshots(t, setupConfig)
require.NoError(t, err)
testcases := map[string]struct {
height uint64
@ -1963,8 +2129,15 @@ func TestLoadSnapshotChunk(t *testing.T) {
func TestOfferSnapshot_Errors(t *testing.T) {
// Set up app before test cases, since it's fairly expensive.
app, teardown := setupBaseAppWithSnapshots(t, 0, 0)
defer teardown()
setupConfig := &setupConfig{
blocks: 0,
blockTxs: 0,
snapshotInterval: 2,
snapshotKeepRecent: 2,
pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
}
app, err := setupBaseAppWithSnapshots(t, setupConfig)
require.NoError(t, err)
m := snapshottypes.Metadata{ChunkHashes: [][]byte{{1}, {2}, {3}}}
metadata, err := m.Marshal()
@ -2018,11 +2191,25 @@ func TestOfferSnapshot_Errors(t *testing.T) {
}
func TestApplySnapshotChunk(t *testing.T) {
source, teardown := setupBaseAppWithSnapshots(t, 4, 10)
defer teardown()
setupConfig1 := &setupConfig{
blocks: 4,
blockTxs: 10,
snapshotInterval: 2,
snapshotKeepRecent: 2,
pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
}
source, err := setupBaseAppWithSnapshots(t, setupConfig1)
require.NoError(t, err)
target, teardown := setupBaseAppWithSnapshots(t, 0, 0)
defer teardown()
setupConfig2 := &setupConfig{
blocks: 0,
blockTxs: 0,
snapshotInterval: 2,
snapshotKeepRecent: 2,
pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
}
target, err := setupBaseAppWithSnapshots(t, setupConfig2)
require.NoError(t, err)
// Fetch latest snapshot to restore
respList := source.ListSnapshots(abci.RequestListSnapshots{})
@ -2102,7 +2289,8 @@ func TestWithRouter(t *testing.T) {
)
bapp.SetTxHandler(txHandler)
}
app := setupBaseApp(t, txHandlerOpt)
app, err := setupBaseApp(t, txHandlerOpt)
require.NoError(t, err)
app.InitChain(abci.RequestInitChain{})
nBlocks := 3
@ -2159,3 +2347,164 @@ func TestBaseApp_EndBlock(t *testing.T) {
require.Equal(t, int64(100), res.GetValidatorUpdates()[0].Power)
require.Equal(t, cp.Block.MaxGas, res.ConsensusParamUpdates.Block.MaxGas)
}
func TestBaseApp_Init(t *testing.T) {
db := dbm.NewMemDB()
name := t.Name()
logger := defaultLogger()
snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), testutil.GetTempDir(t))
require.NoError(t, err)
testCases := map[string]struct {
bapp *baseapp.BaseApp
expectedPruning pruningtypes.PruningOptions
expectedSnapshot snapshottypes.SnapshotOptions
expectedErr error
}{
"snapshot but no pruning": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)),
),
pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
snapshottypes.NewSnapshotOptions(1500, 2),
// if no pruning is set, the default is PruneNothing
nil,
},
"pruning everything only": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningEverything)),
),
pruningtypes.NewPruningOptions(pruningtypes.PruningEverything),
snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 0),
nil,
},
"pruning nothing only": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)),
),
pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 0),
nil,
},
"pruning default only": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningDefault)),
),
pruningtypes.NewPruningOptions(pruningtypes.PruningDefault),
snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 0),
nil,
},
"pruning custom only": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 10)),
),
pruningtypes.NewCustomPruningOptions(10, 10),
snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 0),
nil,
},
"pruning everything and snapshots": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningEverything)),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)),
),
pruningtypes.NewPruningOptions(pruningtypes.PruningEverything),
snapshottypes.NewSnapshotOptions(1500, 2),
nil,
},
"pruning nothing and snapshots": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)),
),
pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
snapshottypes.NewSnapshotOptions(1500, 2),
nil,
},
"pruning default and snapshots": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningDefault)),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)),
),
pruningtypes.NewPruningOptions(pruningtypes.PruningDefault),
snapshottypes.NewSnapshotOptions(1500, 2),
nil,
},
"pruning custom and snapshots": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 10)),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)),
),
pruningtypes.NewCustomPruningOptions(10, 10),
snapshottypes.NewSnapshotOptions(1500, 2),
nil,
},
"error custom pruning 0 interval": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 0)),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)),
),
pruningtypes.NewCustomPruningOptions(10, 0),
snapshottypes.NewSnapshotOptions(1500, 2),
pruningtypes.ErrPruningIntervalZero,
},
"error custom pruning too small interval": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 9)),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)),
),
pruningtypes.NewCustomPruningOptions(10, 9),
snapshottypes.NewSnapshotOptions(1500, 2),
pruningtypes.ErrPruningIntervalTooSmall,
},
"error custom pruning too small keep recent": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(1, 10)),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)),
),
pruningtypes.NewCustomPruningOptions(9, 10),
snapshottypes.NewSnapshotOptions(1500, 2),
pruningtypes.ErrPruningKeepRecentTooSmall,
},
"snapshot zero interval - manager not set": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 10)),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 2)),
),
pruningtypes.NewCustomPruningOptions(10, 10),
snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 0),
nil,
},
"snapshot zero keep recent - allowed": {
baseapp.NewBaseApp(name, logger, db,
baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 10)),
baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 0)),
),
pruningtypes.NewCustomPruningOptions(10, 10),
snapshottypes.NewSnapshotOptions(1500, 0), // 0 snapshot-keep-recent means keep all
nil,
},
}
for _, tc := range testCases {
// Init and validate
require.Equal(t, tc.expectedErr, tc.bapp.Init())
if tc.expectedErr != nil {
continue
}
// Check that settings were set correctly
actualPruning := tc.bapp.CMS().GetPruning()
require.Equal(t, tc.expectedPruning, actualPruning)
snapshotManager := tc.bapp.GetSnapshotManager()
if tc.expectedSnapshot.Interval == snapshottypes.SnapshotIntervalOff {
require.Nil(t, snapshotManager)
continue
}
require.NotNil(t, snapshotManager)
require.Equal(t, tc.expectedSnapshot.Interval, snapshotManager.GetInterval())
require.Equal(t, tc.expectedSnapshot.KeepRecent, snapshotManager.GetKeepRecent())
}
}

View File

@ -7,7 +7,9 @@ import (
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/codec/types"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/snapshots"
snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types"
"github.com/cosmos/cosmos-sdk/store"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/cosmos/cosmos-sdk/types/tx"
@ -17,7 +19,7 @@ import (
// for options that need access to non-exported fields of the BaseApp
// SetPruning sets a pruning option on the multistore associated with the app
func SetPruning(opts sdk.PruningOptions) func(*BaseApp) {
func SetPruning(opts pruningtypes.PruningOptions) func(*BaseApp) {
return func(bapp *BaseApp) { bapp.cms.SetPruning(opts) }
}
@ -69,19 +71,9 @@ func SetInterBlockCache(cache sdk.MultiStorePersistentCache) func(*BaseApp) {
return func(app *BaseApp) { app.setInterBlockCache(cache) }
}
// SetSnapshotInterval sets the snapshot interval.
func SetSnapshotInterval(interval uint64) func(*BaseApp) {
return func(app *BaseApp) { app.SetSnapshotInterval(interval) }
}
// SetSnapshotKeepRecent sets the recent snapshots to keep.
func SetSnapshotKeepRecent(keepRecent uint32) func(*BaseApp) {
return func(app *BaseApp) { app.SetSnapshotKeepRecent(keepRecent) }
}
// SetSnapshotStore sets the snapshot store.
func SetSnapshotStore(snapshotStore *snapshots.Store) func(*BaseApp) {
return func(app *BaseApp) { app.SetSnapshotStore(snapshotStore) }
// SetSnapshot sets the snapshot store.
func SetSnapshot(snapshotStore *snapshots.Store, opts snapshottypes.SnapshotOptions) func(*BaseApp) {
return func(app *BaseApp) { app.SetSnapshot(snapshotStore, opts) }
}
func (app *BaseApp) SetName(name string) {
@ -201,32 +193,17 @@ func (app *BaseApp) SetStoreLoader(loader StoreLoader) {
app.storeLoader = loader
}
// SetSnapshotStore sets the snapshot store.
func (app *BaseApp) SetSnapshotStore(snapshotStore *snapshots.Store) {
// SetSnapshot sets the snapshot store and options.
func (app *BaseApp) SetSnapshot(snapshotStore *snapshots.Store, opts snapshottypes.SnapshotOptions) {
if app.sealed {
panic("SetSnapshotStore() on sealed BaseApp")
panic("SetSnapshot() on sealed BaseApp")
}
if snapshotStore == nil {
if snapshotStore == nil || opts.Interval == snapshottypes.SnapshotIntervalOff {
app.snapshotManager = nil
return
}
app.snapshotManager = snapshots.NewManager(snapshotStore, app.cms, nil)
}
// SetSnapshotInterval sets the snapshot interval.
func (app *BaseApp) SetSnapshotInterval(snapshotInterval uint64) {
if app.sealed {
panic("SetSnapshotInterval() on sealed BaseApp")
}
app.snapshotInterval = snapshotInterval
}
// SetSnapshotKeepRecent sets the number of recent snapshots to keep.
func (app *BaseApp) SetSnapshotKeepRecent(snapshotKeepRecent uint32) {
if app.sealed {
panic("SetSnapshotKeepRecent() on sealed BaseApp")
}
app.snapshotKeepRecent = snapshotKeepRecent
app.cms.SetSnapshotInterval(opts.Interval)
app.snapshotManager = snapshots.NewManager(snapshotStore, opts, app.cms, nil, app.logger)
}
// SetInterfaceRegistry sets the InterfaceRegistry.

View File

@ -1,8 +1,8 @@
package baseapp
import (
"github.com/cosmos/cosmos-sdk/snapshots"
"github.com/cosmos/cosmos-sdk/types"
sdk "github.com/cosmos/cosmos-sdk/types"
)
// TODO: Can be removed once we move all middleware tests into x/auth/middleware
@ -31,10 +31,18 @@ func (app *BaseApp) CMS() types.CommitMultiStore {
return app.cms
}
// GetSnapshotManager() is an exported method to be able to access baseapp's snapshot
// manager in tests.
//
// This method is only accessible in baseapp tests.
func (app *BaseApp) GetSnapshotManager() *snapshots.Manager {
return app.snapshotManager
}
// GetMaximumBlockGas return maximum blocks gas.
//
// This method is only accessible in baseapp tests.
func (app *BaseApp) GetMaximumBlockGas(ctx sdk.Context) uint64 {
func (app *BaseApp) GetMaximumBlockGas(ctx types.Context) uint64 {
return app.getMaximumBlockGas(ctx)
}
@ -48,13 +56,13 @@ func (app *BaseApp) GetName() string {
// CreateQueryContext calls app's createQueryContext.
//
// This method is only accessible in baseapp tests.
func (app *BaseApp) CreateQueryContext(height int64, prove bool) (sdk.Context, error) {
func (app *BaseApp) CreateQueryContext(height int64, prove bool) (types.Context, error) {
return app.createQueryContext(height, prove)
}
// MinGasPrices returns minGasPrices.
//
// This method is only accessible in baseapp tests.
func (app *BaseApp) MinGasPrices() sdk.DecCoins {
func (app *BaseApp) MinGasPrices() types.DecCoins {
return app.minGasPrices
}

29
pruning/README.md Normal file
View File

@ -0,0 +1,29 @@
# Pruning
## Overview
Pruning is the mechanism for deleting old application heights from the disk. Depending on the use case,
nodes may require different pruning strategies. For example, archive nodes must keep all
the states and prune nothing. On the other hand, a regular validator node may want to only keep 100 latest heights for performance reasons.
## Strategies
The strategies are configured in `app.toml`, with the format `pruning = "<strategy>"` where the options are:
- `default`: only the last 362,880 states(approximately 3.5 weeks worth of state) are kept; pruning at 10 block intervals
- `nothing`: all historic states will be saved, nothing will be deleted (i.e. archiving node)
- `everything`: 2 latest states will be kept; pruning at 10 block intervals.
- `custom`: allow pruning options to be manually specified through 'pruning-keep-recent', and 'pruning-interval'
If no strategy is given to the BaseApp, `nothing` is selected. However, we perform validation on the CLI layer to require these to be always set in the config file.
## Custom Pruning
These are applied if and only if the pruning strategy is custom:
- `pruning-keep-recent`: N means to keep all of the last N states
- `pruning-interval`: N means to delete old states from disk every Nth block.
## Relationship to State Sync Snapshots
Snapshot settings are optional. However, if set, they have an effect on how pruning is done by
persisting the heights that are multiples of `state-sync.snapshot-interval` until after the snapshot is complete. See the "Relationship to Pruning" section in `snapshots/README.md` for more details.

11
pruning/export_test.go Normal file
View File

@ -0,0 +1,11 @@
package pruning
var (
PruneHeightsKey = pruneHeightsKey
PruneSnapshotHeightsKey = pruneSnapshotHeightsKey
Int64SliceToBytes = int64SliceToBytes
ListToBytes = listToBytes
LoadPruningHeights = loadPruningHeights
LoadPruningSnapshotHeights = loadPruningSnapshotHeights
)

282
pruning/manager.go Normal file
View File

@ -0,0 +1,282 @@
package pruning
import (
"container/list"
"encoding/binary"
"fmt"
"sync"
"github.com/tendermint/tendermint/libs/log"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/pruning/types"
)
// Manager is an abstraction to handle the logic needed for
// determinging when to prune old heights of the store
// based on the strategy described by the pruning options.
type Manager struct {
db dbm.DB
logger log.Logger
opts types.PruningOptions
snapshotInterval uint64
// Although pruneHeights happen in the same goroutine with the normal execution,
// we sync access to them to avoid soundness issues in the future if concurrency pattern changes.
pruneHeightsMx sync.Mutex
pruneHeights []int64
// Snapshots are taken in a separate goroutine from the regular execution
// and can be delivered asynchrounously via HandleHeightSnapshot.
// Therefore, we sync access to pruneSnapshotHeights with this mutex.
pruneSnapshotHeightsMx sync.Mutex
// These are the heights that are multiples of snapshotInterval and kept for state sync snapshots.
// The heights are added to this list to be pruned when a snapshot is complete.
pruneSnapshotHeights *list.List
}
// NegativeHeightsError is returned when a negative height is provided to the manager.
type NegativeHeightsError struct {
Height int64
}
var _ error = &NegativeHeightsError{}
func (e *NegativeHeightsError) Error() string {
return fmt.Sprintf("failed to get pruned heights: %d", e.Height)
}
var (
pruneHeightsKey = []byte("s/pruneheights")
pruneSnapshotHeightsKey = []byte("s/prunesnapshotheights")
)
// NewManager returns a new Manager with the given db and logger.
// The retuned manager uses a pruning strategy of "nothing" which
// keeps all heights. Users of the Manager may change the strategy
// by calling SetOptions.
func NewManager(db dbm.DB, logger log.Logger) *Manager {
return &Manager{
db: db,
logger: logger,
opts: types.NewPruningOptions(types.PruningNothing),
pruneHeights: []int64{},
pruneSnapshotHeights: list.New(),
}
}
// SetOptions sets the pruning strategy on the manager.
func (m *Manager) SetOptions(opts types.PruningOptions) {
m.opts = opts
}
// GetOptions fetches the pruning strategy from the manager.
func (m *Manager) GetOptions() types.PruningOptions {
return m.opts
}
// GetFlushAndResetPruningHeights returns all heights to be pruned during the next call to Prune().
// It also flushes and resets the pruning heights.
func (m *Manager) GetFlushAndResetPruningHeights() ([]int64, error) {
if m.opts.GetPruningStrategy() == types.PruningNothing {
return []int64{}, nil
}
m.pruneHeightsMx.Lock()
defer m.pruneHeightsMx.Unlock()
// flush the updates to disk so that it is not lost if crash happens.
if err := m.db.SetSync(pruneHeightsKey, int64SliceToBytes(m.pruneHeights)); err != nil {
return nil, err
}
// Return a copy to prevent data races.
pruningHeights := make([]int64, len(m.pruneHeights))
copy(pruningHeights, m.pruneHeights)
m.pruneHeights = m.pruneHeights[:0]
return pruningHeights, nil
}
// HandleHeight determines if previousHeight height needs to be kept for pruning at the right interval prescribed by
// the pruning strategy. Returns previousHeight, if it was kept to be pruned at the next call to Prune(), 0 otherwise.
// previousHeight must be greater than 0 for the handling to take effect since valid heights start at 1 and 0 represents
// the latest height. The latest height cannot be pruned. As a result, if previousHeight is less than or equal to 0, 0 is returned.
func (m *Manager) HandleHeight(previousHeight int64) int64 {
if m.opts.GetPruningStrategy() == types.PruningNothing || previousHeight <= 0 {
return 0
}
defer func() {
m.pruneHeightsMx.Lock()
defer m.pruneHeightsMx.Unlock()
m.pruneSnapshotHeightsMx.Lock()
defer m.pruneSnapshotHeightsMx.Unlock()
// move persisted snapshot heights to pruneHeights which
// represent the heights to be pruned at the next pruning interval.
var next *list.Element
for e := m.pruneSnapshotHeights.Front(); e != nil; e = next {
snHeight := e.Value.(int64)
if snHeight < previousHeight-int64(m.opts.KeepRecent) {
m.pruneHeights = append(m.pruneHeights, snHeight)
// We must get next before removing to be able to continue iterating.
next = e.Next()
m.pruneSnapshotHeights.Remove(e)
} else {
next = e.Next()
}
}
// flush the updates to disk so that they are not lost if crash happens.
if err := m.db.SetSync(pruneHeightsKey, int64SliceToBytes(m.pruneHeights)); err != nil {
panic(err)
}
}()
if int64(m.opts.KeepRecent) < previousHeight {
pruneHeight := previousHeight - int64(m.opts.KeepRecent)
// We consider this height to be pruned iff:
//
// - snapshotInterval is zero as that means that all heights should be pruned.
// - snapshotInterval % (height - KeepRecent) != 0 as that means the height is not
// a 'snapshot' height.
if m.snapshotInterval == 0 || pruneHeight%int64(m.snapshotInterval) != 0 {
m.pruneHeightsMx.Lock()
defer m.pruneHeightsMx.Unlock()
m.pruneHeights = append(m.pruneHeights, pruneHeight)
return pruneHeight
}
}
return 0
}
// HandleHeightSnapshot persists the snapshot height to be pruned at the next appropriate
// height defined by the pruning strategy. Flushes the update to disk and panics if the flush fails
// The input height must be greater than 0 and pruning strategy any but pruning nothing.
// If one of these conditions is not met, this function does nothing.
func (m *Manager) HandleHeightSnapshot(height int64) {
if m.opts.GetPruningStrategy() == types.PruningNothing || height <= 0 {
return
}
m.pruneSnapshotHeightsMx.Lock()
defer m.pruneSnapshotHeightsMx.Unlock()
m.logger.Debug("HandleHeightSnapshot", "height", height)
m.pruneSnapshotHeights.PushBack(height)
// flush the updates to disk so that they are not lost if crash happens.
if err := m.db.SetSync(pruneSnapshotHeightsKey, listToBytes(m.pruneSnapshotHeights)); err != nil {
panic(err)
}
}
// SetSnapshotInterval sets the interval at which the snapshots are taken.
func (m *Manager) SetSnapshotInterval(snapshotInterval uint64) {
m.snapshotInterval = snapshotInterval
}
// ShouldPruneAtHeight return true if the given height should be pruned, false otherwise
func (m *Manager) ShouldPruneAtHeight(height int64) bool {
return m.opts.Interval > 0 && m.opts.GetPruningStrategy() != types.PruningNothing && height%int64(m.opts.Interval) == 0
}
// LoadPruningHeights loads the pruning heights from the database as a crash recovery.
func (m *Manager) LoadPruningHeights(db dbm.DB) error {
if m.opts.GetPruningStrategy() == types.PruningNothing {
return nil
}
loadedPruneHeights, err := loadPruningHeights(db)
if err != nil {
return err
}
if len(loadedPruneHeights) > 0 {
m.pruneHeightsMx.Lock()
defer m.pruneHeightsMx.Unlock()
m.pruneHeights = loadedPruneHeights
}
loadedPruneSnapshotHeights, err := loadPruningSnapshotHeights(db)
if err != nil {
return err
}
if loadedPruneSnapshotHeights.Len() > 0 {
m.pruneSnapshotHeightsMx.Lock()
defer m.pruneSnapshotHeightsMx.Unlock()
m.pruneSnapshotHeights = loadedPruneSnapshotHeights
}
return nil
}
func loadPruningHeights(db dbm.DB) ([]int64, error) {
bz, err := db.Get(pruneHeightsKey)
if err != nil {
return nil, fmt.Errorf("failed to get pruned heights: %w", err)
}
if len(bz) == 0 {
return []int64{}, nil
}
prunedHeights := make([]int64, len(bz)/8)
i, offset := 0, 0
for offset < len(bz) {
h := int64(binary.BigEndian.Uint64(bz[offset : offset+8]))
if h < 0 {
return []int64{}, &NegativeHeightsError{Height: h}
}
prunedHeights[i] = h
i++
offset += 8
}
return prunedHeights, nil
}
func loadPruningSnapshotHeights(db dbm.DB) (*list.List, error) {
bz, err := db.Get(pruneSnapshotHeightsKey)
if err != nil {
return nil, fmt.Errorf("failed to get post-snapshot pruned heights: %w", err)
}
pruneSnapshotHeights := list.New()
if len(bz) == 0 {
return pruneSnapshotHeights, nil
}
i, offset := 0, 0
for offset < len(bz) {
h := int64(binary.BigEndian.Uint64(bz[offset : offset+8]))
if h < 0 {
return nil, &NegativeHeightsError{Height: h}
}
pruneSnapshotHeights.PushBack(h)
i++
offset += 8
}
return pruneSnapshotHeights, nil
}
func int64SliceToBytes(slice []int64) []byte {
bz := make([]byte, 0, len(slice)*8)
for _, ph := range slice {
buf := make([]byte, 8)
binary.BigEndian.PutUint64(buf, uint64(ph))
bz = append(bz, buf...)
}
return bz
}
func listToBytes(list *list.List) []byte {
bz := make([]byte, 0, list.Len()*8)
for e := list.Front(); e != nil; e = e.Next() {
buf := make([]byte, 8)
binary.BigEndian.PutUint64(buf, uint64(e.Value.(int64)))
bz = append(bz, buf...)
}
return bz
}

536
pruning/manager_test.go Normal file
View File

@ -0,0 +1,536 @@
package pruning_test
import (
"container/list"
"errors"
"fmt"
"testing"
"github.com/golang/mock/gomock"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/libs/log"
db "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/pruning"
"github.com/cosmos/cosmos-sdk/pruning/mock"
"github.com/cosmos/cosmos-sdk/pruning/types"
)
const dbErr = "db error"
func TestNewManager(t *testing.T) {
manager := pruning.NewManager(db.NewMemDB(), log.NewNopLogger())
require.NotNil(t, manager)
heights, err := manager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.NotNil(t, heights)
require.Equal(t, types.PruningNothing, manager.GetOptions().GetPruningStrategy())
}
func TestStrategies(t *testing.T) {
testcases := map[string]struct {
strategy types.PruningOptions
snapshotInterval uint64
strategyToAssert types.PruningStrategy
isValid bool
}{
"prune nothing - no snapshot": {
strategy: types.NewPruningOptions(types.PruningNothing),
strategyToAssert: types.PruningNothing,
},
"prune nothing - snapshot": {
strategy: types.NewPruningOptions(types.PruningNothing),
strategyToAssert: types.PruningNothing,
snapshotInterval: 100,
},
"prune default - no snapshot": {
strategy: types.NewPruningOptions(types.PruningDefault),
strategyToAssert: types.PruningDefault,
},
"prune default - snapshot": {
strategy: types.NewPruningOptions(types.PruningDefault),
strategyToAssert: types.PruningDefault,
snapshotInterval: 100,
},
"prune everything - no snapshot": {
strategy: types.NewPruningOptions(types.PruningEverything),
strategyToAssert: types.PruningEverything,
},
"prune everything - snapshot": {
strategy: types.NewPruningOptions(types.PruningEverything),
strategyToAssert: types.PruningEverything,
snapshotInterval: 100,
},
"custom 100-10-15": {
strategy: types.NewCustomPruningOptions(100, 15),
snapshotInterval: 10,
strategyToAssert: types.PruningCustom,
},
"custom 10-10-15": {
strategy: types.NewCustomPruningOptions(10, 15),
snapshotInterval: 10,
strategyToAssert: types.PruningCustom,
},
"custom 100-0-15": {
strategy: types.NewCustomPruningOptions(100, 15),
snapshotInterval: 0,
strategyToAssert: types.PruningCustom,
},
}
manager := pruning.NewManager(db.NewMemDB(), log.NewNopLogger())
require.NotNil(t, manager)
for name, tc := range testcases {
t.Run(name, func(t *testing.T) {
curStrategy := tc.strategy
manager.SetSnapshotInterval(tc.snapshotInterval)
pruneStrategy := curStrategy.GetPruningStrategy()
require.Equal(t, tc.strategyToAssert, pruneStrategy)
// Validate strategy parameters
switch pruneStrategy {
case types.PruningDefault:
require.Equal(t, uint64(362880), curStrategy.KeepRecent)
require.Equal(t, uint64(10), curStrategy.Interval)
case types.PruningNothing:
require.Equal(t, uint64(0), curStrategy.KeepRecent)
require.Equal(t, uint64(0), curStrategy.Interval)
case types.PruningEverything:
require.Equal(t, uint64(2), curStrategy.KeepRecent)
require.Equal(t, uint64(10), curStrategy.Interval)
default:
//
}
manager.SetOptions(curStrategy)
require.Equal(t, tc.strategy, manager.GetOptions())
curKeepRecent := curStrategy.KeepRecent
curInterval := curStrategy.Interval
for curHeight := int64(0); curHeight < 110000; curHeight++ {
handleHeightActual := manager.HandleHeight(curHeight)
shouldPruneAtHeightActual := manager.ShouldPruneAtHeight(curHeight)
curPruningHeihts, err := manager.GetFlushAndResetPruningHeights()
require.Nil(t, err)
curHeightStr := fmt.Sprintf("height: %d", curHeight)
switch curStrategy.GetPruningStrategy() {
case types.PruningNothing:
require.Equal(t, int64(0), handleHeightActual, curHeightStr)
require.False(t, shouldPruneAtHeightActual, curHeightStr)
heights, err := manager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Equal(t, 0, len(heights))
default:
if curHeight > int64(curKeepRecent) && (tc.snapshotInterval != 0 && (curHeight-int64(curKeepRecent))%int64(tc.snapshotInterval) != 0 || tc.snapshotInterval == 0) {
expectedHeight := curHeight - int64(curKeepRecent)
require.Equal(t, curHeight-int64(curKeepRecent), handleHeightActual, curHeightStr)
require.Contains(t, curPruningHeihts, expectedHeight, curHeightStr)
} else {
require.Equal(t, int64(0), handleHeightActual, curHeightStr)
heights, err := manager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Equal(t, 0, len(heights))
}
require.Equal(t, curHeight%int64(curInterval) == 0, shouldPruneAtHeightActual, curHeightStr)
}
heights, err := manager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Equal(t, 0, len(heights))
}
})
}
}
func TestHandleHeight_Inputs(t *testing.T) {
var keepRecent int64 = int64(types.NewPruningOptions(types.PruningEverything).KeepRecent)
testcases := map[string]struct {
height int64
expectedResult int64
strategy types.PruningStrategy
expectedHeights []int64
}{
"previousHeight is negative - prune everything - invalid previousHeight": {
-1,
0,
types.PruningEverything,
[]int64{},
},
"previousHeight is zero - prune everything - invalid previousHeight": {
0,
0,
types.PruningEverything,
[]int64{},
},
"previousHeight is positive but within keep recent- prune everything - not kept": {
keepRecent,
0,
types.PruningEverything,
[]int64{},
},
"previousHeight is positive and greater than keep recent - kept": {
keepRecent + 1,
keepRecent + 1 - keepRecent,
types.PruningEverything,
[]int64{keepRecent + 1 - keepRecent},
},
"pruning nothing, previousHeight is positive and greater than keep recent - not kept": {
keepRecent + 1,
0,
types.PruningNothing,
[]int64{},
},
}
for name, tc := range testcases {
t.Run(name, func(t *testing.T) {
manager := pruning.NewManager(db.NewMemDB(), log.NewNopLogger())
require.NotNil(t, manager)
manager.SetOptions(types.NewPruningOptions(tc.strategy))
handleHeightActual := manager.HandleHeight(tc.height)
require.Equal(t, tc.expectedResult, handleHeightActual)
actualHeights, err := manager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Equal(t, len(tc.expectedHeights), len(actualHeights))
require.Equal(t, tc.expectedHeights, actualHeights)
})
}
}
func TestHandleHeight_FlushLoadFromDisk(t *testing.T) {
testcases := map[string]struct {
previousHeight int64
keepRecent uint64
snapshotInterval uint64
movedSnapshotHeights []int64
expectedHandleHeightResult int64
expectedLoadPruningHeightsResult error
expectedLoadedHeights []int64
}{
"simple flush occurs": {
previousHeight: 11,
keepRecent: 10,
snapshotInterval: 0,
movedSnapshotHeights: []int64{},
expectedHandleHeightResult: 11 - 10,
expectedLoadPruningHeightsResult: nil,
expectedLoadedHeights: []int64{11 - 10},
},
"previous height <= keep recent - no update and no flush": {
previousHeight: 9,
keepRecent: 10,
snapshotInterval: 0,
movedSnapshotHeights: []int64{},
expectedHandleHeightResult: 0,
expectedLoadPruningHeightsResult: nil,
expectedLoadedHeights: []int64{},
},
"previous height alligns with snapshot interval - no update and no flush": {
previousHeight: 12,
keepRecent: 10,
snapshotInterval: 2,
movedSnapshotHeights: []int64{},
expectedHandleHeightResult: 0,
expectedLoadPruningHeightsResult: nil,
expectedLoadedHeights: []int64{},
},
"previous height does not align with snapshot interval - flush": {
previousHeight: 12,
keepRecent: 10,
snapshotInterval: 3,
movedSnapshotHeights: []int64{},
expectedHandleHeightResult: 2,
expectedLoadPruningHeightsResult: nil,
expectedLoadedHeights: []int64{2},
},
"moved snapshot heights - flushed": {
previousHeight: 32,
keepRecent: 10,
snapshotInterval: 5,
movedSnapshotHeights: []int64{15, 20, 25},
expectedHandleHeightResult: 22,
expectedLoadPruningHeightsResult: nil,
expectedLoadedHeights: []int64{15, 20, 22},
},
"previous height alligns with snapshot interval - no update but flush snapshot heights": {
previousHeight: 30,
keepRecent: 10,
snapshotInterval: 5,
movedSnapshotHeights: []int64{15, 20, 25},
expectedHandleHeightResult: 0,
expectedLoadPruningHeightsResult: nil,
expectedLoadedHeights: []int64{15},
},
}
for name, tc := range testcases {
t.Run(name, func(t *testing.T) {
// Setup
db := db.NewMemDB()
manager := pruning.NewManager(db, log.NewNopLogger())
require.NotNil(t, manager)
manager.SetSnapshotInterval(tc.snapshotInterval)
manager.SetOptions(types.NewCustomPruningOptions(uint64(tc.keepRecent), uint64(10)))
for _, snapshotHeight := range tc.movedSnapshotHeights {
manager.HandleHeightSnapshot(snapshotHeight)
}
// Test HandleHeight and flush
handleHeightActual := manager.HandleHeight(tc.previousHeight)
require.Equal(t, tc.expectedHandleHeightResult, handleHeightActual)
loadedPruneHeights, err := pruning.LoadPruningHeights(db)
require.NoError(t, err)
require.Equal(t, len(loadedPruneHeights), len(loadedPruneHeights))
// Test load back
err = manager.LoadPruningHeights(db)
require.NoError(t, err)
heights, err := manager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Equal(t, len(tc.expectedLoadedHeights), len(heights))
require.ElementsMatch(t, tc.expectedLoadedHeights, heights)
})
}
}
func TestHandleHeight_DbErr_Panic(t *testing.T) {
ctrl := gomock.NewController(t)
// Setup
dbMock := mock.NewMockDB(ctrl)
dbMock.EXPECT().SetSync(gomock.Any(), gomock.Any()).Return(errors.New(dbErr)).Times(1)
manager := pruning.NewManager(dbMock, log.NewNopLogger())
manager.SetOptions(types.NewPruningOptions(types.PruningEverything))
require.NotNil(t, manager)
defer func() {
if r := recover(); r == nil {
t.Fail()
}
}()
manager.HandleHeight(10)
}
func TestHandleHeightSnapshot_FlushLoadFromDisk(t *testing.T) {
loadedHeightsMirror := []int64{}
// Setup
db := db.NewMemDB()
manager := pruning.NewManager(db, log.NewNopLogger())
require.NotNil(t, manager)
manager.SetOptions(types.NewPruningOptions(types.PruningEverything))
for snapshotHeight := int64(-1); snapshotHeight < 100; snapshotHeight++ {
// Test flush
manager.HandleHeightSnapshot(snapshotHeight)
// Post test
if snapshotHeight > 0 {
loadedHeightsMirror = append(loadedHeightsMirror, snapshotHeight)
}
loadedSnapshotHeights, err := pruning.LoadPruningSnapshotHeights(db)
require.NoError(t, err)
require.Equal(t, len(loadedHeightsMirror), loadedSnapshotHeights.Len())
// Test load back
err = manager.LoadPruningHeights(db)
require.NoError(t, err)
loadedSnapshotHeights, err = pruning.LoadPruningSnapshotHeights(db)
require.NoError(t, err)
require.Equal(t, len(loadedHeightsMirror), loadedSnapshotHeights.Len())
}
}
func TestHandleHeightSnapshot_DbErr_Panic(t *testing.T) {
ctrl := gomock.NewController(t)
// Setup
dbMock := mock.NewMockDB(ctrl)
dbMock.EXPECT().SetSync(gomock.Any(), gomock.Any()).Return(errors.New(dbErr)).Times(1)
manager := pruning.NewManager(dbMock, log.NewNopLogger())
manager.SetOptions(types.NewPruningOptions(types.PruningEverything))
require.NotNil(t, manager)
defer func() {
if r := recover(); r == nil {
t.Fail()
}
}()
manager.HandleHeightSnapshot(10)
}
func TestFlushLoad(t *testing.T) {
db := db.NewMemDB()
manager := pruning.NewManager(db, log.NewNopLogger())
require.NotNil(t, manager)
curStrategy := types.NewCustomPruningOptions(100, 15)
snapshotInterval := uint64(10)
manager.SetSnapshotInterval(snapshotInterval)
manager.SetOptions(curStrategy)
require.Equal(t, curStrategy, manager.GetOptions())
keepRecent := curStrategy.KeepRecent
heightsToPruneMirror := make([]int64, 0)
for curHeight := int64(0); curHeight < 1000; curHeight++ {
handleHeightActual := manager.HandleHeight(curHeight)
curHeightStr := fmt.Sprintf("height: %d", curHeight)
if curHeight > int64(keepRecent) && (snapshotInterval != 0 && (curHeight-int64(keepRecent))%int64(snapshotInterval) != 0 || snapshotInterval == 0) {
expectedHandleHeight := curHeight - int64(keepRecent)
require.Equal(t, expectedHandleHeight, handleHeightActual, curHeightStr)
heightsToPruneMirror = append(heightsToPruneMirror, expectedHandleHeight)
} else {
require.Equal(t, int64(0), handleHeightActual, curHeightStr)
}
if manager.ShouldPruneAtHeight(curHeight) && curHeight > int64(keepRecent) {
actualHeights, err := manager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Equal(t, len(heightsToPruneMirror), len(actualHeights))
require.Equal(t, heightsToPruneMirror, actualHeights)
err = manager.LoadPruningHeights(db)
require.NoError(t, err)
actualHeights, err = manager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Equal(t, len(heightsToPruneMirror), len(actualHeights))
require.Equal(t, heightsToPruneMirror, actualHeights)
heightsToPruneMirror = make([]int64, 0)
}
}
}
func TestLoadPruningHeights(t *testing.T) {
var (
manager = pruning.NewManager(db.NewMemDB(), log.NewNopLogger())
err error
)
require.NotNil(t, manager)
// must not be PruningNothing
manager.SetOptions(types.NewPruningOptions(types.PruningDefault))
testcases := map[string]struct {
flushedPruningHeights []int64
getFlushedPruningSnapshotHeights func() *list.List
expectedResult error
}{
"negative pruningHeight - error": {
flushedPruningHeights: []int64{10, 0, -1},
expectedResult: &pruning.NegativeHeightsError{Height: -1},
},
"negative snapshotPruningHeight - error": {
getFlushedPruningSnapshotHeights: func() *list.List {
l := list.New()
l.PushBack(int64(5))
l.PushBack(int64(-2))
l.PushBack(int64(3))
return l
},
expectedResult: &pruning.NegativeHeightsError{Height: -2},
},
"both have negative - pruningHeight error": {
flushedPruningHeights: []int64{10, 0, -1},
getFlushedPruningSnapshotHeights: func() *list.List {
l := list.New()
l.PushBack(int64(5))
l.PushBack(int64(-2))
l.PushBack(int64(3))
return l
},
expectedResult: &pruning.NegativeHeightsError{Height: -1},
},
"both non-negative - success": {
flushedPruningHeights: []int64{10, 0, 3},
getFlushedPruningSnapshotHeights: func() *list.List {
l := list.New()
l.PushBack(int64(5))
l.PushBack(int64(0))
l.PushBack(int64(3))
return l
},
},
}
for name, tc := range testcases {
t.Run(name, func(t *testing.T) {
db := db.NewMemDB()
if tc.flushedPruningHeights != nil {
err = db.Set(pruning.PruneHeightsKey, pruning.Int64SliceToBytes(tc.flushedPruningHeights))
require.NoError(t, err)
}
if tc.getFlushedPruningSnapshotHeights != nil {
err = db.Set(pruning.PruneSnapshotHeightsKey, pruning.ListToBytes(tc.getFlushedPruningSnapshotHeights()))
require.NoError(t, err)
}
err = manager.LoadPruningHeights(db)
require.Equal(t, tc.expectedResult, err)
})
}
}
func TestLoadPruningHeights_PruneNothing(t *testing.T) {
var manager = pruning.NewManager(db.NewMemDB(), log.NewNopLogger())
require.NotNil(t, manager)
manager.SetOptions(types.NewPruningOptions(types.PruningNothing))
require.Nil(t, manager.LoadPruningHeights(db.NewMemDB()))
}
func TestGetFlushAndResetPruningHeights_DbErr_Panic(t *testing.T) {
ctrl := gomock.NewController(t)
// Setup
dbMock := mock.NewMockDB(ctrl)
dbMock.EXPECT().SetSync(gomock.Any(), gomock.Any()).Return(errors.New(dbErr)).Times(1)
manager := pruning.NewManager(dbMock, log.NewNopLogger())
manager.SetOptions(types.NewPruningOptions(types.PruningEverything))
require.NotNil(t, manager)
heights, err := manager.GetFlushAndResetPruningHeights()
require.Error(t, err)
require.Nil(t, heights)
}

420
pruning/mock/db_mock.go Normal file
View File

@ -0,0 +1,420 @@
// Code generated by MockGen. DO NOT EDIT.
// Source: /home/roman/projects/cosmos-sdk/vendor/github.com/tendermint/tm-db/types.go
// Package mock_db is a generated GoMock package.
package mock
import (
reflect "reflect"
gomock "github.com/golang/mock/gomock"
db "github.com/tendermint/tm-db"
)
// MockDB is a mock of DB interface.
type MockDB struct {
ctrl *gomock.Controller
recorder *MockDBMockRecorder
}
// MockDBMockRecorder is the mock recorder for MockDB.
type MockDBMockRecorder struct {
mock *MockDB
}
// NewMockDB creates a new mock instance.
func NewMockDB(ctrl *gomock.Controller) *MockDB {
mock := &MockDB{ctrl: ctrl}
mock.recorder = &MockDBMockRecorder{mock}
return mock
}
// EXPECT returns an object that allows the caller to indicate expected use.
func (m *MockDB) EXPECT() *MockDBMockRecorder {
return m.recorder
}
// Close mocks base method.
func (m *MockDB) Close() error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Close")
ret0, _ := ret[0].(error)
return ret0
}
// Close indicates an expected call of Close.
func (mr *MockDBMockRecorder) Close() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockDB)(nil).Close))
}
// Delete mocks base method.
func (m *MockDB) Delete(arg0 []byte) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Delete", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// Delete indicates an expected call of Delete.
func (mr *MockDBMockRecorder) Delete(arg0 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Delete", reflect.TypeOf((*MockDB)(nil).Delete), arg0)
}
// DeleteSync mocks base method.
func (m *MockDB) DeleteSync(arg0 []byte) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "DeleteSync", arg0)
ret0, _ := ret[0].(error)
return ret0
}
// DeleteSync indicates an expected call of DeleteSync.
func (mr *MockDBMockRecorder) DeleteSync(arg0 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteSync", reflect.TypeOf((*MockDB)(nil).DeleteSync), arg0)
}
// Get mocks base method.
func (m *MockDB) Get(arg0 []byte) ([]byte, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Get", arg0)
ret0, _ := ret[0].([]byte)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// Get indicates an expected call of Get.
func (mr *MockDBMockRecorder) Get(arg0 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Get", reflect.TypeOf((*MockDB)(nil).Get), arg0)
}
// Has mocks base method.
func (m *MockDB) Has(key []byte) (bool, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Has", key)
ret0, _ := ret[0].(bool)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// Has indicates an expected call of Has.
func (mr *MockDBMockRecorder) Has(key interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Has", reflect.TypeOf((*MockDB)(nil).Has), key)
}
// Iterator mocks base method.
func (m *MockDB) Iterator(start, end []byte) (db.Iterator, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Iterator", start, end)
ret0, _ := ret[0].(db.Iterator)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// Iterator indicates an expected call of Iterator.
func (mr *MockDBMockRecorder) Iterator(start, end interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Iterator", reflect.TypeOf((*MockDB)(nil).Iterator), start, end)
}
// NewBatch mocks base method.
func (m *MockDB) NewBatch() db.Batch {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "NewBatch")
ret0, _ := ret[0].(db.Batch)
return ret0
}
// NewBatch indicates an expected call of NewBatch.
func (mr *MockDBMockRecorder) NewBatch() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NewBatch", reflect.TypeOf((*MockDB)(nil).NewBatch))
}
// Print mocks base method.
func (m *MockDB) Print() error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Print")
ret0, _ := ret[0].(error)
return ret0
}
// Print indicates an expected call of Print.
func (mr *MockDBMockRecorder) Print() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Print", reflect.TypeOf((*MockDB)(nil).Print))
}
// ReverseIterator mocks base method.
func (m *MockDB) ReverseIterator(start, end []byte) (db.Iterator, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "ReverseIterator", start, end)
ret0, _ := ret[0].(db.Iterator)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// ReverseIterator indicates an expected call of ReverseIterator.
func (mr *MockDBMockRecorder) ReverseIterator(start, end interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReverseIterator", reflect.TypeOf((*MockDB)(nil).ReverseIterator), start, end)
}
// Set mocks base method.
func (m *MockDB) Set(arg0, arg1 []byte) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Set", arg0, arg1)
ret0, _ := ret[0].(error)
return ret0
}
// Set indicates an expected call of Set.
func (mr *MockDBMockRecorder) Set(arg0, arg1 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Set", reflect.TypeOf((*MockDB)(nil).Set), arg0, arg1)
}
// SetSync mocks base method.
func (m *MockDB) SetSync(arg0, arg1 []byte) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "SetSync", arg0, arg1)
ret0, _ := ret[0].(error)
return ret0
}
// SetSync indicates an expected call of SetSync.
func (mr *MockDBMockRecorder) SetSync(arg0, arg1 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetSync", reflect.TypeOf((*MockDB)(nil).SetSync), arg0, arg1)
}
// Stats mocks base method.
func (m *MockDB) Stats() map[string]string {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Stats")
ret0, _ := ret[0].(map[string]string)
return ret0
}
// Stats indicates an expected call of Stats.
func (mr *MockDBMockRecorder) Stats() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stats", reflect.TypeOf((*MockDB)(nil).Stats))
}
// MockBatch is a mock of Batch interface.
type MockBatch struct {
ctrl *gomock.Controller
recorder *MockBatchMockRecorder
}
// MockBatchMockRecorder is the mock recorder for MockBatch.
type MockBatchMockRecorder struct {
mock *MockBatch
}
// NewMockBatch creates a new mock instance.
func NewMockBatch(ctrl *gomock.Controller) *MockBatch {
mock := &MockBatch{ctrl: ctrl}
mock.recorder = &MockBatchMockRecorder{mock}
return mock
}
// EXPECT returns an object that allows the caller to indicate expected use.
func (m *MockBatch) EXPECT() *MockBatchMockRecorder {
return m.recorder
}
// Close mocks base method.
func (m *MockBatch) Close() error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Close")
ret0, _ := ret[0].(error)
return ret0
}
// Close indicates an expected call of Close.
func (mr *MockBatchMockRecorder) Close() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockBatch)(nil).Close))
}
// Delete mocks base method.
func (m *MockBatch) Delete(key []byte) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Delete", key)
ret0, _ := ret[0].(error)
return ret0
}
// Delete indicates an expected call of Delete.
func (mr *MockBatchMockRecorder) Delete(key interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Delete", reflect.TypeOf((*MockBatch)(nil).Delete), key)
}
// Set mocks base method.
func (m *MockBatch) Set(key, value []byte) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Set", key, value)
ret0, _ := ret[0].(error)
return ret0
}
// Set indicates an expected call of Set.
func (mr *MockBatchMockRecorder) Set(key, value interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Set", reflect.TypeOf((*MockBatch)(nil).Set), key, value)
}
// Write mocks base method.
func (m *MockBatch) Write() error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Write")
ret0, _ := ret[0].(error)
return ret0
}
// Write indicates an expected call of Write.
func (mr *MockBatchMockRecorder) Write() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Write", reflect.TypeOf((*MockBatch)(nil).Write))
}
// WriteSync mocks base method.
func (m *MockBatch) WriteSync() error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "WriteSync")
ret0, _ := ret[0].(error)
return ret0
}
// WriteSync indicates an expected call of WriteSync.
func (mr *MockBatchMockRecorder) WriteSync() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteSync", reflect.TypeOf((*MockBatch)(nil).WriteSync))
}
// MockIterator is a mock of Iterator interface.
type MockIterator struct {
ctrl *gomock.Controller
recorder *MockIteratorMockRecorder
}
// MockIteratorMockRecorder is the mock recorder for MockIterator.
type MockIteratorMockRecorder struct {
mock *MockIterator
}
// NewMockIterator creates a new mock instance.
func NewMockIterator(ctrl *gomock.Controller) *MockIterator {
mock := &MockIterator{ctrl: ctrl}
mock.recorder = &MockIteratorMockRecorder{mock}
return mock
}
// EXPECT returns an object that allows the caller to indicate expected use.
func (m *MockIterator) EXPECT() *MockIteratorMockRecorder {
return m.recorder
}
// Close mocks base method.
func (m *MockIterator) Close() error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Close")
ret0, _ := ret[0].(error)
return ret0
}
// Close indicates an expected call of Close.
func (mr *MockIteratorMockRecorder) Close() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockIterator)(nil).Close))
}
// Domain mocks base method.
func (m *MockIterator) Domain() ([]byte, []byte) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Domain")
ret0, _ := ret[0].([]byte)
ret1, _ := ret[1].([]byte)
return ret0, ret1
}
// Domain indicates an expected call of Domain.
func (mr *MockIteratorMockRecorder) Domain() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Domain", reflect.TypeOf((*MockIterator)(nil).Domain))
}
// Error mocks base method.
func (m *MockIterator) Error() error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Error")
ret0, _ := ret[0].(error)
return ret0
}
// Error indicates an expected call of Error.
func (mr *MockIteratorMockRecorder) Error() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Error", reflect.TypeOf((*MockIterator)(nil).Error))
}
// Key mocks base method.
func (m *MockIterator) Key() []byte {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Key")
ret0, _ := ret[0].([]byte)
return ret0
}
// Key indicates an expected call of Key.
func (mr *MockIteratorMockRecorder) Key() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Key", reflect.TypeOf((*MockIterator)(nil).Key))
}
// Next mocks base method.
func (m *MockIterator) Next() {
m.ctrl.T.Helper()
m.ctrl.Call(m, "Next")
}
// Next indicates an expected call of Next.
func (mr *MockIteratorMockRecorder) Next() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockIterator)(nil).Next))
}
// Valid mocks base method.
func (m *MockIterator) Valid() bool {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Valid")
ret0, _ := ret[0].(bool)
return ret0
}
// Valid indicates an expected call of Valid.
func (mr *MockIteratorMockRecorder) Valid() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Valid", reflect.TypeOf((*MockIterator)(nil).Valid))
}
// Value mocks base method.
func (m *MockIterator) Value() []byte {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Value")
ret0, _ := ret[0].([]byte)
return ret0
}
// Value indicates an expected call of Value.
func (mr *MockIteratorMockRecorder) Value() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Value", reflect.TypeOf((*MockIterator)(nil).Value))
}

130
pruning/types/options.go Normal file
View File

@ -0,0 +1,130 @@
package types
import (
"errors"
"fmt"
)
// PruningOptions defines the pruning strategy used when determining which
// heights are removed from disk when committing state.
type PruningOptions struct {
// KeepRecent defines how many recent heights to keep on disk.
KeepRecent uint64
// Interval defines when the pruned heights are removed from disk.
Interval uint64
// Strategy defines the kind of pruning strategy. See below for more information on each.
Strategy PruningStrategy
}
type PruningStrategy int
// Pruning option string constants
const (
PruningOptionDefault = "default"
PruningOptionEverything = "everything"
PruningOptionNothing = "nothing"
PruningOptionCustom = "custom"
)
const (
// PruningDefault defines a pruning strategy where the last 362880 heights are
// kept where to-be pruned heights are pruned at every 10th height.
// The last 362880 heights are kept(approximately 3.5 weeks worth of state) assuming the typical
// block time is 6s. If these values do not match the applications' requirements, use the "custom" option.
PruningDefault PruningStrategy = iota
// PruningEverything defines a pruning strategy where all committed heights are
// deleted, storing only the current height and last 2 states. To-be pruned heights are
// pruned at every 10th height.
PruningEverything
// PruningNothing defines a pruning strategy where all heights are kept on disk.
// This is the only stretegy where KeepEvery=1 is allowed with state-sync snapshots disabled.
PruningNothing
// PruningCustom defines a pruning strategy where the user specifies the pruning.
PruningCustom
// PruningUndefined defines an undefined pruning strategy. It is to be returned by stores that do not support pruning.
PruningUndefined
)
const (
pruneEverythingKeepRecent = 2
pruneEverythingInterval = 10
)
var (
ErrPruningIntervalZero = errors.New("'pruning-interval' must not be 0. If you want to disable pruning, select pruning = \"nothing\"")
ErrPruningIntervalTooSmall = fmt.Errorf("'pruning-interval' must not be less than %d. For the most aggressive pruning, select pruning = \"everything\"", pruneEverythingInterval)
ErrPruningKeepRecentTooSmall = fmt.Errorf("'pruning-keep-recent' must not be less than %d. For the most aggressive pruning, select pruning = \"everything\"", pruneEverythingKeepRecent)
)
func NewPruningOptions(pruningStrategy PruningStrategy) PruningOptions {
switch pruningStrategy {
case PruningDefault:
return PruningOptions{
KeepRecent: 362880,
Interval: 10,
Strategy: PruningDefault,
}
case PruningEverything:
return PruningOptions{
KeepRecent: pruneEverythingKeepRecent,
Interval: pruneEverythingInterval,
Strategy: PruningEverything,
}
case PruningNothing:
return PruningOptions{
KeepRecent: 0,
Interval: 0,
Strategy: PruningNothing,
}
default:
return PruningOptions{
Strategy: PruningCustom,
}
}
}
func NewCustomPruningOptions(keepRecent, interval uint64) PruningOptions {
return PruningOptions{
KeepRecent: keepRecent,
Interval: interval,
Strategy: PruningCustom,
}
}
func (po PruningOptions) GetPruningStrategy() PruningStrategy {
return po.Strategy
}
func (po PruningOptions) Validate() error {
if po.Strategy == PruningNothing {
return nil
}
if po.Interval == 0 {
return ErrPruningIntervalZero
}
if po.Interval < pruneEverythingInterval {
return ErrPruningIntervalTooSmall
}
if po.KeepRecent < pruneEverythingKeepRecent {
return ErrPruningKeepRecentTooSmall
}
return nil
}
func NewPruningOptionsFromString(strategy string) PruningOptions {
switch strategy {
case PruningOptionEverything:
return NewPruningOptions(PruningEverything)
case PruningOptionNothing:
return NewPruningOptions(PruningNothing)
case PruningOptionDefault:
return NewPruningOptions(PruningDefault)
default:
return NewPruningOptions(PruningDefault)
}
}

View File

@ -0,0 +1,65 @@
package types
import (
"testing"
"github.com/stretchr/testify/require"
)
func TestPruningOptions_Validate(t *testing.T) {
testCases := []struct {
opts PruningOptions
expectErr error
}{
{NewPruningOptions(PruningDefault), nil},
{NewPruningOptions(PruningEverything), nil},
{NewPruningOptions(PruningNothing), nil},
{NewPruningOptions(PruningCustom), ErrPruningIntervalZero},
{NewCustomPruningOptions(2, 10), nil},
{NewCustomPruningOptions(100, 15), nil},
{NewCustomPruningOptions(1, 10), ErrPruningKeepRecentTooSmall},
{NewCustomPruningOptions(2, 9), ErrPruningIntervalTooSmall},
{NewCustomPruningOptions(2, 0), ErrPruningIntervalZero},
{NewCustomPruningOptions(2, 0), ErrPruningIntervalZero},
}
for _, tc := range testCases {
err := tc.opts.Validate()
require.Equal(t, tc.expectErr, err, "options: %v, err: %s", tc.opts, err)
}
}
func TestPruningOptions_GetStrategy(t *testing.T) {
testCases := []struct {
opts PruningOptions
expectedStrategy PruningStrategy
}{
{NewPruningOptions(PruningDefault), PruningDefault},
{NewPruningOptions(PruningEverything), PruningEverything},
{NewPruningOptions(PruningNothing), PruningNothing},
{NewPruningOptions(PruningCustom), PruningCustom},
{NewCustomPruningOptions(2, 10), PruningCustom},
}
for _, tc := range testCases {
actualStrategy := tc.opts.GetPruningStrategy()
require.Equal(t, tc.expectedStrategy, actualStrategy)
}
}
func TestNewPruningOptionsFromString(t *testing.T) {
testCases := []struct {
optString string
expect PruningOptions
}{
{PruningOptionDefault, NewPruningOptions(PruningDefault)},
{PruningOptionEverything, NewPruningOptions(PruningEverything)},
{PruningOptionNothing, NewPruningOptions(PruningNothing)},
{"invalid", NewPruningOptions(PruningDefault)},
}
for _, tc := range testCases {
actual := NewPruningOptionsFromString(tc.optString)
require.Equal(t, tc.expect, actual)
}
}

View File

@ -6,7 +6,7 @@ import (
"github.com/spf13/viper"
storetypes "github.com/cosmos/cosmos-sdk/store/types"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/telemetry"
sdk "github.com/cosmos/cosmos-sdk/types"
sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
@ -211,7 +211,7 @@ func DefaultConfig() *Config {
BaseConfig: BaseConfig{
MinGasPrices: defaultMinGasPrices,
InterBlockCache: true,
Pruning: storetypes.PruningOptionDefault,
Pruning: pruningtypes.PruningOptionDefault,
PruningKeepRecent: "0",
PruningInterval: "0",
MinRetainBlocks: 0,
@ -327,9 +327,9 @@ func (c Config) ValidateBasic() error {
if c.BaseConfig.MinGasPrices == "" {
return sdkerrors.ErrAppConfig.Wrap("set min gas price in app.toml or flag or env variable")
}
if c.Pruning == storetypes.PruningOptionEverything && c.StateSync.SnapshotInterval > 0 {
if c.Pruning == pruningtypes.PruningOptionEverything && c.StateSync.SnapshotInterval > 0 {
return sdkerrors.ErrAppConfig.Wrapf(
"cannot enable state sync snapshots with '%s' pruning setting", storetypes.PruningOptionEverything,
"cannot enable state sync snapshots with '%s' pruning setting", pruningtypes.PruningOptionEverything,
)
}

View File

@ -24,8 +24,8 @@ minimum-gas-prices = "{{ .BaseConfig.MinGasPrices }}"
# default: the last 362880 states are kept, pruning at 10 block intervals
# nothing: all historic states will be saved, nothing will be deleted (i.e. archiving node)
# everything: all saved states will be deleted, storing only the current and previous state; pruning at 10 block intervals
# custom: allow pruning options to be manually specified through 'pruning-keep-recent' and 'pruning-interval'
# everything: 2 latest states will be kept; pruning at 10 block intervals.
# custom: allow pruning options to be manually specified through 'pruning-keep-recent', and 'pruning-interval'
pruning = "{{ .BaseConfig.Pruning }}"
# These are applied if and only if the pruning strategy is custom.

View File

@ -6,6 +6,7 @@ import (
protoio "github.com/gogo/protobuf/io"
dbm "github.com/tendermint/tm-db"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types"
storetypes "github.com/cosmos/cosmos-sdk/store/types"
sdk "github.com/cosmos/cosmos-sdk/types"
@ -65,11 +66,11 @@ func (ms multiStore) LastCommitID() storetypes.CommitID {
panic("not implemented")
}
func (ms multiStore) SetPruning(opts sdk.PruningOptions) {
func (ms multiStore) SetPruning(opts pruningtypes.PruningOptions) {
panic("not implemented")
}
func (ms multiStore) GetPruning() sdk.PruningOptions {
func (ms multiStore) GetPruning() pruningtypes.PruningOptions {
panic("not implemented")
}
@ -113,6 +114,14 @@ func (ms multiStore) GetStoreType() storetypes.StoreType {
panic("not implemented")
}
func (ms multiStore) PruneSnapshotHeight(height int64) {
panic("not implemented")
}
func (ms multiStore) SetSnapshotInterval(snapshotInterval uint64) {
panic("not implemented")
}
func (ms multiStore) SetInterBlockCache(_ sdk.MultiStorePersistentCache) {
panic("not implemented")
}

View File

@ -6,23 +6,22 @@ import (
"github.com/spf13/cast"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/server/types"
"github.com/cosmos/cosmos-sdk/store"
storetypes "github.com/cosmos/cosmos-sdk/store/types"
)
// GetPruningOptionsFromFlags parses command flags and returns the correct
// PruningOptions. If a pruning strategy is provided, that will be parsed and
// returned, otherwise, it is assumed custom pruning options are provided.
func GetPruningOptionsFromFlags(appOpts types.AppOptions) (storetypes.PruningOptions, error) {
func GetPruningOptionsFromFlags(appOpts types.AppOptions) (pruningtypes.PruningOptions, error) {
strategy := strings.ToLower(cast.ToString(appOpts.Get(FlagPruning)))
switch strategy {
case storetypes.PruningOptionDefault, storetypes.PruningOptionNothing, storetypes.PruningOptionEverything:
return storetypes.NewPruningOptionsFromString(strategy), nil
case pruningtypes.PruningOptionDefault, pruningtypes.PruningOptionNothing, pruningtypes.PruningOptionEverything:
return pruningtypes.NewPruningOptionsFromString(strategy), nil
case storetypes.PruningOptionCustom:
opts := storetypes.NewPruningOptions(
case pruningtypes.PruningOptionCustom:
opts := pruningtypes.NewCustomPruningOptions(
cast.ToUint64(appOpts.Get(FlagPruningKeepRecent)),
cast.ToUint64(appOpts.Get(FlagPruningInterval)),
)
@ -34,6 +33,6 @@ func GetPruningOptionsFromFlags(appOpts types.AppOptions) (storetypes.PruningOpt
return opts, nil
default:
return store.PruningOptions{}, fmt.Errorf("unknown pruning strategy %s", strategy)
return pruningtypes.PruningOptions{}, fmt.Errorf("unknown pruning strategy %s", strategy)
}
}

View File

@ -6,48 +6,45 @@ import (
"github.com/spf13/viper"
"github.com/stretchr/testify/require"
"github.com/cosmos/cosmos-sdk/store/types"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
)
func TestGetPruningOptionsFromFlags(t *testing.T) {
tests := []struct {
name string
initParams func() *viper.Viper
expectedOptions types.PruningOptions
expectedOptions pruningtypes.PruningOptions
wantErr bool
}{
{
name: FlagPruning,
initParams: func() *viper.Viper {
v := viper.New()
v.Set(FlagPruning, types.PruningOptionNothing)
v.Set(FlagPruning, pruningtypes.PruningOptionNothing)
return v
},
expectedOptions: types.PruneNothing,
expectedOptions: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing),
},
{
name: "custom pruning options",
initParams: func() *viper.Viper {
v := viper.New()
v.Set(FlagPruning, types.PruningOptionCustom)
v.Set(FlagPruning, pruningtypes.PruningOptionCustom)
v.Set(FlagPruningKeepRecent, 1234)
v.Set(FlagPruningInterval, 10)
return v
},
expectedOptions: types.PruningOptions{
KeepRecent: 1234,
Interval: 10,
},
expectedOptions: pruningtypes.NewCustomPruningOptions(1234, 10),
},
{
name: types.PruningOptionDefault,
name: pruningtypes.PruningOptionDefault,
initParams: func() *viper.Viper {
v := viper.New()
v.Set(FlagPruning, types.PruningOptionDefault)
v.Set(FlagPruning, pruningtypes.PruningOptionDefault)
return v
},
expectedOptions: types.PruneDefault,
expectedOptions: pruningtypes.NewPruningOptions(pruningtypes.PruningDefault),
},
}
@ -56,7 +53,7 @@ func TestGetPruningOptionsFromFlags(t *testing.T) {
t.Run(tt.name, func(j *testing.T) {
viper.Reset()
viper.SetDefault(FlagPruning, types.PruningOptionDefault)
viper.SetDefault(FlagPruning, pruningtypes.PruningOptionDefault)
v := tt.initParams()
opts, err := GetPruningOptionsFromFlags(v)

View File

@ -36,7 +36,7 @@ application.
return fmt.Errorf("failed to rollback tendermint state: %w", err)
}
// rollback the multistore
cms := rootmulti.NewStore(db)
cms := rootmulti.NewStore(db, ctx.Logger)
cms.RollbackToVersion(height)
fmt.Printf("Rolled back state to height %d and hash %X", height, hash)

View File

@ -25,13 +25,13 @@ import (
"github.com/cosmos/cosmos-sdk/client"
"github.com/cosmos/cosmos-sdk/client/flags"
"github.com/cosmos/cosmos-sdk/codec"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/server/api"
"github.com/cosmos/cosmos-sdk/server/config"
servergrpc "github.com/cosmos/cosmos-sdk/server/grpc"
"github.com/cosmos/cosmos-sdk/server/rosetta"
crgserver "github.com/cosmos/cosmos-sdk/server/rosetta/lib/server"
"github.com/cosmos/cosmos-sdk/server/types"
storetypes "github.com/cosmos/cosmos-sdk/store/types"
)
const (
@ -86,15 +86,15 @@ func StartCmd(appCreator types.AppCreator, defaultNodeHome string) *cobra.Comman
Long: `Run the full node application with Tendermint in or out of process. By
default, the application will run with Tendermint in process.
Pruning options can be provided via the '--pruning' flag or alternatively with '--pruning-keep-recent'
and 'pruning-interval' together.
Pruning options can be provided via the '--pruning' flag or alternatively with '--pruning-keep-recent', and
'pruning-interval' together.
For '--pruning' the options are as follows:
default: the last 362880 states are kept, pruning at 10 block intervals
nothing: all historic states will be saved, nothing will be deleted (i.e. archiving node)
everything: all saved states will be deleted, storing only the current and previous state; pruning at 10 block intervals
custom: allow pruning options to be manually specified through 'pruning-keep-recent' and 'pruning-interval'
everything: 2 latest states will be kept; pruning at 10 block intervals.
custom: allow pruning options to be manually specified through 'pruning-keep-recent', and 'pruning-interval'
Node halting configurations exist in the form of two flags: '--halt-height' and '--halt-time'. During
the ABCI Commit phase, the node will check if the current block height is greater than or equal to
@ -159,7 +159,7 @@ is performed. Note, when enabled, gRPC will also be automatically enabled.
cmd.Flags().Bool(FlagInterBlockCache, true, "Enable inter-block caching")
cmd.Flags().String(flagCPUProfile, "", "Enable CPU profiling and write to the provided file")
cmd.Flags().Bool(FlagTrace, false, "Provide full stack traces for errors in ABCI Log")
cmd.Flags().String(FlagPruning, storetypes.PruningOptionDefault, "Pruning strategy (default|nothing|everything|custom)")
cmd.Flags().String(FlagPruning, pruningtypes.PruningOptionDefault, "Pruning strategy (default|nothing|everything|custom)")
cmd.Flags().Uint64(FlagPruningKeepRecent, 0, "Number of recent heights to keep on disk (ignored if pruning is not 'custom')")
cmd.Flags().Uint64(FlagPruningInterval, 0, "Height interval at which pruned heights are removed from disk (ignored if pruning is not 'custom')")
cmd.Flags().Uint(FlagInvCheckPeriod, 0, "Assert registered invariants every N blocks")

View File

@ -26,6 +26,7 @@ import (
"github.com/cosmos/cosmos-sdk/simapp"
"github.com/cosmos/cosmos-sdk/simapp/params"
"github.com/cosmos/cosmos-sdk/snapshots"
snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types"
"github.com/cosmos/cosmos-sdk/store"
sdk "github.com/cosmos/cosmos-sdk/types"
authcmd "github.com/cosmos/cosmos-sdk/x/auth/client/cli"
@ -270,6 +271,11 @@ func (a appCreator) newApp(logger log.Logger, db dbm.DB, traceStore io.Writer, a
panic(err)
}
snapshotOptions := snapshottypes.NewSnapshotOptions(
cast.ToUint64(appOpts.Get(server.FlagStateSyncSnapshotInterval)),
cast.ToUint32(appOpts.Get(server.FlagStateSyncSnapshotKeepRecent)),
)
return simapp.NewSimApp(
logger, db, traceStore, true, skipUpgradeHeights,
cast.ToString(appOpts.Get(flags.FlagHome)),
@ -284,9 +290,7 @@ func (a appCreator) newApp(logger log.Logger, db dbm.DB, traceStore io.Writer, a
baseapp.SetInterBlockCache(cache),
baseapp.SetTrace(cast.ToBool(appOpts.Get(server.FlagTrace))),
baseapp.SetIndexEvents(cast.ToStringSlice(appOpts.Get(server.FlagIndexEvents))),
baseapp.SetSnapshotStore(snapshotStore),
baseapp.SetSnapshotInterval(cast.ToUint64(appOpts.Get(server.FlagStateSyncSnapshotInterval))),
baseapp.SetSnapshotKeepRecent(cast.ToUint32(appOpts.Get(server.FlagStateSyncSnapshotKeepRecent))),
baseapp.SetSnapshot(snapshotStore, snapshotOptions),
)
}

View File

@ -28,7 +28,9 @@ filesystem under `<node_home>/data/snapshots/`, with metadata in a LevelDB datab
Snapshots are taken asynchronously, i.e. new blocks will be applied concurrently
with snapshots being taken. This is possible because IAVL supports querying
immutable historical heights.
immutable historical heights. However, this requires heights that are multiples of `state-sync.snapshot-interval`
to be kept until after the snapshot is complete. It is done to prevent a height from being removed
while it is being snapshotted.
When a remote node is state syncing, Tendermint calls the ABCI method
`ListSnapshots` to list available local snapshots and `LoadSnapshotChunk` to
@ -47,6 +49,52 @@ can be trivially forged by an adversary. This was considered out of scope for
the initial implementation, but can be added later without changes to the
ABCI state sync protocol.
## Relationship to Pruning
Snapshot settings are optional. However, if set, they have an effect on how pruning is done by
persisting the heights that are multiples of `state-sync.snapshot-interval` until after the snapshot is complete.
If pruning is enabled (not `pruning = "nothing"`), we avoid pruning heights that are multiples of
`state-sync.snapshot-interval` in the regular logic determined by the
pruning settings and applied after every `Commit()`. This is done to prevent a
height from being removed before a snapshot is complete. Therefore, we keep
such heights until after a snapshot is done. At this point, the height is sent to
the `pruning.Manager` to be pruned according to the pruning settings after the next `Commit()`.
To illustrate, assume that we are currently at height 960 with `pruning-keep-recent = 50`,
`pruning-interval = 10`, and `state-sync.snapshot-interval = 100`. Let's assume that
the snapshot that was triggered at height `900` **just finishes**. Then, we can prune height
`900` right away (that is, when we call `Commit()` at height 960 because 900 is less than `960 - 50 = 910`.
Let's now assume that all conditions stay the same but the snapshot at height 900 is **not complete yet**.
Then, we cannot prune it to avoid deleting a height that is still being snapshotted. Therefore, we keep track
of this height until the snapshot is complete. The height 900 will be pruned at the first height h that satisfied the following conditions:
- the snapshot is complete
- h is a multiple of `pruning-interval`
- snapshot height is less than h - `pruning-keep-recent`
Note that in both examples, if we let current height = C, and previous height P = C - 1, then for every height h that is:
P - `pruning-keep-recent` - `pruning-interval` <= h <= P - `pruning-keep-recent`
we can prune height h. In our first example, all heights 899 - 909 fall in this range and are pruned at height 960 as long as
h is not a snapshot height (E.g. 900).
That is, we always use current height to determine at which height to prune (960) while we use previous
to determine which heights are to be pruned (959 - 50 - 10 = 899-909 = 959 - 50).
## Configuration
- `state-sync.snapshot-interval`
* the interval at which to take snapshots.
* the value of 0 disables snapshots.
* if pruning is enabled, it is done after a snapshot is complete for the heights that are multiples of this interval.
- `state-sync.snapshot-keep-recent`:
* the number of recent snapshots to keep.
* 0 means keep all.
## Snapshot Metadata
The ABCI Protobuf type for a snapshot is listed below (refer to the ABCI spec
@ -181,7 +229,9 @@ concurrently.
During `BaseApp.Commit`, once a state transition has been committed, the height
is checked against the `state-sync.snapshot-interval` setting. If the committed
height should be snapshotted, a goroutine `BaseApp.snapshot()` is spawned that
calls `snapshots.Manager.Create()` to create the snapshot.
calls `snapshots.Manager.Create()` to create the snapshot. Once a snapshot is
complete and if pruning is enabled, the snapshot height is pruned away by the manager
with the call `PruneSnapshotHeight(...)` to the `snapshots.types.Snapshotter`.
`Manager.Create()` will do some basic pre-flight checks, and then start
generating a snapshot by calling `rootmulti.Store.Snapshot()`. The chunk stream

View File

@ -7,16 +7,17 @@ import (
"crypto/sha256"
"errors"
"io"
"os"
"testing"
"time"
protoio "github.com/gogo/protobuf/io"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/libs/log"
db "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/snapshots"
"github.com/cosmos/cosmos-sdk/snapshots/types"
snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types"
"github.com/cosmos/cosmos-sdk/testutil"
sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
)
@ -73,7 +74,7 @@ func snapshotItems(items [][]byte) [][]byte {
zWriter, _ := zlib.NewWriterLevel(bufWriter, 7)
protoWriter := protoio.NewDelimitedWriter(zWriter)
for _, item := range items {
_ = types.WriteExtensionItem(protoWriter, item)
_ = snapshottypes.WriteExtensionItem(protoWriter, item)
}
_ = protoWriter.Close()
_ = zWriter.Close()
@ -94,41 +95,43 @@ func snapshotItems(items [][]byte) [][]byte {
}
type mockSnapshotter struct {
items [][]byte
items [][]byte
prunedHeights map[int64]struct{}
snapshotInterval uint64
}
func (m *mockSnapshotter) Restore(
height uint64, format uint32, protoReader protoio.Reader,
) (types.SnapshotItem, error) {
) (snapshottypes.SnapshotItem, error) {
if format == 0 {
return types.SnapshotItem{}, types.ErrUnknownFormat
return snapshottypes.SnapshotItem{}, snapshottypes.ErrUnknownFormat
}
if m.items != nil {
return types.SnapshotItem{}, errors.New("already has contents")
return snapshottypes.SnapshotItem{}, errors.New("already has contents")
}
m.items = [][]byte{}
for {
item := &types.SnapshotItem{}
item := &snapshottypes.SnapshotItem{}
err := protoReader.ReadMsg(item)
if err == io.EOF {
break
} else if err != nil {
return types.SnapshotItem{}, sdkerrors.Wrap(err, "invalid protobuf message")
return snapshottypes.SnapshotItem{}, sdkerrors.Wrap(err, "invalid protobuf message")
}
payload := item.GetExtensionPayload()
if payload == nil {
return types.SnapshotItem{}, sdkerrors.Wrap(err, "invalid protobuf message")
return snapshottypes.SnapshotItem{}, sdkerrors.Wrap(err, "invalid protobuf message")
}
m.items = append(m.items, payload.Payload)
}
return types.SnapshotItem{}, nil
return snapshottypes.SnapshotItem{}, nil
}
func (m *mockSnapshotter) Snapshot(height uint64, protoWriter protoio.Writer) error {
for _, item := range m.items {
if err := types.WriteExtensionItem(protoWriter, item); err != nil {
if err := snapshottypes.WriteExtensionItem(protoWriter, item); err != nil {
return err
}
}
@ -136,30 +139,40 @@ func (m *mockSnapshotter) Snapshot(height uint64, protoWriter protoio.Writer) er
}
func (m *mockSnapshotter) SnapshotFormat() uint32 {
return types.CurrentFormat
return snapshottypes.CurrentFormat
}
func (m *mockSnapshotter) SupportedFormats() []uint32 {
return []uint32{types.CurrentFormat}
return []uint32{snapshottypes.CurrentFormat}
}
func (m *mockSnapshotter) PruneSnapshotHeight(height int64) {
m.prunedHeights[height] = struct{}{}
}
func (m *mockSnapshotter) GetSnapshotInterval() uint64 {
return m.snapshotInterval
}
func (m *mockSnapshotter) SetSnapshotInterval(snapshotInterval uint64) {
m.snapshotInterval = snapshotInterval
}
// setupBusyManager creates a manager with an empty store that is busy creating a snapshot at height 1.
// The snapshot will complete when the returned closer is called.
func setupBusyManager(t *testing.T) *snapshots.Manager {
// os.MkdirTemp() is used instead of testing.T.TempDir()
// see https://github.com/cosmos/cosmos-sdk/pull/8475 for
// this change's rationale.
tempdir, err := os.MkdirTemp("", "")
require.NoError(t, err)
t.Cleanup(func() { _ = os.RemoveAll(tempdir) })
store, err := snapshots.NewStore(db.NewMemDB(), tempdir)
store, err := snapshots.NewStore(db.NewMemDB(), testutil.GetTempDir(t))
require.NoError(t, err)
hung := newHungSnapshotter()
mgr := snapshots.NewManager(store, hung, nil)
hung.SetSnapshotInterval(opts.Interval)
mgr := snapshots.NewManager(store, opts, hung, nil, log.NewNopLogger())
require.Equal(t, opts.Interval, hung.snapshotInterval)
go func() {
_, err := mgr.Create(1)
require.NoError(t, err)
_, didPruneHeight := hung.prunedHeights[1]
require.True(t, didPruneHeight)
}()
time.Sleep(10 * time.Millisecond)
t.Cleanup(hung.Close)
@ -169,12 +182,15 @@ func setupBusyManager(t *testing.T) *snapshots.Manager {
// hungSnapshotter can be used to test operations in progress. Call close to end the snapshot.
type hungSnapshotter struct {
ch chan struct{}
ch chan struct{}
prunedHeights map[int64]struct{}
snapshotInterval uint64
}
func newHungSnapshotter() *hungSnapshotter {
return &hungSnapshotter{
ch: make(chan struct{}),
ch: make(chan struct{}),
prunedHeights: make(map[int64]struct{}),
}
}
@ -187,8 +203,16 @@ func (m *hungSnapshotter) Snapshot(height uint64, protoWriter protoio.Writer) er
return nil
}
func (m *hungSnapshotter) PruneSnapshotHeight(height int64) {
m.prunedHeights[height] = struct{}{}
}
func (m *hungSnapshotter) SetSnapshotInterval(snapshotInterval uint64) {
m.snapshotInterval = snapshotInterval
}
func (m *hungSnapshotter) Restore(
height uint64, format uint32, protoReader protoio.Reader,
) (types.SnapshotItem, error) {
) (snapshottypes.SnapshotItem, error) {
panic("not implemented")
}

View File

@ -3,36 +3,19 @@ package snapshots
import (
"bytes"
"crypto/sha256"
"errors"
"fmt"
"io"
"math"
"sort"
"sync"
"github.com/tendermint/tendermint/libs/log"
"github.com/cosmos/cosmos-sdk/snapshots/types"
sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
)
const (
opNone operation = ""
opSnapshot operation = "snapshot"
opPrune operation = "prune"
opRestore operation = "restore"
chunkBufferSize = 4
snapshotMaxItemSize = int(64e6) // SDK has no key/value size limit, so we set an arbitrary limit
)
// operation represents a Manager operation. Only one operation can be in progress at a time.
type operation string
// restoreDone represents the result of a restore operation.
type restoreDone struct {
complete bool // if true, restore completed successfully (not prematurely)
err error // if non-nil, restore errored
}
// Manager manages snapshot and restore operations for an app, making sure only a single
// long-running operation is in progress at any given time, and provides convenience methods
// mirroring the ABCI interface.
@ -47,9 +30,13 @@ type restoreDone struct {
// 2) io.ReadCloser streams automatically propagate IO errors, and can pass arbitrary
// errors via io.Pipe.CloseWithError().
type Manager struct {
store *Store
multistore types.Snapshotter
extensions map[string]types.ExtensionSnapshotter
// store is the snapshot store where all completed snapshots are persisted.
store *Store
opts types.SnapshotOptions
// multistore is the store from which snapshots are taken.
multistore types.Snapshotter
logger log.Logger
mtx sync.Mutex
operation operation
@ -59,12 +46,38 @@ type Manager struct {
restoreChunkIndex uint32
}
// operation represents a Manager operation. Only one operation can be in progress at a time.
type operation string
// restoreDone represents the result of a restore operation.
type restoreDone struct {
complete bool // if true, restore completed successfully (not prematurely)
err error // if non-nil, restore errored
}
const (
opNone operation = ""
opSnapshot operation = "snapshot"
opPrune operation = "prune"
opRestore operation = "restore"
chunkBufferSize = 4
snapshotMaxItemSize = int(64e6) // SDK has no key/value size limit, so we set an arbitrary limit
)
var (
ErrOptsZeroSnapshotInterval = errors.New("snaphot-interval must not be 0")
)
// NewManager creates a new manager.
func NewManager(store *Store, multistore types.Snapshotter, extensions map[string]types.ExtensionSnapshotter) *Manager {
func NewManager(store *Store, opts types.SnapshotOptions, multistore types.Snapshotter, extensions map[string]types.ExtensionSnapshotter, logger log.Logger) *Manager {
return &Manager{
store: store,
opts: opts,
multistore: multistore,
extensions: extensions,
logger: logger,
}
}
@ -121,15 +134,22 @@ func (m *Manager) endLocked() {
m.restoreChunkIndex = 0
}
// sortedExtensionNames sort extension names for deterministic iteration.
func (m *Manager) sortedExtensionNames() []string {
names := make([]string, 0, len(m.extensions))
for name := range m.extensions {
names = append(names, name)
}
// GetInterval returns snapshot interval represented in heights.
func (m *Manager) GetInterval() uint64 {
return m.opts.Interval
}
sort.Strings(names)
return names
// GetKeepRecent returns snapshot keep-recent represented in heights.
func (m *Manager) GetKeepRecent() uint32 {
return m.opts.KeepRecent
}
// GetSnapshotBlockRetentionHeights returns the number of heights needed
// for block retention. Blocks since the oldest available snapshot must be
// available for state sync nodes to catch up (oldest because a node may be
// restoring an old snapshot while a new snapshot was taken).
func (m *Manager) GetSnapshotBlockRetentionHeights() int64 {
return int64(m.opts.Interval * uint64(m.opts.KeepRecent))
}
// Create creates a snapshot and returns its metadata.
@ -137,6 +157,9 @@ func (m *Manager) Create(height uint64) (*types.Snapshot, error) {
if m == nil {
return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "no snapshot store configured")
}
defer m.multistore.PruneSnapshotHeight(int64(height))
err := m.begin(opSnapshot)
if err != nil {
return nil, err
@ -367,6 +390,17 @@ func (m *Manager) RestoreChunk(chunk []byte) (bool, error) {
return false, nil
}
// sortedExtensionNames sort extension names for deterministic iteration.
func (m *Manager) sortedExtensionNames() []string {
names := make([]string, 0, len(m.extensions))
for name := range m.extensions {
names = append(names, name)
}
sort.Strings(names)
return names
}
// IsFormatSupported returns if the snapshotter supports restoration from given format.
func IsFormatSupported(snapshotter types.ExtensionSnapshotter, format uint32) bool {
for _, i := range snapshotter.SupportedFormats() {
@ -376,3 +410,50 @@ func IsFormatSupported(snapshotter types.ExtensionSnapshotter, format uint32) bo
}
return false
}
// SnapshotIfApplicable takes a snapshot of the current state if we are on a snapshot height.
// It also prunes any old snapshots.
func (m *Manager) SnapshotIfApplicable(height int64) {
if m == nil {
return
}
if !m.shouldTakeSnapshot(height) {
m.logger.Debug("snapshot is skipped", "height", height)
return
}
m.snapshot(height)
}
// shouldTakeSnapshot returns true is snapshot should be taken at height.
func (m *Manager) shouldTakeSnapshot(height int64) bool {
return m.opts.Interval > 0 && uint64(height)%m.opts.Interval == 0
}
func (m *Manager) snapshot(height int64) {
m.logger.Info("creating state snapshot", "height", height)
if height <= 0 {
m.logger.Error("snapshot height must be positive", "height", height)
return
}
snapshot, err := m.Create(uint64(height))
if err != nil {
m.logger.Error("failed to create state snapshot", "height", height, "err", err)
return
}
m.logger.Info("completed state snapshot", "height", height, "format", snapshot.Format)
if m.opts.KeepRecent > 0 {
m.logger.Debug("pruning state snapshots")
pruned, err := m.Prune(m.opts.KeepRecent)
if err != nil {
m.logger.Error("Failed to prune state snapshots", "err", err)
return
}
m.logger.Debug("pruned state snapshots", "pruned", pruned)
}
}

View File

@ -6,14 +6,20 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/libs/log"
"github.com/cosmos/cosmos-sdk/snapshots"
"github.com/cosmos/cosmos-sdk/snapshots/types"
)
var opts = types.NewSnapshotOptions(1500, 2)
func TestManager_List(t *testing.T) {
store := setupStore(t)
manager := snapshots.NewManager(store, nil, nil)
snapshotter := &mockSnapshotter{}
snapshotter.SetSnapshotInterval(opts.Interval)
manager := snapshots.NewManager(store, opts, snapshotter, nil, log.NewNopLogger())
require.Equal(t, opts.Interval, snapshotter.GetSnapshotInterval())
mgrList, err := manager.List()
require.NoError(t, err)
@ -32,7 +38,7 @@ func TestManager_List(t *testing.T) {
func TestManager_LoadChunk(t *testing.T) {
store := setupStore(t)
manager := snapshots.NewManager(store, nil, nil)
manager := snapshots.NewManager(store, opts, &mockSnapshotter{}, nil, log.NewNopLogger())
// Existing chunk should return body
chunk, err := manager.LoadChunk(2, 1, 1)
@ -59,10 +65,11 @@ func TestManager_Take(t *testing.T) {
{7, 8, 9},
}
snapshotter := &mockSnapshotter{
items: items,
items: items,
prunedHeights: make(map[int64]struct{}),
}
expectChunks := snapshotItems(items)
manager := snapshots.NewManager(store, snapshotter, nil)
manager := snapshots.NewManager(store, opts, snapshotter, nil, log.NewNopLogger())
// nil manager should return error
_, err := (*snapshots.Manager)(nil).Create(1)
@ -71,10 +78,15 @@ func TestManager_Take(t *testing.T) {
// creating a snapshot at a lower height than the latest should error
_, err = manager.Create(3)
require.Error(t, err)
_, didPruneHeight := snapshotter.prunedHeights[3]
require.True(t, didPruneHeight)
// creating a snapshot at a higher height should be fine, and should return it
snapshot, err := manager.Create(5)
require.NoError(t, err)
_, didPruneHeight = snapshotter.prunedHeights[5]
require.True(t, didPruneHeight)
assert.Equal(t, &types.Snapshot{
Height: 5,
Format: snapshotter.SnapshotFormat(),
@ -98,7 +110,9 @@ func TestManager_Take(t *testing.T) {
func TestManager_Prune(t *testing.T) {
store := setupStore(t)
manager := snapshots.NewManager(store, nil, nil)
snapshotter := &mockSnapshotter{}
snapshotter.SetSnapshotInterval(opts.Interval)
manager := snapshots.NewManager(store, opts, snapshotter, nil, log.NewNopLogger())
pruned, err := manager.Prune(2)
require.NoError(t, err)
@ -116,8 +130,10 @@ func TestManager_Prune(t *testing.T) {
func TestManager_Restore(t *testing.T) {
store := setupStore(t)
target := &mockSnapshotter{}
manager := snapshots.NewManager(store, target, nil)
target := &mockSnapshotter{
prunedHeights: make(map[int64]struct{}),
}
manager := snapshots.NewManager(store, opts, target, nil, log.NewNopLogger())
expectItems := [][]byte{
{1, 2, 3},
@ -165,6 +181,8 @@ func TestManager_Restore(t *testing.T) {
// While the restore is in progress, any other operations fail
_, err = manager.Create(4)
require.Error(t, err)
_, didPruneHeight := target.prunedHeights[4]
require.True(t, didPruneHeight)
_, err = manager.Prune(1)
require.Error(t, err)

View File

@ -4,7 +4,6 @@ import (
"bytes"
"errors"
"io"
"os"
"path/filepath"
"testing"
"time"
@ -19,14 +18,7 @@ import (
)
func setupStore(t *testing.T) *snapshots.Store {
// os.MkdirTemp() is used instead of testing.T.TempDir()
// see https://github.com/cosmos/cosmos-sdk/pull/8475 for
// this change's rationale.
tempdir, err := os.MkdirTemp("", "")
require.NoError(t, err)
t.Cleanup(func() { _ = os.RemoveAll(tempdir) })
store, err := snapshots.NewStore(db.NewMemDB(), tempdir)
store, err := snapshots.NewStore(db.NewMemDB(), testutil.GetTempDir(t))
require.NoError(t, err)
_, err = store.Save(1, 1, makeChunks([][]byte{

View File

@ -0,0 +1,22 @@
package types
// SnapshotOptions defines the snapshot strategy used when determining which
// heights are snapshotted for state sync.
type SnapshotOptions struct {
// Interval defines at which heights the snapshot is taken.
Interval uint64
// KeepRecent defines how many snapshots to keep in heights.
KeepRecent uint32
}
// SnapshotIntervalOff represents the snapshot interval, at which
// no snapshots are taken.
const SnapshotIntervalOff uint64 = 0
func NewSnapshotOptions(interval uint64, keepRecent uint32) SnapshotOptions {
return SnapshotOptions{
Interval: interval,
KeepRecent: keepRecent,
}
}

View File

@ -11,7 +11,18 @@ type Snapshotter interface {
// Snapshot writes snapshot items into the protobuf writer.
Snapshot(height uint64, protoWriter protoio.Writer) error
// Restore restores a state snapshot from the protobuf items read from the reader.
// PruneSnapshotHeight prunes the given height according to the prune strategy.
// If PruneNothing, this is a no-op.
// If other strategy, this height is persisted until it is
// less than <current height> - KeepRecent and <current height> % Interval == 0
PruneSnapshotHeight(height int64)
// SetSnapshotInterval sets the interval at which the snapshots are taken.
// It is used by the store that implements the Snapshotter interface
// to determine which heights to retain until after the snapshot is complete.
SetSnapshotInterval(snapshotInterval uint64)
// Restore restores a state snapshot, taking snapshot chunk readers as input.
// If the ready channel is non-nil, it returns a ready signal (by being closed) once the
// restorer is ready to accept chunks.
Restore(height uint64, format uint32, protoReader protoio.Reader) (SnapshotItem, error)

View File

@ -12,6 +12,7 @@ import (
tmcrypto "github.com/tendermint/tendermint/proto/tendermint/crypto"
dbm "github.com/tendermint/tm-db"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/store/cachekv"
"github.com/cosmos/cosmos-sdk/store/listenkv"
"github.com/cosmos/cosmos-sdk/store/tracekv"
@ -128,13 +129,13 @@ func (st *Store) LastCommitID() types.CommitID {
// SetPruning panics as pruning options should be provided at initialization
// since IAVl accepts pruning options directly.
func (st *Store) SetPruning(_ types.PruningOptions) {
func (st *Store) SetPruning(_ pruningtypes.PruningOptions) {
panic("cannot set pruning options on an initialized IAVL store")
}
// SetPruning panics as pruning options should be provided at initialization
// since IAVl accepts pruning options directly.
func (st *Store) GetPruning() types.PruningOptions {
func (st *Store) GetPruning() pruningtypes.PruningOptions {
panic("cannot get pruning options on an initialized IAVL store")
}

View File

@ -5,6 +5,7 @@ import (
dbm "github.com/tendermint/tm-db"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/store/cachekv"
"github.com/cosmos/cosmos-sdk/store/dbadapter"
"github.com/cosmos/cosmos-sdk/store/listenkv"
@ -54,10 +55,12 @@ func (s Store) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types
// Commit performs a no-op as entries are persistent between commitments.
func (s *Store) Commit() (id types.CommitID) { return }
func (s *Store) SetPruning(pruning types.PruningOptions) {}
func (s *Store) SetPruning(pruning pruningtypes.PruningOptions) {}
// GetPruning is a no-op as pruning options cannot be directly set on this store.
// They must be set on the root commit multi-store.
func (s *Store) GetPruning() types.PruningOptions { return types.PruningOptions{} }
func (s *Store) GetPruning() pruningtypes.PruningOptions {
return pruningtypes.NewPruningOptions(pruningtypes.PruningUndefined)
}
func (s Store) LastCommitID() (id types.CommitID) { return }

View File

@ -6,7 +6,6 @@ import (
// Import cosmos-sdk/types/store.go for convenience.
type (
PruningOptions = types.PruningOptions
Store = types.Store
Committer = types.Committer
CommitStore = types.CommitStore

View File

@ -3,6 +3,8 @@ package rootmulti
import (
"github.com/cosmos/cosmos-sdk/store/dbadapter"
"github.com/cosmos/cosmos-sdk/store/types"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
)
var commithash = []byte("FAKE_HASH")
@ -30,8 +32,10 @@ func (cdsa commitDBStoreAdapter) LastCommitID() types.CommitID {
}
}
func (cdsa commitDBStoreAdapter) SetPruning(_ types.PruningOptions) {}
func (cdsa commitDBStoreAdapter) SetPruning(_ pruningtypes.PruningOptions) {}
// GetPruning is a no-op as pruning options cannot be directly set on this store.
// They must be set on the root commit multi-store.
func (cdsa commitDBStoreAdapter) GetPruning() types.PruningOptions { return types.PruningOptions{} }
func (cdsa commitDBStoreAdapter) GetPruning() pruningtypes.PruningOptions {
return pruningtypes.NewPruningOptions(pruningtypes.PruningUndefined)
}

View File

@ -5,6 +5,7 @@ import (
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/store/iavl"
@ -57,7 +58,7 @@ func TestVerifyIAVLStoreQueryProof(t *testing.T) {
func TestVerifyMultiStoreQueryProof(t *testing.T) {
// Create main tree for testing.
db := dbm.NewMemDB()
store := NewStore(db)
store := NewStore(db, log.NewNopLogger())
iavlStoreKey := types.NewKVStoreKey("iavlStoreKey")
store.MountStoreWithDB(iavlStoreKey, types.StoreTypeIAVL, nil)
@ -112,7 +113,7 @@ func TestVerifyMultiStoreQueryProof(t *testing.T) {
func TestVerifyMultiStoreQueryProofAbsence(t *testing.T) {
// Create main tree for testing.
db := dbm.NewMemDB()
store := NewStore(db)
store := NewStore(db, log.NewNopLogger())
iavlStoreKey := types.NewKVStoreKey("iavlStoreKey")
store.MountStoreWithDB(iavlStoreKey, types.StoreTypeIAVL, nil)

View File

@ -18,11 +18,12 @@ import (
"github.com/cosmos/cosmos-sdk/store/iavl"
"github.com/cosmos/cosmos-sdk/store/rootmulti"
"github.com/cosmos/cosmos-sdk/store/types"
"github.com/tendermint/tendermint/libs/log"
dbm "github.com/tendermint/tm-db"
)
func newMultiStoreWithGeneratedData(db dbm.DB, stores uint8, storeKeys uint64) *rootmulti.Store {
multiStore := rootmulti.NewStore(db)
multiStore := rootmulti.NewStore(db, log.NewNopLogger())
r := rand.New(rand.NewSource(49872768940)) // Fixed seed for deterministic tests
keys := []*types.KVStoreKey{}
@ -54,7 +55,7 @@ func newMultiStoreWithGeneratedData(db dbm.DB, stores uint8, storeKeys uint64) *
}
func newMultiStoreWithMixedMounts(db dbm.DB) *rootmulti.Store {
store := rootmulti.NewStore(db)
store := rootmulti.NewStore(db, log.NewNopLogger())
store.MountStoreWithDB(types.NewKVStoreKey("iavl1"), types.StoreTypeIAVL, nil)
store.MountStoreWithDB(types.NewKVStoreKey("iavl2"), types.StoreTypeIAVL, nil)
store.MountStoreWithDB(types.NewKVStoreKey("iavl3"), types.StoreTypeIAVL, nil)
@ -234,7 +235,7 @@ func benchmarkMultistoreSnapshot(b *testing.B, stores uint8, storeKeys uint64) {
b.StartTimer()
for i := 0; i < b.N; i++ {
target := rootmulti.NewStore(dbm.NewMemDB())
target := rootmulti.NewStore(dbm.NewMemDB(), log.NewNopLogger())
for _, key := range source.StoreKeysByName() {
target.MountStoreWithDB(key, types.StoreTypeIAVL, nil)
}
@ -269,7 +270,7 @@ func benchmarkMultistoreSnapshotRestore(b *testing.B, stores uint8, storeKeys ui
b.StartTimer()
for i := 0; i < b.N; i++ {
target := rootmulti.NewStore(dbm.NewMemDB())
target := rootmulti.NewStore(dbm.NewMemDB(), log.NewNopLogger())
for _, key := range source.StoreKeysByName() {
target.MountStoreWithDB(key, types.StoreTypeIAVL, nil)
}

View File

@ -1,7 +1,6 @@
package rootmulti
import (
"encoding/binary"
"fmt"
"io"
"math"
@ -14,8 +13,11 @@ import (
gogotypes "github.com/gogo/protobuf/types"
"github.com/pkg/errors"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/pruning"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types"
"github.com/cosmos/cosmos-sdk/store/cachemulti"
"github.com/cosmos/cosmos-sdk/store/dbadapter"
@ -30,7 +32,6 @@ import (
const (
latestVersionKey = "s/latest"
pruneHeightsKey = "s/pruneheights"
commitInfoKeyFmt = "s/%d" // s/<version>
)
@ -39,14 +40,14 @@ const (
// the CommitMultiStore interface.
type Store struct {
db dbm.DB
logger log.Logger
lastCommitInfo *types.CommitInfo
pruningOpts types.PruningOptions
pruningManager *pruning.Manager
iavlCacheSize int
storesParams map[types.StoreKey]storeParams
stores map[types.StoreKey]types.CommitKVStore
keysByName map[string]types.StoreKey
lazyLoading bool
pruneHeights []int64
initialVersion int64
removalMap map[types.StoreKey]bool
@ -68,30 +69,36 @@ var (
// store will be created with a PruneNothing pruning strategy by default. After
// a store is created, KVStores must be mounted and finally LoadLatestVersion or
// LoadVersion must be called.
func NewStore(db dbm.DB) *Store {
func NewStore(db dbm.DB, logger log.Logger) *Store {
return &Store{
db: db,
pruningOpts: types.PruneNothing,
iavlCacheSize: iavl.DefaultIAVLCacheSize,
storesParams: make(map[types.StoreKey]storeParams),
stores: make(map[types.StoreKey]types.CommitKVStore),
keysByName: make(map[string]types.StoreKey),
pruneHeights: make([]int64, 0),
listeners: make(map[types.StoreKey][]types.WriteListener),
removalMap: make(map[types.StoreKey]bool),
db: db,
logger: logger,
iavlCacheSize: iavl.DefaultIAVLCacheSize,
storesParams: make(map[types.StoreKey]storeParams),
stores: make(map[types.StoreKey]types.CommitKVStore),
keysByName: make(map[string]types.StoreKey),
listeners: make(map[types.StoreKey][]types.WriteListener),
removalMap: make(map[types.StoreKey]bool),
pruningManager: pruning.NewManager(db, logger),
}
}
// GetPruning fetches the pruning strategy from the root store.
func (rs *Store) GetPruning() types.PruningOptions {
return rs.pruningOpts
func (rs *Store) GetPruning() pruningtypes.PruningOptions {
return rs.pruningManager.GetOptions()
}
// SetPruning sets the pruning strategy on the root store and all the sub-stores.
// Note, calling SetPruning on the root store prior to LoadVersion or
// LoadLatestVersion performs a no-op as the stores aren't mounted yet.
func (rs *Store) SetPruning(pruningOpts types.PruningOptions) {
rs.pruningOpts = pruningOpts
func (rs *Store) SetPruning(pruningOpts pruningtypes.PruningOptions) {
rs.pruningManager.SetOptions(pruningOpts)
}
// SetSnapshotInterval sets the interval at which the snapshots are taken.
// It is used by the store to determine which heights to retain until after the snapshot is complete.
func (rs *Store) SetSnapshotInterval(snapshotInterval uint64) {
rs.pruningManager.SetSnapshotInterval(snapshotInterval)
}
func (rs *Store) SetIAVLCacheSize(cacheSize int) {
@ -262,9 +269,8 @@ func (rs *Store) loadVersion(ver int64, upgrades *types.StoreUpgrades) error {
rs.stores = newStores
// load any pruned heights we missed from disk to be pruned on the next run
ph, err := getPruningHeights(rs.db)
if err == nil && len(ph) > 0 {
rs.pruneHeights = ph
if err := rs.pruningManager.LoadPruningHeights(rs.db); err != nil {
return err
}
return nil
@ -309,6 +315,14 @@ func moveKVStoreData(oldDB types.KVStore, newDB types.KVStore) error {
return deleteKVStore(oldDB)
}
// PruneSnapshotHeight prunes the given height according to the prune strategy.
// If PruneNothing, this is a no-op.
// If other strategy, this height is persisted until it is
// less than <current height> - KeepRecent and <current height> % Interval == 0
func (rs *Store) PruneSnapshotHeight(height int64) {
rs.pruningManager.HandleHeightSnapshot(height)
}
// SetInterBlockCache sets the Store's internal inter-block (persistent) cache.
// When this is defined, all CommitKVStores will be wrapped with their respective
// inter-block cache.
@ -403,6 +417,7 @@ func (rs *Store) Commit() types.CommitID {
}
rs.lastCommitInfo = commitStores(version, rs.stores, rs.removalMap)
defer rs.flushMetadata(rs.db, version, rs.lastCommitInfo)
// remove remnants of removed stores
for sk := range rs.removalMap {
@ -412,54 +427,19 @@ func (rs *Store) Commit() types.CommitID {
delete(rs.keysByName, sk.Name())
}
}
// reset the removalMap
rs.removalMap = make(map[types.StoreKey]bool)
// Determine if pruneHeight height needs to be added to the list of heights to
// be pruned, where pruneHeight = (commitHeight - 1) - KeepRecent.
if rs.pruningOpts.Interval > 0 && int64(rs.pruningOpts.KeepRecent) < previousHeight {
pruneHeight := previousHeight - int64(rs.pruningOpts.KeepRecent)
rs.pruneHeights = append(rs.pruneHeights, pruneHeight)
if err := rs.handlePruning(version); err != nil {
panic(err)
}
// batch prune if the current height is a pruning interval height
if rs.pruningOpts.Interval > 0 && version%int64(rs.pruningOpts.Interval) == 0 {
rs.pruneStores()
}
flushMetadata(rs.db, version, rs.lastCommitInfo, rs.pruneHeights)
return types.CommitID{
Version: version,
Hash: rs.lastCommitInfo.Hash(),
}
}
// pruneStores will batch delete a list of heights from each mounted sub-store.
// Afterwards, pruneHeights is reset.
func (rs *Store) pruneStores() {
if len(rs.pruneHeights) == 0 {
return
}
for key, store := range rs.stores {
if store.GetStoreType() == types.StoreTypeIAVL {
// If the store is wrapped with an inter-block cache, we must first unwrap
// it to get the underlying IAVL store.
store = rs.GetCommitKVStore(key)
if err := store.(*iavl.Store).DeleteVersions(rs.pruneHeights...); err != nil {
if errCause := errors.Cause(err); errCause != nil && errCause != iavltree.ErrVersionDoesNotExist {
panic(err)
}
}
}
}
rs.pruneHeights = make([]int64, 0)
}
// CacheWrap implements CacheWrapper/Store/CommitStore.
func (rs *Store) CacheWrap() types.CacheWrap {
return rs.CacheMultiStore().(types.CacheWrap)
@ -553,7 +533,51 @@ func (rs *Store) GetKVStore(key types.StoreKey) types.KVStore {
return store
}
// GetStoreByName performs a lookup of a StoreKey given a store name typically
func (rs *Store) handlePruning(version int64) error {
rs.pruningManager.HandleHeight(version - 1) // we should never prune the current version.
if !rs.pruningManager.ShouldPruneAtHeight(version) {
return nil
}
rs.logger.Info("prune start", "height", version)
defer rs.logger.Info("prune end", "height", version)
return rs.pruneStores()
}
func (rs *Store) pruneStores() error {
pruningHeights, err := rs.pruningManager.GetFlushAndResetPruningHeights()
if err != nil {
return err
}
if len(pruningHeights) == 0 {
rs.logger.Debug("pruning skipped; no heights to prune")
return nil
}
rs.logger.Debug("pruning heights", "heights", pruningHeights)
for key, store := range rs.stores {
// If the store is wrapped with an inter-block cache, we must first unwrap
// it to get the underlying IAVL store.
if store.GetStoreType() != types.StoreTypeIAVL {
continue
}
store = rs.GetCommitKVStore(key)
err := store.(*iavl.Store).DeleteVersions(pruningHeights...)
if err == nil {
continue
}
if errCause := errors.Cause(err); errCause != nil && errCause != iavltree.ErrVersionDoesNotExist {
return err
}
}
return nil
}
// getStoreByName performs a lookup of a StoreKey given a store name typically
// provided in a path. The StoreKey is then used to perform a lookup and return
// a Store. If the Store is wrapped in an inter-block cache, it will be unwrapped
// prior to being returned. If the StoreKey does not exist, nil is returned.
@ -666,7 +690,7 @@ func (rs *Store) Snapshot(height uint64, protoWriter protoio.Writer) error {
if height == 0 {
return sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot snapshot height 0")
}
if height > uint64(rs.LastCommitID().Version) {
if height > uint64(getLatestVersion(rs.db)) {
return sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot snapshot future height %v", height)
}
@ -819,7 +843,7 @@ loop:
importer.Close()
}
flushMetadata(rs.db, int64(height), rs.buildCommitInfo(int64(height)), []int64{})
rs.flushMetadata(rs.db, int64(height), rs.buildCommitInfo(int64(height)))
return snapshotItem, rs.LoadLatestVersion()
}
@ -910,9 +934,11 @@ func (rs *Store) RollbackToVersion(target int64) int64 {
return current
}
for ; current > target; current-- {
rs.pruneHeights = append(rs.pruneHeights, current)
rs.pruningManager.HandleHeight(current)
}
if err := rs.pruneStores(); err != nil {
panic(err)
}
rs.pruneStores()
// update latest height
bz, err := gogotypes.StdInt64Marshal(current)
@ -924,6 +950,25 @@ func (rs *Store) RollbackToVersion(target int64) int64 {
return current
}
func (rs *Store) flushMetadata(db dbm.DB, version int64, cInfo *types.CommitInfo) {
rs.logger.Debug("flushing metadata", "height", version)
batch := db.NewBatch()
defer batch.Close()
if cInfo != nil {
flushCommitInfo(batch, version, cInfo)
} else {
rs.logger.Debug("commitInfo is nil, not flushed", "height", version)
}
flushLatestVersion(batch, version)
if err := batch.WriteSync(); err != nil {
panic(fmt.Errorf("error on batch write %w", err))
}
rs.logger.Debug("flushing metadata finished", "height", version)
}
type storeParams struct {
key types.StoreKey
db dbm.DB
@ -996,7 +1041,7 @@ func getCommitInfo(db dbm.DB, ver int64) (*types.CommitInfo, error) {
return cInfo, nil
}
func setCommitInfo(batch dbm.Batch, version int64, cInfo *types.CommitInfo) {
func flushCommitInfo(batch dbm.Batch, version int64, cInfo *types.CommitInfo) {
bz, err := cInfo.Marshal()
if err != nil {
panic(err)
@ -1006,7 +1051,7 @@ func setCommitInfo(batch dbm.Batch, version int64, cInfo *types.CommitInfo) {
batch.Set([]byte(cInfoKey), bz)
}
func setLatestVersion(batch dbm.Batch, version int64) {
func flushLatestVersion(batch dbm.Batch, version int64) {
bz, err := gogotypes.StdInt64Marshal(version)
if err != nil {
panic(err)
@ -1014,47 +1059,3 @@ func setLatestVersion(batch dbm.Batch, version int64) {
batch.Set([]byte(latestVersionKey), bz)
}
func setPruningHeights(batch dbm.Batch, pruneHeights []int64) {
bz := make([]byte, 0)
for _, ph := range pruneHeights {
buf := make([]byte, 8)
binary.BigEndian.PutUint64(buf, uint64(ph))
bz = append(bz, buf...)
}
batch.Set([]byte(pruneHeightsKey), bz)
}
func getPruningHeights(db dbm.DB) ([]int64, error) {
bz, err := db.Get([]byte(pruneHeightsKey))
if err != nil {
return nil, fmt.Errorf("failed to get pruned heights: %w", err)
}
if len(bz) == 0 {
return nil, errors.New("no pruned heights found")
}
prunedHeights := make([]int64, len(bz)/8)
i, offset := 0, 0
for offset < len(bz) {
prunedHeights[i] = int64(binary.BigEndian.Uint64(bz[offset : offset+8]))
i++
offset += 8
}
return prunedHeights, nil
}
func flushMetadata(db dbm.DB, version int64, cInfo *types.CommitInfo, pruneHeights []int64) {
batch := db.NewBatch()
defer batch.Close()
setCommitInfo(batch, version, cInfo)
setLatestVersion(batch, version)
setPruningHeights(batch, pruneHeights)
if err := batch.Write(); err != nil {
panic(fmt.Errorf("error on batch write %w", err))
}
}

View File

@ -8,10 +8,12 @@ import (
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/codec"
codecTypes "github.com/cosmos/cosmos-sdk/codec/types"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/store/cachemulti"
"github.com/cosmos/cosmos-sdk/store/iavl"
sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps"
@ -22,13 +24,13 @@ import (
func TestStoreType(t *testing.T) {
db := dbm.NewMemDB()
store := NewStore(db)
store := NewStore(db, log.NewNopLogger())
store.MountStoreWithDB(types.NewKVStoreKey("store1"), types.StoreTypeIAVL, db)
}
func TestGetCommitKVStore(t *testing.T) {
var db dbm.DB = dbm.NewMemDB()
ms := newMultiStoreWithMounts(db, types.PruneDefault)
ms := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningDefault))
err := ms.LoadLatestVersion()
require.Nil(t, err)
@ -45,7 +47,7 @@ func TestGetCommitKVStore(t *testing.T) {
func TestStoreMount(t *testing.T) {
db := dbm.NewMemDB()
store := NewStore(db)
store := NewStore(db, log.NewNopLogger())
key1 := types.NewKVStoreKey("store1")
key2 := types.NewKVStoreKey("store2")
@ -61,7 +63,7 @@ func TestStoreMount(t *testing.T) {
func TestCacheMultiStore(t *testing.T) {
var db dbm.DB = dbm.NewMemDB()
ms := newMultiStoreWithMounts(db, types.PruneNothing)
ms := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
cacheMulti := ms.CacheMultiStore()
require.IsType(t, cachemulti.Store{}, cacheMulti)
@ -69,7 +71,7 @@ func TestCacheMultiStore(t *testing.T) {
func TestCacheMultiStoreWithVersion(t *testing.T) {
var db dbm.DB = dbm.NewMemDB()
ms := newMultiStoreWithMounts(db, types.PruneNothing)
ms := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err := ms.LoadLatestVersion()
require.Nil(t, err)
@ -106,7 +108,7 @@ func TestCacheMultiStoreWithVersion(t *testing.T) {
func TestHashStableWithEmptyCommit(t *testing.T) {
var db dbm.DB = dbm.NewMemDB()
ms := newMultiStoreWithMounts(db, types.PruneNothing)
ms := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err := ms.LoadLatestVersion()
require.Nil(t, err)
@ -130,7 +132,7 @@ func TestHashStableWithEmptyCommit(t *testing.T) {
func TestMultistoreCommitLoad(t *testing.T) {
var db dbm.DB = dbm.NewMemDB()
store := newMultiStoreWithMounts(db, types.PruneNothing)
store := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err := store.LoadLatestVersion()
require.Nil(t, err)
@ -155,7 +157,7 @@ func TestMultistoreCommitLoad(t *testing.T) {
}
// Load the latest multistore again and check version.
store = newMultiStoreWithMounts(db, types.PruneNothing)
store = newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err = store.LoadLatestVersion()
require.Nil(t, err)
commitID = getExpectedCommitID(store, nCommits)
@ -168,7 +170,7 @@ func TestMultistoreCommitLoad(t *testing.T) {
// Load an older multistore and check version.
ver := nCommits - 1
store = newMultiStoreWithMounts(db, types.PruneNothing)
store = newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err = store.LoadVersion(ver)
require.Nil(t, err)
commitID = getExpectedCommitID(store, ver)
@ -177,7 +179,7 @@ func TestMultistoreCommitLoad(t *testing.T) {
func TestMultistoreLoadWithUpgrade(t *testing.T) {
var db dbm.DB = dbm.NewMemDB()
store := newMultiStoreWithMounts(db, types.PruneNothing)
store := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err := store.LoadLatestVersion()
require.Nil(t, err)
@ -212,7 +214,7 @@ func TestMultistoreLoadWithUpgrade(t *testing.T) {
checkContains(t, ci.StoreInfos, []string{"store1", "store2", "store3"})
// Load without changes and make sure it is sensible
store = newMultiStoreWithMounts(db, types.PruneNothing)
store = newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err = store.LoadLatestVersion()
require.Nil(t, err)
@ -225,7 +227,7 @@ func TestMultistoreLoadWithUpgrade(t *testing.T) {
require.Equal(t, v2, s2.Get(k2))
// now, let's load with upgrades...
restore, upgrades := newMultiStoreWithModifiedMounts(db, types.PruneNothing)
restore, upgrades := newMultiStoreWithModifiedMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err = restore.LoadLatestVersionAndUpgrade(upgrades)
require.Nil(t, err)
@ -270,7 +272,7 @@ func TestMultistoreLoadWithUpgrade(t *testing.T) {
migratedID := restore.Commit()
require.Equal(t, migratedID.Version, int64(2))
reload, _ := newMultiStoreWithModifiedMounts(db, types.PruneNothing)
reload, _ := newMultiStoreWithModifiedMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err = reload.LoadLatestVersion()
require.Nil(t, err)
require.Equal(t, migratedID, reload.LastCommitID())
@ -319,10 +321,7 @@ func TestParsePath(t *testing.T) {
func TestMultiStoreRestart(t *testing.T) {
db := dbm.NewMemDB()
pruning := types.PruningOptions{
KeepRecent: 2,
Interval: 1,
}
pruning := pruningtypes.NewCustomPruningOptions(2, 1)
multi := newMultiStoreWithMounts(db, pruning)
err := multi.LoadLatestVersion()
require.Nil(t, err)
@ -401,7 +400,7 @@ func TestMultiStoreRestart(t *testing.T) {
func TestMultiStoreQuery(t *testing.T) {
db := dbm.NewMemDB()
multi := newMultiStoreWithMounts(db, types.PruneNothing)
multi := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err := multi.LoadLatestVersion()
require.Nil(t, err)
@ -428,7 +427,7 @@ func TestMultiStoreQuery(t *testing.T) {
ver := cid.Version
// Reload multistore from database
multi = newMultiStoreWithMounts(db, types.PruneNothing)
multi = newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err = multi.LoadLatestVersion()
require.Nil(t, err)
@ -473,15 +472,15 @@ func TestMultiStore_Pruning(t *testing.T) {
testCases := []struct {
name string
numVersions int64
po types.PruningOptions
po pruningtypes.PruningOptions
deleted []int64
saved []int64
}{
{"prune nothing", 10, types.PruneNothing, nil, []int64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}},
{"prune everything", 10, types.PruneEverything, []int64{1, 2, 3, 4, 5, 6, 7, 8, 9}, []int64{10}},
{"prune some; no batch", 10, types.NewPruningOptions(2, 1), []int64{1, 2, 4, 5, 7}, []int64{3, 6, 8, 9, 10}},
{"prune some; small batch", 10, types.NewPruningOptions(2, 3), []int64{1, 2, 4, 5}, []int64{3, 6, 7, 8, 9, 10}},
{"prune some; large batch", 10, types.NewPruningOptions(2, 11), nil, []int64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}},
{"prune nothing", 10, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), nil, []int64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}},
{"prune everything", 10, pruningtypes.NewPruningOptions(pruningtypes.PruningEverything), []int64{1, 2, 3, 4, 5, 6, 7, 8, 9}, []int64{10}},
{"prune some; no batch", 10, pruningtypes.NewCustomPruningOptions(2, 1), []int64{1, 2, 4, 5, 7}, []int64{3, 6, 8, 9, 10}},
{"prune some; small batch", 10, pruningtypes.NewCustomPruningOptions(2, 3), []int64{1, 2, 4, 5}, []int64{3, 6, 7, 8, 9, 10}},
{"prune some; large batch", 10, pruningtypes.NewCustomPruningOptions(2, 11), nil, []int64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}},
}
for _, tc := range testCases {
@ -509,9 +508,64 @@ func TestMultiStore_Pruning(t *testing.T) {
}
}
func TestMultiStore_Pruning_SameHeightsTwice(t *testing.T) {
const (
numVersions int64 = 10
keepRecent uint64 = 2
interval uint64 = 10
)
expectedHeights := []int64{}
for i := int64(1); i < numVersions-int64(keepRecent); i++ {
expectedHeights = append(expectedHeights, i)
}
db := dbm.NewMemDB()
ms := newMultiStoreWithMounts(db, pruningtypes.NewCustomPruningOptions(keepRecent, interval))
require.NoError(t, ms.LoadLatestVersion())
var lastCommitInfo types.CommitID
for i := int64(0); i < numVersions; i++ {
lastCommitInfo = ms.Commit()
}
require.Equal(t, numVersions, lastCommitInfo.Version)
for v := int64(1); v < numVersions-int64(keepRecent); v++ {
err := ms.LoadVersion(v)
require.Error(t, err, "expected error when loading pruned height: %d", v)
}
for v := int64(numVersions - int64(keepRecent)); v < numVersions; v++ {
err := ms.LoadVersion(v)
require.NoError(t, err, "expected no error when loading height: %d", v)
}
// Get latest
err := ms.LoadVersion(numVersions - 1)
require.NoError(t, err)
// Ensure already pruned heights were loaded
heights, err := ms.pruningManager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Equal(t, expectedHeights, heights)
require.NoError(t, ms.pruningManager.LoadPruningHeights(db))
// Test pruning the same heights again
lastCommitInfo = ms.Commit()
require.Equal(t, numVersions, lastCommitInfo.Version)
// Ensure that can commit one more height with no panic
lastCommitInfo = ms.Commit()
require.Equal(t, numVersions+1, lastCommitInfo.Version)
}
func TestMultiStore_PruningRestart(t *testing.T) {
db := dbm.NewMemDB()
ms := newMultiStoreWithMounts(db, types.NewPruningOptions(2, 11))
ms := newMultiStoreWithMounts(db, pruningtypes.NewCustomPruningOptions(2, 11))
ms.SetSnapshotInterval(3)
require.NoError(t, ms.LoadLatestVersion())
// Commit enough to build up heights to prune, where on the next block we should
@ -523,19 +577,30 @@ func TestMultiStore_PruningRestart(t *testing.T) {
pruneHeights := []int64{1, 2, 4, 5, 7}
// ensure we've persisted the current batch of heights to prune to the store's DB
ph, err := getPruningHeights(ms.db)
err := ms.pruningManager.LoadPruningHeights(ms.db)
require.NoError(t, err)
require.Equal(t, []int64{1, 2, 3, 4, 5, 6, 7}, ph)
actualHeightsToPrune, err := ms.pruningManager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Equal(t, len(pruneHeights), len(actualHeightsToPrune))
require.Equal(t, pruneHeights, actualHeightsToPrune)
// "restart"
ms = newMultiStoreWithMounts(db, types.NewPruningOptions(2, 11))
ms = newMultiStoreWithMounts(db, pruningtypes.NewCustomPruningOptions(2, 11))
ms.SetSnapshotInterval(3)
err = ms.LoadLatestVersion()
require.NoError(t, err)
require.Equal(t, []int64{1, 2, 3, 4, 5, 6, 7}, ms.pruneHeights)
actualHeightsToPrune, err = ms.pruningManager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Equal(t, pruneHeights, actualHeightsToPrune)
// commit one more block and ensure the heights have been pruned
ms.Commit()
require.Empty(t, ms.pruneHeights)
actualHeightsToPrune, err = ms.pruningManager.GetFlushAndResetPruningHeights()
require.NoError(t, err)
require.Empty(t, actualHeightsToPrune)
for _, v := range pruneHeights {
_, err := ms.CacheMultiStoreWithVersion(v)
@ -545,7 +610,7 @@ func TestMultiStore_PruningRestart(t *testing.T) {
func TestSetInitialVersion(t *testing.T) {
db := dbm.NewMemDB()
multi := newMultiStoreWithMounts(db, types.PruneNothing)
multi := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
require.NoError(t, multi.LoadLatestVersion())
@ -563,7 +628,7 @@ func TestSetInitialVersion(t *testing.T) {
func TestAddListenersAndListeningEnabled(t *testing.T) {
db := dbm.NewMemDB()
multi := newMultiStoreWithMounts(db, types.PruneNothing)
multi := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
testKey := types.NewKVStoreKey("listening_test_key")
enabled := multi.ListeningEnabled(testKey)
require.False(t, enabled)
@ -594,7 +659,7 @@ var (
func TestGetListenWrappedKVStore(t *testing.T) {
buf := new(bytes.Buffer)
var db dbm.DB = dbm.NewMemDB()
ms := newMultiStoreWithMounts(db, types.PruneNothing)
ms := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
ms.LoadLatestVersion()
mockListeners := []types.WriteListener{types.NewStoreKVPairWriteListener(buf, testMarshaller)}
ms.AddListeners(testStoreKey1, mockListeners)
@ -637,6 +702,7 @@ func TestGetListenWrappedKVStore(t *testing.T) {
StoreKey: testStoreKey2.Name(),
Delete: false,
})
require.NoError(t, err)
kvPairSet2Bytes := buf.Bytes()
buf.Reset()
require.Equal(t, expectedOutputKVPairSet2, kvPairSet2Bytes)
@ -648,6 +714,7 @@ func TestGetListenWrappedKVStore(t *testing.T) {
StoreKey: testStoreKey2.Name(),
Delete: true,
})
require.NoError(t, err)
kvPairDelete2Bytes := buf.Bytes()
buf.Reset()
require.Equal(t, expectedOutputKVPairDelete2, kvPairDelete2Bytes)
@ -668,7 +735,7 @@ func TestGetListenWrappedKVStore(t *testing.T) {
func TestCacheWraps(t *testing.T) {
db := dbm.NewMemDB()
multi := newMultiStoreWithMounts(db, types.PruneNothing)
multi := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
cacheWrapper := multi.CacheWrap()
require.IsType(t, cachemulti.Store{}, cacheWrapper)
@ -682,7 +749,7 @@ func TestCacheWraps(t *testing.T) {
func TestTraceConcurrency(t *testing.T) {
db := dbm.NewMemDB()
multi := newMultiStoreWithMounts(db, types.PruneNothing)
multi := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err := multi.LoadLatestVersion()
require.NoError(t, err)
@ -732,7 +799,7 @@ func TestTraceConcurrency(t *testing.T) {
func TestCommitOrdered(t *testing.T) {
var db dbm.DB = dbm.NewMemDB()
multi := newMultiStoreWithMounts(db, types.PruneNothing)
multi := newMultiStoreWithMounts(db, pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
err := multi.LoadLatestVersion()
require.Nil(t, err)
@ -773,9 +840,9 @@ var (
testStoreKey3 = types.NewKVStoreKey("store3")
)
func newMultiStoreWithMounts(db dbm.DB, pruningOpts types.PruningOptions) *Store {
store := NewStore(db)
store.pruningOpts = pruningOpts
func newMultiStoreWithMounts(db dbm.DB, pruningOpts pruningtypes.PruningOptions) *Store {
store := NewStore(db, log.NewNopLogger())
store.SetPruning(pruningOpts)
store.MountStoreWithDB(testStoreKey1, types.StoreTypeIAVL, nil)
store.MountStoreWithDB(testStoreKey2, types.StoreTypeIAVL, nil)
@ -784,9 +851,9 @@ func newMultiStoreWithMounts(db dbm.DB, pruningOpts types.PruningOptions) *Store
return store
}
func newMultiStoreWithModifiedMounts(db dbm.DB, pruningOpts types.PruningOptions) (*Store, *types.StoreUpgrades) {
store := NewStore(db)
store.pruningOpts = pruningOpts
func newMultiStoreWithModifiedMounts(db dbm.DB, pruningOpts pruningtypes.PruningOptions) (*Store, *types.StoreUpgrades) {
store := NewStore(db, log.NewNopLogger())
store.SetPruning(pruningOpts)
store.MountStoreWithDB(types.NewKVStoreKey("store1"), types.StoreTypeIAVL, nil)
store.MountStoreWithDB(types.NewKVStoreKey("restore2"), types.StoreTypeIAVL, nil)

View File

@ -1,6 +1,7 @@
package store
import (
"github.com/tendermint/tendermint/libs/log"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/store/cache"
@ -9,7 +10,7 @@ import (
)
func NewCommitMultiStore(db dbm.DB) types.CommitMultiStore {
return rootmulti.NewStore(db)
return rootmulti.NewStore(db, log.NewNopLogger())
}
func NewCommitKVStoreCacheManager() types.MultiStorePersistentCache {

View File

@ -3,6 +3,7 @@ package transient
import (
dbm "github.com/tendermint/tm-db"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/store/dbadapter"
"github.com/cosmos/cosmos-sdk/store/types"
)
@ -27,11 +28,13 @@ func (ts *Store) Commit() (id types.CommitID) {
return
}
func (ts *Store) SetPruning(_ types.PruningOptions) {}
func (ts *Store) SetPruning(_ pruningtypes.PruningOptions) {}
// GetPruning is a no-op as pruning options cannot be directly set on this store.
// They must be set on the root commit multi-store.
func (ts *Store) GetPruning() types.PruningOptions { return types.PruningOptions{} }
func (ts *Store) GetPruning() pruningtypes.PruningOptions {
return pruningtypes.NewPruningOptions(pruningtypes.PruningUndefined)
}
// Implements CommitStore
func (ts *Store) LastCommitID() (id types.CommitID) {

View File

@ -6,8 +6,9 @@ import (
"github.com/stretchr/testify/require"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/store/transient"
types "github.com/cosmos/cosmos-sdk/store/v2alpha1"
"github.com/cosmos/cosmos-sdk/store/v2alpha1/transient"
)
var k, v = []byte("hello"), []byte("world")
@ -26,7 +27,7 @@ func TestTransientStore(t *testing.T) {
require.Nil(t, tstore.Get(k))
// no-op
tstore.SetPruning(types.PruningOptions{})
tstore.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningUndefined))
emptyCommitID := tstore.LastCommitID()
require.Equal(t, emptyCommitID.Version, int64(0))

View File

@ -1,71 +0,0 @@
package types
import (
"fmt"
)
// Pruning option string constants
const (
PruningOptionDefault = "default"
PruningOptionEverything = "everything"
PruningOptionNothing = "nothing"
PruningOptionCustom = "custom"
)
var (
// PruneDefault defines a pruning strategy where the last 362880 heights are
// kept in addition to every 100th and where to-be pruned heights are pruned
// at every 10th height. The last 362880 heights are kept assuming the typical
// block time is 5s and typical unbonding period is 21 days. If these values
// do not match the applications' requirements, use the "custom" option.
PruneDefault = NewPruningOptions(362880, 10)
// PruneEverything defines a pruning strategy where all committed heights are
// deleted, storing only the current and previous height and where to-be pruned
// heights are pruned at every 10th height.
PruneEverything = NewPruningOptions(2, 10)
// PruneNothing defines a pruning strategy where all heights are kept on disk.
PruneNothing = NewPruningOptions(0, 0)
)
// PruningOptions defines the pruning strategy used when determining which
// heights are removed from disk when committing state.
type PruningOptions struct {
// KeepRecent defines how many recent heights to keep on disk.
KeepRecent uint64
// Interval defines when the pruned heights are removed from disk.
Interval uint64
}
func NewPruningOptions(keepRecent, interval uint64) PruningOptions {
return PruningOptions{
KeepRecent: keepRecent,
Interval: interval,
}
}
func (po PruningOptions) Validate() error {
if po.KeepRecent > 0 && po.Interval == 0 {
return fmt.Errorf("invalid 'Interval' when pruning recent heights: %d", po.Interval)
}
return nil
}
func NewPruningOptionsFromString(strategy string) PruningOptions {
switch strategy {
case PruningOptionEverything:
return PruneEverything
case PruningOptionNothing:
return PruneNothing
case PruningOptionDefault:
return PruneDefault
default:
return PruneDefault
}
}

View File

@ -1,26 +0,0 @@
package types
import (
"testing"
"github.com/stretchr/testify/require"
)
func TestPruningOptions_Validate(t *testing.T) {
testCases := []struct {
keepRecent uint64
interval uint64
expectErr bool
}{
{100, 10, false}, // default
{0, 10, false}, // everything
{0, 0, false}, // nothing
{100, 0, true}, // invalid interval
}
for _, tc := range testCases {
po := NewPruningOptions(tc.keepRecent, tc.interval)
err := po.Validate()
require.Equal(t, tc.expectErr, err != nil, "options: %v, err: %s", po, err)
}
}

View File

@ -8,6 +8,7 @@ import (
tmstrings "github.com/tendermint/tendermint/libs/strings"
dbm "github.com/tendermint/tm-db"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types"
"github.com/cosmos/cosmos-sdk/types/kv"
)
@ -22,8 +23,8 @@ type Committer interface {
Commit() CommitID
LastCommitID() CommitID
SetPruning(PruningOptions)
GetPruning() PruningOptions
SetPruning(pruningtypes.PruningOptions)
GetPruning() pruningtypes.PruningOptions
}
// Stores of MultiStore must implement CommitStore.

View File

@ -5,20 +5,21 @@ import (
"testing"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/libs/log"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/store/rootmulti"
"github.com/cosmos/cosmos-sdk/store/types"
sdk "github.com/cosmos/cosmos-sdk/store/types"
)
func initTestStores(t *testing.T) (types.KVStore, types.KVStore) {
func initTestStores(t *testing.T) (sdk.KVStore, sdk.KVStore) {
db := dbm.NewMemDB()
ms := rootmulti.NewStore(db)
ms := rootmulti.NewStore(db, log.NewNopLogger())
key1 := types.NewKVStoreKey("store1")
key2 := types.NewKVStoreKey("store2")
require.NotPanics(t, func() { ms.MountStoreWithDB(key1, types.StoreTypeIAVL, db) })
require.NotPanics(t, func() { ms.MountStoreWithDB(key2, types.StoreTypeIAVL, db) })
key1 := sdk.NewKVStoreKey("store1")
key2 := sdk.NewKVStoreKey("store2")
require.NotPanics(t, func() { ms.MountStoreWithDB(key1, sdk.StoreTypeIAVL, db) })
require.NotPanics(t, func() { ms.MountStoreWithDB(key2, sdk.StoreTypeIAVL, db) })
require.NoError(t, ms.LoadLatestVersion())
return ms.GetKVStore(key1), ms.GetKVStore(key2)
}
@ -31,27 +32,27 @@ func TestDiffKVStores(t *testing.T) {
store1.Set(k1, v1)
store2.Set(k1, v1)
kvAs, kvBs := types.DiffKVStores(store1, store2, nil)
kvAs, kvBs := sdk.DiffKVStores(store1, store2, nil)
require.Equal(t, 0, len(kvAs))
require.Equal(t, len(kvAs), len(kvBs))
// delete k1 from store2, which is now empty
store2.Delete(k1)
kvAs, kvBs = types.DiffKVStores(store1, store2, nil)
kvAs, kvBs = sdk.DiffKVStores(store1, store2, nil)
require.Equal(t, 1, len(kvAs))
require.Equal(t, len(kvAs), len(kvBs))
// set k1 in store2, different value than what store1 holds for k1
v2 := []byte("v2")
store2.Set(k1, v2)
kvAs, kvBs = types.DiffKVStores(store1, store2, nil)
kvAs, kvBs = sdk.DiffKVStores(store1, store2, nil)
require.Equal(t, 1, len(kvAs))
require.Equal(t, len(kvAs), len(kvBs))
// add k2 to store2
k2 := []byte("k2")
store2.Set(k2, v2)
kvAs, kvBs = types.DiffKVStores(store1, store2, nil)
kvAs, kvBs = sdk.DiffKVStores(store1, store2, nil)
require.Equal(t, 2, len(kvAs))
require.Equal(t, len(kvAs), len(kvBs))
@ -65,7 +66,7 @@ func TestDiffKVStores(t *testing.T) {
k1Prefixed := append(prefix, k1...)
store1.Set(k1Prefixed, v1)
store2.Set(k1Prefixed, v2)
kvAs, kvBs = types.DiffKVStores(store1, store2, [][]byte{prefix})
kvAs, kvBs = sdk.DiffKVStores(store1, store2, [][]byte{prefix})
require.Equal(t, 0, len(kvAs))
require.Equal(t, len(kvAs), len(kvBs))
}
@ -73,16 +74,16 @@ func TestDiffKVStores(t *testing.T) {
func TestPrefixEndBytes(t *testing.T) {
t.Parallel()
bs1 := []byte{0x23, 0xA5, 0x06}
require.True(t, bytes.Equal([]byte{0x23, 0xA5, 0x07}, types.PrefixEndBytes(bs1)))
require.True(t, bytes.Equal([]byte{0x23, 0xA5, 0x07}, sdk.PrefixEndBytes(bs1)))
bs2 := []byte{0x23, 0xA5, 0xFF}
require.True(t, bytes.Equal([]byte{0x23, 0xA6}, types.PrefixEndBytes(bs2)))
require.Nil(t, types.PrefixEndBytes([]byte{0xFF}))
require.Nil(t, types.PrefixEndBytes(nil))
require.True(t, bytes.Equal([]byte{0x23, 0xA6}, sdk.PrefixEndBytes(bs2)))
require.Nil(t, sdk.PrefixEndBytes([]byte{0xFF}))
require.Nil(t, sdk.PrefixEndBytes(nil))
}
func TestInclusiveEndBytes(t *testing.T) {
t.Parallel()
require.True(t, bytes.Equal([]byte{0x00}, types.InclusiveEndBytes(nil)))
require.True(t, bytes.Equal([]byte{0x00}, sdk.InclusiveEndBytes(nil)))
bs := []byte("test")
require.True(t, bytes.Equal(append(bs, byte(0x00)), types.InclusiveEndBytes(bs)))
require.True(t, bytes.Equal(append(bs, byte(0x00)), sdk.InclusiveEndBytes(bs)))
}

View File

@ -3,6 +3,7 @@ package mem
import (
dbm "github.com/cosmos/cosmos-sdk/db"
"github.com/cosmos/cosmos-sdk/db/memdb"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/store/types"
"github.com/cosmos/cosmos-sdk/store/v2alpha1/dbadapter"
)
@ -38,7 +39,7 @@ func (s *Store) Commit() (id types.CommitID) {
return
}
func (s *Store) SetPruning(pruning types.PruningOptions) {}
func (s *Store) GetPruning() types.PruningOptions { return types.PruningOptions{} }
func (s *Store) SetPruning(pruningtypes.PruningOptions) {}
func (s *Store) GetPruning() pruningtypes.PruningOptions { return pruningtypes.PruningOptions{} }
func (s Store) LastCommitID() (id types.CommitID) { return }

View File

@ -11,6 +11,7 @@ import (
"github.com/cosmos/cosmos-sdk/store/rootmulti"
"github.com/cosmos/cosmos-sdk/store/types"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/libs/log"
dbm "github.com/tendermint/tm-db"
)
@ -19,7 +20,7 @@ func TestMigrationV2(t *testing.T) {
// setup a rootmulti store
db := dbm.NewMemDB()
v1Store := rootmulti.NewStore(db)
v1Store := rootmulti.NewStore(db, log.NewNopLogger())
// mount the kvStores
var keys []*types.KVStoreKey
@ -94,7 +95,7 @@ func TestMigrationV2(t *testing.T) {
func TestMigrateV2ForEmptyStore(t *testing.T) {
// setup a rootmulti store
db := dbm.NewMemDB()
v1Store := rootmulti.NewStore(db)
v1Store := rootmulti.NewStore(db, log.NewNopLogger())
err := v1Store.LoadLatestVersion()
require.Nil(t, err)
db2 := memdb.NewDB()

View File

@ -17,6 +17,7 @@ import (
dbm "github.com/cosmos/cosmos-sdk/db"
"github.com/cosmos/cosmos-sdk/db/memdb"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/snapshots"
snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types"
"github.com/cosmos/cosmos-sdk/store/types"
@ -24,7 +25,7 @@ import (
func multiStoreConfig(t *testing.T, stores int) StoreConfig {
opts := DefaultStoreConfig()
opts.Pruning = types.PruneNothing
opts.Pruning = pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)
for i := 0; i < stores; i++ {
sKey := types.NewKVStoreKey(fmt.Sprintf("store%d", i))

View File

@ -13,6 +13,7 @@ import (
dbm "github.com/cosmos/cosmos-sdk/db"
prefixdb "github.com/cosmos/cosmos-sdk/db/prefix"
util "github.com/cosmos/cosmos-sdk/internal"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps"
"github.com/cosmos/cosmos-sdk/store/listenkv"
"github.com/cosmos/cosmos-sdk/store/prefix"
@ -56,7 +57,7 @@ func ErrStoreNotFound(skey string) error {
// StoreConfig is used to define a schema and other options and pass them to the MultiStore constructor.
type StoreConfig struct {
// Version pruning options for backing DBs.
Pruning types.PruningOptions
Pruning pruningtypes.PruningOptions
// The minimum allowed version number.
InitialVersion uint64
// The backing DB to use for the state commitment Merkle tree data.
@ -92,7 +93,7 @@ type Store struct {
mtx sync.RWMutex
// Copied from StoreConfig
Pruning types.PruningOptions
Pruning pruningtypes.PruningOptions
InitialVersion uint64 // if
*traceListenMixin
@ -152,7 +153,7 @@ func newTraceListenMixin() *traceListenMixin {
// pruning with PruneDefault, no listeners and no tracer.
func DefaultStoreConfig() StoreConfig {
return StoreConfig{
Pruning: types.PruneDefault,
Pruning: pruningtypes.NewPruningOptions(pruningtypes.PruningDefault),
prefixRegistry: prefixRegistry{
StoreSchema: StoreSchema{},
},
@ -175,12 +176,12 @@ func validSubStoreType(sst types.StoreType) bool {
}
// Returns true iff both schema maps match exactly (including mem/tran stores)
func (this StoreSchema) equal(that StoreSchema) bool {
if len(this) != len(that) {
func (ss StoreSchema) equal(that StoreSchema) bool {
if len(ss) != len(that) {
return false
}
for key, val := range that {
myval, has := this[key]
myval, has := ss[key]
if !has {
return false
}
@ -248,7 +249,7 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) {
}
// Version sets of each DB must match
if !versions.Equal(scVersions) {
err = fmt.Errorf("Storage and StateCommitment DB have different version history") //nolint:stylecheck
err = fmt.Errorf("different version history between Storage and StateCommitment DB ")
return
}
err = opts.StateCommitmentDB.Revert()
@ -689,6 +690,20 @@ func (rs *Store) CacheMultiStore() types.CacheMultiStore {
}
}
// PruneSnapshotHeight prunes the given height according to the prune strategy.
// If PruneNothing, this is a no-op.
// If other strategy, this height is persisted until it is
// less than <current height> - KeepRecent and <current height> % Interval == 0
func (rs *Store) PruneSnapshotHeight(height int64) {
panic("not implemented")
}
// SetSnapshotInterval sets the interval at which the snapshots are taken.
// It is used by the store to determine which heights to retain until after the snapshot is complete.
func (rs *Store) SetSnapshotInterval(snapshotInterval uint64) {
panic("not implemented")
}
// parsePath expects a format like /<storeName>[/<subpath>]
// Must start with /, subpath may be empty
// Returns error if it doesn't start with /
@ -769,7 +784,7 @@ func (rs *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) {
// TODO: actual IBC compatible proof. This is a placeholder so unit tests can pass
res.ProofOps, err = substore.GetProof(res.Key)
if err != nil {
return sdkerrors.QueryResult(fmt.Errorf("Merkle proof creation failed for key: %v", res.Key), false) //nolint: stylecheck // proper name
return sdkerrors.QueryResult(fmt.Errorf("merkle proof creation failed for key: %v", res.Key), false)
}
case "/subspace":
@ -894,5 +909,5 @@ func (tlm *traceListenMixin) wrapTraceListen(store types.KVStore, skey types.Sto
return store
}
func (s *Store) GetPruning() types.PruningOptions { return s.Pruning }
func (s *Store) SetPruning(po types.PruningOptions) { s.Pruning = po }
func (s *Store) GetPruning() pruningtypes.PruningOptions { return s.Pruning }
func (s *Store) SetPruning(po pruningtypes.PruningOptions) { s.Pruning = po }

View File

@ -13,12 +13,12 @@ import (
codecTypes "github.com/cosmos/cosmos-sdk/codec/types"
dbm "github.com/cosmos/cosmos-sdk/db"
"github.com/cosmos/cosmos-sdk/db/memdb"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
types "github.com/cosmos/cosmos-sdk/store/v2alpha1"
"github.com/cosmos/cosmos-sdk/types/kv"
)
var (
cacheSize = 100
alohaData = map[string]string{
"hello": "goodbye",
"aloha": "shalom",
@ -40,7 +40,7 @@ func simpleStoreConfig(t *testing.T) StoreConfig {
func storeConfig123(t *testing.T) StoreConfig {
opts := DefaultStoreConfig()
opts.Pruning = types.PruneNothing
opts.Pruning = pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)
require.NoError(t, opts.RegisterSubstore(skey_1.Name(), types.StoreTypePersistent))
require.NoError(t, opts.RegisterSubstore(skey_2.Name(), types.StoreTypePersistent))
require.NoError(t, opts.RegisterSubstore(skey_3.Name(), types.StoreTypePersistent))
@ -101,7 +101,7 @@ func TestConstructors(t *testing.T) {
require.NoError(t, store.Close())
t.Run("fail to load if InitialVersion > lowest existing version", func(t *testing.T) {
opts := StoreConfig{InitialVersion: 5, Pruning: types.PruneNothing}
opts := StoreConfig{InitialVersion: 5, Pruning: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)}
store, err = NewStore(db, opts)
require.Error(t, err)
db.Close()
@ -247,7 +247,7 @@ func TestCommit(t *testing.T) {
}
}
basicOpts := simpleStoreConfig(t)
basicOpts.Pruning = types.PruneNothing
basicOpts.Pruning = pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)
t.Run("sanity tests for Merkle hashing", func(t *testing.T) {
testBasic(basicOpts)
})
@ -286,7 +286,7 @@ func TestCommit(t *testing.T) {
}
opts := simpleStoreConfig(t)
opts.Pruning = types.PruneNothing
opts.Pruning = pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)
// Ensure Store's commit is rolled back in each failure case...
t.Run("recover after failed Commit", func(t *testing.T) {
@ -349,7 +349,7 @@ func TestCommit(t *testing.T) {
t.Run("height overflow triggers failure", func(t *testing.T) {
opts.StateCommitmentDB = nil
opts.InitialVersion = math.MaxInt64
opts.Pruning = types.PruneNothing
opts.Pruning = pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)
store, err := NewStore(memdb.NewDB(), opts)
require.NoError(t, err)
require.Equal(t, int64(math.MaxInt64), store.Commit().Version)
@ -360,7 +360,7 @@ func TestCommit(t *testing.T) {
t.Run("first commit version matches InitialVersion", func(t *testing.T) {
opts = simpleStoreConfig(t)
opts.InitialVersion = 5
opts.Pruning = types.PruneNothing
opts.Pruning = pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)
opts.StateCommitmentDB = memdb.NewDB()
store, err := NewStore(memdb.NewDB(), opts)
require.NoError(t, err)
@ -395,13 +395,13 @@ func sliceToSet(slice []uint64) map[uint64]struct{} {
func TestPruning(t *testing.T) {
// Save versions up to 10 and verify pruning at final commit
testCases := []struct {
types.PruningOptions
pruningtypes.PruningOptions
kept []uint64
}{
{types.PruningOptions{2, 10}, []uint64{8, 9, 10}},
{types.PruningOptions{0, 10}, []uint64{10}},
{types.PruneEverything, []uint64{8, 9, 10}},
{types.PruneNothing, []uint64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}},
{pruningtypes.NewCustomPruningOptions(2, 10), []uint64{8, 9, 10}},
{pruningtypes.NewCustomPruningOptions(0, 10), []uint64{10}},
{pruningtypes.NewPruningOptions(pruningtypes.PruningEverything), []uint64{8, 9, 10}},
{pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), []uint64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}},
}
for tci, tc := range testCases {
@ -443,7 +443,7 @@ func TestPruning(t *testing.T) {
db := memdb.NewDB()
opts := simpleStoreConfig(t)
opts.Pruning = types.PruningOptions{0, 10}
opts.Pruning = pruningtypes.NewCustomPruningOptions(0, 10)
store, err := NewStore(db, opts)
require.NoError(t, err)
@ -689,7 +689,7 @@ func TestGetVersion(t *testing.T) {
require.Panics(t, func() { subview.Set([]byte{1}, []byte{1}) })
require.Panics(t, func() { subview.Delete([]byte{0}) })
// nonexistent version shouldn't be accessible
view, err = store.GetVersion(cid.Version + 1)
_, err = store.GetVersion(cid.Version + 1)
require.Equal(t, ErrVersionDoesNotExist, err)
substore := store.GetKVStore(skey_1)
@ -750,7 +750,7 @@ func TestMultiStoreMigration(t *testing.T) {
t.Run("basic migration", func(t *testing.T) {
// now, let's load with upgrades...
opts.Upgrades = []types.StoreUpgrades{
types.StoreUpgrades{
{
Added: []string{skey_4.Name()},
Renamed: []types.StoreRename{{
OldKey: skey_2.Name(),

View File

@ -86,7 +86,7 @@ func (st *viewSubstore) CacheWrapWithListeners(storeKey types.StoreKey, listener
func (s *viewStore) getMerkleRoots() (ret map[string][]byte, err error) {
ret = map[string][]byte{}
for key, _ := range s.schema {
for key := range s.schema {
sub, has := s.substoreCache[key]
if !has {
sub, err = s.getSubstore(key)

View File

@ -3,6 +3,7 @@ package transient
import (
dbm "github.com/cosmos/cosmos-sdk/db"
"github.com/cosmos/cosmos-sdk/db/memdb"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/store/types"
"github.com/cosmos/cosmos-sdk/store/v2alpha1/dbadapter"
)
@ -40,7 +41,7 @@ func (ts *Store) Commit() (id types.CommitID) {
return
}
func (ts *Store) SetPruning(types.PruningOptions) {}
func (ts *Store) GetPruning() types.PruningOptions { return types.PruningOptions{} }
func (ts *Store) SetPruning(pruningtypes.PruningOptions) {}
func (ts *Store) GetPruning() pruningtypes.PruningOptions { return pruningtypes.PruningOptions{} }
func (ts *Store) LastCommitID() (id types.CommitID) { return }

View File

@ -9,13 +9,12 @@ import (
// Re-export relevant original store types
type (
StoreKey = v1.StoreKey
StoreType = v1.StoreType
CommitID = v1.CommitID
StoreUpgrades = v1.StoreUpgrades
StoreRename = v1.StoreRename
Iterator = v1.Iterator
PruningOptions = v1.PruningOptions
StoreKey = v1.StoreKey
StoreType = v1.StoreType
CommitID = v1.CommitID
StoreUpgrades = v1.StoreUpgrades
StoreRename = v1.StoreRename
Iterator = v1.Iterator
TraceContext = v1.TraceContext
WriteListener = v1.WriteListener
@ -46,10 +45,6 @@ const (
)
var (
PruneDefault = v1.PruneDefault
PruneEverything = v1.PruneEverything
PruneNothing = v1.PruneNothing
NewKVStoreKey = v1.NewKVStoreKey
PrefixEndBytes = v1.PrefixEndBytes
KVStorePrefixIterator = v1.KVStorePrefixIterator

View File

@ -67,8 +67,21 @@ func WriteToNewTempFile(t testing.TB, s string) *os.File {
func TempFile(t testing.TB) *os.File {
t.Helper()
fp, err := os.CreateTemp(t.TempDir(), "")
fp, err := os.CreateTemp(GetTempDir(t), "")
require.NoError(t, err)
return fp
}
// GetTempDir returns a writable temporary director for the test to use.
func GetTempDir(t testing.TB) string {
t.Helper()
// os.MkDir() is used instead of testing.T.TempDir()
// see https://github.com/cosmos/cosmos-sdk/pull/8475 and
// https://github.com/cosmos/cosmos-sdk/pull/10341 for
// this change's rationale.
tempdir, err := os.MkdirTemp("", "")
require.NoError(t, err)
t.Cleanup(func() { _ = os.RemoveAll(tempdir) })
return tempdir
}

View File

@ -32,13 +32,13 @@ import (
"github.com/cosmos/cosmos-sdk/crypto/hd"
"github.com/cosmos/cosmos-sdk/crypto/keyring"
cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/server"
"github.com/cosmos/cosmos-sdk/server/api"
srvconfig "github.com/cosmos/cosmos-sdk/server/config"
servertypes "github.com/cosmos/cosmos-sdk/server/types"
"github.com/cosmos/cosmos-sdk/simapp"
"github.com/cosmos/cosmos-sdk/simapp/params"
storetypes "github.com/cosmos/cosmos-sdk/store/types"
"github.com/cosmos/cosmos-sdk/testutil"
sdk "github.com/cosmos/cosmos-sdk/types"
authtypes "github.com/cosmos/cosmos-sdk/x/auth/types"
@ -61,7 +61,7 @@ func NewAppConstructor(encodingCfg params.EncodingConfig) AppConstructor {
val.Ctx.Logger, dbm.NewMemDB(), nil, true, make(map[int64]bool), val.Ctx.Config.RootDir, 0,
encodingCfg,
simapp.EmptyAppOptions{},
baseapp.SetPruning(storetypes.NewPruningOptionsFromString(val.AppConfig.Pruning)),
baseapp.SetPruning(pruningtypes.NewPruningOptionsFromString(val.AppConfig.Pruning)),
baseapp.SetMinGasPrices(val.AppConfig.MinGasPrices),
)
}
@ -119,7 +119,7 @@ func DefaultConfig() Config {
AccountTokens: sdk.TokensFromConsensusPower(1000, sdk.DefaultPowerReduction),
StakingTokens: sdk.TokensFromConsensusPower(500, sdk.DefaultPowerReduction),
BondedTokens: sdk.TokensFromConsensusPower(100, sdk.DefaultPowerReduction),
PruningStrategy: storetypes.PruningOptionNothing,
PruningStrategy: pruningtypes.PruningOptionNothing,
CleanupDir: true,
SigningAlgo: string(hd.Secp256k1Type),
KeyringOptions: []keyring.Option{},

View File

@ -9,10 +9,6 @@ import (
"github.com/cosmos/cosmos-sdk/types/kv"
)
type (
PruningOptions = types.PruningOptions
)
type (
Store = types.Store
Committer = types.Committer
@ -155,15 +151,15 @@ type (
GasConfig = types.GasConfig
)
func NewGasMeter(limit Gas) GasMeter {
return types.NewGasMeter(limit)
}
type (
ErrorOutOfGas = types.ErrorOutOfGas
ErrorGasOverflow = types.ErrorGasOverflow
)
func NewGasMeter(limit Gas) GasMeter {
return types.NewGasMeter(limit)
}
func NewInfiniteGasMeter() GasMeter {
return types.NewInfiniteGasMeter()
}

View File

@ -4,6 +4,7 @@ import (
"testing"
"github.com/stretchr/testify/suite"
"github.com/tendermint/tendermint/libs/log"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/store/rootmulti"
@ -108,7 +109,7 @@ func (s *storeTestSuite) TestDiffKVStores() {
func (s *storeTestSuite) initTestStores() (types.KVStore, types.KVStore) {
db := dbm.NewMemDB()
ms := rootmulti.NewStore(db)
ms := rootmulti.NewStore(db, log.NewNopLogger())
key1 := types.NewKVStoreKey("store1")
key2 := types.NewKVStoreKey("store2")

View File

@ -14,6 +14,7 @@ import (
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/baseapp"
pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types"
"github.com/cosmos/cosmos-sdk/server"
"github.com/cosmos/cosmos-sdk/store/rootmulti"
storetypes "github.com/cosmos/cosmos-sdk/store/types"
@ -34,8 +35,8 @@ func defaultLogger() log.Logger {
}
func initStore(t *testing.T, db dbm.DB, storeKey string, k, v []byte) {
rs := rootmulti.NewStore(db)
rs.SetPruning(storetypes.PruneNothing)
rs := rootmulti.NewStore(db, log.NewNopLogger())
rs.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
key := sdk.NewKVStoreKey(storeKey)
rs.MountStoreWithDB(key, storetypes.StoreTypeIAVL, nil)
err := rs.LoadLatestVersion()
@ -51,8 +52,8 @@ func initStore(t *testing.T, db dbm.DB, storeKey string, k, v []byte) {
}
func checkStore(t *testing.T, db dbm.DB, ver int64, storeKey string, k, v []byte) {
rs := rootmulti.NewStore(db)
rs.SetPruning(storetypes.PruneNothing)
rs := rootmulti.NewStore(db, log.NewNopLogger())
rs.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))
key := sdk.NewKVStoreKey(storeKey)
rs.MountStoreWithDB(key, storetypes.StoreTypeIAVL, nil)
err := rs.LoadLatestVersion()
@ -122,7 +123,7 @@ func TestSetLoader(t *testing.T) {
initStore(t, db, tc.origStoreKey, k, v)
// load the app with the existing db
opts := []func(*baseapp.BaseApp){baseapp.SetPruning(storetypes.PruneNothing)}
opts := []func(*baseapp.BaseApp){baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))}
origapp := baseapp.NewBaseApp(t.Name(), defaultLogger(), db, opts...)
origapp.MountStores(sdk.NewKVStoreKey(tc.origStoreKey))