fix(sims): TestAppSimulationAfterImport and legacy proposal handling (#21800)

This commit is contained in:
Alexander Peters 2024-09-18 17:03:03 +02:00 committed by GitHub
parent 356df96770
commit cc8366c3ee
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 201 additions and 60 deletions

View File

@ -52,9 +52,6 @@ test-sim-multi-seed-short:
@cd ${CURRENT_DIR}/simapp && go test -failfast -mod=readonly -timeout 30m -tags='sims' -run TestFullAppSimulation \
-NumBlocks=50 -Period=10 -FauxMerkle=true
test-v2-sim-wip:
@echo "Running v2 simapp. This may take awhile!"
@cd ${CURRENT_DIR}/simapp/v2 && go test -failfast -mod=readonly -timeout 30m -tags='sims' -run TestSimsAppV2
test-sim-benchmark-invariants:
@echo "Running simulation invariant benchmarks..."

View File

@ -74,7 +74,7 @@ var (
)
func TestAppImportExport(t *testing.T) {
simsx.Run(t, NewSimApp, setupStateFactory, func(t testing.TB, ti simsx.TestInstance[*SimApp]) {
simsx.Run(t, NewSimApp, setupStateFactory, func(t testing.TB, ti simsx.TestInstance[*SimApp], _ []simtypes.Account) {
app := ti.App
t.Log("exporting genesis...\n")
exported, err := app.ExportAppStateAndValidators(false, exportWithValidatorSet, exportAllModules)
@ -116,35 +116,39 @@ func TestAppImportExport(t *testing.T) {
// set up a new node instance, Init chain from exported genesis
// run new instance for n blocks
func TestAppSimulationAfterImport(t *testing.T) {
simsx.Run(t, NewSimApp, setupStateFactory, func(t testing.TB, ti simsx.TestInstance[*SimApp]) {
simsx.Run(t, NewSimApp, setupStateFactory, func(t testing.TB, ti simsx.TestInstance[*SimApp], accs []simtypes.Account) {
app := ti.App
t.Log("exporting genesis...\n")
exported, err := app.ExportAppStateAndValidators(false, exportWithValidatorSet, exportAllModules)
require.NoError(t, err)
t.Log("importing genesis...\n")
importGenesisStateFactory := func(app *SimApp) simsx.SimStateFactory {
return simsx.SimStateFactory{
Codec: app.AppCodec(),
AppStateFn: func(r *rand.Rand, accs []simtypes.Account, config simtypes.Config) (json.RawMessage, []simtypes.Account, string, time.Time) {
AppStateFn: func(r *rand.Rand, _ []simtypes.Account, config simtypes.Config) (json.RawMessage, []simtypes.Account, string, time.Time) {
t.Log("importing genesis...\n")
genesisTimestamp := time.Unix(config.GenesisTime, 0)
_, err = app.InitChain(&abci.InitChainRequest{
AppStateBytes: exported.AppState,
ChainId: simsx.SimAppChainID,
InitialHeight: exported.Height,
Time: genesisTimestamp,
})
if IsEmptyValidatorSetErr(err) {
t.Skip("Skipping simulation as all validators have been unbonded")
return nil, nil, "", time.Time{}
}
acc, err := simtestutil.AccountsFromAppState(app.AppCodec(), exported.AppState)
require.NoError(t, err)
genesisTimestamp := time.Unix(config.GenesisTime, 0)
return exported.AppState, acc, config.ChainID, genesisTimestamp
// use accounts from initial run
return exported.AppState, accs, config.ChainID, genesisTimestamp
},
BlockedAddr: must(BlockedAddresses(app.AuthKeeper.AddressCodec())),
AccountSource: app.AuthKeeper,
BalanceSource: app.BankKeeper,
}
}
ti.Cfg.InitialBlockHeight = int(exported.Height)
simsx.RunWithSeed(t, ti.Cfg, NewSimApp, importGenesisStateFactory, ti.Cfg.Seed, ti.Cfg.FuzzSeed)
})
}
@ -191,7 +195,7 @@ func TestAppStateDeterminism(t *testing.T) {
var mx sync.Mutex
appHashResults := make(map[int64][][]byte)
appSimLogger := make(map[int64][]simulation.LogWriter)
captureAndCheckHash := func(t testing.TB, ti simsx.TestInstance[*SimApp]) {
captureAndCheckHash := func(t testing.TB, ti simsx.TestInstance[*SimApp], _ []simtypes.Account) {
seed, appHash := ti.Cfg.Seed, ti.App.LastCommitID().Hash
mx.Lock()
otherHashes, execWriters := appHashResults[seed], appSimLogger[seed]

View File

@ -269,6 +269,9 @@ func NewChainDataSource(
for i, a := range oldSimAcc {
acc[i] = SimAccount{Account: a, r: r, bank: bank}
index[a.AddressBech32] = i
if a.AddressBech32 == "" {
panic("test account has empty bech32 address")
}
}
return &ChainDataSource{
r: r,

View File

@ -6,13 +6,41 @@ import (
sdk "github.com/cosmos/cosmos-sdk/types"
)
// SimMsgFactoryX is an interface for creating and handling fuzz test-like simulation messages in the system.
type SimMsgFactoryX interface {
// MsgType returns an empty instance of the concrete message type that the factory provides.
// This instance is primarily used for deduplication and reporting purposes.
// The result must not be nil
MsgType() sdk.Msg
// Create returns a FactoryMethod implementation which is responsible for constructing new instances of the message
// on each invocation.
Create() FactoryMethod
// DeliveryResultHandler returns a SimDeliveryResultHandler instance which processes the delivery
// response error object. While most simulation factories anticipate successful message delivery,
// certain factories employ this handler to validate execution errors, thereby covering negative
// test scenarios.
DeliveryResultHandler() SimDeliveryResultHandler
}
type (
// FactoryMethod method signature that is implemented by the concrete message factories
// FactoryMethod is a method signature implemented by concrete message factories for SimMsgFactoryX
//
// This factory method is responsible for creating a new `sdk.Msg` instance and determining the
// proposed signers who are expected to successfully sign the message for delivery.
//
// Parameters:
// - ctx: The context for the operation
// - testData: A pointer to a `ChainDataSource` which provides helper methods and simple access to accounts
// and balances within the chain.
// - reporter: An instance of `SimulationReporter` used to report the results of the simulation.
// If no valid message can be provided, the factory method should call `reporter.Skip("some comment")`
// with both `signer` and `msg` set to nil.
//
// Returns:
// - signer: A slice of `SimAccount` representing the proposed signers.
// - msg: An instance of `sdk.Msg` representing the message to be delivered.
FactoryMethod func(ctx context.Context, testData *ChainDataSource, reporter SimulationReporter) (signer []SimAccount, msg sdk.Msg)
// FactoryMethodWithFutureOps extended message factory method for the gov module or others that have to schedule operations for a future block.

View File

@ -1,6 +1,7 @@
package simsx
import (
"cmp"
"context"
"iter"
"maps"
@ -36,7 +37,7 @@ type (
)
// WeightedProposalMsgIter iterator for weighted gov proposal payload messages
type WeightedProposalMsgIter = iter.Seq2[uint32, SimMsgFactoryX]
type WeightedProposalMsgIter = iter.Seq2[uint32, FactoryMethod]
var _ Registry = &WeightedOperationRegistryAdapter{}
@ -108,6 +109,7 @@ type HasFutureOpsRegistry interface {
SetFutureOpsRegistry(FutureOpsRegistry)
}
// msg factory to legacy Operation type
func legacyOperationAdapter(l regCommon, fx SimMsgFactoryX) simtypes.Operation {
return func(
r *rand.Rand, app AppEntrypoint, ctx sdk.Context,
@ -167,15 +169,15 @@ func (s UniqueTypeRegistry) Add(weight uint32, f SimMsgFactoryX) {
}
// Iterator returns an iterator function for a Go for loop sorted by weight desc.
func (s UniqueTypeRegistry) Iterator() iter.Seq2[uint32, SimMsgFactoryX] {
func (s UniqueTypeRegistry) Iterator() WeightedProposalMsgIter {
x := maps.Values(s)
sortedWeightedFactory := slices.SortedFunc(x, func(a, b WeightedFactory) int {
return a.Compare(b)
})
return func(yield func(uint32, SimMsgFactoryX) bool) {
return func(yield func(uint32, FactoryMethod) bool) {
for _, v := range sortedWeightedFactory {
if !yield(v.Weight, v.Factory) {
if !yield(v.Weight, v.Factory.Create()) {
return
}
}
@ -204,3 +206,63 @@ func (f WeightedFactory) Compare(b WeightedFactory) int {
return strings.Compare(sdk.MsgTypeURL(f.Factory.MsgType()), sdk.MsgTypeURL(b.Factory.MsgType()))
}
}
// WeightedFactoryMethod is a data tuple used for registering legacy proposal operations
type WeightedFactoryMethod struct {
Weight uint32
Factory FactoryMethod
}
type WeightedFactoryMethods []WeightedFactoryMethod
// NewWeightedFactoryMethods constructor
func NewWeightedFactoryMethods() WeightedFactoryMethods {
return make(WeightedFactoryMethods, 0)
}
// Add adds a new WeightedFactoryMethod to the WeightedFactoryMethods slice.
// If weight is zero or f is nil, it returns without making any changes.
func (s *WeightedFactoryMethods) Add(weight uint32, f FactoryMethod) {
if weight == 0 {
return
}
if f == nil {
panic("message factory must not be nil")
}
*s = append(*s, WeightedFactoryMethod{Weight: weight, Factory: f})
}
// Iterator returns an iterator function for a Go for loop sorted by weight desc.
func (s WeightedFactoryMethods) Iterator() WeightedProposalMsgIter {
slices.SortFunc(s, func(e, e2 WeightedFactoryMethod) int {
return cmp.Compare(e.Weight, e2.Weight)
})
return func(yield func(uint32, FactoryMethod) bool) {
for _, v := range s {
if !yield(v.Weight, v.Factory) {
return
}
}
}
}
// legacy operation to Msg factory type
func legacyToMsgFactoryAdapter(fn simtypes.MsgSimulatorFnX) FactoryMethod {
return func(ctx context.Context, testData *ChainDataSource, reporter SimulationReporter) (signer []SimAccount, msg sdk.Msg) {
msg, err := fn(ctx, testData.r, testData.AllAccounts(), testData.AddressCodec())
if err != nil {
reporter.Skip(err.Error())
return nil, nil
}
return []SimAccount{}, msg
}
}
// AppendIterators takes multiple WeightedProposalMsgIter and returns a single iterator that sequentially yields items after each one.
func AppendIterators(iterators ...WeightedProposalMsgIter) WeightedProposalMsgIter {
return func(yield func(uint32, FactoryMethod) bool) {
for _, it := range iterators {
it(yield)
}
}
}

View File

@ -116,20 +116,21 @@ func TestSimsMsgRegistryAdapter(t *testing.T) {
}
func TestUniqueTypeRegistry(t *testing.T) {
f1 := SimMsgFactoryFn[*testdata.TestMsg](func(ctx context.Context, testData *ChainDataSource, reporter SimulationReporter) (signer []SimAccount, msg *testdata.TestMsg) {
exampleFactory := SimMsgFactoryFn[*testdata.TestMsg](func(ctx context.Context, testData *ChainDataSource, reporter SimulationReporter) (signer []SimAccount, msg *testdata.TestMsg) {
return []SimAccount{}, nil
})
specs := map[string]struct {
src []WeightedFactory
exp []WeightedFactory
exp []WeightedFactoryMethod
expErr bool
}{
"unique": {
src: []WeightedFactory{{Weight: 1, Factory: f1}},
exp: []WeightedFactory{{Weight: 1, Factory: f1}},
src: []WeightedFactory{{Weight: 1, Factory: exampleFactory}},
exp: []WeightedFactoryMethod{{Weight: 1, Factory: exampleFactory.Create()}},
},
"duplicate": {
src: []WeightedFactory{{Weight: 1, Factory: f1}, {Weight: 2, Factory: f1}},
src: []WeightedFactory{{Weight: 1, Factory: exampleFactory}, {Weight: 2, Factory: exampleFactory}},
expErr: true,
},
}
@ -148,11 +149,56 @@ func TestUniqueTypeRegistry(t *testing.T) {
reg.Add(v.Weight, v.Factory)
}
// then
var got []WeightedFactory
for w, f := range reg.Iterator() {
got = append(got, WeightedFactory{Weight: w, Factory: f})
}
got := readAll(reg.Iterator())
require.Len(t, got, len(spec.exp))
})
}
}
func TestWeightedFactories(t *testing.T) {
r := NewWeightedFactoryMethods()
f1 := func(ctx context.Context, testData *ChainDataSource, reporter SimulationReporter) (signer []SimAccount, msg sdk.Msg) {
panic("not implemented")
}
f2 := func(ctx context.Context, testData *ChainDataSource, reporter SimulationReporter) (signer []SimAccount, msg sdk.Msg) {
panic("not implemented")
}
r.Add(1, f1)
r.Add(2, f2)
got := readAll(r.Iterator())
require.Len(t, got, 2)
assert.Equal(t, uint32(1), r[0].Weight)
assert.Equal(t, uint32(2), r[1].Weight)
}
func TestAppendIterators(t *testing.T) {
r1 := NewWeightedFactoryMethods()
r1.Add(2, func(ctx context.Context, testData *ChainDataSource, reporter SimulationReporter) (signer []SimAccount, msg sdk.Msg) {
panic("not implemented")
})
r1.Add(2, func(ctx context.Context, testData *ChainDataSource, reporter SimulationReporter) (signer []SimAccount, msg sdk.Msg) {
panic("not implemented")
})
r1.Add(3, func(ctx context.Context, testData *ChainDataSource, reporter SimulationReporter) (signer []SimAccount, msg sdk.Msg) {
panic("not implemented")
})
r2 := NewUniqueTypeRegistry()
r2.Add(1, SimMsgFactoryFn[*testdata.TestMsg](func(ctx context.Context, testData *ChainDataSource, reporter SimulationReporter) (signer []SimAccount, msg *testdata.TestMsg) {
panic("not implemented")
}))
// when
all := readAll(AppendIterators(r1.Iterator(), r2.Iterator()))
// then
require.Len(t, all, 4)
gotWeights := Collect(all, func(a WeightedFactoryMethod) uint32 { return a.Weight })
assert.Equal(t, []uint32{2, 2, 3, 1}, gotWeights)
}
func readAll(iterator WeightedProposalMsgIter) []WeightedFactoryMethod {
var ret []WeightedFactoryMethod
for w, f := range iterator {
ret = append(ret, WeightedFactoryMethod{Weight: w, Factory: f})
}
return ret
}

View File

@ -240,6 +240,9 @@ func (s *ExecutionSummary) String() string {
for _, key := range keys {
sb.WriteString(fmt.Sprintf("%s: %d\n", key, s.counts[key]))
}
if len(s.skipReasons) != 0 {
sb.WriteString("\nSkip reasons:\n")
}
for m, c := range s.skipReasons {
values := maps.Values(c)
keys := maps.Keys(c)

View File

@ -4,7 +4,6 @@ import (
"encoding/json"
"fmt"
"io"
"iter"
"os"
"path/filepath"
"testing"
@ -82,7 +81,7 @@ func Run[T SimulationApp](
baseAppOptions ...func(*baseapp.BaseApp),
) T,
setupStateFactory func(app T) SimStateFactory,
postRunActions ...func(t testing.TB, app TestInstance[T]),
postRunActions ...func(t testing.TB, app TestInstance[T], accs []simtypes.Account),
) {
t.Helper()
RunWithSeeds(t, appFactory, setupStateFactory, defaultSeeds, nil, postRunActions...)
@ -110,7 +109,7 @@ func RunWithSeeds[T SimulationApp](
setupStateFactory func(app T) SimStateFactory,
seeds []int64,
fuzzSeed []byte,
postRunActions ...func(t testing.TB, app TestInstance[T]),
postRunActions ...func(t testing.TB, app TestInstance[T], accs []simtypes.Account),
) {
t.Helper()
cfg := cli.NewConfigFromFlags()
@ -137,7 +136,7 @@ func RunWithSeed[T SimulationApp](
setupStateFactory func(app T) SimStateFactory,
seed int64,
fuzzSeed []byte,
postRunActions ...func(t testing.TB, app TestInstance[T]),
postRunActions ...func(t testing.TB, app TestInstance[T], accs []simtypes.Account),
) {
tb.Helper()
// setup environment
@ -154,7 +153,7 @@ func RunWithSeed[T SimulationApp](
app := testInstance.App
stateFactory := setupStateFactory(app)
ops, reporter := prepareWeightedOps(app.SimulationManager(), stateFactory, tCfg, testInstance.App.TxConfig(), runLogger)
simParams, err := simulation.SimulateFromSeedX(tb, runLogger, WriteToDebugLog(runLogger), app.GetBaseApp(), stateFactory.AppStateFn, simtypes.RandomAccounts, ops, stateFactory.BlockedAddr, tCfg, stateFactory.Codec, testInstance.ExecLogWriter)
simParams, accs, err := simulation.SimulateFromSeedX(tb, runLogger, WriteToDebugLog(runLogger), app.GetBaseApp(), stateFactory.AppStateFn, simtypes.RandomAccounts, ops, stateFactory.BlockedAddr, tCfg, stateFactory.Codec, testInstance.ExecLogWriter)
require.NoError(tb, err)
err = simtestutil.CheckExportSimulation(app, tCfg, simParams)
require.NoError(tb, err)
@ -164,7 +163,7 @@ func RunWithSeed[T SimulationApp](
// not using tb.Log to always print the summary
fmt.Printf("+++ DONE (seed: %d): \n%s\n", seed, reporter.Summary().String())
for _, step := range postRunActions {
step(tb, testInstance)
step(tb, testInstance, accs)
}
require.NoError(tb, app.Close())
}
@ -174,7 +173,7 @@ type (
WeightedOperationsX(weight WeightSource, reg Registry)
}
HasWeightedOperationsXWithProposals interface {
WeightedOperationsX(weights WeightSource, reg Registry, proposals iter.Seq2[uint32, SimMsgFactoryX],
WeightedOperationsX(weights WeightSource, reg Registry, proposals WeightedProposalMsgIter,
legacyProposals []simtypes.WeightedProposalContent) //nolint: staticcheck // used for legacy proposal types
}
HasProposalMsgsX interface {
@ -254,26 +253,22 @@ func prepareWeightedOps(
reporter := NewBasicSimulationReporter()
pReg := make(UniqueTypeRegistry)
wProps := make([]simtypes.WeightedProposalMsg, 0, len(sm.Modules))
wContent := make([]simtypes.WeightedProposalContent, 0) //nolint:staticcheck // required for legacy type
legacyPReg := NewWeightedFactoryMethods()
// add gov proposals types
for _, m := range sm.Modules {
switch xm := m.(type) {
case HasProposalMsgsX:
xm.ProposalMsgsX(weights, pReg)
case HasLegacyProposalMsgs:
wProps = append(wProps, xm.ProposalMsgs(simState)...)
for _, p := range xm.ProposalMsgs(simState) {
weight := weights.Get(p.AppParamsKey(), uint32(p.DefaultWeight()))
legacyPReg.Add(weight, legacyToMsgFactoryAdapter(p.MsgSimulatorFn()))
}
case HasLegacyProposalContents:
wContent = append(wContent, xm.ProposalContents(simState)...)
}
}
if len(wProps) != 0 {
panic("legacy proposals are not empty")
}
if len(wContent) != 0 {
panic("legacy proposal contents are not empty")
}
oReg := NewSimsMsgRegistryAdapter(reporter, stateFact.AccountSource, stateFact.BalanceSource, txConfig, logger)
wOps := make([]simtypes.WeightedOperation, 0, len(sm.Modules))
@ -283,7 +278,7 @@ func prepareWeightedOps(
case HasWeightedOperationsX:
xm.WeightedOperationsX(weights, oReg)
case HasWeightedOperationsXWithProposals:
xm.WeightedOperationsX(weights, oReg, pReg.Iterator(), nil)
xm.WeightedOperationsX(weights, oReg, AppendIterators(legacyPReg.Iterator(), pReg.Iterator()), wContent)
case HasLegacyWeightedOperations:
wOps = append(wOps, xm.WeightedOperations(simState)...)
}

View File

@ -115,9 +115,9 @@ func MsgSubmitLegacyProposalFactory(k *keeper.Keeper, contentSimFn simtypes.Cont
})
}
func MsgSubmitProposalFactory(k *keeper.Keeper, payloadFactory simsx.SimMsgFactoryX) simsx.SimMsgFactoryX {
func MsgSubmitProposalFactory(k *keeper.Keeper, payloadFactory simsx.FactoryMethod) simsx.SimMsgFactoryX {
return simsx.NewSimMsgFactoryWithFutureOps[*v1.MsgSubmitProposal](func(ctx context.Context, testData *simsx.ChainDataSource, reporter simsx.SimulationReporter, fOpsReg simsx.FutureOpsRegistry) ([]simsx.SimAccount, *v1.MsgSubmitProposal) {
_, proposalMsg := payloadFactory.Create()(ctx, testData, reporter)
_, proposalMsg := payloadFactory(ctx, testData, reporter)
return submitProposalWithVotesScheduled(ctx, k, testData, reporter, fOpsReg, proposalMsg)
})
}

View File

@ -92,9 +92,9 @@ func updateValidators(
if update.Power == 0 {
if _, ok := current[str]; !ok {
tb.Fatalf("tried to delete a nonexistent validator: %s", str)
tb.Logf("tried to delete a nonexistent validator: %s", str)
continue
}
event("end_block", "validator_updates", "kicked")
delete(current, str)
} else if _, ok := current[str]; ok {
@ -187,15 +187,17 @@ func RandomRequestFinalizeBlock(
params.evidenceFraction = 0.9
}
totalBlocksProcessed := len(pastTimes)
startHeight := blockHeight - int64(totalBlocksProcessed) + 1
for r.Float64() < params.EvidenceFraction() {
vals := voteInfos
height := blockHeight
misbehaviorTime := time
if r.Float64() < params.PastEvidenceFraction() && height > 1 {
height = int64(r.Intn(int(height)-1)) + 1 // CometBFT starts at height 1
// array indices offset by one
misbehaviorTime = pastTimes[height-1]
vals = pastVoteInfos[height-1]
if r.Float64() < params.PastEvidenceFraction() && totalBlocksProcessed > 1 {
n := int64(r.Intn(totalBlocksProcessed))
misbehaviorTime = pastTimes[n]
vals = pastVoteInfos[n]
height = startHeight + n
}
validator := vals[r.Intn(len(vals))].Validator

View File

@ -48,6 +48,7 @@ func initChain(
ChainId: chainID,
ConsensusParams: consensusParams,
Time: genesisTimestamp,
InitialHeight: int64(config.InitialBlockHeight),
}
res, err := app.InitChain(&req)
if err != nil {
@ -60,7 +61,7 @@ func initChain(
// SimulateFromSeed tests an application by running the provided
// operations, testing the provided invariants, but using the provided config.Seed.
func SimulateFromSeed( // exists for backwards compatibility only
func SimulateFromSeed(
tb testing.TB,
logger corelog.Logger,
w io.Writer,
@ -72,7 +73,7 @@ func SimulateFromSeed( // exists for backwards compatibility only
config simtypes.Config,
cdc codec.JSONCodec,
addressCodec address.Codec,
) (exportedParams Params, err error) {
) (exportedParams Params, accs []simtypes.Account, err error) {
tb.Helper()
mode, _, _ := getTestingMode(tb)
return SimulateFromSeedX(tb, logger, w, app, appStateFn, randAccFn, ops, blockedAddrs, config, cdc, NewLogWriter(mode))
@ -92,7 +93,7 @@ func SimulateFromSeedX(
config simtypes.Config,
cdc codec.JSONCodec,
logWriter LogWriter,
) (exportedParams Params, err error) {
) (exportedParams Params, accs []simtypes.Account, err error) {
tb.Helper()
defer func() {
if err != nil {
@ -110,7 +111,7 @@ func SimulateFromSeedX(
logger.Debug("Randomized simulation setup", "params", mustMarshalJSONIndent(params))
timeDiff := maxTimePerBlock - minTimePerBlock
accs := randAccFn(r, params.NumKeys())
accs = randAccFn(r, params.NumKeys())
eventStats := NewEventStats()
// Second variable to keep pending validator set (delayed one block since
@ -119,7 +120,7 @@ func SimulateFromSeedX(
// At least 2 accounts must be added here, otherwise when executing SimulateMsgSend
// two accounts will be selected to meet the conditions from != to and it will fall into an infinite loop.
if len(accs) <= 1 {
return params, errors.New("at least two genesis accounts are required")
return params, accs, errors.New("at least two genesis accounts are required")
}
config.ChainID = chainID
@ -187,7 +188,7 @@ func SimulateFromSeedX(
}
if _, err := app.FinalizeBlock(finalizeBlockReq); err != nil {
return params, fmt.Errorf("block finalization failed at height %d: %w", blockHeight, err)
return params, accs, fmt.Errorf("block finalization failed at height %d: %+w", blockHeight, err)
}
for blockHeight < int64(config.NumBlocks+config.InitialBlockHeight) {
@ -199,7 +200,7 @@ func SimulateFromSeedX(
res, err := app.FinalizeBlock(finalizeBlockReq)
if err != nil {
return params, fmt.Errorf("block finalization failed at height %d: %w", blockHeight, err)
return params, accs, fmt.Errorf("block finalization failed at height %d: %w", blockHeight, err)
}
ctx := app.NewContextLegacy(false, cmtproto.Header{
@ -246,7 +247,7 @@ func SimulateFromSeedX(
if config.Commit {
app.SimWriteState()
if _, err := app.Commit(); err != nil {
return params, fmt.Errorf("commit failed at height %d: %w", blockHeight, err)
return params, accs, fmt.Errorf("commit failed at height %d: %w", blockHeight, err)
}
}
@ -278,7 +279,7 @@ func SimulateFromSeedX(
} else {
eventStats.Print(w)
}
return exportedParams, err
return exportedParams, accs, err
}
type blockSimFn func(