Skip to content

Commit

Permalink
feat: export genesis in simapp v2 (#21199)
Browse files Browse the repository at this point in the history
Co-authored-by: marbar3778 <marbar3778@yahoo.com>
  • Loading branch information
randygrok and tac0turtle committed Aug 16, 2024
1 parent e2ec889 commit aeeaca6
Show file tree
Hide file tree
Showing 15 changed files with 426 additions and 67 deletions.
5 changes: 5 additions & 0 deletions runtime/v2/app.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,11 @@ func (a *App[T]) LoadHeight(height uint64) error {
return a.db.LoadVersion(height)
}

// LoadLatestHeight loads the latest height.
func (a *App[T]) LoadLatestHeight() (uint64, error) {
return a.db.GetLatestVersion()
}

// Close is called in start cmd to gracefully cleanup resources.
func (a *App[T]) Close() error {
return nil
Expand Down
15 changes: 14 additions & 1 deletion runtime/v2/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ func (a *AppBuilder[T]) RegisterModules(modules map[string]appmodulev2.AppModule

// RegisterStores registers the provided store keys.
// This method should only be used for registering extra stores
// wiich is necessary for modules that not registered using the app config.
// which is necessary for modules that not registered using the app config.
// To be used in combination of RegisterModules.
func (a *AppBuilder[T]) RegisterStores(keys ...string) {
a.app.storeKeys = append(a.app.storeKeys, keys...)
Expand Down Expand Up @@ -175,6 +175,19 @@ func (a *AppBuilder[T]) Build(opts ...AppBuilderOption[T]) (*App[T], error) {
}
return nil
},
ExportGenesis: func(ctx context.Context, version uint64) ([]byte, error) {
genesisJson, err := a.app.moduleManager.ExportGenesisForModules(ctx)
if err != nil {
return nil, fmt.Errorf("failed to export genesis: %w", err)
}

bz, err := json.Marshal(genesisJson)
if err != nil {
return nil, fmt.Errorf("failed to marshal genesis: %w", err)
}

return bz, nil
},
}

appManager, err := appManagerBuilder.Build()
Expand Down
32 changes: 9 additions & 23 deletions runtime/v2/manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,16 +203,13 @@ func (m *MM[T]) ExportGenesisForModules(
return nil, err
}

type genesisResult struct {
bz json.RawMessage
err error
}

type ModuleI interface {
ExportGenesis(ctx context.Context) (json.RawMessage, error)
}

channels := make(map[string]chan genesisResult)
genesisData := make(map[string]json.RawMessage)

// TODO: make async export genesis https://github.com/cosmos/cosmos-sdk/issues/21303
for _, moduleName := range modulesToExport {
mod := m.modules[moduleName]
var moduleI ModuleI
Expand All @@ -221,27 +218,16 @@ func (m *MM[T]) ExportGenesisForModules(
moduleI = module.(ModuleI)
} else if module, hasABCIGenesis := mod.(appmodulev2.HasGenesis); hasABCIGenesis {
moduleI = module.(ModuleI)
} else {
continue
}

channels[moduleName] = make(chan genesisResult)
go func(moduleI ModuleI, ch chan genesisResult) {
jm, err := moduleI.ExportGenesis(ctx)
if err != nil {
ch <- genesisResult{nil, err}
return
}
ch <- genesisResult{jm, nil}
}(moduleI, channels[moduleName])
}

genesisData := make(map[string]json.RawMessage)
for moduleName := range channels {
res := <-channels[moduleName]
if res.err != nil {
return nil, fmt.Errorf("genesis export error in %s: %w", moduleName, res.err)
res, err := moduleI.ExportGenesis(ctx)
if err != nil {
return nil, err
}

genesisData[moduleName] = res.bz
genesisData[moduleName] = res
}

return genesisData, nil
Expand Down
19 changes: 18 additions & 1 deletion server/v2/appmanager/appmanager.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,24 @@ func (a AppManager[T]) InitGenesis(

// ExportGenesis exports the genesis state of the application.
func (a AppManager[T]) ExportGenesis(ctx context.Context, version uint64) ([]byte, error) {
bz, err := a.exportGenesis(ctx, version)
zeroState, err := a.db.StateAt(version)
if err != nil {
return nil, fmt.Errorf("unable to get latest state: %w", err)
}

bz := make([]byte, 0)
_, err = a.stf.RunWithCtx(ctx, zeroState, func(ctx context.Context) error {
if a.exportGenesis == nil {
return errors.New("export genesis function not set")
}

bz, err = a.exportGenesis(ctx, version)
if err != nil {
return fmt.Errorf("failed to export genesis state: %w", err)
}

return nil
})
if err != nil {
return nil, fmt.Errorf("failed to export genesis state: %w", err)
}
Expand Down
2 changes: 1 addition & 1 deletion server/v2/cometbft/go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ require (
cosmossdk.io/errors v1.0.1
cosmossdk.io/log v1.4.0
cosmossdk.io/server/v2 v2.0.0-00010101000000-000000000000
cosmossdk.io/server/v2/appmanager v0.0.0-00010101000000-000000000000
cosmossdk.io/server/v2/appmanager v0.0.0-20240802110823-cffeedff643d
cosmossdk.io/store/v2 v2.0.0-00010101000000-000000000000
cosmossdk.io/x/consensus v0.0.0-00010101000000-000000000000
github.com/cometbft/cometbft v1.0.0-rc1
Expand Down
7 changes: 3 additions & 4 deletions simapp/app.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@ import (
circuittypes "cosmossdk.io/x/circuit/types"
"cosmossdk.io/x/consensus"
consensusparamkeeper "cosmossdk.io/x/consensus/keeper"
consensusparamtypes "cosmossdk.io/x/consensus/types"
consensustypes "cosmossdk.io/x/consensus/types"
distr "cosmossdk.io/x/distribution"
distrkeeper "cosmossdk.io/x/distribution/keeper"
Expand Down Expand Up @@ -265,7 +264,7 @@ func NewSimApp(
keys := storetypes.NewKVStoreKeys(
authtypes.StoreKey, banktypes.StoreKey, stakingtypes.StoreKey,
minttypes.StoreKey, distrtypes.StoreKey, slashingtypes.StoreKey,
govtypes.StoreKey, consensusparamtypes.StoreKey, upgradetypes.StoreKey, feegrant.StoreKey,
govtypes.StoreKey, consensustypes.StoreKey, upgradetypes.StoreKey, feegrant.StoreKey,
evidencetypes.StoreKey, circuittypes.StoreKey,
authzkeeper.StoreKey, nftkeeper.StoreKey, group.StoreKey, pooltypes.StoreKey,
accounts.StoreKey, epochstypes.StoreKey,
Expand All @@ -288,7 +287,7 @@ func NewSimApp(
cometService := runtime.NewContextAwareCometInfoService()

// set the BaseApp's parameter store
app.ConsensusParamsKeeper = consensusparamkeeper.NewKeeper(appCodec, runtime.NewEnvironment(runtime.NewKVStoreService(keys[consensusparamtypes.StoreKey]), logger.With(log.ModuleKey, "x/consensus")), authtypes.NewModuleAddress(govtypes.ModuleName).String())
app.ConsensusParamsKeeper = consensusparamkeeper.NewKeeper(appCodec, runtime.NewEnvironment(runtime.NewKVStoreService(keys[consensustypes.StoreKey]), logger.With(log.ModuleKey, "x/consensus")), authtypes.NewModuleAddress(govtypes.ModuleName).String())
bApp.SetParamStore(app.ConsensusParamsKeeper.ParamsStore)

// add keepers
Expand Down Expand Up @@ -519,7 +518,7 @@ func NewSimApp(
group.ModuleName,
upgradetypes.ModuleName,
vestingtypes.ModuleName,
consensusparamtypes.ModuleName,
consensustypes.ModuleName,
circuittypes.ModuleName,
pooltypes.ModuleName,
epochstypes.ModuleName,
Expand Down
2 changes: 0 additions & 2 deletions simapp/v2/app_di.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ import (
"cosmossdk.io/depinject"
"cosmossdk.io/log"
"cosmossdk.io/runtime/v2"
serverv2 "cosmossdk.io/server/v2"
"cosmossdk.io/x/accounts"
authkeeper "cosmossdk.io/x/auth/keeper"
authzkeeper "cosmossdk.io/x/authz/keeper"
Expand Down Expand Up @@ -92,7 +91,6 @@ func NewSimApp[T transaction.Tx](
logger log.Logger,
viper *viper.Viper,
) *SimApp[T] {
viper.Set(serverv2.FlagHome, DefaultNodeHome) // TODO possibly set earlier when viper is created
var (
app = &SimApp[T]{}
appBuilder *runtime.AppBuilder[T]
Expand Down
155 changes: 155 additions & 0 deletions simapp/v2/app_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,155 @@
package simapp

import (
"context"
"crypto/sha256"
"encoding/json"
"testing"
"time"

"github.com/cometbft/cometbft/types"
"github.com/spf13/viper"
"github.com/stretchr/testify/require"

app2 "cosmossdk.io/core/app"
"cosmossdk.io/core/comet"
context2 "cosmossdk.io/core/context"
"cosmossdk.io/core/store"
"cosmossdk.io/core/transaction"
"cosmossdk.io/log"
sdkmath "cosmossdk.io/math"
serverv2 "cosmossdk.io/server/v2"
comettypes "cosmossdk.io/server/v2/cometbft/types"
"cosmossdk.io/store/v2/db"
authtypes "cosmossdk.io/x/auth/types"
banktypes "cosmossdk.io/x/bank/types"

"github.com/cosmos/cosmos-sdk/crypto/keys/secp256k1"
"github.com/cosmos/cosmos-sdk/testutil/mock"
simtestutil "github.com/cosmos/cosmos-sdk/testutil/sims"
sdk "github.com/cosmos/cosmos-sdk/types"
)

func NewTestApp(t *testing.T) (*SimApp[transaction.Tx], context.Context) {
t.Helper()

logger := log.NewTestLogger(t)

vp := viper.New()
vp.Set("store.app-db-backend", string(db.DBTypeGoLevelDB))
vp.Set(serverv2.FlagHome, t.TempDir())

app := NewSimApp[transaction.Tx](logger, vp)
genesis := app.ModuleManager().DefaultGenesis()

privVal := mock.NewPV()
pubKey, err := privVal.GetPubKey()
require.NoError(t, err)

// create validator set with single validator
validator := types.NewValidator(pubKey, 1)
valSet := types.NewValidatorSet([]*types.Validator{validator})

// generate genesis account
senderPrivKey := secp256k1.GenPrivKey()
acc := authtypes.NewBaseAccount(senderPrivKey.PubKey().Address().Bytes(), senderPrivKey.PubKey(), 0, 0)
balance := banktypes.Balance{
Address: acc.GetAddress().String(),
Coins: sdk.NewCoins(sdk.NewCoin(sdk.DefaultBondDenom, sdkmath.NewInt(100000000000000))),
}

genesis, err = simtestutil.GenesisStateWithValSet(
app.AppCodec(),
genesis,
valSet,
[]authtypes.GenesisAccount{acc},
balance,
)
require.NoError(t, err)

genesisBytes, err := json.Marshal(genesis)
require.NoError(t, err)

st := app.GetStore().(comettypes.Store)
ci, err := st.LastCommitID()
require.NoError(t, err)

bz := sha256.Sum256([]byte{})

ctx := context.Background()

_, newState, err := app.InitGenesis(
ctx,
&app2.BlockRequest[transaction.Tx]{
Time: time.Now(),
Hash: bz[:],
ChainId: "theChain",
AppHash: ci.Hash,
IsGenesis: true,
},
genesisBytes,
nil,
)
require.NoError(t, err)

changes, err := newState.GetStateChanges()
require.NoError(t, err)

_, err = st.Commit(&store.Changeset{Changes: changes})
require.NoError(t, err)

return app, ctx
}

func MoveNextBlock(t *testing.T, app *SimApp[transaction.Tx], ctx context.Context) {
t.Helper()

bz := sha256.Sum256([]byte{})

st := app.GetStore().(comettypes.Store)
ci, err := st.LastCommitID()
require.NoError(t, err)

height, err := app.LoadLatestHeight()
require.NoError(t, err)

// TODO: this is a hack to set the comet info in the context for distribution module dependency.
ctx = context.WithValue(ctx, context2.CometInfoKey, comet.Info{
Evidence: nil,
ValidatorsHash: nil,
ProposerAddress: nil,
LastCommit: comet.CommitInfo{},
})

_, newState, err := app.DeliverBlock(
ctx,
&app2.BlockRequest[transaction.Tx]{
Height: height + 1,
Time: time.Now(),
Hash: bz[:],
AppHash: ci.Hash,
})
require.NoError(t, err)

changes, err := newState.GetStateChanges()
require.NoError(t, err)

_, err = st.Commit(&store.Changeset{Changes: changes})
require.NoError(t, err)
}

func TestSimAppExportAndBlockedAddrs_WithOneBlockProduced(t *testing.T) {
app, ctx := NewTestApp(t)

MoveNextBlock(t, app, ctx)

_, err := app.ExportAppStateAndValidators(nil)
require.NoError(t, err)
}

func TestSimAppExportAndBlockedAddrs_NoBlocksProduced(t *testing.T) {
app, _ := NewTestApp(t)

_, err := app.ExportAppStateAndValidators(nil)
require.NoError(t, err)
}
27 changes: 23 additions & 4 deletions simapp/v2/export.go
Original file line number Diff line number Diff line change
@@ -1,10 +1,29 @@
package simapp

import (
servertypes "github.com/cosmos/cosmos-sdk/server/types"
"context"

v2 "github.com/cosmos/cosmos-sdk/x/genutil/v2"
)

// ExportAppStateAndValidators exports the state of the application for a genesis file.
func (app *SimApp[T]) ExportAppStateAndValidators(forZeroHeight bool, jailAllowedAddrs, modulesToExport []string) (servertypes.ExportedApp, error) {
panic("not implemented")
// ExportAppStateAndValidators exports the state of the application for a genesis
// file.
func (app *SimApp[T]) ExportAppStateAndValidators(jailAllowedAddrs []string) (v2.ExportedApp, error) {
// as if they could withdraw from the start of the next block
ctx := context.Background()

latestHeight, err := app.LoadLatestHeight()
if err != nil {
return v2.ExportedApp{}, err
}

genesis, err := app.ExportGenesis(ctx, latestHeight)
if err != nil {
return v2.ExportedApp{}, err
}

return v2.ExportedApp{
AppState: genesis,
Height: int64(latestHeight),
}, nil
}
Loading

0 comments on commit aeeaca6

Please sign in to comment.