Refactor x/staking Unbonding Validator Queue (#6844)
* init commit * revise GetValidatorQueueKey * add order test * update UnbondAllMatureValidators * update APIs * fix build * cl++
This commit is contained in:
parent
fd6beba238
commit
9a3fd7cac5
|
@ -200,6 +200,7 @@ invalid or incomplete requests.
|
|||
|
||||
### State Machine Breaking
|
||||
|
||||
* (x/staking) [\#6844](https://github.com/cosmos/cosmos-sdk/pull/6844) Validators are now inserted into the unbonding queue based on their unbonding time and height. The relevant keeper APIs are modified to reflect these changes by now also requiring a height.
|
||||
* (x/bank) [\#6518](https://github.com/cosmos/cosmos-sdk/pull/6518) Support for global and per-denomination send enabled flags.
|
||||
* Existing send_enabled global flag has been moved into a Params structure as `default_send_enabled`.
|
||||
* An array of: `{denom: string, enabled: bool}` is added to bank Params to support per-denomination override of global default value.
|
||||
|
|
|
@ -48,7 +48,7 @@ func InitGenesis(
|
|||
|
||||
// update timeslice if necessary
|
||||
if validator.IsUnbonding() {
|
||||
keeper.InsertValidatorQueue(ctx, validator)
|
||||
keeper.InsertUnbondingValidatorQueue(ctx, validator)
|
||||
}
|
||||
|
||||
switch validator.GetStatus() {
|
||||
|
|
|
@ -528,7 +528,7 @@ func TestUndelegateFromUnbondedValidator(t *testing.T) {
|
|||
|
||||
// unbond the validator
|
||||
ctx = ctx.WithBlockTime(validator.UnbondingTime)
|
||||
app.StakingKeeper.UnbondAllMatureValidatorQueue(ctx)
|
||||
app.StakingKeeper.UnbondAllMatureValidators(ctx)
|
||||
|
||||
// Make sure validator is still in state because there is still an outstanding delegation
|
||||
validator, found = app.StakingKeeper.GetValidator(ctx, addrVals[0])
|
||||
|
@ -619,7 +619,7 @@ func TestUnbondingAllDelegationFromValidator(t *testing.T) {
|
|||
|
||||
// unbond the validator
|
||||
ctx = ctx.WithBlockTime(validator.UnbondingTime)
|
||||
app.StakingKeeper.UnbondAllMatureValidatorQueue(ctx)
|
||||
app.StakingKeeper.UnbondAllMatureValidators(ctx)
|
||||
|
||||
// validator should now be deleted from state
|
||||
_, found = app.StakingKeeper.GetValidator(ctx, addrVals[0])
|
||||
|
|
|
@ -26,8 +26,8 @@ func (k Keeper) BlockValidatorUpdates(ctx sdk.Context) []abci.ValidatorUpdate {
|
|||
// UnbondAllMatureValidatorQueue).
|
||||
validatorUpdates := k.ApplyAndReturnValidatorSetUpdates(ctx)
|
||||
|
||||
// Unbond all mature validators from the unbonding queue.
|
||||
k.UnbondAllMatureValidatorQueue(ctx)
|
||||
// unbond all mature validators from the unbonding queue
|
||||
k.UnbondAllMatureValidators(ctx)
|
||||
|
||||
// Remove all mature unbonding delegations from the ubd queue.
|
||||
matureUnbonds := k.DequeueAllMatureUBDQueue(ctx, ctx.BlockHeader().Time)
|
||||
|
@ -283,7 +283,7 @@ func (k Keeper) beginUnbondingValidator(ctx sdk.Context, validator types.Validat
|
|||
k.SetValidatorByPowerIndex(ctx, validator)
|
||||
|
||||
// Adds to unbonding validator queue
|
||||
k.InsertValidatorQueue(ctx, validator)
|
||||
k.InsertUnbondingValidatorQueue(ctx, validator)
|
||||
|
||||
// trigger hook
|
||||
k.AfterValidatorBeginUnbonding(ctx, validator.GetConsAddr(), validator.OperatorAddress)
|
||||
|
|
|
@ -358,112 +358,118 @@ func (k Keeper) GetLastValidators(ctx sdk.Context) (validators []types.Validator
|
|||
return validators[:i] // trim
|
||||
}
|
||||
|
||||
//_______________________________________________________________________
|
||||
// Validator Queue
|
||||
|
||||
// gets a specific validator queue timeslice. A timeslice is a slice of ValAddresses corresponding to unbonding validators
|
||||
// that expire at a certain time.
|
||||
func (k Keeper) GetValidatorQueueTimeSlice(ctx sdk.Context, timestamp time.Time) []sdk.ValAddress {
|
||||
// GetUnbondingValidators returns a slice of mature validator addresses that
|
||||
// complete their unbonding at a given time and height.
|
||||
func (k Keeper) GetUnbondingValidators(ctx sdk.Context, endTime time.Time, endHeight int64) []sdk.ValAddress {
|
||||
store := ctx.KVStore(k.storeKey)
|
||||
|
||||
bz := store.Get(types.GetValidatorQueueTimeKey(timestamp))
|
||||
bz := store.Get(types.GetValidatorQueueKey(endTime, endHeight))
|
||||
if bz == nil {
|
||||
return []sdk.ValAddress{}
|
||||
}
|
||||
|
||||
va := sdk.ValAddresses{}
|
||||
k.cdc.MustUnmarshalBinaryBare(bz, &va)
|
||||
addrs := sdk.ValAddresses{}
|
||||
k.cdc.MustUnmarshalBinaryBare(bz, &addrs)
|
||||
|
||||
return va.Addresses
|
||||
return addrs.Addresses
|
||||
}
|
||||
|
||||
// Sets a specific validator queue timeslice.
|
||||
func (k Keeper) SetValidatorQueueTimeSlice(ctx sdk.Context, timestamp time.Time, keys []sdk.ValAddress) {
|
||||
// SetUnbondingValidatorsQueue sets a given slice of validator addresses into
|
||||
// the unbonding validator queue by a given height and time.
|
||||
func (k Keeper) SetUnbondingValidatorsQueue(ctx sdk.Context, endTime time.Time, endHeight int64, addrs []sdk.ValAddress) {
|
||||
store := ctx.KVStore(k.storeKey)
|
||||
bz := k.cdc.MustMarshalBinaryBare(&sdk.ValAddresses{Addresses: keys})
|
||||
store.Set(types.GetValidatorQueueTimeKey(timestamp), bz)
|
||||
bz := k.cdc.MustMarshalBinaryBare(&sdk.ValAddresses{Addresses: addrs})
|
||||
store.Set(types.GetValidatorQueueKey(endTime, endHeight), bz)
|
||||
}
|
||||
|
||||
// Deletes a specific validator queue timeslice.
|
||||
func (k Keeper) DeleteValidatorQueueTimeSlice(ctx sdk.Context, timestamp time.Time) {
|
||||
// InsertUnbondingValidatorQueue inserts a given unbonding validator address into
|
||||
// the unbonding validator queue for a given height and time.
|
||||
func (k Keeper) InsertUnbondingValidatorQueue(ctx sdk.Context, val types.Validator) {
|
||||
addrs := k.GetUnbondingValidators(ctx, val.UnbondingTime, val.UnbondingHeight)
|
||||
addrs = append(addrs, val.OperatorAddress)
|
||||
k.SetUnbondingValidatorsQueue(ctx, val.UnbondingTime, val.UnbondingHeight, addrs)
|
||||
}
|
||||
|
||||
// DeleteValidatorQueueTimeSlice deletes all entries in the queue indexed by a
|
||||
// given height and time.
|
||||
func (k Keeper) DeleteValidatorQueueTimeSlice(ctx sdk.Context, endTime time.Time, endHeight int64) {
|
||||
store := ctx.KVStore(k.storeKey)
|
||||
store.Delete(types.GetValidatorQueueTimeKey(timestamp))
|
||||
store.Delete(types.GetValidatorQueueKey(endTime, endHeight))
|
||||
}
|
||||
|
||||
// Insert an validator address to the appropriate timeslice in the validator queue
|
||||
func (k Keeper) InsertValidatorQueue(ctx sdk.Context, val types.Validator) {
|
||||
timeSlice := k.GetValidatorQueueTimeSlice(ctx, val.UnbondingTime)
|
||||
timeSlice = append(timeSlice, val.OperatorAddress)
|
||||
k.SetValidatorQueueTimeSlice(ctx, val.UnbondingTime, timeSlice)
|
||||
}
|
||||
|
||||
// Delete a validator address from the validator queue
|
||||
// DeleteValidatorQueue removes a validator by address from the unbonding queue
|
||||
// indexed by a given height and time.
|
||||
func (k Keeper) DeleteValidatorQueue(ctx sdk.Context, val types.Validator) {
|
||||
timeSlice := k.GetValidatorQueueTimeSlice(ctx, val.UnbondingTime)
|
||||
newTimeSlice := []sdk.ValAddress{}
|
||||
addrs := k.GetUnbondingValidators(ctx, val.UnbondingTime, val.UnbondingHeight)
|
||||
newAddrs := []sdk.ValAddress{}
|
||||
|
||||
for _, addr := range timeSlice {
|
||||
for _, addr := range addrs {
|
||||
if !bytes.Equal(addr, val.OperatorAddress) {
|
||||
newTimeSlice = append(newTimeSlice, addr)
|
||||
newAddrs = append(newAddrs, addr)
|
||||
}
|
||||
}
|
||||
|
||||
if len(newTimeSlice) == 0 {
|
||||
k.DeleteValidatorQueueTimeSlice(ctx, val.UnbondingTime)
|
||||
if len(newAddrs) == 0 {
|
||||
k.DeleteValidatorQueueTimeSlice(ctx, val.UnbondingTime, val.UnbondingHeight)
|
||||
} else {
|
||||
k.SetValidatorQueueTimeSlice(ctx, val.UnbondingTime, newTimeSlice)
|
||||
k.SetUnbondingValidatorsQueue(ctx, val.UnbondingTime, val.UnbondingHeight, newAddrs)
|
||||
}
|
||||
}
|
||||
|
||||
// Returns all the validator queue timeslices from time 0 until endTime
|
||||
func (k Keeper) ValidatorQueueIterator(ctx sdk.Context, endTime time.Time) sdk.Iterator {
|
||||
// ValidatorQueueIterator returns an interator ranging over validators that are
|
||||
// unbonding whose unbonding completion occurs at the given height and time.
|
||||
func (k Keeper) ValidatorQueueIterator(ctx sdk.Context, endTime time.Time, endHeight int64) sdk.Iterator {
|
||||
store := ctx.KVStore(k.storeKey)
|
||||
return store.Iterator(types.ValidatorQueueKey, sdk.InclusiveEndBytes(types.GetValidatorQueueTimeKey(endTime)))
|
||||
return store.Iterator(types.ValidatorQueueKey, sdk.InclusiveEndBytes(types.GetValidatorQueueKey(endTime, endHeight)))
|
||||
}
|
||||
|
||||
// Returns a concatenated list of all the timeslices before currTime, and deletes the timeslices from the queue
|
||||
func (k Keeper) GetAllMatureValidatorQueue(ctx sdk.Context, currTime time.Time) (matureValsAddrs []sdk.ValAddress) {
|
||||
// gets an iterator for all timeslices from time 0 until the current Blockheader time
|
||||
validatorTimesliceIterator := k.ValidatorQueueIterator(ctx, ctx.BlockHeader().Time)
|
||||
defer validatorTimesliceIterator.Close()
|
||||
|
||||
for ; validatorTimesliceIterator.Valid(); validatorTimesliceIterator.Next() {
|
||||
timeslice := sdk.ValAddresses{}
|
||||
k.cdc.MustUnmarshalBinaryBare(validatorTimesliceIterator.Value(), ×lice)
|
||||
|
||||
matureValsAddrs = append(matureValsAddrs, timeslice.Addresses...)
|
||||
}
|
||||
|
||||
return matureValsAddrs
|
||||
}
|
||||
|
||||
// Unbonds all the unbonding validators that have finished their unbonding period
|
||||
func (k Keeper) UnbondAllMatureValidatorQueue(ctx sdk.Context) {
|
||||
// UnbondAllMatureValidators unbonds all the mature unbonding validators that
|
||||
// have finished their unbonding period.
|
||||
func (k Keeper) UnbondAllMatureValidators(ctx sdk.Context) {
|
||||
store := ctx.KVStore(k.storeKey)
|
||||
|
||||
validatorTimesliceIterator := k.ValidatorQueueIterator(ctx, ctx.BlockHeader().Time)
|
||||
defer validatorTimesliceIterator.Close()
|
||||
blockTime := ctx.BlockTime()
|
||||
blockHeight := ctx.BlockHeight()
|
||||
|
||||
for ; validatorTimesliceIterator.Valid(); validatorTimesliceIterator.Next() {
|
||||
timeslice := sdk.ValAddresses{}
|
||||
k.cdc.MustUnmarshalBinaryBare(validatorTimesliceIterator.Value(), ×lice)
|
||||
// unbondingValIterator will contains all validator addresses indexed under
|
||||
// the ValidatorQueueKey prefix. Note, the entire index key is composed as
|
||||
// ValidatorQueueKey | timeBzLen (8-byte big endian) | timeBz | heightBz (8-byte big endian),
|
||||
// so it may be possible that certain validator addresses that are iterated
|
||||
// over are not ready to unbond, so an explicit check is required.
|
||||
unbondingValIterator := k.ValidatorQueueIterator(ctx, blockTime, blockHeight)
|
||||
defer unbondingValIterator.Close()
|
||||
|
||||
for _, valAddr := range timeslice.Addresses {
|
||||
val, found := k.GetValidator(ctx, valAddr)
|
||||
if !found {
|
||||
panic("validator in the unbonding queue was not found")
|
||||
}
|
||||
|
||||
if !val.IsUnbonding() {
|
||||
panic("unexpected validator in unbonding queue; status was not unbonding")
|
||||
}
|
||||
|
||||
val = k.UnbondingToUnbonded(ctx, val)
|
||||
if val.GetDelegatorShares().IsZero() {
|
||||
k.RemoveValidator(ctx, val.OperatorAddress)
|
||||
}
|
||||
for ; unbondingValIterator.Valid(); unbondingValIterator.Next() {
|
||||
key := unbondingValIterator.Key()
|
||||
keyTime, keyHeight, err := types.ParseValidatorQueueKey(key)
|
||||
if err != nil {
|
||||
panic(fmt.Errorf("failed to parse unbonding key: %w", err))
|
||||
}
|
||||
|
||||
store.Delete(validatorTimesliceIterator.Key())
|
||||
// All addresses for the given key have the same unbonding height and time.
|
||||
// We only unbond if the height and time are less than the current height
|
||||
// and time.
|
||||
if keyHeight <= blockHeight && (keyTime.Before(blockTime) || keyTime.Equal(blockTime)) {
|
||||
addrs := sdk.ValAddresses{}
|
||||
k.cdc.MustUnmarshalBinaryBare(unbondingValIterator.Value(), &addrs)
|
||||
|
||||
for _, valAddr := range addrs.Addresses {
|
||||
val, found := k.GetValidator(ctx, valAddr)
|
||||
if !found {
|
||||
panic("validator in the unbonding queue was not found")
|
||||
}
|
||||
|
||||
if !val.IsUnbonding() {
|
||||
panic("unexpected validator in unbonding queue; status was not unbonding")
|
||||
}
|
||||
|
||||
val = k.UnbondingToUnbonded(ctx, val)
|
||||
if val.GetDelegatorShares().IsZero() {
|
||||
k.RemoveValidator(ctx, val.OperatorAddress)
|
||||
}
|
||||
}
|
||||
|
||||
store.Delete(key)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,7 +1,9 @@
|
|||
package types
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
|
@ -120,13 +122,49 @@ func ParseValidatorPowerRankKey(key []byte) (operAddr []byte) {
|
|||
return operAddr
|
||||
}
|
||||
|
||||
// gets the prefix for all unbonding delegations from a delegator
|
||||
func GetValidatorQueueTimeKey(timestamp time.Time) []byte {
|
||||
bz := sdk.FormatTimeBytes(timestamp)
|
||||
return append(ValidatorQueueKey, bz...)
|
||||
// GetValidatorQueueKey returns the prefix key used for getting a set of unbonding
|
||||
// validators whose unbonding completion occurs at the given time and height.
|
||||
func GetValidatorQueueKey(timestamp time.Time, height int64) []byte {
|
||||
heightBz := sdk.Uint64ToBigEndian(uint64(height))
|
||||
timeBz := sdk.FormatTimeBytes(timestamp)
|
||||
timeBzL := len(timeBz)
|
||||
prefixL := len(ValidatorQueueKey)
|
||||
|
||||
bz := make([]byte, prefixL+8+timeBzL+8)
|
||||
|
||||
// copy the prefix
|
||||
copy(bz[:prefixL], ValidatorQueueKey)
|
||||
|
||||
// copy the encoded time bytes length
|
||||
copy(bz[prefixL:prefixL+8], sdk.Uint64ToBigEndian(uint64(timeBzL)))
|
||||
|
||||
// copy the encoded time bytes
|
||||
copy(bz[prefixL+8:prefixL+8+timeBzL], timeBz)
|
||||
|
||||
// copy the encoded height
|
||||
copy(bz[prefixL+8+timeBzL:], heightBz)
|
||||
|
||||
return bz
|
||||
}
|
||||
|
||||
//______________________________________________________________________________
|
||||
// ParseValidatorQueueKey returns the encoded time and height from a key created
|
||||
// from GetValidatorQueueKey.
|
||||
func ParseValidatorQueueKey(bz []byte) (time.Time, int64, error) {
|
||||
prefixL := len(ValidatorQueueKey)
|
||||
if prefix := bz[:prefixL]; !bytes.Equal(prefix, ValidatorQueueKey) {
|
||||
return time.Time{}, 0, fmt.Errorf("invalid prefix; expected: %X, got: %X", ValidatorQueueKey, prefix)
|
||||
}
|
||||
|
||||
timeBzL := sdk.BigEndianToUint64(bz[prefixL : prefixL+8])
|
||||
ts, err := sdk.ParseTimeBytes(bz[prefixL+8 : prefixL+8+int(timeBzL)])
|
||||
if err != nil {
|
||||
return time.Time{}, 0, err
|
||||
}
|
||||
|
||||
height := sdk.BigEndianToUint64(bz[prefixL+8+int(timeBzL):])
|
||||
|
||||
return ts, int64(height), nil
|
||||
}
|
||||
|
||||
// gets the key for delegator bond with validator
|
||||
// VALUE: staking/Delegation
|
||||
|
@ -139,8 +177,6 @@ func GetDelegationsKey(delAddr sdk.AccAddress) []byte {
|
|||
return append(DelegationKey, delAddr.Bytes()...)
|
||||
}
|
||||
|
||||
//______________________________________________________________________________
|
||||
|
||||
// gets the key for an unbonding delegation by delegator and validator addr
|
||||
// VALUE: staking/UnbondingDelegation
|
||||
func GetUBDKey(delAddr sdk.AccAddress, valAddr sdk.ValAddress) []byte {
|
||||
|
@ -168,8 +204,6 @@ func GetUBDKeyFromValIndexKey(indexKey []byte) []byte {
|
|||
return GetUBDKey(delAddr, valAddr)
|
||||
}
|
||||
|
||||
//______________
|
||||
|
||||
// gets the prefix for all unbonding delegations from a delegator
|
||||
func GetUBDsKey(delAddr sdk.AccAddress) []byte {
|
||||
return append(UnbondingDelegationKey, delAddr.Bytes()...)
|
||||
|
@ -186,10 +220,8 @@ func GetUnbondingDelegationTimeKey(timestamp time.Time) []byte {
|
|||
return append(UnbondingQueueKey, bz...)
|
||||
}
|
||||
|
||||
//________________________________________________________________________________
|
||||
|
||||
// gets the key for a redelegation
|
||||
// VALUE: staking/RedelegationKey
|
||||
// GetREDKey returns a key prefix for indexing a redelegation from a delegator
|
||||
// and source validator to a destination validator.
|
||||
func GetREDKey(delAddr sdk.AccAddress, valSrcAddr, valDstAddr sdk.ValAddress) []byte {
|
||||
key := make([]byte, 1+sdk.AddrLen*3)
|
||||
|
||||
|
@ -258,38 +290,38 @@ func GetREDKeyFromValDstIndexKey(indexKey []byte) []byte {
|
|||
return GetREDKey(delAddr, valSrcAddr, valDstAddr)
|
||||
}
|
||||
|
||||
// gets the prefix for all unbonding delegations from a delegator
|
||||
// GetRedelegationTimeKey returns a key prefix for indexing an unbonding
|
||||
// redelegation based on a completion time.
|
||||
func GetRedelegationTimeKey(timestamp time.Time) []byte {
|
||||
bz := sdk.FormatTimeBytes(timestamp)
|
||||
return append(RedelegationQueueKey, bz...)
|
||||
}
|
||||
|
||||
//______________
|
||||
|
||||
// gets the prefix keyspace for redelegations from a delegator
|
||||
// GetREDsKey returns a key prefix for indexing a redelegation from a delegator
|
||||
// address.
|
||||
func GetREDsKey(delAddr sdk.AccAddress) []byte {
|
||||
return append(RedelegationKey, delAddr.Bytes()...)
|
||||
}
|
||||
|
||||
// gets the prefix keyspace for all redelegations redelegating away from a source validator
|
||||
// GetREDsFromValSrcIndexKey returns a key prefix for indexing a redelegation to
|
||||
// a source validator.
|
||||
func GetREDsFromValSrcIndexKey(valSrcAddr sdk.ValAddress) []byte {
|
||||
return append(RedelegationByValSrcIndexKey, valSrcAddr.Bytes()...)
|
||||
}
|
||||
|
||||
// gets the prefix keyspace for all redelegations redelegating towards a destination validator
|
||||
// GetREDsToValDstIndexKey returns a key prefix for indexing a redelegation to a
|
||||
// destination (target) validator.
|
||||
func GetREDsToValDstIndexKey(valDstAddr sdk.ValAddress) []byte {
|
||||
return append(RedelegationByValDstIndexKey, valDstAddr.Bytes()...)
|
||||
}
|
||||
|
||||
// gets the prefix keyspace for all redelegations redelegating towards a destination validator
|
||||
// from a particular delegator
|
||||
// GetREDsByDelToValDstIndexKey returns a key prefix for indexing a redelegation
|
||||
// from an address to a source validator.
|
||||
func GetREDsByDelToValDstIndexKey(delAddr sdk.AccAddress, valDstAddr sdk.ValAddress) []byte {
|
||||
return append(GetREDsToValDstIndexKey(valDstAddr), delAddr.Bytes()...)
|
||||
}
|
||||
|
||||
//________________________________________________________________________________
|
||||
|
||||
// GetHistoricalInfoKey gets the key for the historical info
|
||||
// GetHistoricalInfoKey returns a key prefix for indexing HistoricalInfo objects.
|
||||
func GetHistoricalInfoKey(height int64) []byte {
|
||||
return append(HistoricalInfoKey, []byte(strconv.FormatInt(height, 10))...)
|
||||
}
|
||||
|
|
|
@ -1,11 +1,13 @@
|
|||
package types
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/hex"
|
||||
"math/big"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/tendermint/tendermint/crypto/ed25519"
|
||||
|
||||
sdk "github.com/cosmos/cosmos-sdk/types"
|
||||
|
@ -43,7 +45,7 @@ func TestGetValidatorPowerRank(t *testing.T) {
|
|||
for i, tt := range tests {
|
||||
got := hex.EncodeToString(getValidatorPowerRank(tt.validator))
|
||||
|
||||
assert.Equal(t, tt.wantHex, got, "Keys did not match on test case %d", i)
|
||||
require.Equal(t, tt.wantHex, got, "Keys did not match on test case %d", i)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -64,7 +66,7 @@ func TestGetREDByValDstIndexKey(t *testing.T) {
|
|||
for i, tt := range tests {
|
||||
got := hex.EncodeToString(GetREDByValDstIndexKey(tt.delAddr, tt.valSrcAddr, tt.valDstAddr))
|
||||
|
||||
assert.Equal(t, tt.wantHex, got, "Keys did not match on test case %d", i)
|
||||
require.Equal(t, tt.wantHex, got, "Keys did not match on test case %d", i)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -85,6 +87,32 @@ func TestGetREDByValSrcIndexKey(t *testing.T) {
|
|||
for i, tt := range tests {
|
||||
got := hex.EncodeToString(GetREDByValSrcIndexKey(tt.delAddr, tt.valSrcAddr, tt.valDstAddr))
|
||||
|
||||
assert.Equal(t, tt.wantHex, got, "Keys did not match on test case %d", i)
|
||||
require.Equal(t, tt.wantHex, got, "Keys did not match on test case %d", i)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGetValidatorQueueKey(t *testing.T) {
|
||||
ts := time.Now()
|
||||
height := int64(1024)
|
||||
|
||||
bz := GetValidatorQueueKey(ts, height)
|
||||
rTs, rHeight, err := ParseValidatorQueueKey(bz)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, ts.UTC(), rTs.UTC())
|
||||
require.Equal(t, rHeight, height)
|
||||
}
|
||||
|
||||
func TestTestGetValidatorQueueKeyOrder(t *testing.T) {
|
||||
ts := time.Now().UTC()
|
||||
height := int64(1000)
|
||||
|
||||
endKey := GetValidatorQueueKey(ts, height)
|
||||
|
||||
keyA := GetValidatorQueueKey(ts.Add(-10*time.Minute), height-10)
|
||||
keyB := GetValidatorQueueKey(ts.Add(-5*time.Minute), height+50)
|
||||
keyC := GetValidatorQueueKey(ts.Add(10*time.Minute), height+100)
|
||||
|
||||
require.Equal(t, -1, bytes.Compare(keyA, endKey)) // keyA <= endKey
|
||||
require.Equal(t, -1, bytes.Compare(keyB, endKey)) // keyB <= endKey
|
||||
require.Equal(t, 1, bytes.Compare(keyC, endKey)) // keyB >= endKey
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue