Add a flag to export for zero-height start (#2827)

Closes #2812

This PR adds the flag --for-zero-height to gaiad export, which runs several alterations to the application state to prepare for restarting a new chain in a consistent fashion.

It also:

* Moves Gaia's export code to cmd/gaia/app/export.go for cleaner separation.
* Fixes an inconsistency where we treated the initChainer as happening at height -1 - it should now happen at height 0, since the first header sent by Tendermint has height 1.
* Runs the runtime invariant checks on start (in initChainer)
* Adds a few auxiliary functions to clear slashing periods
* Removes the Height field from Delegation objects in x/stake, which was not used anywhere
This commit is contained in:
Christopher Goes 2018-11-26 13:21:23 +01:00 committed by Jae Kwon
parent 7cb1ba625e
commit ad121f1498
32 changed files with 534 additions and 110 deletions

View File

@ -162,6 +162,24 @@ jobs:
export PATH="$GOBIN:$PATH"
make test_sim_gaia_import_export
test_sim_gaia_simulation_after_import:
<<: *defaults
parallelism: 1
steps:
- attach_workspace:
at: /tmp/workspace
- checkout
- run:
name: dependencies
command: |
export PATH="$GOBIN:$PATH"
make get_vendor_deps
- run:
name: Test Gaia import/export simulation
command: |
export PATH="$GOBIN:$PATH"
make test_sim_gaia_simulation_after_import
test_sim_gaia_multi_seed:
<<: *defaults
parallelism: 1
@ -301,6 +319,9 @@ workflows:
- test_sim_gaia_import_export:
requires:
- setup_dependencies
- test_sim_gaia_simulation_after_import:
requires:
- setup_dependencies
- test_sim_gaia_multi_seed:
requires:
- setup_dependencies

View File

@ -184,6 +184,10 @@ test_sim_gaia_import_export:
@echo "Running Gaia import/export simulation. This may take several minutes..."
@bash scripts/import-export-sim.sh 50
test_sim_gaia_simulation_after_import:
@echo "Running Gaia simulation-after-import. This may take several minutes..."
@bash scripts/simulation-after-import.sh 50
test_sim_gaia_multi_seed:
@echo "Running multi-seed Gaia simulation. This may take awhile!"
@bash scripts/multisim.sh 25

View File

@ -44,6 +44,7 @@ FEATURES
for getting governance parameters.
* [app] \#2663 - Runtime-assertable invariants
* [app] \#2791 Support export at a specific height, with `gaiad export --height=HEIGHT`.
* [app] \#2812 Support export alterations to prepare for restarting at zero-height
* SDK
* [simulator] \#2682 MsgEditValidator now looks at the validator's max rate, thus it now succeeds a significant portion of the time

View File

@ -1,7 +1,6 @@
package app
import (
"encoding/json"
"fmt"
"io"
"os"
@ -22,7 +21,6 @@ import (
cmn "github.com/tendermint/tendermint/libs/common"
dbm "github.com/tendermint/tendermint/libs/db"
"github.com/tendermint/tendermint/libs/log"
tmtypes "github.com/tendermint/tendermint/types"
)
const (
@ -218,18 +216,8 @@ func (app *GaiaApp) EndBlocker(ctx sdk.Context, req abci.RequestEndBlock) abci.R
}
}
// custom logic for gaia initialization
func (app *GaiaApp) initChainer(ctx sdk.Context, req abci.RequestInitChain) abci.ResponseInitChain {
stateJSON := req.AppStateBytes
// TODO is this now the whole genesis file?
var genesisState GenesisState
err := app.cdc.UnmarshalJSON(stateJSON, &genesisState)
if err != nil {
panic(err) // TODO https://github.com/cosmos/cosmos-sdk/issues/468
// return sdk.ErrGenesisParse("").TraceCause(err, "")
}
// initialize store from a genesis state
func (app *GaiaApp) initFromGenesisState(ctx sdk.Context, genesisState GenesisState) []abci.ValidatorUpdate {
// sort by account number to maintain consistency
sort.Slice(genesisState.Accounts, func(i, j int) bool {
return genesisState.Accounts[i].AccountNumber < genesisState.Accounts[j].AccountNumber
@ -276,6 +264,22 @@ func (app *GaiaApp) initChainer(ctx sdk.Context, req abci.RequestInitChain) abci
validators = app.stakeKeeper.ApplyAndReturnValidatorSetUpdates(ctx)
}
return validators
}
// custom logic for gaia initialization
func (app *GaiaApp) initChainer(ctx sdk.Context, req abci.RequestInitChain) abci.ResponseInitChain {
stateJSON := req.AppStateBytes
// TODO is this now the whole genesis file?
var genesisState GenesisState
err := app.cdc.UnmarshalJSON(stateJSON, &genesisState)
if err != nil {
panic(err) // TODO https://github.com/cosmos/cosmos-sdk/issues/468
// return sdk.ErrGenesisParse("").TraceCause(err, "")
}
validators := app.initFromGenesisState(ctx, genesisState)
// sanity check
if len(req.Validators) > 0 {
@ -292,40 +296,14 @@ func (app *GaiaApp) initChainer(ctx sdk.Context, req abci.RequestInitChain) abci
}
}
// assert runtime invariants
app.assertRuntimeInvariants()
return abci.ResponseInitChain{
Validators: validators,
}
}
// export the state of gaia for a genesis file
func (app *GaiaApp) ExportAppStateAndValidators() (appState json.RawMessage, validators []tmtypes.GenesisValidator, err error) {
ctx := app.NewContext(true, abci.Header{})
// iterate to get the accounts
accounts := []GenesisAccount{}
appendAccount := func(acc auth.Account) (stop bool) {
account := NewGenesisAccountI(acc)
accounts = append(accounts, account)
return false
}
app.accountKeeper.IterateAccounts(ctx, appendAccount)
genState := NewGenesisState(
accounts,
auth.ExportGenesis(ctx, app.feeCollectionKeeper),
stake.ExportGenesis(ctx, app.stakeKeeper),
mint.ExportGenesis(ctx, app.mintKeeper),
distr.ExportGenesis(ctx, app.distrKeeper),
gov.ExportGenesis(ctx, app.govKeeper),
slashing.ExportGenesis(ctx, app.slashingKeeper),
)
appState, err = codec.MarshalJSONIndent(app.cdc, genState)
if err != nil {
return nil, nil, err
}
validators = stake.WriteValidators(ctx, app.stakeKeeper)
return appState, validators, nil
}
// load a particular height
func (app *GaiaApp) LoadHeight(height int64) error {
return app.LoadVersion(height, app.keyMain)

View File

@ -49,6 +49,6 @@ func TestGaiadExport(t *testing.T) {
// Making a new app object with the db, so that initchain hasn't been called
newGapp := NewGaiaApp(log.NewTMLogger(log.NewSyncWriter(os.Stdout)), db, nil)
_, _, err := newGapp.ExportAppStateAndValidators()
_, _, err := newGapp.ExportAppStateAndValidators(false)
require.NoError(t, err, "ExportAppStateAndValidators should not have an error")
}

156
cmd/gaia/app/export.go Normal file
View File

@ -0,0 +1,156 @@
package app
import (
"encoding/json"
"fmt"
"github.com/cosmos/cosmos-sdk/codec"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/cosmos/cosmos-sdk/x/auth"
distr "github.com/cosmos/cosmos-sdk/x/distribution"
"github.com/cosmos/cosmos-sdk/x/gov"
"github.com/cosmos/cosmos-sdk/x/mint"
"github.com/cosmos/cosmos-sdk/x/slashing"
stake "github.com/cosmos/cosmos-sdk/x/stake"
abci "github.com/tendermint/tendermint/abci/types"
tmtypes "github.com/tendermint/tendermint/types"
)
// export the state of gaia for a genesis file
func (app *GaiaApp) ExportAppStateAndValidators(forZeroHeight bool) (
appState json.RawMessage, validators []tmtypes.GenesisValidator, err error) {
// as if they could withdraw from the start of the next block
ctx := app.NewContext(true, abci.Header{Height: app.LastBlockHeight()})
if forZeroHeight {
app.prepForZeroHeightGenesis(ctx)
}
// iterate to get the accounts
accounts := []GenesisAccount{}
appendAccount := func(acc auth.Account) (stop bool) {
account := NewGenesisAccountI(acc)
accounts = append(accounts, account)
return false
}
app.accountKeeper.IterateAccounts(ctx, appendAccount)
genState := NewGenesisState(
accounts,
auth.ExportGenesis(ctx, app.feeCollectionKeeper),
stake.ExportGenesis(ctx, app.stakeKeeper),
mint.ExportGenesis(ctx, app.mintKeeper),
distr.ExportGenesis(ctx, app.distrKeeper),
gov.ExportGenesis(ctx, app.govKeeper),
slashing.ExportGenesis(ctx, app.slashingKeeper),
)
appState, err = codec.MarshalJSONIndent(app.cdc, genState)
if err != nil {
return nil, nil, err
}
validators = stake.WriteValidators(ctx, app.stakeKeeper)
return appState, validators, nil
}
// prepare for fresh start at zero height
func (app *GaiaApp) prepForZeroHeightGenesis(ctx sdk.Context) {
/* TODO XXX check some invariants */
height := ctx.BlockHeight()
valAccum := sdk.ZeroDec()
vdiIter := func(_ int64, vdi distr.ValidatorDistInfo) bool {
lastValPower := app.stakeKeeper.GetLastValidatorPower(ctx, vdi.OperatorAddr)
valAccum = valAccum.Add(vdi.GetValAccum(height, sdk.NewDecFromInt(lastValPower)))
return false
}
app.distrKeeper.IterateValidatorDistInfos(ctx, vdiIter)
lastTotalPower := sdk.NewDecFromInt(app.stakeKeeper.GetLastTotalPower(ctx))
totalAccum := app.distrKeeper.GetFeePool(ctx).GetTotalValAccum(height, lastTotalPower)
if !totalAccum.Equal(valAccum) {
panic(fmt.Errorf("validator accum invariance: \n\tfee pool totalAccum: %v"+
"\n\tvalidator accum \t%v\n", totalAccum.String(), valAccum.String()))
}
fmt.Printf("accum invariant ok!\n")
/* END TODO XXX */
/* Handle fee distribution state. */
// withdraw all delegator & validator rewards
vdiIter = func(_ int64, valInfo distr.ValidatorDistInfo) (stop bool) {
err := app.distrKeeper.WithdrawValidatorRewardsAll(ctx, valInfo.OperatorAddr)
if err != nil {
panic(err)
}
return false
}
app.distrKeeper.IterateValidatorDistInfos(ctx, vdiIter)
ddiIter := func(_ int64, distInfo distr.DelegationDistInfo) (stop bool) {
err := app.distrKeeper.WithdrawDelegationReward(
ctx, distInfo.DelegatorAddr, distInfo.ValOperatorAddr)
if err != nil {
panic(err)
}
return false
}
app.distrKeeper.IterateDelegationDistInfos(ctx, ddiIter)
// delete all distribution infos
// these will be recreated in InitGenesis
app.distrKeeper.RemoveValidatorDistInfos(ctx)
app.distrKeeper.RemoveDelegationDistInfos(ctx)
// assert that the fee pool is empty
feePool := app.distrKeeper.GetFeePool(ctx)
if !feePool.TotalValAccum.Accum.IsZero() {
panic("unexpected leftover validator accum")
}
bondDenom := app.stakeKeeper.GetParams(ctx).BondDenom
if !feePool.ValPool.AmountOf(bondDenom).IsZero() {
panic(fmt.Sprintf("unexpected leftover validator pool coins: %v",
feePool.ValPool.AmountOf(bondDenom).String()))
}
// reset fee pool height, save fee pool
feePool.TotalValAccum.UpdateHeight = 0
app.distrKeeper.SetFeePool(ctx, feePool)
/* Handle stake state. */
// iterate through validators by power descending, reset bond height, update bond intra-tx counter
store := ctx.KVStore(app.keyStake)
iter := sdk.KVStoreReversePrefixIterator(store, stake.ValidatorsByPowerIndexKey)
counter := int16(0)
for ; iter.Valid(); iter.Next() {
addr := sdk.ValAddress(iter.Value())
validator, found := app.stakeKeeper.GetValidator(ctx, addr)
if !found {
panic("expected validator, not found")
}
validator.BondHeight = 0
validator.BondIntraTxCounter = counter
validator.UnbondingHeight = 0
app.stakeKeeper.SetValidator(ctx, validator)
counter++
}
iter.Close()
/* Handle slashing state. */
// we have to clear the slashing periods, since they reference heights
app.slashingKeeper.DeleteValidatorSlashingPeriods(ctx)
// reset start height on signing infos
app.slashingKeeper.IterateValidatorSigningInfos(ctx, func(addr sdk.ConsAddress, info slashing.ValidatorSigningInfo) (stop bool) {
info.StartHeight = 0
app.slashingKeeper.SetValidatorSigningInfo(ctx, addr, info)
return false
})
}

View File

@ -137,7 +137,7 @@ func appStateFn(r *rand.Rand, accs []simulation.Account) json.RawMessage {
validator := stake.NewValidator(valAddr, accs[i].PubKey, stake.Description{})
validator.Tokens = sdk.NewDec(amount)
validator.DelegatorShares = sdk.NewDec(amount)
delegation := stake.Delegation{accs[i].Address, valAddr, sdk.NewDec(amount), 0}
delegation := stake.Delegation{accs[i].Address, valAddr, sdk.NewDec(amount)}
validators = append(validators, validator)
delegations = append(delegations, delegation)
}
@ -210,7 +210,7 @@ func BenchmarkFullGaiaSimulation(b *testing.B) {
// Run randomized simulation
// TODO parameterize numbers, save for a later PR
err := simulation.SimulateFromSeed(
_, err := simulation.SimulateFromSeed(
b, app.BaseApp, appStateFn, seed,
testAndRunTxs(app),
[]simulation.RandSetup{},
@ -253,7 +253,7 @@ func TestFullGaiaSimulation(t *testing.T) {
require.Equal(t, "GaiaApp", app.Name())
// Run randomized simulation
err := simulation.SimulateFromSeed(
_, err := simulation.SimulateFromSeed(
t, app.BaseApp, appStateFn, seed,
testAndRunTxs(app),
[]simulation.RandSetup{},
@ -295,7 +295,7 @@ func TestGaiaImportExport(t *testing.T) {
require.Equal(t, "GaiaApp", app.Name())
// Run randomized simulation
err := simulation.SimulateFromSeed(
_, err := simulation.SimulateFromSeed(
t, app.BaseApp, appStateFn, seed,
testAndRunTxs(app),
[]simulation.RandSetup{},
@ -315,7 +315,7 @@ func TestGaiaImportExport(t *testing.T) {
fmt.Printf("Exporting genesis...\n")
appState, _, err := app.ExportAppStateAndValidators()
appState, _, err := app.ExportAppStateAndValidators(false)
if err != nil {
panic(err)
}
@ -330,15 +330,16 @@ func TestGaiaImportExport(t *testing.T) {
}()
newApp := NewGaiaApp(log.NewNopLogger(), newDB, nil)
require.Equal(t, "GaiaApp", newApp.Name())
request := abci.RequestInitChain{
AppStateBytes: appState,
var genesisState GenesisState
err = app.cdc.UnmarshalJSON(appState, &genesisState)
if err != nil {
panic(err)
}
newApp.InitChain(request)
newApp.Commit()
ctxB := newApp.NewContext(true, abci.Header{})
newApp.initFromGenesisState(ctxB, genesisState)
fmt.Printf("Comparing stores...\n")
ctxA := app.NewContext(true, abci.Header{})
ctxB := newApp.NewContext(true, abci.Header{})
type StoreKeysPrefixes struct {
A sdk.StoreKey
B sdk.StoreKey
@ -369,6 +370,87 @@ func TestGaiaImportExport(t *testing.T) {
}
func TestGaiaSimulationAfterImport(t *testing.T) {
if !enabled {
t.Skip("Skipping Gaia simulation after import")
}
// Setup Gaia application
var logger log.Logger
if verbose {
logger = log.TestingLogger()
} else {
logger = log.NewNopLogger()
}
dir, _ := ioutil.TempDir("", "goleveldb-gaia-sim")
db, _ := dbm.NewGoLevelDB("Simulation", dir)
defer func() {
db.Close()
os.RemoveAll(dir)
}()
app := NewGaiaApp(logger, db, nil)
require.Equal(t, "GaiaApp", app.Name())
// Run randomized simulation
stopEarly, err := simulation.SimulateFromSeed(
t, app.BaseApp, appStateFn, seed,
testAndRunTxs(app),
[]simulation.RandSetup{},
invariants(app),
numBlocks,
blockSize,
commit,
)
if commit {
// for memdb:
// fmt.Println("Database Size", db.Stats()["database.size"])
fmt.Println("GoLevelDB Stats")
fmt.Println(db.Stats()["leveldb.stats"])
fmt.Println("GoLevelDB cached block size", db.Stats()["leveldb.cachedblock"])
}
require.Nil(t, err)
if stopEarly {
// we can't export or import a zero-validator genesis
fmt.Printf("We can't export or import a zero-validator genesis, exiting test...\n")
return
}
fmt.Printf("Exporting genesis...\n")
appState, _, err := app.ExportAppStateAndValidators(true)
if err != nil {
panic(err)
}
fmt.Printf("Importing genesis...\n")
newDir, _ := ioutil.TempDir("", "goleveldb-gaia-sim-2")
newDB, _ := dbm.NewGoLevelDB("Simulation-2", dir)
defer func() {
newDB.Close()
os.RemoveAll(newDir)
}()
newApp := NewGaiaApp(log.NewNopLogger(), newDB, nil)
require.Equal(t, "GaiaApp", newApp.Name())
newApp.InitChain(abci.RequestInitChain{
AppStateBytes: appState,
})
// Run randomized simulation on imported app
_, err = simulation.SimulateFromSeed(
t, newApp.BaseApp, appStateFn, seed,
testAndRunTxs(newApp),
[]simulation.RandSetup{},
invariants(newApp),
numBlocks,
blockSize,
commit,
)
require.Nil(t, err)
}
// TODO: Make another test for the fuzzer itself, which just has noOp txs
// and doesn't depend on gaia
func TestAppStateDeterminism(t *testing.T) {

View File

@ -62,9 +62,8 @@ func newApp(logger log.Logger, db dbm.DB, traceStore io.Writer) abci.Application
}
func exportAppStateAndTMValidators(
logger log.Logger, db dbm.DB, traceStore io.Writer, height int64) (
json.RawMessage, []tmtypes.GenesisValidator, error) {
logger log.Logger, db dbm.DB, traceStore io.Writer, height int64, forZeroHeight bool,
) (json.RawMessage, []tmtypes.GenesisValidator, error) {
gApp := app.NewGaiaApp(logger, db, traceStore)
if height != -1 {
err := gApp.LoadHeight(height)
@ -72,5 +71,5 @@ func exportAppStateAndTMValidators(
return nil, nil, err
}
}
return gApp.ExportAppStateAndValidators()
return gApp.ExportAppStateAndValidators(forZeroHeight)
}

View File

@ -124,7 +124,7 @@ func newApp(logger log.Logger, db dbm.DB, storeTracer io.Writer) abci.Applicatio
return app.NewBasecoinApp(logger, db, baseapp.SetPruning(viper.GetString("pruning")))
}
func exportAppStateAndTMValidators(logger log.Logger, db dbm.DB, storeTracer io.Writer, _ int64) (
func exportAppStateAndTMValidators(logger log.Logger, db dbm.DB, storeTracer io.Writer, _ int64, _ bool) (
json.RawMessage, []tmtypes.GenesisValidator, error) {
bapp := app.NewBasecoinApp(logger, db)
return bapp.ExportAppStateAndValidators()

View File

@ -128,7 +128,7 @@ func newApp(logger log.Logger, db dbm.DB, _ io.Writer) abci.Application {
return app.NewDemocoinApp(logger, db)
}
func exportAppStateAndTMValidators(logger log.Logger, db dbm.DB, _ io.Writer, _ int64) (
func exportAppStateAndTMValidators(logger log.Logger, db dbm.DB, _ io.Writer, _ int64, _ bool) (
json.RawMessage, []tmtypes.GenesisValidator, error) {
dapp := app.NewDemocoinApp(logger, db)
return dapp.ExportAppStateAndValidators()

View File

@ -142,7 +142,13 @@ gaiad export > [filename].json
You can also export state from a particular height (at the end of processing the block of that height):
```bash
gaiad export --height=[height] > [filename].json
gaiad export --height [height] > [filename].json
```
If you plan to start a new network from the exported state, export with the `--for-zero-height` flag:
```bash
gaiad export --height [height] --for-zero-height > [filename].json
```
## Upgrade to Validator Node

View File

@ -0,0 +1,54 @@
#!/bin/bash
seeds=(1 2 4 7 9 20 32 123 124 582 1893 2989 3012 4728 37827 981928 87821 891823782 989182 89182391 \
11 22 44 77 99 2020 3232 123123 124124 582582 18931893 29892989 30123012 47284728 37827)
blocks=$1
echo "Running multi-seed import-export simulation with seeds ${seeds[@]}"
echo "Running $blocks blocks per seed"
echo "Edit scripts/simulation-after-import.sh to add new seeds. Keeping parameters in the file makes failures easy to reproduce."
echo "This script will kill all sub-simulations on SIGINT/SIGTERM (i.e. Ctrl-C)."
trap 'kill $(jobs -pr)' SIGINT SIGTERM
tmpdir=$(mktemp -d)
echo "Using temporary log directory: $tmpdir"
sim() {
seed=$1
echo "Running simulation after import with seed $seed. This may take awhile!"
file="$tmpdir/gaia-simulation-seed-$seed-date-$(date -Iseconds -u).stdout"
echo "Writing stdout to $file..."
go test ./cmd/gaia/app -run TestGaiaSimulationAfterImport -SimulationEnabled=true -SimulationNumBlocks=$blocks \
-SimulationBlockSize=200 -SimulationCommit=true -SimulationSeed=$seed -v -timeout 24h > $file
}
i=0
pids=()
for seed in ${seeds[@]}; do
sim $seed &
pids[${i}]=$!
i=$(($i+1))
sleep 10 # start in order, nicer logs
done
echo "Simulation processes spawned, waiting for completion..."
code=0
i=0
for pid in ${pids[*]}; do
wait $pid
last=$?
seed=${seeds[${i}]}
if [ $last -ne 0 ]
then
echo "Import/export simulation with seed $seed failed!"
code=1
else
echo "Import/export simulation with seed $seed OK"
fi
i=$(($i+1))
done
exit $code

View File

@ -19,7 +19,7 @@ type (
// 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) (json.RawMessage, []tmtypes.GenesisValidator, error)
AppExporter func(log.Logger, dbm.DB, io.Writer, int64, bool) (json.RawMessage, []tmtypes.GenesisValidator, error)
)
func openDB(rootDir string) (dbm.DB, error) {

View File

@ -14,7 +14,8 @@ import (
)
const (
flagHeight = "height"
flagHeight = "height"
flagForZeroHeight = "for-zero-height"
)
// ExportCmd dumps app state to JSON.
@ -50,7 +51,8 @@ func ExportCmd(ctx *Context, cdc *codec.Codec, appExporter AppExporter) *cobra.C
return err
}
height := viper.GetInt64(flagHeight)
appState, validators, err := appExporter(ctx.Logger, db, traceWriter, height)
forZeroHeight := viper.GetBool(flagForZeroHeight)
appState, validators, err := appExporter(ctx.Logger, db, traceWriter, height, forZeroHeight)
if err != nil {
return errors.Errorf("error exporting state: %v\n", err)
}
@ -73,6 +75,7 @@ func ExportCmd(ctx *Context, cdc *codec.Codec, appExporter AppExporter) *cobra.C
},
}
cmd.Flags().Int64(flagHeight, -1, "Export state from a particular height (-1 means latest height)")
cmd.Flags().Bool(flagForZeroHeight, false, "Export state to start at height zero (perform preproccessing)")
return cmd
}

View File

@ -42,6 +42,34 @@ func (k Keeper) RemoveDelegationDistInfo(ctx sdk.Context, delAddr sdk.AccAddress
store.Delete(GetDelegationDistInfoKey(delAddr, valOperatorAddr))
}
// remove all delegation distribution infos
func (k Keeper) RemoveDelegationDistInfos(ctx sdk.Context) {
store := ctx.KVStore(k.storeKey)
iter := sdk.KVStorePrefixIterator(store, DelegationDistInfoKey)
defer iter.Close()
for ; iter.Valid(); iter.Next() {
store.Delete(iter.Key())
}
}
// iterate over all the validator distribution infos
func (k Keeper) IterateDelegationDistInfos(ctx sdk.Context,
fn func(index int64, distInfo types.DelegationDistInfo) (stop bool)) {
store := ctx.KVStore(k.storeKey)
iter := sdk.KVStorePrefixIterator(store, DelegationDistInfoKey)
defer iter.Close()
index := int64(0)
for ; iter.Valid(); iter.Next() {
var ddi types.DelegationDistInfo
k.cdc.MustUnmarshalBinaryLengthPrefixed(iter.Value(), &ddi)
if fn(index, ddi) {
return
}
index++
}
}
//___________________________________________________________________________________________
// get the delegator withdraw address, return the delegator address if not set

View File

@ -47,6 +47,34 @@ func (k Keeper) RemoveValidatorDistInfo(ctx sdk.Context, valAddr sdk.ValAddress)
store.Delete(GetValidatorDistInfoKey(valAddr))
}
// remove all validator distribution infos
func (k Keeper) RemoveValidatorDistInfos(ctx sdk.Context) {
store := ctx.KVStore(k.storeKey)
iter := sdk.KVStorePrefixIterator(store, ValidatorDistInfoKey)
defer iter.Close()
for ; iter.Valid(); iter.Next() {
store.Delete(iter.Key())
}
}
// iterate over all the validator distribution infos
func (k Keeper) IterateValidatorDistInfos(ctx sdk.Context,
fn func(index int64, distInfo types.ValidatorDistInfo) (stop bool)) {
store := ctx.KVStore(k.storeKey)
iter := sdk.KVStorePrefixIterator(store, ValidatorDistInfoKey)
defer iter.Close()
index := int64(0)
for ; iter.Valid(); iter.Next() {
var vdi types.ValidatorDistInfo
k.cdc.MustUnmarshalBinaryLengthPrefixed(iter.Value(), &vdi)
if fn(index, vdi) {
return
}
index++
}
}
// Get the calculated accum of a validator at the current block
// without affecting the state.
func (k Keeper) GetValidatorAccum(ctx sdk.Context, operatorAddr sdk.ValAddress) (sdk.Dec, sdk.Error) {
@ -74,6 +102,10 @@ func (k Keeper) WithdrawValidatorRewardsAll(ctx sdk.Context, operatorAddr sdk.Va
accAddr := sdk.AccAddress(operatorAddr.Bytes())
withdraw := k.withdrawDelegationRewardsAll(ctx, accAddr)
//if withdraw.AmountOf {
//return types.ErrNoValidatorDistInfo(k.codespace)
//}
// withdrawal validator commission rewards
valInfo := k.GetValidatorDistInfo(ctx, operatorAddr)
wc := k.GetWithdrawContext(ctx, operatorAddr)

View File

@ -7,12 +7,14 @@ import (
sdk "github.com/cosmos/cosmos-sdk/types"
distr "github.com/cosmos/cosmos-sdk/x/distribution"
"github.com/cosmos/cosmos-sdk/x/mock/simulation"
"github.com/cosmos/cosmos-sdk/x/stake"
abci "github.com/tendermint/tendermint/abci/types"
)
// AllInvariants runs all invariants of the distribution module
// Currently: total supply, positive power
func AllInvariants(d distr.Keeper, sk distr.StakeKeeper) simulation.Invariant {
func AllInvariants(d distr.Keeper, stk stake.Keeper) simulation.Invariant {
sk := distr.StakeKeeper(stk)
return func(app *baseapp.BaseApp) error {
err := ValAccumInvariants(d, sk)(app)
if err != nil {
@ -22,6 +24,10 @@ func AllInvariants(d distr.Keeper, sk distr.StakeKeeper) simulation.Invariant {
if err != nil {
return err
}
err = CanWithdrawInvariant(d, stk)(app)
if err != nil {
return err
}
return nil
}
}
@ -130,3 +136,49 @@ func DelAccumInvariants(k distr.Keeper, sk distr.StakeKeeper) simulation.Invaria
return nil
}
}
// CanWithdrawInvariant checks that current rewards can be completely withdrawn
func CanWithdrawInvariant(k distr.Keeper, sk stake.Keeper) simulation.Invariant {
return func(app *baseapp.BaseApp) error {
mockHeader := abci.Header{Height: app.LastBlockHeight() + 1}
ctx := app.NewContext(false, mockHeader)
// we don't want to write the changes
ctx, _ = ctx.CacheContext()
// withdraw all delegator & validator rewards
vdiIter := func(_ int64, valInfo distr.ValidatorDistInfo) (stop bool) {
err := k.WithdrawValidatorRewardsAll(ctx, valInfo.OperatorAddr)
if err != nil {
panic(err)
}
return false
}
k.IterateValidatorDistInfos(ctx, vdiIter)
ddiIter := func(_ int64, distInfo distr.DelegationDistInfo) (stop bool) {
err := k.WithdrawDelegationReward(
ctx, distInfo.DelegatorAddr, distInfo.ValOperatorAddr)
if err != nil {
panic(err)
}
return false
}
k.IterateDelegationDistInfos(ctx, ddiIter)
// assert that the fee pool is empty
feePool := k.GetFeePool(ctx)
if !feePool.TotalValAccum.Accum.IsZero() {
return fmt.Errorf("unexpected leftover validator accum")
}
bondDenom := sk.GetParams(ctx).BondDenom
if !feePool.ValPool.AmountOf(bondDenom).IsZero() {
return fmt.Errorf("unexpected leftover validator pool coins: %v",
feePool.ValPool.AmountOf(bondDenom).String())
}
// all ok
return nil
}
}

View File

@ -95,6 +95,7 @@ func BenchmarkBlockProvision(b *testing.B) {
r1 := rand.New(s1)
minter.AnnualProvisions = sdk.NewDec(r1.Int63n(1000000))
// run the Fib function b.N times
for n := 0; n < b.N; n++ {
minter.BlockProvision(params)
}

View File

@ -167,10 +167,11 @@ func RandomRequestBeginBlock(r *rand.Rand, params Params,
time := header.Time
vals := voteInfos
if r.Float64() < params.PastEvidenceFraction {
height = int64(r.Intn(int(header.Height) - 1))
time = pastTimes[height]
vals = pastVoteInfos[height]
if r.Float64() < params.PastEvidenceFraction && header.Height > 1 {
height = int64(r.Intn(int(header.Height)-1)) + 1 // Tendermint starts at height 1
// array indices offset by one
time = pastTimes[height-1]
vals = pastVoteInfos[height-1]
}
validator := vals[r.Intn(len(vals))].Validator

View File

@ -27,7 +27,7 @@ type AppStateFn func(r *rand.Rand, accs []Account) json.RawMessage
// Simulate tests application by sending random messages.
func Simulate(t *testing.T, app *baseapp.BaseApp,
appStateFn AppStateFn, ops WeightedOperations, setups []RandSetup,
invariants Invariants, numBlocks int, blockSize int, commit bool) error {
invariants Invariants, numBlocks int, blockSize int, commit bool) (bool, error) {
time := time.Now().UnixNano()
return SimulateFromSeed(t, app, appStateFn, time, ops,
@ -57,10 +57,9 @@ func initChain(r *rand.Rand, params Params, accounts []Account,
func SimulateFromSeed(tb testing.TB, app *baseapp.BaseApp,
appStateFn AppStateFn, seed int64, ops WeightedOperations,
setups []RandSetup, invariants Invariants,
numBlocks int, blockSize int, commit bool) (simError error) {
numBlocks int, blockSize int, commit bool) (stopEarly bool, simError error) {
// in case we have to end early, don't os.Exit so that we can run cleanup code.
stopEarly := false
testingMode, t, b := getTestingMode(tb)
fmt.Printf("Starting SimulateFromSeed with randomness "+
"created with seed %d\n", int(seed))
@ -217,14 +216,14 @@ func SimulateFromSeed(tb testing.TB, app *baseapp.BaseApp,
if stopEarly {
eventStats.Print()
return simError
return true, simError
}
fmt.Printf("\nSimulation complete. Final height (blocks): %d, "+
"final time (seconds), : %v, operations ran %d\n",
header.Height, header.Time, opCount)
eventStats.Print()
return nil
return false, nil
}
//______________________________________________________________________________

View File

@ -7,10 +7,10 @@ import (
// GenesisState - all slashing state that must be provided at genesis
type GenesisState struct {
Params Params
SigningInfos map[string]ValidatorSigningInfo
MissedBlocks map[string][]MissedBlock
SlashingPeriods []ValidatorSlashingPeriod
Params Params `json:"params"`
SigningInfos map[string]ValidatorSigningInfo `json:"signing_infos"`
MissedBlocks map[string][]MissedBlock `json:"missed_blocks"`
SlashingPeriods []ValidatorSlashingPeriod `json:"slashing_periods"`
}
// MissedBlock
@ -41,7 +41,7 @@ func InitGenesis(ctx sdk.Context, keeper Keeper, data GenesisState, sdata types.
if err != nil {
panic(err)
}
keeper.setValidatorSigningInfo(ctx, address, info)
keeper.SetValidatorSigningInfo(ctx, address, info)
}
for addr, array := range data.MissedBlocks {
@ -70,12 +70,12 @@ func ExportGenesis(ctx sdk.Context, keeper Keeper) (data GenesisState) {
signingInfos := make(map[string]ValidatorSigningInfo)
missedBlocks := make(map[string][]MissedBlock)
keeper.iterateValidatorSigningInfos(ctx, func(address sdk.ConsAddress, info ValidatorSigningInfo) (stop bool) {
keeper.IterateValidatorSigningInfos(ctx, func(address sdk.ConsAddress, info ValidatorSigningInfo) (stop bool) {
bechAddr := address.String()
signingInfos[bechAddr] = info
localMissedBlocks := []MissedBlock{}
keeper.iterateValidatorMissedBlockBitArray(ctx, address, func(index int64, missed bool) (stop bool) {
keeper.IterateValidatorMissedBlockBitArray(ctx, address, func(index int64, missed bool) (stop bool) {
localMissedBlocks = append(localMissedBlocks, MissedBlock{index, missed})
return false
})
@ -85,7 +85,7 @@ func ExportGenesis(ctx sdk.Context, keeper Keeper) (data GenesisState) {
})
slashingPeriods := []ValidatorSlashingPeriod{}
keeper.iterateValidatorSlashingPeriods(ctx, func(slashingPeriod ValidatorSlashingPeriod) (stop bool) {
keeper.IterateValidatorSlashingPeriods(ctx, func(slashingPeriod ValidatorSlashingPeriod) (stop bool) {
slashingPeriods = append(slashingPeriods, slashingPeriod)
return false
})

View File

@ -60,7 +60,7 @@ func TestJailedValidatorDelegations(t *testing.T) {
JailedUntil: time.Unix(0, 0),
MissedBlocksCounter: int64(0),
}
slashingKeeper.setValidatorSigningInfo(ctx, consAddr, newInfo)
slashingKeeper.SetValidatorSigningInfo(ctx, consAddr, newInfo)
// delegate tokens to the validator
delAddr := sdk.AccAddress(addrs[2])

View File

@ -18,7 +18,7 @@ func (k Keeper) onValidatorBonded(ctx sdk.Context, address sdk.ConsAddress, _ sd
JailedUntil: time.Unix(0, 0),
MissedBlocksCounter: 0,
}
k.setValidatorSigningInfo(ctx, address, signingInfo)
k.SetValidatorSigningInfo(ctx, address, signingInfo)
}
// Create a new slashing period when a validator is bonded

View File

@ -96,7 +96,7 @@ func (k Keeper) handleDoubleSign(ctx sdk.Context, addr crypto.Address, infractio
panic(fmt.Sprintf("Expected signing info for validator %s but not found", consAddr))
}
signInfo.JailedUntil = time.Add(k.DoubleSignUnbondDuration(ctx))
k.setValidatorSigningInfo(ctx, consAddr, signInfo)
k.SetValidatorSigningInfo(ctx, consAddr, signInfo)
}
// handle a validator signature, must be called once per validator per block
@ -168,7 +168,7 @@ func (k Keeper) handleValidatorSignature(ctx sdk.Context, addr crypto.Address, p
}
// Set the updated signing info
k.setValidatorSigningInfo(ctx, consAddr, signInfo)
k.SetValidatorSigningInfo(ctx, consAddr, signInfo)
}
func (k Keeper) addPubkey(ctx sdk.Context, pubkey crypto.PubKey) {

View File

@ -21,7 +21,7 @@ func (k Keeper) getValidatorSigningInfo(ctx sdk.Context, address sdk.ConsAddress
}
// Stored by *validator* address (not operator address)
func (k Keeper) iterateValidatorSigningInfos(ctx sdk.Context, handler func(address sdk.ConsAddress, info ValidatorSigningInfo) (stop bool)) {
func (k Keeper) IterateValidatorSigningInfos(ctx sdk.Context, handler func(address sdk.ConsAddress, info ValidatorSigningInfo) (stop bool)) {
store := ctx.KVStore(k.storeKey)
iter := sdk.KVStorePrefixIterator(store, ValidatorSigningInfoKey)
defer iter.Close()
@ -36,7 +36,7 @@ func (k Keeper) iterateValidatorSigningInfos(ctx sdk.Context, handler func(addre
}
// Stored by *validator* address (not operator address)
func (k Keeper) setValidatorSigningInfo(ctx sdk.Context, address sdk.ConsAddress, info ValidatorSigningInfo) {
func (k Keeper) SetValidatorSigningInfo(ctx sdk.Context, address sdk.ConsAddress, info ValidatorSigningInfo) {
store := ctx.KVStore(k.storeKey)
bz := k.cdc.MustMarshalBinaryLengthPrefixed(info)
store.Set(GetValidatorSigningInfoKey(address), bz)
@ -56,7 +56,7 @@ func (k Keeper) getValidatorMissedBlockBitArray(ctx sdk.Context, address sdk.Con
}
// Stored by *validator* address (not operator address)
func (k Keeper) iterateValidatorMissedBlockBitArray(ctx sdk.Context, address sdk.ConsAddress, handler func(index int64, missed bool) (stop bool)) {
func (k Keeper) IterateValidatorMissedBlockBitArray(ctx sdk.Context, address sdk.ConsAddress, handler func(index int64, missed bool) (stop bool)) {
store := ctx.KVStore(k.storeKey)
index := int64(0)
// Array may be sparse

View File

@ -19,7 +19,7 @@ func TestGetSetValidatorSigningInfo(t *testing.T) {
JailedUntil: time.Unix(2, 0),
MissedBlocksCounter: int64(10),
}
keeper.setValidatorSigningInfo(ctx, sdk.ConsAddress(addrs[0]), newInfo)
keeper.SetValidatorSigningInfo(ctx, sdk.ConsAddress(addrs[0]), newInfo)
info, found = keeper.getValidatorSigningInfo(ctx, sdk.ConsAddress(addrs[0]))
require.True(t, found)
require.Equal(t, info.StartHeight, int64(4))

View File

@ -54,7 +54,7 @@ func (k Keeper) getValidatorSlashingPeriodForHeight(ctx sdk.Context, address sdk
// Iterate over all slashing periods in the store, calling on each
// decode slashing period a provided handler function
// Stop if the provided handler function returns true
func (k Keeper) iterateValidatorSlashingPeriods(ctx sdk.Context, handler func(slashingPeriod ValidatorSlashingPeriod) (stop bool)) {
func (k Keeper) IterateValidatorSlashingPeriods(ctx sdk.Context, handler func(slashingPeriod ValidatorSlashingPeriod) (stop bool)) {
store := ctx.KVStore(k.storeKey)
iter := sdk.KVStorePrefixIterator(store, ValidatorSlashingPeriodKey)
defer iter.Close()
@ -66,6 +66,16 @@ func (k Keeper) iterateValidatorSlashingPeriods(ctx sdk.Context, handler func(sl
}
}
// Delete all slashing periods in the store.
func (k Keeper) DeleteValidatorSlashingPeriods(ctx sdk.Context) {
store := ctx.KVStore(k.storeKey)
iter := sdk.KVStorePrefixIterator(store, ValidatorSlashingPeriodKey)
for ; iter.Valid(); iter.Next() {
store.Delete(iter.Key())
}
iter.Close()
}
// Stored by validator Tendermint address (not operator address)
// This function sets a validator slashing period for a particular validator,
// start height, end height, and current slashed-so-far total, or updates

View File

@ -21,8 +21,8 @@ func InitGenesis(ctx sdk.Context, keeper Keeper, data types.GenesisState) (res [
// We need to pretend to be "n blocks before genesis", where "n" is the validator update delay,
// so that e.g. slashing periods are correctly initialized for the validator set
// e.g. with a one-block offset - the first TM block is at height 0, so state updates applied from genesis.json are in block -1.
ctx = ctx.WithBlockHeight(-types.ValidatorUpdateDelay)
// e.g. with a one-block offset - the first TM block is at height 1, so state updates applied from genesis.json are in block 0.
ctx = ctx.WithBlockHeight(1 - types.ValidatorUpdateDelay)
keeper.SetPool(ctx, data.Pool)
keeper.SetParams(ctx, data.Params)
@ -72,6 +72,13 @@ func InitGenesis(ctx sdk.Context, keeper Keeper, data types.GenesisState) (res [
if data.Exported {
for _, lv := range data.LastValidatorPowers {
keeper.SetLastValidatorPower(ctx, lv.Address, lv.Power)
validator, found := keeper.GetValidator(ctx, lv.Address)
if !found {
panic("expected validator, not found")
}
update := validator.ABCIValidatorUpdate()
update.Power = lv.Power.Int64() // keep the next-val-set offset, use the last power for the first block
res = append(res, update)
}
} else {
res = keeper.ApplyAndReturnValidatorSetUpdates(ctx)

View File

@ -341,8 +341,6 @@ func TestIncrementsMsgDelegate(t *testing.T) {
expDelegatorShares := int64(i+2) * bondAmount // (1 self delegation)
expDelegatorAcc := sdk.NewInt(initBond - expBond)
require.Equal(t, bond.Height, int64(i), "Incorrect bond height")
gotBond := bond.Shares.RoundInt64()
gotDelegatorShares := validator.DelegatorShares.RoundInt64()
gotDelegatorAcc := accMapper.GetAccount(ctx, delegatorAddr).GetCoins().AmountOf(params.BondDenom)

View File

@ -415,7 +415,6 @@ func (k Keeper) Delegate(ctx sdk.Context, delAddr sdk.AccAddress, bondAmt sdk.Co
// Update delegation
delegation.Shares = delegation.Shares.Add(newShares)
delegation.Height = ctx.BlockHeight()
k.SetDelegation(ctx, delegation)
return newShares, nil
@ -462,8 +461,7 @@ func (k Keeper) unbond(ctx sdk.Context, delAddr sdk.AccAddress, valAddr sdk.ValA
k.RemoveDelegation(ctx, delegation)
} else {
// Update height
delegation.Height = ctx.BlockHeight()
// update the delegation
k.SetDelegation(ctx, delegation)
}
@ -600,7 +598,7 @@ func (k Keeper) BeginRedelegation(ctx sdk.Context, delAddr sdk.AccAddress,
}
rounded := returnAmount.TruncateInt()
if rounded.IsZero() {
if rounded.IsZero() { //TODO design consideration
return types.Redelegation{}, types.ErrVerySmallRedelegation(k.Codespace())
}
returnCoin := sdk.NewCoin(k.BondDenom(ctx), rounded)

View File

@ -56,11 +56,11 @@ func TestDelegation(t *testing.T) {
require.True(t, bond1to1.Equal(resBond))
// add some more records
bond1to2 := types.Delegation{addrDels[0], addrVals[1], sdk.NewDec(9), 0}
bond1to3 := types.Delegation{addrDels[0], addrVals[2], sdk.NewDec(9), 1}
bond2to1 := types.Delegation{addrDels[1], addrVals[0], sdk.NewDec(9), 2}
bond2to2 := types.Delegation{addrDels[1], addrVals[1], sdk.NewDec(9), 3}
bond2to3 := types.Delegation{addrDels[1], addrVals[2], sdk.NewDec(9), 4}
bond1to2 := types.Delegation{addrDels[0], addrVals[1], sdk.NewDec(9)}
bond1to3 := types.Delegation{addrDels[0], addrVals[2], sdk.NewDec(9)}
bond2to1 := types.Delegation{addrDels[1], addrVals[0], sdk.NewDec(9)}
bond2to2 := types.Delegation{addrDels[1], addrVals[1], sdk.NewDec(9)}
bond2to3 := types.Delegation{addrDels[1], addrVals[2], sdk.NewDec(9)}
keeper.SetDelegation(ctx, bond1to2)
keeper.SetDelegation(ctx, bond1to3)
keeper.SetDelegation(ctx, bond2to1)

View File

@ -33,19 +33,16 @@ type Delegation struct {
DelegatorAddr sdk.AccAddress `json:"delegator_addr"`
ValidatorAddr sdk.ValAddress `json:"validator_addr"`
Shares sdk.Dec `json:"shares"`
Height int64 `json:"height"` // Last height bond updated
}
type delegationValue struct {
Shares sdk.Dec
Height int64
}
// return the delegation without fields contained within the key for the store
func MustMarshalDelegation(cdc *codec.Codec, delegation Delegation) []byte {
val := delegationValue{
delegation.Shares,
delegation.Height,
}
return cdc.MustMarshalBinaryLengthPrefixed(val)
}
@ -81,7 +78,6 @@ func UnmarshalDelegation(cdc *codec.Codec, key, value []byte) (delegation Delega
DelegatorAddr: delAddr,
ValidatorAddr: valAddr,
Shares: storeValue.Shares,
Height: storeValue.Height,
}, nil
}
@ -89,7 +85,6 @@ func UnmarshalDelegation(cdc *codec.Codec, key, value []byte) (delegation Delega
func (d Delegation) Equal(d2 Delegation) bool {
return bytes.Equal(d.DelegatorAddr, d2.DelegatorAddr) &&
bytes.Equal(d.ValidatorAddr, d2.ValidatorAddr) &&
d.Height == d2.Height &&
d.Shares.Equal(d2.Shares)
}
@ -109,7 +104,6 @@ func (d Delegation) HumanReadableString() (string, error) {
resp += fmt.Sprintf("Delegator: %s\n", d.DelegatorAddr)
resp += fmt.Sprintf("Validator: %s\n", d.ValidatorAddr)
resp += fmt.Sprintf("Shares: %s\n", d.Shares.String())
resp += fmt.Sprintf("Height: %d", d.Height)
return resp, nil
}