Add height in exported genesis (#7089)

* Add height in exported genesis

* +1

* Add test

* Refactor ctx in setupApp

* Use amino in export

* Use tmjson

* Add custom initialVersion (set to 0 for now)

* Add comment

* Add mount in initChainer

* app.LastBlockheight

* InitializeAndSeal in InitChain?

* Revert create store with initial version

* Update to latest iavl

* Check height in test

* Make it work

* Add more tests

* Rename interface

* Use struct isntead of 6 args

* Fix lint

* Remove stray fmt

* Revert go mod/sum

* Install iavl rc3

* Update comments

* Add fee in network

* Typo

* Fix logic in commit

* Fix tests

* Only set initial version on > 1

* Genesis block num = 1

* Fresh chain, genesis block = 0

* Add comments

* Revert Mutable/ImmutableTree

* Allow for zero height

* Fix restart

* Add comments

* Add comments, fix test

* Fix remaining one test

* Add panic test

* Update comment

* Add test for --height

* No cast

* Add check that genesis file exists

* Remove duplicate imports

* Fail early

Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com>
Co-authored-by: Jack Zampolin <jack.zampolin@gmail.com>
Co-authored-by: Cory <cjlevinson@gmail.com>
This commit is contained in:
Amaury Martiny 2020-09-03 12:11:46 +02:00 committed by GitHub
parent 9e85e81e0e
commit 3b9b58c931
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 422 additions and 109 deletions

View File

@ -23,8 +23,21 @@ import (
// InitChain implements the ABCI interface. It runs the initialization logic
// directly on the CommitMultiStore.
func (app *BaseApp) InitChain(req abci.RequestInitChain) (res abci.ResponseInitChain) {
// On a new chain, we consider the init chain block height as 0, even though
// req.InitialHeight is 1 by default.
initHeader := tmproto.Header{ChainID: req.ChainId, Time: req.Time}
// If req.InitialHeight is > 1, then we set the initial version in the
// stores.
if req.InitialHeight > 1 {
app.initialHeight = req.InitialHeight
initHeader = tmproto.Header{ChainID: req.ChainId, Height: req.InitialHeight, Time: req.Time}
err := app.cms.SetInitialVersion(req.InitialHeight)
if err != nil {
panic(err)
}
}
// initialize the deliver state and check state with a correct header
app.setDeliverState(initHeader)
app.setCheckState(initHeader)
@ -60,13 +73,13 @@ func (app *BaseApp) InitChain(req abci.RequestInitChain) (res abci.ResponseInitC
sort.Sort(abci.ValidatorUpdates(res.Validators))
for i := range res.Validators {
if proto.Equal(&res.Validators[i], &req.Validators[i]) {
if !proto.Equal(&res.Validators[i], &req.Validators[i]) {
panic(fmt.Errorf("genesisValidators[%d] != req.Validators[%d] ", i, i))
}
}
}
// NOTE: We don't commit, but BeginBlock for block 1 starts from this
// NOTE: We don't commit, but BeginBlock for block `initial_height` starts from this
// deliverState.
return res
}
@ -97,7 +110,7 @@ func (app *BaseApp) FilterPeerByAddrPort(info string) abci.ResponseQuery {
return abci.ResponseQuery{}
}
// FilterPeerByIDfilters peers by node ID.
// FilterPeerByID filters peers by node ID.
func (app *BaseApp) FilterPeerByID(info string) abci.ResponseQuery {
if app.idPeerFilter != nil {
return app.idPeerFilter(info)

View File

@ -81,6 +81,9 @@ type BaseApp struct { // nolint: maligned
// transaction. This is mainly used for DoS and spam prevention.
minGasPrices sdk.DecCoins
// initialHeight is the initial height at which we start the baseapp
initialHeight int64
// flag for sealing options and parameters to a BaseApp
sealed bool
@ -206,12 +209,6 @@ func (app *BaseApp) MountMemoryStores(keys map[string]*sdk.MemoryStoreKey) {
}
}
// MountStoreWithDB mounts a store to the provided key in the BaseApp
// multistore, using a specified DB.
func (app *BaseApp) MountStoreWithDB(key sdk.StoreKey, typ sdk.StoreType, db dbm.DB) {
app.cms.MountStoreWithDB(key, typ, db)
}
// MountStore mounts a store to the provided key in the BaseApp multistore,
// using the default DB.
func (app *BaseApp) MountStore(key sdk.StoreKey, typ sdk.StoreType) {
@ -422,9 +419,23 @@ func (app *BaseApp) validateHeight(req abci.RequestBeginBlock) error {
return fmt.Errorf("invalid height: %d", req.Header.Height)
}
prevHeight := app.LastBlockHeight()
if req.Header.Height != prevHeight+1 {
return fmt.Errorf("invalid height: %d; expected: %d", req.Header.Height, prevHeight+1)
// expectedHeight holds the expected height to validate.
var expectedHeight int64
if app.LastBlockHeight() == 0 && app.initialHeight > 1 {
// In this case, we're validating the first block of the chain (no
// previous commit). The height we're expecting is the initial height.
expectedHeight = app.initialHeight
} else {
// This case can means two things:
// - either there was already a previous commit in the store, in which
// case we increment the version from there,
// - or there was no previous commit, and initial version was not set,
// in which case we start at version 1.
expectedHeight = app.LastBlockHeight() + 1
}
if req.Header.Height != expectedHeight {
return fmt.Errorf("invalid height: %d; expected: %d", req.Header.Height, expectedHeight)
}
return nil

View File

@ -535,6 +535,52 @@ func TestInitChainer(t *testing.T) {
require.Equal(t, value, res.Value)
}
func TestInitChain_WithInitialHeight(t *testing.T) {
name := t.Name()
db := dbm.NewMemDB()
logger := defaultLogger()
app := NewBaseApp(name, logger, db, nil)
app.InitChain(
abci.RequestInitChain{
InitialHeight: 3,
},
)
app.Commit()
require.Equal(t, int64(3), app.LastBlockHeight())
}
func TestBeginBlock_WithInitialHeight(t *testing.T) {
name := t.Name()
db := dbm.NewMemDB()
logger := defaultLogger()
app := NewBaseApp(name, logger, db, nil)
app.InitChain(
abci.RequestInitChain{
InitialHeight: 3,
},
)
require.PanicsWithError(t, "invalid height: 4; expected: 3", func() {
app.BeginBlock(abci.RequestBeginBlock{
Header: tmproto.Header{
Height: 4,
},
})
})
app.BeginBlock(abci.RequestBeginBlock{
Header: tmproto.Header{
Height: 3,
},
})
app.Commit()
require.Equal(t, int64(3), app.LastBlockHeight())
}
// Simple tx with a list of Msgs.
type txTest struct {
Msgs []sdk.Msg
@ -1292,7 +1338,6 @@ func TestCustomRunTxPanicHandler(t *testing.T) {
anteOpt := func(bapp *BaseApp) {
bapp.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) {
panic(sdkerrors.Wrap(anteErr, "anteHandler"))
return
})
}
routerOpt := func(bapp *BaseApp) {

View File

@ -3,12 +3,12 @@ package server
// DONTCOVER
import (
"encoding/json"
"fmt"
"io/ioutil"
"os"
"github.com/spf13/cobra"
tmjson "github.com/tendermint/tendermint/libs/json"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
tmtypes "github.com/tendermint/tendermint/types"
@ -35,6 +35,10 @@ func ExportCmd(appExporter types.AppExporter, defaultNodeHome string) *cobra.Com
homeDir, _ := cmd.Flags().GetString(flags.FlagHome)
config.SetRoot(homeDir)
if _, err := os.Stat(config.GenesisFile()); os.IsNotExist(err) {
return err
}
db, err := openDB(config.RootDir)
if err != nil {
return err
@ -64,7 +68,7 @@ func ExportCmd(appExporter types.AppExporter, defaultNodeHome string) *cobra.Com
forZeroHeight, _ := cmd.Flags().GetBool(FlagForZeroHeight)
jailAllowedAddrs, _ := cmd.Flags().GetStringSlice(FlagJailAllowedAddrs)
appState, validators, cp, err := appExporter(serverCtx.Logger, db, traceWriter, height, forZeroHeight, jailAllowedAddrs)
exported, err := appExporter(serverCtx.Logger, db, traceWriter, height, forZeroHeight, jailAllowedAddrs)
if err != nil {
return fmt.Errorf("error exporting state: %v", err)
}
@ -74,29 +78,30 @@ func ExportCmd(appExporter types.AppExporter, defaultNodeHome string) *cobra.Com
return err
}
doc.AppState = appState
doc.Validators = validators
doc.AppState = exported.AppState
doc.Validators = exported.Validators
doc.InitialHeight = exported.Height
doc.ConsensusParams = &tmproto.ConsensusParams{
Block: tmproto.BlockParams{
MaxBytes: cp.Block.MaxBytes,
MaxGas: cp.Block.MaxGas,
MaxBytes: exported.ConsensusParams.Block.MaxBytes,
MaxGas: exported.ConsensusParams.Block.MaxGas,
TimeIotaMs: doc.ConsensusParams.Block.TimeIotaMs,
},
Evidence: tmproto.EvidenceParams{
MaxAgeNumBlocks: cp.Evidence.MaxAgeNumBlocks,
MaxAgeDuration: cp.Evidence.MaxAgeDuration,
MaxNum: cp.Evidence.MaxNum,
ProofTrialPeriod: cp.Evidence.ProofTrialPeriod,
MaxAgeNumBlocks: exported.ConsensusParams.Evidence.MaxAgeNumBlocks,
MaxAgeDuration: exported.ConsensusParams.Evidence.MaxAgeDuration,
MaxNum: exported.ConsensusParams.Evidence.MaxNum,
ProofTrialPeriod: exported.ConsensusParams.Evidence.ProofTrialPeriod,
},
Validator: tmproto.ValidatorParams{
PubKeyTypes: cp.Validator.PubKeyTypes,
PubKeyTypes: exported.ConsensusParams.Validator.PubKeyTypes,
},
}
// NOTE: for now we're just using standard JSON marshaling for the root GenesisDoc.
// These types are in Tendermint, don't support proto and as far as we know, don't need it.
// All of the protobuf/amino state is inside AppState
encoded, err := json.MarshalIndent(doc, "", " ")
// NOTE: Tendermint uses a custom JSON decoder for GenesisDoc
// (except for stuff inside AppState). Inside AppState, we're free
// to encode as protobuf or amino.
encoded, err := tmjson.Marshal(doc)
if err != nil {
return err
}

View File

@ -1,4 +1,4 @@
package server
package server_test
import (
"bytes"
@ -10,15 +10,20 @@ import (
"path"
"testing"
"github.com/spf13/cobra"
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
tmjson "github.com/tendermint/tendermint/libs/json"
"github.com/tendermint/tendermint/libs/log"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
tmtypes "github.com/tendermint/tendermint/types"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/client"
"github.com/cosmos/cosmos-sdk/client/flags"
"github.com/cosmos/cosmos-sdk/server"
"github.com/cosmos/cosmos-sdk/server/types"
"github.com/cosmos/cosmos-sdk/simapp"
"github.com/cosmos/cosmos-sdk/testutil"
"github.com/cosmos/cosmos-sdk/types/errors"
@ -29,40 +34,7 @@ func TestExportCmd_ConsensusParams(t *testing.T) {
tempDir, clean := testutil.NewTestCaseDir(t)
defer clean()
err := createConfigFolder(tempDir)
if err != nil {
t.Fatalf("error creating config folder: %s", err)
}
db := dbm.NewMemDB()
app := simapp.NewSimApp(log.NewTMLogger(log.NewSyncWriter(os.Stdout)), db, nil, true, map[int64]bool{}, tempDir, 0, simapp.MakeEncodingConfig())
serverCtx := NewDefaultContext()
serverCtx.Config.RootDir = tempDir
clientCtx := client.Context{}.WithJSONMarshaler(app.AppCodec())
genDoc := newDefaultGenesisDoc()
err = saveGenesisFile(genDoc, serverCtx.Config.GenesisFile())
app.InitChain(
abci.RequestInitChain{
Validators: []abci.ValidatorUpdate{},
ConsensusParams: simapp.DefaultConsensusParams,
AppStateBytes: genDoc.AppState,
},
)
app.Commit()
cmd := ExportCmd(
func(logger log.Logger, db dbm.DB, writer io.Writer, i int64, b bool, strings []string) (json.RawMessage, []tmtypes.GenesisValidator, *abci.ConsensusParams, error) {
return app.ExportAppStateAndValidators(true, []string{})
}, tempDir)
ctx := context.Background()
ctx = context.WithValue(ctx, client.ClientContextKey, &clientCtx)
ctx = context.WithValue(ctx, ServerContextKey, serverCtx)
_, ctx, genDoc, cmd := setupApp(t, tempDir)
output := &bytes.Buffer{}
cmd.SetOut(output)
@ -70,7 +42,7 @@ func TestExportCmd_ConsensusParams(t *testing.T) {
require.NoError(t, cmd.ExecuteContext(ctx))
var exportedGenDoc tmtypes.GenesisDoc
err = json.Unmarshal(output.Bytes(), &exportedGenDoc)
err := tmjson.Unmarshal(output.Bytes(), &exportedGenDoc)
if err != nil {
t.Fatalf("error unmarshaling exported genesis doc: %s", err)
}
@ -85,6 +57,131 @@ func TestExportCmd_ConsensusParams(t *testing.T) {
require.Equal(t, simapp.DefaultConsensusParams.Validator.PubKeyTypes, exportedGenDoc.ConsensusParams.Validator.PubKeyTypes)
}
func TestExportCmd_HomeDir(t *testing.T) {
tempDir, clean := testutil.NewTestCaseDir(t)
defer clean()
_, ctx, _, cmd := setupApp(t, tempDir)
cmd.SetArgs([]string{fmt.Sprintf("--%s=%s", flags.FlagHome, "foobar")})
err := cmd.ExecuteContext(ctx)
require.EqualError(t, err, "stat foobar/config/genesis.json: no such file or directory")
}
func TestExportCmd_Height(t *testing.T) {
testCases := []struct {
name string
flags []string
fastForward int64
expHeight int64
}{
{
"should export correct height",
[]string{},
5, 6,
},
{
"should export correct height with --height",
[]string{
fmt.Sprintf("--%s=%d", server.FlagHeight, 3),
},
5, 4,
},
{
"should export height 0 with --for-zero-height",
[]string{
fmt.Sprintf("--%s=%s", server.FlagForZeroHeight, "true"),
},
2, 0,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
tempDir, clean := testutil.NewTestCaseDir(t)
defer clean()
app, ctx, _, cmd := setupApp(t, tempDir)
// Fast forward to block `tc.fastForward`.
for i := int64(2); i <= tc.fastForward; i++ {
app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: i}})
app.Commit()
}
output := &bytes.Buffer{}
cmd.SetOut(output)
args := append(tc.flags, fmt.Sprintf("--%s=%s", flags.FlagHome, tempDir))
cmd.SetArgs(args)
require.NoError(t, cmd.ExecuteContext(ctx))
var exportedGenDoc tmtypes.GenesisDoc
err := tmjson.Unmarshal(output.Bytes(), &exportedGenDoc)
if err != nil {
t.Fatalf("error unmarshaling exported genesis doc: %s", err)
}
require.Equal(t, tc.expHeight, exportedGenDoc.InitialHeight)
})
}
}
func setupApp(t *testing.T, tempDir string) (*simapp.SimApp, context.Context, *tmtypes.GenesisDoc, *cobra.Command) {
err := createConfigFolder(tempDir)
if err != nil {
t.Fatalf("error creating config folder: %s", err)
}
logger := log.NewTMLogger(log.NewSyncWriter(os.Stdout))
db := dbm.NewMemDB()
encCfg := simapp.MakeEncodingConfig()
app := simapp.NewSimApp(logger, db, nil, true, map[int64]bool{}, tempDir, 0, encCfg)
serverCtx := server.NewDefaultContext()
serverCtx.Config.RootDir = tempDir
clientCtx := client.Context{}.WithJSONMarshaler(app.AppCodec())
genDoc := newDefaultGenesisDoc()
err = saveGenesisFile(genDoc, serverCtx.Config.GenesisFile())
require.NoError(t, err)
app.InitChain(
abci.RequestInitChain{
Validators: []abci.ValidatorUpdate{},
ConsensusParams: simapp.DefaultConsensusParams,
AppStateBytes: genDoc.AppState,
},
)
app.Commit()
cmd := server.ExportCmd(
func(_ log.Logger, _ dbm.DB, _ io.Writer, height int64, forZeroHeight bool, jailAllowedAddrs []string) (types.ExportedApp, error) {
encCfg := simapp.MakeEncodingConfig()
var simApp *simapp.SimApp
if height != -1 {
simApp = simapp.NewSimApp(logger, db, nil, false, map[int64]bool{}, "", 0, encCfg)
if err := simApp.LoadHeight(height); err != nil {
return types.ExportedApp{}, err
}
} else {
simApp = simapp.NewSimApp(logger, db, nil, true, map[int64]bool{}, "", 0, encCfg)
}
return simApp.ExportAppStateAndValidators(forZeroHeight, jailAllowedAddrs)
}, tempDir)
ctx := context.Background()
ctx = context.WithValue(ctx, client.ClientContextKey, &clientCtx)
ctx = context.WithValue(ctx, server.ServerContextKey, serverCtx)
return app, ctx, genDoc, cmd
}
func createConfigFolder(dir string) error {
return os.Mkdir(path.Join(dir, "config"), 0700)
}

View File

@ -99,6 +99,10 @@ func (ms multiStore) SetInterBlockCache(_ sdk.MultiStorePersistentCache) {
panic("not implemented")
}
func (ms multiStore) SetInitialVersion(version int64) error {
panic("not implemented")
}
var _ sdk.KVStore = kvStore{}
type kvStore struct {

View File

@ -42,7 +42,20 @@ type (
// application using various configurations.
AppCreator func(log.Logger, dbm.DB, io.Writer, AppOptions) Application
// ExportedApp represents an exported app state, along with
// validators, consensus params and latest app height.
ExportedApp struct {
// AppState is the application state as JSON.
AppState json.RawMessage
// Validators is the exported validator set.
Validators []tmtypes.GenesisValidator
// Height is the app's latest block height.
Height int64
// ConsensusParams are the exported consensus params for ABCI.
ConsensusParams *abci.ConsensusParams
}
// AppExporter is a function that dumps all app state to
// JSON-serializable structure and returns the current validator set.
AppExporter func(log.Logger, dbm.DB, io.Writer, int64, bool, []string) (json.RawMessage, []tmtypes.GenesisValidator, *abci.ConsensusParams, error)
AppExporter func(log.Logger, dbm.DB, io.Writer, int64, bool, []string) (ExportedApp, error)
)

View File

@ -383,15 +383,17 @@ func NewSimApp(
if err := app.LoadLatestVersion(); err != nil {
tmos.Exit(err.Error())
}
}
// Initialize and seal the capability keeper so all persistent capabilities
// are loaded in-memory and prevent any further modules from creating scoped
// sub-keepers.
// This must be done during creation of baseapp rather than in InitChain so
// that in-memory capabilities get regenerated on app restart
ctx := app.BaseApp.NewUncachedContext(true, tmproto.Header{})
app.CapabilityKeeper.InitializeAndSeal(ctx)
// Initialize and seal the capability keeper so all persistent capabilities
// are loaded in-memory and prevent any further modules from creating scoped
// sub-keepers.
// This must be done during creation of baseapp rather than in InitChain so
// that in-memory capabilities get regenerated on app restart.
// Note that since this reads from the store, we can only perform it when
// `loadLatest` is set to true.
ctx := app.BaseApp.NewUncachedContext(true, tmproto.Header{})
app.CapabilityKeeper.InitializeAndSeal(ctx)
}
app.ScopedIBCKeeper = scopedIBCKeeper
app.ScopedTransferKeeper = scopedTransferKeeper

View File

@ -31,7 +31,7 @@ func TestSimAppExport(t *testing.T) {
// Making a new app object with the db, so that initchain hasn't been called
app2 := NewSimApp(log.NewTMLogger(log.NewSyncWriter(os.Stdout)), db, nil, true, map[int64]bool{}, DefaultNodeHome, 0, MakeEncodingConfig())
_, _, _, err = app2.ExportAppStateAndValidators(false, []string{})
_, err = app2.ExportAppStateAndValidators(false, []string{})
require.NoError(t, err, "ExportAppStateAndValidators should not have an error")
}

View File

@ -4,10 +4,9 @@ import (
"encoding/json"
"log"
abci "github.com/tendermint/tendermint/abci/types"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
tmtypes "github.com/tendermint/tendermint/types"
servertypes "github.com/cosmos/cosmos-sdk/server/types"
sdk "github.com/cosmos/cosmos-sdk/types"
slashingtypes "github.com/cosmos/cosmos-sdk/x/slashing/types"
"github.com/cosmos/cosmos-sdk/x/staking"
@ -19,23 +18,31 @@ import (
// file.
func (app *SimApp) ExportAppStateAndValidators(
forZeroHeight bool, jailAllowedAddrs []string,
) (appState json.RawMessage, validators []tmtypes.GenesisValidator, cp *abci.ConsensusParams, err error) {
) (servertypes.ExportedApp, error) {
// as if they could withdraw from the start of the next block
ctx := app.NewContext(true, tmproto.Header{Height: app.LastBlockHeight()})
// We export at last height + 1, because that's the height at which
// Tendermint will start InitChain.
height := app.LastBlockHeight() + 1
if forZeroHeight {
height = 0
app.prepForZeroHeightGenesis(ctx, jailAllowedAddrs)
}
genState := app.mm.ExportGenesis(ctx, app.appCodec)
appState, err = json.MarshalIndent(genState, "", " ")
appState, err := json.MarshalIndent(genState, "", " ")
if err != nil {
return nil, nil, nil, err
return servertypes.ExportedApp{}, err
}
validators = staking.WriteValidators(ctx, app.StakingKeeper)
return appState, validators, app.BaseApp.GetConsensusParams(ctx), nil
validators := staking.WriteValidators(ctx, app.StakingKeeper)
return servertypes.ExportedApp{
AppState: appState,
Validators: validators,
Height: height,
ConsensusParams: app.BaseApp.GetConsensusParams(ctx),
}, nil
}
// prepare for fresh start at zero height

View File

@ -131,7 +131,7 @@ func TestAppImportExport(t *testing.T) {
fmt.Printf("exporting genesis...\n")
appState, _, consensusParams, err := app.ExportAppStateAndValidators(false, []string{})
exported, err := app.ExportAppStateAndValidators(false, []string{})
require.NoError(t, err)
fmt.Printf("importing genesis...\n")
@ -148,13 +148,13 @@ func TestAppImportExport(t *testing.T) {
require.Equal(t, "SimApp", newApp.Name())
var genesisState GenesisState
err = json.Unmarshal(appState, &genesisState)
err = json.Unmarshal(exported.AppState, &genesisState)
require.NoError(t, err)
ctxA := app.NewContext(true, tmproto.Header{Height: app.LastBlockHeight()})
ctxB := newApp.NewContext(true, tmproto.Header{Height: app.LastBlockHeight()})
newApp.mm.InitGenesis(ctxB, app.AppCodec(), genesisState)
newApp.StoreConsensusParams(ctxB, consensusParams)
newApp.StoreConsensusParams(ctxB, exported.ConsensusParams)
fmt.Printf("comparing stores...\n")
@ -232,7 +232,7 @@ func TestAppSimulationAfterImport(t *testing.T) {
fmt.Printf("exporting genesis...\n")
appState, _, _, err := app.ExportAppStateAndValidators(true, []string{})
exported, err := app.ExportAppStateAndValidators(true, []string{})
require.NoError(t, err)
fmt.Printf("importing genesis...\n")
@ -249,7 +249,7 @@ func TestAppSimulationAfterImport(t *testing.T) {
require.Equal(t, "SimApp", newApp.Name())
newApp.InitChain(abci.RequestInitChain{
AppStateBytes: appState,
AppStateBytes: exported.AppState,
})
_, _, err = simulation.SimulateFromSeed(

View File

@ -2,7 +2,6 @@ package cmd
import (
"context"
"encoding/json"
"io"
"os"
@ -11,10 +10,8 @@ import (
"github.com/spf13/cast"
"github.com/spf13/cobra"
abci "github.com/tendermint/tendermint/abci/types"
tmcli "github.com/tendermint/tendermint/libs/cli"
"github.com/tendermint/tendermint/libs/log"
tmtypes "github.com/tendermint/tendermint/types"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/baseapp"
@ -97,7 +94,7 @@ func initRootCmd(rootCmd *cobra.Command, encodingConfig params.EncodingConfig) {
debug.Cmd(),
)
server.AddCommands(rootCmd, simapp.DefaultNodeHome, newApp, exportAppStateAndTMValidators)
server.AddCommands(rootCmd, simapp.DefaultNodeHome, newApp, createSimappAndExport)
// add keybase, auxiliary RPC, query, and tx child commands
rootCmd.AddCommand(
@ -191,10 +188,11 @@ func newApp(logger log.Logger, db dbm.DB, traceStore io.Writer, appOpts serverty
)
}
func exportAppStateAndTMValidators(
// createSimappAndExport creates a new simapp (optionally at a given height)
// and exports state.
func createSimappAndExport(
logger log.Logger, db dbm.DB, traceStore io.Writer, height int64, forZeroHeight bool, jailAllowedAddrs []string,
) (json.RawMessage, []tmtypes.GenesisValidator, *abci.ConsensusParams, error) {
) (servertypes.ExportedApp, error) {
encCfg := simapp.MakeEncodingConfig() // Ideally, we would reuse the one created by NewRootCmd.
encCfg.Marshaler = codec.NewProtoCodec(encCfg.InterfaceRegistry)
var simApp *simapp.SimApp
@ -202,7 +200,7 @@ func exportAppStateAndTMValidators(
simApp = simapp.NewSimApp(logger, db, traceStore, false, map[int64]bool{}, "", uint(1), encCfg)
if err := simApp.LoadHeight(height); err != nil {
return nil, nil, nil, err
return servertypes.ExportedApp{}, err
}
} else {
simApp = simapp.NewSimApp(logger, db, traceStore, true, map[int64]bool{}, "", uint(1), encCfg)

View File

@ -1,12 +1,10 @@
package simapp
import (
"encoding/json"
abci "github.com/tendermint/tendermint/abci/types"
tmtypes "github.com/tendermint/tendermint/types"
"github.com/cosmos/cosmos-sdk/codec"
"github.com/cosmos/cosmos-sdk/server/types"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/cosmos/cosmos-sdk/types/module"
)
@ -36,7 +34,7 @@ type App interface {
// Exports the state of the application for a genesis file.
ExportAppStateAndValidators(
forZeroHeight bool, jailAllowedAddrs []string,
) (json.RawMessage, []tmtypes.GenesisValidator, *abci.ConsensusParams, error)
) (types.ExportedApp, error)
// All the registered module account addreses.
ModuleAccountAddrs() map[string]bool

View File

@ -79,12 +79,12 @@ func CheckExportSimulation(
) error {
if config.ExportStatePath != "" {
fmt.Println("exporting app state...")
appState, _, _, err := app.ExportAppStateAndValidators(false, nil)
exported, err := app.ExportAppStateAndValidators(false, nil)
if err != nil {
return err
}
if err := ioutil.WriteFile(config.ExportStatePath, []byte(appState), 0600); err != nil {
if err := ioutil.WriteFile(config.ExportStatePath, []byte(exported.AppState), 0600); err != nil {
return err
}
}

View File

@ -26,10 +26,11 @@ const (
)
var (
_ types.KVStore = (*Store)(nil)
_ types.CommitStore = (*Store)(nil)
_ types.CommitKVStore = (*Store)(nil)
_ types.Queryable = (*Store)(nil)
_ types.KVStore = (*Store)(nil)
_ types.CommitStore = (*Store)(nil)
_ types.CommitKVStore = (*Store)(nil)
_ types.Queryable = (*Store)(nil)
_ types.StoreWithInitialVersion = (*Store)(nil)
)
// Store Implements types.KVStore and CommitKVStore.
@ -109,7 +110,7 @@ func (st *Store) Commit() types.CommitID {
}
}
// Implements Committer.
// LastCommitID implements Committer.
func (st *Store) LastCommitID() types.CommitID {
return types.CommitID{
Version: st.tree.Version(),
@ -205,6 +206,12 @@ func (st *Store) ReverseIterator(start, end []byte) types.Iterator {
return newIAVLIterator(iTree, start, end, false)
}
// SetInitialVersion sets the initial version of the IAVL tree. It is used when
// starting a new chain at an arbitrary height.
func (st *Store) SetInitialVersion(version int64) {
st.tree.SetInitialVersion(uint64(version))
}
// Handle gatest the latest height, if height is 0
func getHeight(tree Tree, req abci.RequestQuery) int64 {
height := req.Height

View File

@ -526,3 +526,54 @@ func BenchmarkIAVLIteratorNext(b *testing.B) {
}
}
}
func TestSetInitialVersion(t *testing.T) {
testCases := []struct {
name string
storeFn func(db *dbm.MemDB) *Store
expPanic bool
}{
{
"works with a mutable tree",
func(db *dbm.MemDB) *Store {
tree, err := iavl.NewMutableTree(db, cacheSize)
require.NoError(t, err)
store := UnsafeNewStore(tree)
return store
}, false,
},
{
"throws error on immutable tree",
func(db *dbm.MemDB) *Store {
tree, err := iavl.NewMutableTree(db, cacheSize)
require.NoError(t, err)
store := UnsafeNewStore(tree)
_, version, err := store.tree.SaveVersion()
require.NoError(t, err)
require.Equal(t, int64(1), version)
store, err = store.GetImmutable(1)
require.NoError(t, err)
return store
}, true,
},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.name, func(t *testing.T) {
db := dbm.NewMemDB()
store := tc.storeFn(db)
if tc.expPanic {
require.Panics(t, func() { store.SetInitialVersion(5) })
} else {
store.SetInitialVersion(5)
cid := store.Commit()
require.Equal(t, int64(5), cid.GetVersion())
}
})
}
}

View File

@ -30,6 +30,7 @@ type (
GetVersioned(key []byte, version int64) (int64, []byte)
GetVersionedWithProof(key []byte, version int64) ([]byte, *iavl.RangeProof, error)
GetImmutable(version int64) (*iavl.ImmutableTree, error)
SetInitialVersion(version uint64)
}
// immutableTree is a simple wrapper around a reference to an iavl.ImmutableTree
@ -60,6 +61,10 @@ func (it *immutableTree) DeleteVersions(_ ...int64) error {
panic("cannot call 'DeleteVersions' on an immutable IAVL tree")
}
func (it *immutableTree) SetInitialVersion(_ uint64) {
panic("cannot call 'SetInitialVersion' on an immutable IAVL tree")
}
func (it *immutableTree) VersionExists(version int64) bool {
return it.Version() == version
}

View File

@ -40,6 +40,7 @@ type Store struct {
keysByName map[string]types.StoreKey
lazyLoading bool
pruneHeights []int64
initialVersion int64
traceWriter io.Writer
traceContext types.TraceContext
@ -298,8 +299,22 @@ func (rs *Store) LastCommitID() types.CommitID {
// Commit implements Committer/CommitStore.
func (rs *Store) Commit() types.CommitID {
previousHeight := rs.lastCommitInfo.Version
version := previousHeight + 1
var previousHeight, version int64
if rs.lastCommitInfo.GetVersion() == 0 && rs.initialVersion > 1 {
// This case means that no commit has been made in the store, we
// start from initialVersion.
version = rs.initialVersion
} else {
// This case can means two things:
// - either there was already a previous commit in the store, in which
// case we increment the version from there,
// - or there was no previous commit, and initial version was not set,
// in which case we start at version 1.
previousHeight = rs.lastCommitInfo.GetVersion()
version = previousHeight + 1
}
rs.lastCommitInfo = commitStores(version, rs.stores)
// Determine if pruneHeight height needs to be added to the list of heights to
@ -501,6 +516,22 @@ func (rs *Store) Query(req abci.RequestQuery) abci.ResponseQuery {
return res
}
// SetInitialVersion sets the initial version of the IAVL tree. It is used when
// starting a new chain at an arbitrary height.
func (rs *Store) SetInitialVersion(version int64) error {
rs.initialVersion = version
// Loop through all the stores, if it's an IAVL store, then set initial
// version on it.
for _, commitKVStore := range rs.stores {
if storeWithVersion, ok := commitKVStore.(types.StoreWithInitialVersion); ok {
storeWithVersion.SetInitialVersion(version)
}
}
return nil
}
// parsePath expects a format like /<storeName>[/<subpath>]
// Must start with /, subpath may be empty
// Returns error if it doesn't start with /

View File

@ -505,6 +505,17 @@ func TestMultiStore_PruningRestart(t *testing.T) {
}
}
func TestSetInitialVersion(t *testing.T) {
db := dbm.NewMemDB()
multi := newMultiStoreWithMounts(db, types.PruneNothing)
multi.SetInitialVersion(5)
require.Equal(t, int64(5), multi.initialVersion)
multi.Commit()
require.Equal(t, int64(5), multi.LastCommitID().Version)
}
//-----------------------------------------------------------------------
// utils

View File

@ -165,6 +165,10 @@ type CommitMultiStore interface {
// Set an inter-block (persistent) cache that maintains a mapping from
// StoreKeys to CommitKVStores.
SetInterBlockCache(MultiStorePersistentCache)
// SetInitialVersion sets the initial version of the IAVL tree. It is used when
// starting a new chain at an arbitrary height.
SetInitialVersion(version int64) error
}
//---------subsp-------------------------------
@ -394,3 +398,11 @@ type MultiStorePersistentCache interface {
// Reset the entire set of internal caches.
Reset()
}
// StoreWithInitialVersion is a store that can have an arbitrary initial
// version.
type StoreWithInitialVersion interface {
// SetInitialVersion sets the initial version of the IAVL tree. It is used when
// starting a new chain at an arbitrary height.
SetInitialVersion(version int64)
}

View File

@ -301,8 +301,11 @@ func New(t *testing.T, cfg Config) *Network {
require.NoError(t, err)
memo := fmt.Sprintf("%s@%s:%s", nodeIDs[i], p2pURL.Hostname(), p2pURL.Port())
fee := sdk.NewCoins(sdk.NewCoin(fmt.Sprintf("%stoken", nodeDirName), sdk.NewInt(0)))
txBuilder := cfg.TxConfig.NewTxBuilder()
require.NoError(t, txBuilder.SetMsgs(createValMsg))
txBuilder.SetFeeAmount(fee) // Arbitrary fee
txBuilder.SetGasLimit(1000000) // Need at least 100386
txBuilder.SetMemo(memo)
txFactory := tx.Factory{}

View File

@ -232,7 +232,7 @@ func (svd SigVerificationDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simul
if err != nil {
return ctx, sdkerrors.Wrapf(
sdkerrors.ErrUnauthorized,
"signature verification failed; please verify account number (%d) and chain-id (%s)", acc.GetAccountNumber(), ctx.ChainID())
"signature verification failed; please verify account number (%d) and chain-id (%s)", accNum, chainID)
}
}
}