mirror of
				https://github.com/0glabs/0g-chain.git
				synced 2025-11-04 00:37:28 +00:00 
			
		
		
		
	Swap module simulations: swap msg operations (#972)
* simulate MsgSwapExactForTokens * extract deadline generation to custom method * simulate MsgSwapForExactTokens * implement simulation decoder * decoder test * add fee to msgs * update comments
This commit is contained in:
		
							parent
							
								
									b2bfe05170
								
							
						
					
					
						commit
						c252a1b99b
					
				@ -15,6 +15,8 @@ const (
 | 
			
		||||
	DefaultWeightMsgClaimReward           int = 20
 | 
			
		||||
	DefaultWeightMsgDeposit               int = 20
 | 
			
		||||
	DefaultWeightMsgWithdraw              int = 20
 | 
			
		||||
	DefaultWeightMsgSwapExactForTokens    int = 20
 | 
			
		||||
	DefaultWeightMsgSwapForExactTokens    int = 20
 | 
			
		||||
	DefaultWeightMsgIssue                 int = 20
 | 
			
		||||
	DefaultWeightMsgRedeem                int = 20
 | 
			
		||||
	DefaultWeightMsgBlock                 int = 20
 | 
			
		||||
 | 
			
		||||
@ -1,13 +1,29 @@
 | 
			
		||||
package simulation
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"github.com/tendermint/tendermint/libs/kv"
 | 
			
		||||
	"bytes"
 | 
			
		||||
	"fmt"
 | 
			
		||||
 | 
			
		||||
	"github.com/cosmos/cosmos-sdk/codec"
 | 
			
		||||
	"github.com/tendermint/tendermint/libs/kv"
 | 
			
		||||
 | 
			
		||||
	"github.com/kava-labs/kava/x/swap/types"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// DecodeStore unmarshals the KVPair's Value to the module's corresponding type
 | 
			
		||||
func DecodeStore(cdc *codec.Codec, kvA, kvB kv.Pair) string {
 | 
			
		||||
	// TODO: as store keys are added to the module, test marshal/unmarshal of each key prefix
 | 
			
		||||
	return ""
 | 
			
		||||
	switch {
 | 
			
		||||
	case bytes.Equal(kvA.Key[:1], types.PoolKeyPrefix):
 | 
			
		||||
		var poolRecordA, poolRecordB types.PoolRecord
 | 
			
		||||
		cdc.MustUnmarshalBinaryLengthPrefixed(kvA.Value, &poolRecordA)
 | 
			
		||||
		cdc.MustUnmarshalBinaryLengthPrefixed(kvB.Value, &poolRecordB)
 | 
			
		||||
		return fmt.Sprintf("%v\n%v", poolRecordA, poolRecordB)
 | 
			
		||||
	case bytes.Equal(kvA.Key[:1], types.DepositorPoolSharesPrefix):
 | 
			
		||||
		var shareRecordA, shareRecordB types.ShareRecord
 | 
			
		||||
		cdc.MustUnmarshalBinaryLengthPrefixed(kvA.Value, &shareRecordA)
 | 
			
		||||
		cdc.MustUnmarshalBinaryLengthPrefixed(kvB.Value, &shareRecordB)
 | 
			
		||||
		return fmt.Sprintf("%v\n%v", shareRecordA, shareRecordB)
 | 
			
		||||
	default:
 | 
			
		||||
		panic(fmt.Sprintf("invalid %s key prefix %X", types.ModuleName, kvA.Key[:1]))
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										62
									
								
								x/swap/simulation/decoder_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										62
									
								
								x/swap/simulation/decoder_test.go
									
									
									
									
									
										Normal file
									
								
							@ -0,0 +1,62 @@
 | 
			
		||||
package simulation
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"fmt"
 | 
			
		||||
	"testing"
 | 
			
		||||
 | 
			
		||||
	"github.com/cosmos/cosmos-sdk/codec"
 | 
			
		||||
	"github.com/stretchr/testify/require"
 | 
			
		||||
 | 
			
		||||
	sdk "github.com/cosmos/cosmos-sdk/types"
 | 
			
		||||
	"github.com/tendermint/tendermint/crypto"
 | 
			
		||||
	"github.com/tendermint/tendermint/libs/kv"
 | 
			
		||||
 | 
			
		||||
	"github.com/kava-labs/kava/x/swap/types"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
func makeTestCodec() (cdc *codec.Codec) {
 | 
			
		||||
	cdc = codec.New()
 | 
			
		||||
	sdk.RegisterCodec(cdc)
 | 
			
		||||
	types.RegisterCodec(cdc)
 | 
			
		||||
	return
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestDecodeSwapStore(t *testing.T) {
 | 
			
		||||
	cdc := makeTestCodec()
 | 
			
		||||
 | 
			
		||||
	depositor := sdk.AccAddress(crypto.AddressHash([]byte("DepositorAddress")))
 | 
			
		||||
	reserves := sdk.NewCoins(
 | 
			
		||||
		sdk.NewCoin("ukava", sdk.NewInt(100000000)),
 | 
			
		||||
		sdk.NewCoin("usdx", sdk.NewInt(200000000)),
 | 
			
		||||
	)
 | 
			
		||||
	shares := sdk.NewInt(123456)
 | 
			
		||||
 | 
			
		||||
	poolRecord := types.NewPoolRecord(reserves, shares)
 | 
			
		||||
	shareRecord := types.NewShareRecord(depositor, poolRecord.PoolID, shares)
 | 
			
		||||
 | 
			
		||||
	kvPairs := kv.Pairs{
 | 
			
		||||
		kv.Pair{Key: types.PoolKeyPrefix, Value: cdc.MustMarshalBinaryLengthPrefixed(poolRecord)},
 | 
			
		||||
		kv.Pair{Key: types.DepositorPoolSharesPrefix, Value: cdc.MustMarshalBinaryLengthPrefixed(shareRecord)},
 | 
			
		||||
		kv.Pair{Key: []byte{0x99}, Value: []byte{0x99}},
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	tests := []struct {
 | 
			
		||||
		name        string
 | 
			
		||||
		expectedLog string
 | 
			
		||||
	}{
 | 
			
		||||
		{"PoolRecord", fmt.Sprintf("%v\n%v", poolRecord, poolRecord)},
 | 
			
		||||
		{"ShareRecord", fmt.Sprintf("%v\n%v", shareRecord, shareRecord)},
 | 
			
		||||
		{"other", ""},
 | 
			
		||||
	}
 | 
			
		||||
	for i, tt := range tests {
 | 
			
		||||
		i, tt := i, tt
 | 
			
		||||
		t.Run(tt.name, func(t *testing.T) {
 | 
			
		||||
			switch i {
 | 
			
		||||
			case len(tests) - 1:
 | 
			
		||||
				require.Panics(t, func() { DecodeStore(cdc, kvPairs[i], kvPairs[i]) }, tt.name)
 | 
			
		||||
			default:
 | 
			
		||||
				require.Equal(t, tt.expectedLog, DecodeStore(cdc, kvPairs[i], kvPairs[i]), tt.name)
 | 
			
		||||
			}
 | 
			
		||||
		})
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
@ -17,7 +17,7 @@ import (
 | 
			
		||||
var (
 | 
			
		||||
	//nolint
 | 
			
		||||
	accs            []simulation.Account
 | 
			
		||||
	consistentPools = [2][2]string{{"ukava", "usdx"}, {"bnb", "stake"}}
 | 
			
		||||
	consistentPools = [2][2]string{{"ukava", "usdx"}, {"bnb", "xrp"}}
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// GenSwapFee generates a random SwapFee in range [0.01, 1.00]
 | 
			
		||||
 | 
			
		||||
@ -27,8 +27,10 @@ var (
 | 
			
		||||
 | 
			
		||||
// Simulation operation weights constants
 | 
			
		||||
const (
 | 
			
		||||
	OpWeightMsgDeposit  = "op_weight_msg_deposit"
 | 
			
		||||
	OpWeightMsgWithdraw = "op_weight_msg_withdraw"
 | 
			
		||||
	OpWeightMsgDeposit            = "op_weight_msg_deposit"
 | 
			
		||||
	OpWeightMsgWithdraw           = "op_weight_msg_withdraw"
 | 
			
		||||
	OpWeightMsgSwapExactForTokens = "op_weight_msg_swap_exact_for_tokens"
 | 
			
		||||
	OpWeightMsgSwapForExactTokens = "op_weight_msg_swap_for_exact_tokens"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// WeightedOperations returns all the operations from the module with their respective weights
 | 
			
		||||
@ -37,6 +39,8 @@ func WeightedOperations(
 | 
			
		||||
) simulation.WeightedOperations {
 | 
			
		||||
	var weightMsgDeposit int
 | 
			
		||||
	var weightMsgWithdraw int
 | 
			
		||||
	var weightMsgSwapExactForTokens int
 | 
			
		||||
	var weightMsgSwapForExactTokens int
 | 
			
		||||
 | 
			
		||||
	appParams.GetOrGenerate(cdc, OpWeightMsgDeposit, &weightMsgDeposit, nil,
 | 
			
		||||
		func(_ *rand.Rand) {
 | 
			
		||||
@ -50,6 +54,18 @@ func WeightedOperations(
 | 
			
		||||
		},
 | 
			
		||||
	)
 | 
			
		||||
 | 
			
		||||
	appParams.GetOrGenerate(cdc, OpWeightMsgSwapExactForTokens, &weightMsgSwapExactForTokens, nil,
 | 
			
		||||
		func(_ *rand.Rand) {
 | 
			
		||||
			weightMsgSwapExactForTokens = appparams.DefaultWeightMsgSwapExactForTokens
 | 
			
		||||
		},
 | 
			
		||||
	)
 | 
			
		||||
 | 
			
		||||
	appParams.GetOrGenerate(cdc, OpWeightMsgSwapForExactTokens, &weightMsgSwapForExactTokens, nil,
 | 
			
		||||
		func(_ *rand.Rand) {
 | 
			
		||||
			weightMsgSwapForExactTokens = appparams.DefaultWeightMsgSwapForExactTokens
 | 
			
		||||
		},
 | 
			
		||||
	)
 | 
			
		||||
 | 
			
		||||
	return simulation.WeightedOperations{
 | 
			
		||||
		simulation.NewWeightedOperation(
 | 
			
		||||
			weightMsgDeposit,
 | 
			
		||||
@ -59,6 +75,14 @@ func WeightedOperations(
 | 
			
		||||
			weightMsgWithdraw,
 | 
			
		||||
			SimulateMsgWithdraw(ak, k),
 | 
			
		||||
		),
 | 
			
		||||
		simulation.NewWeightedOperation(
 | 
			
		||||
			weightMsgSwapExactForTokens,
 | 
			
		||||
			SimulateMsgSwapExactForTokens(ak, k),
 | 
			
		||||
		),
 | 
			
		||||
		simulation.NewWeightedOperation(
 | 
			
		||||
			weightMsgSwapForExactTokens,
 | 
			
		||||
			SimulateMsgSwapForExactTokens(ak, k),
 | 
			
		||||
		),
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -98,16 +122,8 @@ func SimulateMsgDeposit(ak types.AccountKeeper, k keeper.Keeper) simulation.Oper
 | 
			
		||||
		}
 | 
			
		||||
		slippage := slippageRaw.ToDec().Quo(sdk.NewDec(100))
 | 
			
		||||
 | 
			
		||||
		// Set up deadline
 | 
			
		||||
		durationNanoseconds, err := RandIntInclusive(r,
 | 
			
		||||
			sdk.NewInt((time.Second * 10).Nanoseconds()), // ten seconds
 | 
			
		||||
			sdk.NewInt((time.Hour * 24).Nanoseconds()),   // one day
 | 
			
		||||
		)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			panic(err)
 | 
			
		||||
		}
 | 
			
		||||
		extraTime := time.Duration(durationNanoseconds.Int64())
 | 
			
		||||
		deadline := blockTime.Add(extraTime).Unix()
 | 
			
		||||
		// Generate random deadline
 | 
			
		||||
		deadline := genRandDeadline(r, blockTime)
 | 
			
		||||
 | 
			
		||||
		depositorAcc := ak.GetAccount(ctx, depositor.Address)
 | 
			
		||||
		depositorCoins := depositorAcc.SpendableCoins(blockTime)
 | 
			
		||||
@ -201,8 +217,8 @@ func SimulateMsgWithdraw(ak types.AccountKeeper, k keeper.Keeper) simulation.Ope
 | 
			
		||||
		}
 | 
			
		||||
		coinsOwned := denominatedPool.ShareValue(shareRecord.SharesOwned)
 | 
			
		||||
 | 
			
		||||
		// Get random amount of shares between 2-50% of the total
 | 
			
		||||
		sharePercentage, err := RandIntInclusive(r, sdk.NewInt(2), sdk.NewInt(50))
 | 
			
		||||
		// Get random amount of shares between 2-10% of the total
 | 
			
		||||
		sharePercentage, err := RandIntInclusive(r, sdk.NewInt(2), sdk.NewInt(10))
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			panic(err)
 | 
			
		||||
		}
 | 
			
		||||
@ -219,17 +235,9 @@ func SimulateMsgWithdraw(ak types.AccountKeeper, k keeper.Keeper) simulation.Ope
 | 
			
		||||
		minTokenAmtB := amtTokenBOwned.Mul(oneLessThanSharePercentage).Quo(sdk.NewInt(100))
 | 
			
		||||
		minTokenB := sdk.NewCoin(poolRecord.ReservesB.Denom, minTokenAmtB)
 | 
			
		||||
 | 
			
		||||
		// Set up deadline
 | 
			
		||||
		// Generate random deadline
 | 
			
		||||
		blockTime := ctx.BlockHeader().Time
 | 
			
		||||
		durationNanoseconds, err := RandIntInclusive(r,
 | 
			
		||||
			sdk.NewInt((time.Second * 10).Nanoseconds()), // ten seconds
 | 
			
		||||
			sdk.NewInt((time.Hour * 24).Nanoseconds()),   // one day
 | 
			
		||||
		)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			panic(err)
 | 
			
		||||
		}
 | 
			
		||||
		extraTime := time.Duration(durationNanoseconds.Int64())
 | 
			
		||||
		deadline := blockTime.Add(extraTime).Unix()
 | 
			
		||||
		deadline := genRandDeadline(r, blockTime)
 | 
			
		||||
 | 
			
		||||
		// Construct MsgWithdraw
 | 
			
		||||
		msg := types.NewMsgWithdraw(withdrawerAcc.GetAddress(), shares, minTokenA, minTokenB, deadline)
 | 
			
		||||
@ -258,6 +266,204 @@ func SimulateMsgWithdraw(ak types.AccountKeeper, k keeper.Keeper) simulation.Ope
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// SimulateMsgSwapExactForTokens generates a MsgSwapExactForTokens
 | 
			
		||||
func SimulateMsgSwapExactForTokens(ak types.AccountKeeper, k keeper.Keeper) simulation.Operation {
 | 
			
		||||
	return func(
 | 
			
		||||
		r *rand.Rand, app *baseapp.BaseApp, ctx sdk.Context, accs []simulation.Account, chainID string,
 | 
			
		||||
	) (simulation.OperationMsg, []simulation.FutureOperation, error) {
 | 
			
		||||
 | 
			
		||||
		poolRecords := k.GetAllPools(ctx)
 | 
			
		||||
		r.Shuffle(len(poolRecords), func(i, j int) {
 | 
			
		||||
			poolRecords[i], poolRecords[j] = poolRecords[j], poolRecords[i]
 | 
			
		||||
		})
 | 
			
		||||
 | 
			
		||||
		// Find an account-pool pair for which trade is possible
 | 
			
		||||
		trader, poolRecord, found := findValidAccountPoolRecordPair(accs, poolRecords, func(acc simulation.Account, poolRecord types.PoolRecord) bool {
 | 
			
		||||
			traderAcc := ak.GetAccount(ctx, acc.Address)
 | 
			
		||||
			balanceTokenA := traderAcc.GetCoins().AmountOf(poolRecord.ReservesA.Denom)
 | 
			
		||||
			balanceTokenB := traderAcc.GetCoins().AmountOf(poolRecord.ReservesB.Denom)
 | 
			
		||||
			if !balanceTokenA.IsPositive() || !balanceTokenB.IsPositive() {
 | 
			
		||||
				return false
 | 
			
		||||
			}
 | 
			
		||||
			return true
 | 
			
		||||
		})
 | 
			
		||||
		if !found {
 | 
			
		||||
			return simulation.NewOperationMsgBasic(types.ModuleName, "no-operation (no valid pool record and trader)", "", false, nil), nil, nil
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Select input token
 | 
			
		||||
		randInt, err := RandInt(r, sdk.OneInt(), sdk.NewInt(9))
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			panic(err)
 | 
			
		||||
		}
 | 
			
		||||
		inputToken := poolRecord.ReservesA
 | 
			
		||||
		outputToken := poolRecord.ReservesB
 | 
			
		||||
		if randInt.Int64()%2 == 0 {
 | 
			
		||||
			inputToken = poolRecord.ReservesB
 | 
			
		||||
			outputToken = poolRecord.ReservesA
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Select entity (trader account or pool) with smaller token amount
 | 
			
		||||
		traderAcc := ak.GetAccount(ctx, trader.Address)
 | 
			
		||||
		maxTradeAmount := inputToken.Amount
 | 
			
		||||
		if traderAcc.GetCoins().AmountOf(inputToken.Denom).LT(inputToken.Amount) {
 | 
			
		||||
			maxTradeAmount = traderAcc.GetCoins().AmountOf(inputToken.Denom)
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Exact input token is between 2-10% of the max trade amount
 | 
			
		||||
		percentage, err := RandIntInclusive(r, sdk.NewInt(2), sdk.NewInt(10))
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			panic(err)
 | 
			
		||||
		}
 | 
			
		||||
		tradeAmount := maxTradeAmount.Mul(percentage).Quo(sdk.NewInt(100))
 | 
			
		||||
		exactInputToken := sdk.NewCoin(inputToken.Denom, tradeAmount)
 | 
			
		||||
 | 
			
		||||
		// Calculate expected output coin
 | 
			
		||||
		globalSwapFee := k.GetSwapFee(ctx)
 | 
			
		||||
		tradeAmountAfterFee := exactInputToken.Amount.ToDec().Mul(sdk.OneDec().Sub(globalSwapFee)).TruncateInt()
 | 
			
		||||
 | 
			
		||||
		var outputAmt big.Int
 | 
			
		||||
		outputAmt.Mul(outputToken.Amount.BigInt(), tradeAmountAfterFee.BigInt())
 | 
			
		||||
		outputAmt.Quo(&outputAmt, inputToken.Amount.Add(tradeAmountAfterFee).BigInt())
 | 
			
		||||
		expectedOutTokenAmount := sdk.NewIntFromBigInt(&outputAmt)
 | 
			
		||||
		expectedOutputToken := sdk.NewCoin(outputToken.Denom, expectedOutTokenAmount)
 | 
			
		||||
 | 
			
		||||
		// Get random slippage amount between 50-100%
 | 
			
		||||
		slippageRaw, err := RandIntInclusive(r, sdk.NewInt(50), sdk.NewInt(99))
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			panic(err)
 | 
			
		||||
		}
 | 
			
		||||
		slippage := slippageRaw.ToDec().Quo(sdk.NewDec(100))
 | 
			
		||||
 | 
			
		||||
		// Generate random deadline
 | 
			
		||||
		blockTime := ctx.BlockHeader().Time
 | 
			
		||||
		deadline := genRandDeadline(r, blockTime)
 | 
			
		||||
 | 
			
		||||
		// Construct MsgSwapExactForTokens
 | 
			
		||||
		msg := types.NewMsgSwapExactForTokens(traderAcc.GetAddress(), exactInputToken, expectedOutputToken, slippage, deadline)
 | 
			
		||||
		err = msg.ValidateBasic()
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return noOpMsg, nil, nil
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		tx := helpers.GenTx(
 | 
			
		||||
			[]sdk.Msg{msg},
 | 
			
		||||
			sdk.NewCoins(),
 | 
			
		||||
			helpers.DefaultGenTxGas,
 | 
			
		||||
			chainID,
 | 
			
		||||
			[]uint64{traderAcc.GetAccountNumber()},
 | 
			
		||||
			[]uint64{traderAcc.GetSequence()},
 | 
			
		||||
			trader.PrivKey,
 | 
			
		||||
		)
 | 
			
		||||
 | 
			
		||||
		_, result, err := app.Deliver(tx)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			// to aid debugging, add the stack trace to the comment field of the returned opMsg
 | 
			
		||||
			return simulation.NewOperationMsg(msg, false, fmt.Sprintf("%+v", err)), nil, err
 | 
			
		||||
		}
 | 
			
		||||
		return simulation.NewOperationMsg(msg, true, result.Log), nil, nil
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// SimulateMsgSwapForExactTokens generates a MsgSwapForExactTokens
 | 
			
		||||
func SimulateMsgSwapForExactTokens(ak types.AccountKeeper, k keeper.Keeper) simulation.Operation {
 | 
			
		||||
	return func(
 | 
			
		||||
		r *rand.Rand, app *baseapp.BaseApp, ctx sdk.Context, accs []simulation.Account, chainID string,
 | 
			
		||||
	) (simulation.OperationMsg, []simulation.FutureOperation, error) {
 | 
			
		||||
 | 
			
		||||
		poolRecords := k.GetAllPools(ctx)
 | 
			
		||||
		r.Shuffle(len(poolRecords), func(i, j int) {
 | 
			
		||||
			poolRecords[i], poolRecords[j] = poolRecords[j], poolRecords[i]
 | 
			
		||||
		})
 | 
			
		||||
 | 
			
		||||
		// Find an account-pool pair for which trade is possible
 | 
			
		||||
		trader, poolRecord, found := findValidAccountPoolRecordPair(accs, poolRecords, func(acc simulation.Account, poolRecord types.PoolRecord) bool {
 | 
			
		||||
			traderAcc := ak.GetAccount(ctx, acc.Address)
 | 
			
		||||
			balanceTokenA := traderAcc.GetCoins().AmountOf(poolRecord.ReservesA.Denom)
 | 
			
		||||
			balanceTokenB := traderAcc.GetCoins().AmountOf(poolRecord.ReservesB.Denom)
 | 
			
		||||
			if !balanceTokenA.IsPositive() || !balanceTokenB.IsPositive() {
 | 
			
		||||
				return false
 | 
			
		||||
			}
 | 
			
		||||
			return true
 | 
			
		||||
		})
 | 
			
		||||
		if !found {
 | 
			
		||||
			return simulation.NewOperationMsgBasic(types.ModuleName, "no-operation (no valid pool record and trader)", "", false, nil), nil, nil
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Select input token
 | 
			
		||||
		randInt, err := RandInt(r, sdk.OneInt(), sdk.NewInt(9))
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			panic(err)
 | 
			
		||||
		}
 | 
			
		||||
		inputToken := poolRecord.ReservesA
 | 
			
		||||
		outputToken := poolRecord.ReservesB
 | 
			
		||||
		if randInt.Int64()%2 == 0 {
 | 
			
		||||
			inputToken = poolRecord.ReservesB
 | 
			
		||||
			outputToken = poolRecord.ReservesA
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Select entity (trader account or pool) with smaller token amount
 | 
			
		||||
		traderAcc := ak.GetAccount(ctx, trader.Address)
 | 
			
		||||
		maxTradeAmount := inputToken.Amount
 | 
			
		||||
		if traderAcc.GetCoins().AmountOf(inputToken.Denom).LT(inputToken.Amount) {
 | 
			
		||||
			maxTradeAmount = traderAcc.GetCoins().AmountOf(inputToken.Denom)
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Expected input token is between 2-10% of the max trade amount
 | 
			
		||||
		percentage, err := RandIntInclusive(r, sdk.NewInt(2), sdk.NewInt(10))
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			panic(err)
 | 
			
		||||
		}
 | 
			
		||||
		tradeAmount := maxTradeAmount.Mul(percentage).Quo(sdk.NewInt(100))
 | 
			
		||||
		expectedInputToken := sdk.NewCoin(inputToken.Denom, tradeAmount)
 | 
			
		||||
 | 
			
		||||
		// Calculate exact output coin
 | 
			
		||||
		globalSwapFee := k.GetSwapFee(ctx)
 | 
			
		||||
		tradeAmountAfterFee := expectedInputToken.Amount.ToDec().Mul(sdk.OneDec().Sub(globalSwapFee)).TruncateInt()
 | 
			
		||||
 | 
			
		||||
		var outputAmt big.Int
 | 
			
		||||
		outputAmt.Mul(outputToken.Amount.BigInt(), tradeAmountAfterFee.BigInt())
 | 
			
		||||
		outputAmt.Quo(&outputAmt, inputToken.Amount.Add(tradeAmountAfterFee).BigInt())
 | 
			
		||||
		outputTokenAmount := sdk.NewIntFromBigInt(&outputAmt)
 | 
			
		||||
		exactOutputToken := sdk.NewCoin(outputToken.Denom, outputTokenAmount)
 | 
			
		||||
 | 
			
		||||
		// Get random slippage amount between 50-100%
 | 
			
		||||
		slippageRaw, err := RandIntInclusive(r, sdk.NewInt(50), sdk.NewInt(99))
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			panic(err)
 | 
			
		||||
		}
 | 
			
		||||
		slippage := slippageRaw.ToDec().Quo(sdk.NewDec(100))
 | 
			
		||||
 | 
			
		||||
		// Generate random deadline
 | 
			
		||||
		blockTime := ctx.BlockHeader().Time
 | 
			
		||||
		deadline := genRandDeadline(r, blockTime)
 | 
			
		||||
 | 
			
		||||
		// Construct MsgSwapForExactTokens
 | 
			
		||||
		msg := types.NewMsgSwapForExactTokens(traderAcc.GetAddress(), expectedInputToken, exactOutputToken, slippage, deadline)
 | 
			
		||||
		err = msg.ValidateBasic()
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return noOpMsg, nil, nil
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		tx := helpers.GenTx(
 | 
			
		||||
			[]sdk.Msg{msg},
 | 
			
		||||
			sdk.NewCoins(),
 | 
			
		||||
			helpers.DefaultGenTxGas,
 | 
			
		||||
			chainID,
 | 
			
		||||
			[]uint64{traderAcc.GetAccountNumber()},
 | 
			
		||||
			[]uint64{traderAcc.GetSequence()},
 | 
			
		||||
			trader.PrivKey,
 | 
			
		||||
		)
 | 
			
		||||
 | 
			
		||||
		_, result, err := app.Deliver(tx)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			// to aid debugging, add the stack trace to the comment field of the returned opMsg
 | 
			
		||||
			return simulation.NewOperationMsg(msg, false, fmt.Sprintf("%+v", err)), nil, err
 | 
			
		||||
		}
 | 
			
		||||
		return simulation.NewOperationMsg(msg, true, result.Log), nil, nil
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// From a set of coins return a coin of the specified denom with 1-10% of the total amount
 | 
			
		||||
func randCoinFromCoins(r *rand.Rand, coins sdk.Coins, denom string) sdk.Coin {
 | 
			
		||||
	percentOfBalance, err := RandIntInclusive(r, sdk.OneInt(), sdk.NewInt(10))
 | 
			
		||||
@ -309,6 +515,19 @@ func findValidAccountPoolRecordPair(accounts []simulation.Account, pools types.P
 | 
			
		||||
	return simulation.Account{}, types.PoolRecord{}, false
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func genRandDeadline(r *rand.Rand, blockTime time.Time) int64 {
 | 
			
		||||
	// Set up deadline
 | 
			
		||||
	durationNanoseconds, err := RandIntInclusive(r,
 | 
			
		||||
		sdk.NewInt((time.Second * 10).Nanoseconds()), // ten seconds
 | 
			
		||||
		sdk.NewInt((time.Hour * 24).Nanoseconds()),   // one day
 | 
			
		||||
	)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		panic(err)
 | 
			
		||||
	}
 | 
			
		||||
	extraTime := time.Duration(durationNanoseconds.Int64())
 | 
			
		||||
	return blockTime.Add(extraTime).Unix()
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// RandIntInclusive randomly generates an sdk.Int in the range [inclusiveMin, inclusiveMax]. It works for negative and positive integers.
 | 
			
		||||
func RandIntInclusive(r *rand.Rand, inclusiveMin, inclusiveMax sdk.Int) (sdk.Int, error) {
 | 
			
		||||
	if inclusiveMin.GT(inclusiveMax) {
 | 
			
		||||
 | 
			
		||||
		Loading…
	
		Reference in New Issue
	
	Block a user