diff --git a/docs/quality_assurance.md b/docs/quality_assurance.md
index 333031c228..45691f7528 100644
--- a/docs/quality_assurance.md
+++ b/docs/quality_assurance.md
@@ -81,7 +81,7 @@ The main concern addressed in this section is the correctness of the provider ch
| 4.02 | Liveness of redelegations
- redelegations entries are eventually removed from `Redelegations` | `Scheduled` | `NA` | `Scheduled` | `Scheduled` | `Scheduled` | `NA` |
| 4.03 | Liveness of validator unbondings
- unbonding validators with no delegations are eventually removed from `Validators` | `Scheduled` | `NA` | `NA` | `Done` | `Scheduled` | `NA` |
| 4.04 | Unbonding operations (undelegations, redelegations, validator unbondings) should eventually complete even if the CCV channel is never established (due to error)
- expected outcome: the channel initialization sub-protocol eventually times out, which leads to the consumer chain removal | `Scheduled` | `NA` | `Done` [TestUndelegationDuringInit](../tests/e2e/unbonding_test.go#145) | `Future work` | `Scheduled` | `Done` |
-| 4.05 | Unbonding operations (undelegations, redelegations, validator unbondings) should eventually complete even if one of the clients expire
- expected outcome: the pending VSC packets eventually timeout, which leads to the consumer chain removal
- requires https://github.com/cosmos/interchain-security/issues/283 | `Scheduled` | `NA` | `Scheduled` | `Future work` | `Scheduled` | `NA` |
+| 4.05 | Unbonding operations (undelegations, redelegations, validator unbondings) should eventually complete even if one of the clients expire
- expected outcome: the pending VSC packets eventually timeout, which leads to the consumer chain removal | `Scheduled` | `NA` | `Done` [TestUndelegationVscTimeout](../tests/e2e/unbonding.go#127) | `Future work` | `Scheduled` | `NA` |
| 4.06 | A validator cannot get slashed more than once for double signing, regardless of how many times it double signs on different chains (consumers or provider) | `Scheduled` | `NA` |`Done`
[TestHandleSlashPacketErrors](../tests/e2e/slashing_test.go#L317) | `Done` | `Scheduled` | `NA` |
| 4.07 | A validator cannot get slashed multiple times for downtime on the same consumer chain without requesting to `Unjail` itself on the provider chain in between | `Scheduled` | `NA` | `Partial coverage`
[TestSendSlashPacket](../tests/e2e/slashing_test.go#L648) | `Partial coverage` | `Scheduled` | `NA` |
| 4.08 | A validator can be slashed multiple times for downtime on different chains | `Scheduled` | `NA` | `Future work` | `NA` | `Scheduled` | `NA` |
diff --git a/proto/interchain_security/ccv/provider/v1/provider.proto b/proto/interchain_security/ccv/provider/v1/provider.proto
index decc5e95d0..753d1a28e4 100644
--- a/proto/interchain_security/ccv/provider/v1/provider.proto
+++ b/proto/interchain_security/ccv/provider/v1/provider.proto
@@ -66,6 +66,12 @@ message Params {
// The channel initialization (IBC channel opening handshake) will timeout after this duration
google.protobuf.Duration init_timeout_period = 4
[(gogoproto.nullable) = false, (gogoproto.stdduration) = true];
+ // The VSC packets sent by the provider will timeout after this duration.
+ // Note that unlike ccv_timeout_period which is an IBC param,
+ // the vsc_timeout_period is a provider-side param that enables the provider
+ // to timeout VSC packets even when a consumer chain is not live.
+ google.protobuf.Duration vsc_timeout_period = 5
+ [(gogoproto.nullable) = false, (gogoproto.stdduration) = true];
}
message HandshakeMetadata {
diff --git a/tests/e2e/unbonding.go b/tests/e2e/unbonding.go
index d8df431c27..566a0a130e 100644
--- a/tests/e2e/unbonding.go
+++ b/tests/e2e/unbonding.go
@@ -9,102 +9,135 @@ import (
ccv "github.com/cosmos/interchain-security/x/ccv/types"
)
-// TestUndelegationProviderFirst checks that an unbonding operation completes
-// when the unbonding period elapses first on the provider chain
-func (s *CCVTestSuite) TestUndelegationProviderFirst() {
- s.SetupCCVChannel()
- s.SetupTransferChannel()
+// TestUndelegationNormalOperation tests that undelegations complete after
+// the unbonding period elapses on both the consumer and provider, without
+// VSC packets timing out.
+func (s *CCVTestSuite) TestUndelegationNormalOperation() {
+ unbondConsumer := func(expectedPackets int) {
+ // relay 1 VSC packet from provider to consumer
+ relayAllCommittedPackets(s, s.providerChain, s.path, ccv.ProviderPortID, s.path.EndpointB.ChannelID, expectedPackets)
+ // increment time so that the unbonding period ends on the consumer
+ incrementTimeByUnbondingPeriod(s, Consumer)
+ // relay 1 VSCMatured packet from consumer to provider
+ relayAllCommittedPackets(s, s.consumerChain, s.path, ccv.ConsumerPortID, s.path.EndpointA.ChannelID, expectedPackets)
+ }
- // delegate bondAmt and undelegate 1/2 of it
- bondAmt := sdk.NewInt(10000000)
- delAddr := s.providerChain.SenderAccount.GetAddress()
- initBalance, valsetUpdateID := delegateAndUndelegate(s, delAddr, bondAmt, 2)
- // - check that staking unbonding op was created and onHold is true
- checkStakingUnbondingOps(s, 1, true, true)
- // - check that CCV unbonding op was created
- checkCCVUnbondingOp(s, s.providerCtx(), s.consumerChain.ChainID, valsetUpdateID, true)
+ testCases := []struct {
+ name string
+ shareDiv int64
+ unbond func(expBalance, balance sdk.Int)
+ }{
+ {
+ "provider unbonding period elapses first", 2, func(expBalance, balance sdk.Int) {
+ // increment time so that the unbonding period ends on the provider
+ incrementTimeByUnbondingPeriod(s, Provider)
- // call NextBlock on the provider (which increments the height)
- s.providerChain.NextBlock()
+ // check that onHold is true
+ checkStakingUnbondingOps(s, 1, true, true, "unbonding should be on hold")
- // relay 1 VSC packet from provider to consumer
- relayAllCommittedPackets(s, s.providerChain, s.path, ccv.ProviderPortID, s.path.EndpointB.ChannelID, 1)
+ // check that the unbonding is not complete
+ s.Require().Equal(expBalance, balance, "unexpected balance after provider unbonding")
- // increment time so that the unbonding period ends on the provider
- incrementTimeByUnbondingPeriod(s, Provider)
+ // undelegation complete on consumer
+ unbondConsumer(1)
+ },
+ },
+ {
+ "consumer unbonding period elapses first", 2, func(expBalance, balance sdk.Int) {
+ // undelegation complete on consumer
+ unbondConsumer(1)
- // check that onHold is true
- checkStakingUnbondingOps(s, 1, true, true)
- // check that the unbonding is not complete
- s.Require().True(getBalance(s, s.providerCtx(), delAddr).Equal(initBalance.Sub(bondAmt)))
+ // check that onHold is false
+ checkStakingUnbondingOps(s, 1, true, false, "unbonding should be not be on hold")
- // increment time so that the unbonding period ends on the consumer
- incrementTimeByUnbondingPeriod(s, Consumer)
+ // check that the unbonding is not complete
+ s.Require().Equal(expBalance, balance, "unexpected balance after consumer unbonding")
- // relay 1 VSCMatured packet from consumer to provider
- relayAllCommittedPackets(s, s.consumerChain, s.path, ccv.ConsumerPortID, s.path.EndpointA.ChannelID, 1)
+ // increment time so that the unbonding period ends on the provider
+ incrementTimeByUnbondingPeriod(s, Provider)
+ },
+ },
+ {
+ "no valset changes", 1, func(expBalance, balance sdk.Int) {
+ // undelegation complete on consumer
+ unbondConsumer(1)
- // check that the unbonding operation completed
- // - check that ccv unbonding op has been deleted
- checkCCVUnbondingOp(s, s.providerCtx(), s.consumerChain.ChainID, valsetUpdateID, false)
- // - check that staking unbonding op has been deleted
- checkStakingUnbondingOps(s, valsetUpdateID, false, false)
- // - check that half the delegated coins have been returned
- s.Require().True(getBalance(s, s.providerCtx(), delAddr).Equal(initBalance.Sub(bondAmt.Quo(sdk.NewInt(2)))))
-}
+ // check that onHold is false
+ checkStakingUnbondingOps(s, 1, true, false, "unbonding should be not be on hold")
-// TestUndelegationConsumerFirst checks that an unbonding operation completes
-// when the unbonding period elapses first on the consumer chain
-func (s *CCVTestSuite) TestUndelegationConsumerFirst() {
- s.SetupCCVChannel()
- s.SetupTransferChannel()
+ // check that the unbonding is not complete
+ s.Require().Equal(expBalance, balance, "unexpected balance after consumer unbonding")
- // delegate bondAmt and undelegate 1/2 of it
- bondAmt := sdk.NewInt(10000000)
- delAddr := s.providerChain.SenderAccount.GetAddress()
- initBalance, valsetUpdateID := delegateAndUndelegate(s, delAddr, bondAmt, 2)
- // - check that staking unbonding op was created and onHold is true
- checkStakingUnbondingOps(s, 1, true, true)
- // - check that CCV unbonding op was created
- checkCCVUnbondingOp(s, s.providerCtx(), s.consumerChain.ChainID, valsetUpdateID, true)
+ // increment time so that the unbonding period ends on the provider
+ incrementTimeByUnbondingPeriod(s, Provider)
+ },
+ },
+ }
- // call NextBlock on the provider (which increments the height)
- s.providerChain.NextBlock()
+ for i, tc := range testCases {
+ providerKeeper := s.providerApp.GetProviderKeeper()
+ consumerKeeper := s.consumerApp.GetConsumerKeeper()
+ stakingKeeper := s.providerApp.GetE2eStakingKeeper()
- // relay 1 VSC packet from provider to consumer
- relayAllCommittedPackets(s, s.providerChain, s.path, ccv.ProviderPortID, s.path.EndpointB.ChannelID, 1)
+ s.SetupCCVChannel()
- // increment time so that the unbonding period ends on the consumer
- incrementTimeByUnbondingPeriod(s, Consumer)
+ // set VSC timeout period to not trigger the removal of the consumer chain
+ providerUnbondingPeriod := stakingKeeper.UnbondingTime(s.providerCtx())
+ consumerUnbondingPeriod := consumerKeeper.GetUnbondingPeriod(s.consumerCtx())
+ providerKeeper.SetVscTimeoutPeriod(s.providerCtx(), providerUnbondingPeriod+consumerUnbondingPeriod+24*time.Hour)
- // relay 1 VSCMatured packet from consumer to provider
- relayAllCommittedPackets(s, s.consumerChain, s.path, ccv.ConsumerPortID, s.path.EndpointA.ChannelID, 1)
+ // delegate bondAmt and undelegate tc.shareDiv of it
+ bondAmt := sdk.NewInt(10000000)
+ delAddr := s.providerChain.SenderAccount.GetAddress()
+ initBalance, valsetUpdateID := delegateAndUndelegate(s, delAddr, bondAmt, tc.shareDiv)
+ // - check that staking unbonding op was created and onHold is true
+ checkStakingUnbondingOps(s, 1, true, true, "test: "+tc.name)
+ // - check that CCV unbonding op was created
+ checkCCVUnbondingOp(s, s.providerCtx(), s.consumerChain.ChainID, valsetUpdateID, true, "test: "+tc.name)
- // check that the unbonding is not complete
- s.Require().True(getBalance(s, s.providerCtx(), delAddr).Equal(initBalance.Sub(bondAmt)))
+ // call NextBlock on the provider (which increments the height)
+ s.providerChain.NextBlock()
- // increment time so that the unbonding period ends on the provider
- incrementTimeByUnbondingPeriod(s, Provider)
+ // unbond both on provider and consumer and check that
+ // the balance remains unchanged in between
+ tc.unbond(initBalance.Sub(bondAmt), getBalance(s, s.providerCtx(), delAddr))
+
+ // check that the unbonding operation completed
+ // - check that ccv unbonding op has been deleted
+ checkCCVUnbondingOp(s, s.providerCtx(), s.consumerChain.ChainID, valsetUpdateID, false, "test: "+tc.name)
+ // - check that staking unbonding op has been deleted
+ checkStakingUnbondingOps(s, valsetUpdateID, false, false, "test: "+tc.name)
+ // - check that necessary delegated coins have been returned
+ unbondAmt := bondAmt.Sub(bondAmt.Quo(sdk.NewInt(tc.shareDiv)))
+ s.Require().Equal(
+ initBalance.Sub(unbondAmt),
+ getBalance(s, s.providerCtx(), delAddr),
+ "unexpected initial balance after unbonding; test: %s", tc.name,
+ )
- // check that the unbonding operation completed
- // - check that ccv unbonding op has been deleted
- checkCCVUnbondingOp(s, s.providerCtx(), s.consumerChain.ChainID, valsetUpdateID, false)
- // - check that staking unbonding op has been deleted
- checkStakingUnbondingOps(s, valsetUpdateID, false, false)
- // - check that half the delegated coins have been returned
- s.Require().True(getBalance(s, s.providerCtx(), delAddr).Equal(initBalance.Sub(bondAmt.Quo(sdk.NewInt(2)))))
+ if i+1 < len(testCases) {
+ // reset suite to reset provider client
+ s.SetupTest()
+ }
+ }
}
-// TestUndelegationNoValsetChange checks that an unbonding operation completes
-// even when the validator set is not changed
-func (s *CCVTestSuite) TestUndelegationNoValsetChange() {
+// TestUndelegationVscTimeout tests that an undelegation
+// completes after vscTimeoutPeriod even if it does not
+// reach maturity on the consumer chain. In this case,
+// the consumer chain is removed.
+func (s *CCVTestSuite) TestUndelegationVscTimeout() {
+ providerKeeper := s.providerApp.GetProviderKeeper()
+
s.SetupCCVChannel()
- s.SetupTransferChannel()
- // delegate bondAmt and undelegate all of it
+ // set VSC timeout period to trigger the removal of the consumer chain
+ vscTimeout := providerKeeper.GetVscTimeoutPeriod(s.providerCtx())
+
+ // delegate bondAmt and undelegate 1/2 of it
bondAmt := sdk.NewInt(10000000)
delAddr := s.providerChain.SenderAccount.GetAddress()
- initBalance, valsetUpdateID := delegateAndUndelegate(s, delAddr, bondAmt, 1)
+ initBalance, valsetUpdateID := delegateAndUndelegate(s, delAddr, bondAmt, 2)
// - check that staking unbonding op was created and onHold is true
checkStakingUnbondingOps(s, 1, true, true)
// - check that CCV unbonding op was created
@@ -113,28 +146,41 @@ func (s *CCVTestSuite) TestUndelegationNoValsetChange() {
// call NextBlock on the provider (which increments the height)
s.providerChain.NextBlock()
- // relay 1 VSC packet from provider to consumer
- relayAllCommittedPackets(s, s.providerChain, s.path, ccv.ProviderPortID, s.path.EndpointB.ChannelID, 1)
+ // increment time so that the unbonding period ends on the provider
+ incrementTimeByUnbondingPeriod(s, Provider)
+
+ // check that onHold is true
+ checkStakingUnbondingOps(s, 1, true, true, "unbonding should be on hold")
// check that the unbonding is not complete
- s.Require().True(getBalance(s, s.providerCtx(), delAddr).Equal(initBalance.Sub(bondAmt)))
+ s.Require().Equal(
+ initBalance.Sub(bondAmt),
+ getBalance(s, s.providerCtx(), delAddr),
+ "unexpected balance after provider unbonding")
- // increment time so that the unbonding period ends on the consumer
- incrementTimeByUnbondingPeriod(s, Consumer)
+ // increment time
+ incrementTimeBy(s, vscTimeout)
- // relay 1 VSCMatured packet from consumer to provider
- relayAllCommittedPackets(s, s.consumerChain, s.path, ccv.ConsumerPortID, s.path.EndpointA.ChannelID, 1)
+ // check whether the chain was removed
+ chainID := s.consumerChain.ChainID
+ _, found := providerKeeper.GetConsumerClientId(s.providerCtx(), chainID)
+ s.Require().Equal(false, found, "consumer chain was not removed")
- // increment time so that the unbonding period ends on the provider
- incrementTimeByUnbondingPeriod(s, Provider)
+ // check if the chain was properly removed
+ s.checkConsumerChainIsRemoved(chainID, false, true)
// check that the unbonding operation completed
// - check that ccv unbonding op has been deleted
checkCCVUnbondingOp(s, s.providerCtx(), s.consumerChain.ChainID, valsetUpdateID, false)
// - check that staking unbonding op has been deleted
checkStakingUnbondingOps(s, valsetUpdateID, false, false)
- // - check that all the delegated coins have been returned
- s.Require().True(getBalance(s, s.providerCtx(), delAddr).Equal(initBalance))
+ // - check that necessary delegated coins have been returned
+ unbondAmt := bondAmt.Sub(bondAmt.Quo(sdk.NewInt(2)))
+ s.Require().Equal(
+ initBalance.Sub(unbondAmt),
+ getBalance(s, s.providerCtx(), delAddr),
+ "unexpected initial balance after VSC timeout",
+ )
}
// TestUndelegationDuringInit checks that before the CCV channel is established
@@ -337,8 +383,14 @@ func (s *CCVTestSuite) TestRedelegationProviderFirst() {
s.SetupTransferChannel()
providerKeeper := s.providerApp.GetProviderKeeper()
+ consumerKeeper := s.consumerApp.GetConsumerKeeper()
stakingKeeper := s.providerApp.GetE2eStakingKeeper()
+ // set VSC timeout period to not trigger the removal of the consumer chain
+ providerUnbondingPeriod := stakingKeeper.UnbondingTime(s.providerCtx())
+ consumerUnbondingPeriod := consumerKeeper.GetUnbondingPeriod(s.consumerCtx())
+ providerKeeper.SetVscTimeoutPeriod(s.providerCtx(), providerUnbondingPeriod+consumerUnbondingPeriod+24*time.Hour)
+
// Setup delegator, bond amount, and src/dst validators
bondAmt := sdk.NewInt(10000000)
delAddr := s.providerChain.SenderAccount.GetAddress()
diff --git a/x/ccv/provider/keeper/keeper.go b/x/ccv/provider/keeper/keeper.go
index 6115f6c2b9..4d5ff66628 100644
--- a/x/ccv/provider/keeper/keeper.go
+++ b/x/ccv/provider/keeper/keeper.go
@@ -5,6 +5,7 @@ import (
"encoding/binary"
"encoding/json"
"fmt"
+ "time"
"github.com/cosmos/cosmos-sdk/codec"
sdk "github.com/cosmos/cosmos-sdk/types"
@@ -359,18 +360,15 @@ func (k Keeper) SetUnbondingOpIndex(ctx sdk.Context, chainID string, valsetUpdat
// IterateOverUnbondingOpIndex iterates over the unbonding indexes for a given chain id.
func (k Keeper) IterateOverUnbondingOpIndex(ctx sdk.Context, chainID string, cb func(vscID uint64, ubdIndex []uint64) bool) {
store := ctx.KVStore(k.storeKey)
- iterationPrefix := append([]byte{types.UnbondingOpIndexBytePrefix}, types.HashString(chainID)...)
- iterator := sdk.KVStorePrefixIterator(store, iterationPrefix)
-
+ iterator := sdk.KVStorePrefixIterator(store, types.ChainIdWithLenKey(types.UnbondingOpIndexBytePrefix, chainID))
defer iterator.Close()
+
for ; iterator.Valid(); iterator.Next() {
// parse key to get the current VSC ID
- var vscID uint64
- vscBytes, err := types.ParseUnbondingOpIndexKey(iterator.Key())
+ _, vscID, err := types.ParseUnbondingOpIndexKey(iterator.Key())
if err != nil {
- panic(err)
+ panic(fmt.Errorf("failed to parse UnbondingOpIndexKey: %w", err))
}
- vscID = binary.BigEndian.Uint64(vscBytes)
var index ccv.UnbondingOpsIndex
if err = index.Unmarshal(iterator.Value()); err != nil {
@@ -784,8 +782,6 @@ func (k Keeper) DeleteConsumerClientId(ctx sdk.Context, chainID string) {
store.Delete(types.ChainToClientKey(chainID))
}
-// ------
-
// SetInitTimeoutTimestamp sets the init timeout timestamp for the given chain ID
func (k Keeper) SetInitTimeoutTimestamp(ctx sdk.Context, chainID string, ts uint64) {
store := ctx.KVStore(k.storeKey)
@@ -825,3 +821,53 @@ func (k Keeper) IterateInitTimeoutTimestamp(ctx sdk.Context, cb func(chainID str
}
}
}
+
+// SetVscSendTimestamp sets the VSC send timestamp
+// for a VSCPacket with ID vscID sent to a chain with ID chainID
+func (k Keeper) SetVscSendTimestamp(
+ ctx sdk.Context,
+ chainID string,
+ vscID uint64,
+ timestamp time.Time,
+) {
+ store := ctx.KVStore(k.storeKey)
+
+ // Convert timestamp into bytes for storage
+ timeBz := sdk.FormatTimeBytes(timestamp)
+
+ store.Set(types.VscSendingTimestampKey(chainID, vscID), timeBz)
+}
+
+// DeleteVscSendTimestamp removes from the store a specific VSC send timestamp
+// for the given chainID and vscID.
+func (k Keeper) DeleteVscSendTimestamp(ctx sdk.Context, chainID string, vscID uint64) {
+ store := ctx.KVStore(k.storeKey)
+ store.Delete(types.VscSendingTimestampKey(chainID, vscID))
+}
+
+// IterateVscSendTimestamps iterates in order (lowest first)
+// over the vsc send timestamps of the given chainID.
+func (k Keeper) IterateVscSendTimestamps(
+ ctx sdk.Context,
+ chainID string,
+ cb func(vscID uint64, ts time.Time) bool,
+) {
+ store := ctx.KVStore(k.storeKey)
+ iterator := sdk.KVStorePrefixIterator(store, types.ChainIdWithLenKey(types.VscSendTimestampBytePrefix, chainID))
+ defer iterator.Close()
+
+ for ; iterator.Valid(); iterator.Next() {
+ key := iterator.Key()
+ _, vscID, err := types.ParseVscSendingTimestampKey(key)
+ if err != nil {
+ panic(fmt.Errorf("failed to parse VscSendTimestampKey: %w", err))
+ }
+ ts, err := sdk.ParseTimeBytes(iterator.Value())
+ if err != nil {
+ panic(fmt.Errorf("failed to parse timestamp value: %w", err))
+ }
+ if !cb(vscID, ts) {
+ return
+ }
+ }
+}
diff --git a/x/ccv/provider/keeper/keeper_test.go b/x/ccv/provider/keeper/keeper_test.go
index dd5e94a615..df91c5a4c4 100644
--- a/x/ccv/provider/keeper/keeper_test.go
+++ b/x/ccv/provider/keeper/keeper_test.go
@@ -2,6 +2,7 @@ package keeper_test
import (
"testing"
+ "time"
evidencetypes "github.com/cosmos/cosmos-sdk/x/evidence/types"
"github.com/golang/mock/gomock"
@@ -336,3 +337,60 @@ func TestInitTimeoutTimestamp(t *testing.T) {
_, found = providerKeeper.GetInitTimeoutTimestamp(ctx, tc[1].chainID)
require.False(t, found)
}
+
+// TestVscSendTimestamp tests the set, deletion, and iteration methods for VSC timeout timestamps
+func TestVscSendTimestamp(t *testing.T) {
+ providerKeeper, ctx, ctrl, _ := testkeeper.GetProviderKeeperAndCtx(t, testkeeper.NewInMemKeeperParams(t))
+ defer ctrl.Finish()
+
+ now := ctx.BlockTime()
+
+ testCases := []struct {
+ chainID string
+ ts time.Time
+ vscID uint64
+ }{
+ {chainID: "chain", ts: now.Add(time.Hour), vscID: 1},
+ {chainID: "chain", ts: now.Add(2 * time.Hour), vscID: 2},
+ {chainID: "chain1", ts: now.Add(time.Hour), vscID: 1},
+ {chainID: "chain2", ts: now.Add(time.Hour), vscID: 1},
+ }
+
+ i := 0
+ chainID := "chain"
+ providerKeeper.IterateVscSendTimestamps(ctx, chainID, func(_ uint64, _ time.Time) bool {
+ i++
+ return true
+ })
+ require.Equal(t, 0, i)
+
+ for _, tc := range testCases {
+ providerKeeper.SetVscSendTimestamp(ctx, tc.chainID, tc.vscID, tc.ts)
+ }
+
+ i = 0
+ providerKeeper.IterateVscSendTimestamps(ctx, testCases[0].chainID, func(vscID uint64, ts time.Time) bool {
+ require.Equal(t, vscID, testCases[i].vscID)
+ require.Equal(t, ts, testCases[i].ts)
+ i++
+ return true
+ })
+ require.Equal(t, 2, i)
+
+ // delete VSC send timestamps
+ var ids []uint64
+ providerKeeper.IterateVscSendTimestamps(ctx, testCases[0].chainID, func(vscID uint64, _ time.Time) bool {
+ ids = append(ids, vscID)
+ return true
+ })
+ for _, vscID := range ids {
+ providerKeeper.DeleteVscSendTimestamp(ctx, testCases[0].chainID, vscID)
+ }
+
+ i = 0
+ providerKeeper.IterateVscSendTimestamps(ctx, testCases[0].chainID, func(_ uint64, _ time.Time) bool {
+ i++
+ return true
+ })
+ require.Equal(t, 0, i)
+}
diff --git a/x/ccv/provider/keeper/params.go b/x/ccv/provider/keeper/params.go
index 323f860258..9ce00d2f81 100644
--- a/x/ccv/provider/keeper/params.go
+++ b/x/ccv/provider/keeper/params.go
@@ -40,6 +40,18 @@ func (k Keeper) GetInitTimeoutPeriod(ctx sdk.Context) time.Duration {
return p
}
+// GetVscTimeoutPeriod returns the vsc timeout period
+func (k Keeper) GetVscTimeoutPeriod(ctx sdk.Context) time.Duration {
+ var p time.Duration
+ k.paramSpace.Get(ctx, types.KeyVscTimeoutPeriod, &p)
+ return p
+}
+
+// SetVscTimeoutPeriod sets the vsc timeout period
+func (k Keeper) SetVscTimeoutPeriod(ctx sdk.Context, period time.Duration) {
+ k.paramSpace.Set(ctx, types.KeyVscTimeoutPeriod, period)
+}
+
// GetParams returns the paramset for the provider module
func (k Keeper) GetParams(ctx sdk.Context) types.Params {
return types.NewParams(
@@ -47,6 +59,7 @@ func (k Keeper) GetParams(ctx sdk.Context) types.Params {
k.GetTrustingPeriodFraction(ctx),
k.GetCCVTimeoutPeriod(ctx),
k.GetInitTimeoutPeriod(ctx),
+ k.GetVscTimeoutPeriod(ctx),
)
}
diff --git a/x/ccv/provider/keeper/params_test.go b/x/ccv/provider/keeper/params_test.go
index 62623d4b1f..02da2aa9de 100644
--- a/x/ccv/provider/keeper/params_test.go
+++ b/x/ccv/provider/keeper/params_test.go
@@ -28,7 +28,7 @@ func TestParams(t *testing.T) {
newParams := types.NewParams(ibctmtypes.NewClientState("", ibctmtypes.DefaultTrustLevel, 0, 0,
time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
- types.DefaultTrustingPeriodFraction, ccvtypes.DefaultCCVTimeoutPeriod, types.DefaultInitTimeoutPeriod)
+ types.DefaultTrustingPeriodFraction, ccvtypes.DefaultCCVTimeoutPeriod, types.DefaultInitTimeoutPeriod, types.DefaultVscTimeoutPeriod)
providerKeeper.SetParams(ctx, newParams)
params = providerKeeper.GetParams(ctx)
require.Equal(t, newParams, params)
diff --git a/x/ccv/provider/keeper/proposal.go b/x/ccv/provider/keeper/proposal.go
index bd4961817e..9844ab55ac 100644
--- a/x/ccv/provider/keeper/proposal.go
+++ b/x/ccv/provider/keeper/proposal.go
@@ -130,6 +130,7 @@ func (k Keeper) StopConsumerChain(ctx sdk.Context, chainID string, lockUbd, clos
k.DeleteConsumerClientId(ctx, chainID)
k.DeleteConsumerGenesis(ctx, chainID)
k.DeleteLockUnbondingOnTimeout(ctx, chainID)
+ k.DeleteInitTimeoutTimestamp(ctx, chainID)
// close channel and delete the mappings between chain ID and channel ID
if channelID, found := k.GetChainToChannel(ctx, chainID); found {
@@ -138,6 +139,16 @@ func (k Keeper) StopConsumerChain(ctx sdk.Context, chainID string, lockUbd, clos
}
k.DeleteChainToChannel(ctx, chainID)
k.DeleteChannelToChain(ctx, channelID)
+
+ // delete VSC send timestamps
+ var ids []uint64
+ k.IterateVscSendTimestamps(ctx, chainID, func(vscID uint64, ts time.Time) bool {
+ ids = append(ids, vscID)
+ return true
+ })
+ for _, vscID := range ids {
+ k.DeleteVscSendTimestamp(ctx, chainID, vscID)
+ }
}
k.DeleteInitChainHeight(ctx, chainID)
diff --git a/x/ccv/provider/keeper/proposal_test.go b/x/ccv/provider/keeper/proposal_test.go
index 8f164d2cfb..7f6ad1058e 100644
--- a/x/ccv/provider/keeper/proposal_test.go
+++ b/x/ccv/provider/keeper/proposal_test.go
@@ -375,7 +375,7 @@ func TestHandleConsumerRemovalProposal(t *testing.T) {
found := providerKeeper.GetPendingConsumerRemovalProp(ctx, tc.prop.ChainId, tc.prop.StopTime)
require.False(t, found)
- testConsumerStateIsCleaned(t, ctx, providerKeeper, tc.prop.ChainId, "channelID")
+ testProviderStateIsCleaned(t, ctx, providerKeeper, tc.prop.ChainId, "channelID")
} else {
// Proposal should be stored as pending
found := providerKeeper.GetPendingConsumerRemovalProp(ctx, tc.prop.ChainId, tc.prop.StopTime)
@@ -446,14 +446,14 @@ func TestStopConsumerChain(t *testing.T) {
require.NoError(t, err)
}
- testConsumerStateIsCleaned(t, ctx, providerKeeper, "chainID", "channelID")
+ testProviderStateIsCleaned(t, ctx, providerKeeper, "chainID", "channelID")
ctrl.Finish()
}
}
-// testConsumerStateIsCleaned executes test assertions for a stopped consumer chain's state being cleaned.
-func testConsumerStateIsCleaned(t *testing.T, ctx sdk.Context, providerKeeper providerkeeper.Keeper,
+// testProviderStateIsCleaned executes test assertions for the proposer's state being cleaned after a stopped consumer chain.
+func testProviderStateIsCleaned(t *testing.T, ctx sdk.Context, providerKeeper providerkeeper.Keeper,
expectedChainID string, expectedChannelID string) {
_, found := providerKeeper.GetConsumerClientId(ctx, expectedChainID)
@@ -468,6 +468,14 @@ func testConsumerStateIsCleaned(t *testing.T, ctx sdk.Context, providerKeeper pr
require.False(t, found)
acks := providerKeeper.GetSlashAcks(ctx, expectedChainID)
require.Empty(t, acks)
+ _, found = providerKeeper.GetInitTimeoutTimestamp(ctx, expectedChainID)
+ require.False(t, found)
+ found = false
+ providerKeeper.IterateVscSendTimestamps(ctx, expectedChainID, func(_ uint64, _ time.Time) bool {
+ found = true
+ return false
+ })
+ require.False(t, found)
}
// TestPendingConsumerRemovalPropDeletion tests the getting/setting
@@ -628,6 +636,7 @@ func TestMakeConsumerGenesis(t *testing.T) {
TrustingPeriodFraction: providertypes.DefaultTrustingPeriodFraction,
CcvTimeoutPeriod: ccvtypes.DefaultCCVTimeoutPeriod,
InitTimeoutPeriod: types.DefaultInitTimeoutPeriod,
+ VscTimeoutPeriod: types.DefaultVscTimeoutPeriod,
}
providerKeeper.SetParams(ctx, moduleParams)
defer ctrl.Finish()
diff --git a/x/ccv/provider/keeper/relay.go b/x/ccv/provider/keeper/relay.go
index 7c6166e2a3..9c6bd12398 100644
--- a/x/ccv/provider/keeper/relay.go
+++ b/x/ccv/provider/keeper/relay.go
@@ -66,6 +66,9 @@ func (k Keeper) OnRecvVSCMaturedPacket(
// clean up index
k.DeleteUnbondingOpIndex(ctx, chainID, data.ValsetUpdateId)
+ // remove the VSC timeout timestamp for this chainID and vscID
+ k.DeleteVscSendTimestamp(ctx, chainID, data.ValsetUpdateId)
+
ack := channeltypes.NewResultAcknowledgement([]byte{byte(1)})
return ack
}
@@ -162,6 +165,8 @@ func (k Keeper) sendValidatorUpdates(ctx sdk.Context) {
if err != nil {
panic(fmt.Errorf("packet could not be sent over IBC: %w", err))
}
+ // set the VSC send timestamp for this packet
+ k.SetVscSendTimestamp(ctx, chainID, packetData.ValsetUpdateId, ctx.BlockTime())
} else {
// store the packet data to be sent once the CCV channel is established
k.AppendPendingVSC(ctx, chainID, packetData)
@@ -189,6 +194,10 @@ func (k Keeper) SendPendingVSCPackets(ctx sdk.Context, chainID, channelID string
if err != nil {
panic(fmt.Errorf("packet could not be sent over IBC: %w", err))
}
+ // set the VSC send timestamp for this packet;
+ // note that the VSC send timestamp are set when the packets
+ // are actually sent over IBC
+ k.SetVscSendTimestamp(ctx, chainID, data.ValsetUpdateId, ctx.BlockTime())
}
}
@@ -300,24 +309,65 @@ func (k Keeper) HandleSlashPacket(ctx sdk.Context, chainID string, data ccv.Slas
// EndBlockCIS contains the EndBlock logic needed for
// the Consumer Chain Removal sub-protocol
func (k Keeper) EndBlockCCR(ctx sdk.Context) {
- currentTime := uint64(ctx.BlockTime().UnixNano())
+ currentTime := ctx.BlockTime()
+ currentTimeUint64 := uint64(currentTime.UnixNano())
// iterate over initTimeoutTimestamps
- var removedChainIds []string
+ var chainIdsToRemove []string
k.IterateInitTimeoutTimestamp(ctx, func(chainID string, ts uint64) bool {
- if currentTime > ts {
- // initTimeout expired:
- // stop the consumer chain and unlock the unbonding
- err := k.StopConsumerChain(ctx, chainID, false, true)
- if err != nil {
- panic(fmt.Errorf("consumer chain failed to stop: %w", err))
- }
- removedChainIds = append(removedChainIds, chainID)
+ if currentTimeUint64 > ts {
+ // initTimeout expired
+ chainIdsToRemove = append(chainIdsToRemove, chainID)
+ // continue to iterate through all timed out consumers
+ return true
+ }
+ // break iteration since the timeout timestamps are in order
+ return false
+ })
+ // remove consumers that timed out
+ for _, chainID := range chainIdsToRemove {
+ // stop the consumer chain and unlock the unbonding.
+ // Note that the CCV channel was not established,
+ // thus closeChan is irrelevant
+ err := k.StopConsumerChain(ctx, chainID, false, false)
+ if err != nil {
+ panic(fmt.Errorf("consumer chain failed to stop: %w", err))
}
+ }
+
+ // empty slice
+ chainIdsToRemove = nil
+
+ // Iterate over all consumers with established CCV channels and
+ // check if the first vscSendTimestamp in iterator + VscTimeoutPeriod
+ // exceed the current block time.
+ // Checking the first send timestamp for each chain is sufficient since
+ // timestamps are ordered by vsc ID.
+ k.IterateChannelToChain(ctx, func(ctx sdk.Context, _, chainID string) bool {
+ k.IterateVscSendTimestamps(ctx, chainID, func(_ uint64, ts time.Time) bool {
+ timeoutTimestamp := ts.Add(k.GetParams(ctx).VscTimeoutPeriod)
+ if currentTime.After(timeoutTimestamp) {
+ // vscTimeout expired
+ chainIdsToRemove = append(chainIdsToRemove, chainID)
+ }
+ // break iteration since the send timestamps are in order
+ return false
+ })
+ // continue to iterate through all consumers
return true
})
- // remove the init timeout timestamps for the stopped consumers
- for _, chainID := range removedChainIds {
- k.DeleteInitTimeoutTimestamp(ctx, chainID)
+ // remove consumers that timed out
+ for _, chainID := range chainIdsToRemove {
+ // stop the consumer chain and use lockUnbondingOnTimeout
+ // to decide whether to lock the unbonding
+ err := k.StopConsumerChain(
+ ctx,
+ chainID,
+ k.GetLockUnbondingOnTimeout(ctx, chainID),
+ true,
+ )
+ if err != nil {
+ panic(fmt.Errorf("consumer chain failed to stop: %w", err))
+ }
}
}
diff --git a/x/ccv/provider/types/genesis_test.go b/x/ccv/provider/types/genesis_test.go
index 5c4fe8bf25..87aa1f8ff5 100644
--- a/x/ccv/provider/types/genesis_test.go
+++ b/x/ccv/provider/types/genesis_test.go
@@ -79,7 +79,7 @@ func TestValidateGenesisState(t *testing.T) {
nil,
types.NewParams(ibctmtypes.NewClientState("", ibctmtypes.DefaultTrustLevel, 0, 0,
time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
- 3, time.Hour, time.Hour),
+ 3, time.Hour, time.Hour, time.Hour),
),
true,
},
@@ -95,7 +95,7 @@ func TestValidateGenesisState(t *testing.T) {
nil,
types.NewParams(ibctmtypes.NewClientState("", ibctmtypes.DefaultTrustLevel, 0, 0,
0, clienttypes.Height{}, nil, []string{"ibc", "upgradedIBCState"}, true, false),
- types.DefaultTrustingPeriodFraction, ccv.DefaultCCVTimeoutPeriod, types.DefaultInitTimeoutPeriod),
+ types.DefaultTrustingPeriodFraction, ccv.DefaultCCVTimeoutPeriod, types.DefaultInitTimeoutPeriod, types.DefaultVscTimeoutPeriod),
),
false,
},
@@ -113,7 +113,8 @@ func TestValidateGenesisState(t *testing.T) {
time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
0, // 0 trusting period fraction here
ccv.DefaultCCVTimeoutPeriod,
- types.DefaultInitTimeoutPeriod),
+ types.DefaultInitTimeoutPeriod,
+ types.DefaultVscTimeoutPeriod),
),
false,
},
@@ -131,7 +132,8 @@ func TestValidateGenesisState(t *testing.T) {
time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
types.DefaultTrustingPeriodFraction,
0, // 0 ccv timeout here
- types.DefaultInitTimeoutPeriod),
+ types.DefaultInitTimeoutPeriod,
+ types.DefaultVscTimeoutPeriod),
),
false,
},
@@ -149,7 +151,27 @@ func TestValidateGenesisState(t *testing.T) {
time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
types.DefaultTrustingPeriodFraction,
ccv.DefaultCCVTimeoutPeriod,
- 0), // 0 init timeout here
+ 0, // 0 init timeout here
+ types.DefaultVscTimeoutPeriod),
+ ),
+ false,
+ },
+ {
+ "invalid params, zero vsc timeout",
+ types.NewGenesisState(
+ 0,
+ nil,
+ []types.ConsumerState{{ChainId: "chainid-1", ChannelId: "channelid"}},
+ nil,
+ nil,
+ nil,
+ nil,
+ types.NewParams(ibctmtypes.NewClientState("", ibctmtypes.DefaultTrustLevel, 0, 0,
+ time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
+ types.DefaultTrustingPeriodFraction,
+ ccv.DefaultCCVTimeoutPeriod,
+ types.DefaultInitTimeoutPeriod,
+ 0), // 0 vsc timeout here
),
false,
},
diff --git a/x/ccv/provider/types/keys.go b/x/ccv/provider/types/keys.go
index fb86846a39..3fbf69ac64 100644
--- a/x/ccv/provider/types/keys.go
+++ b/x/ccv/provider/types/keys.go
@@ -2,13 +2,11 @@ package types
import (
"bytes"
- "crypto/sha256"
"encoding/binary"
"fmt"
"time"
sdk "github.com/cosmos/cosmos-sdk/types"
- sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
)
type Status int
@@ -88,15 +86,14 @@ const (
// PendingVSCsBytePrefix is the byte prefix that will store pending ValidatorSetChangePacket data
PendingVSCsBytePrefix
+ // VscSendTimestampBytePrefix is the byte prefix for storing
+ // the list of VSC sending timestamps for a given consumer chainID.
+ VscSendTimestampBytePrefix
+
// LockUnbondingOnTimeoutBytePrefix is the byte prefix that will store the consumer chain id which unbonding operations are locked on CCV channel timeout
LockUnbondingOnTimeoutBytePrefix
)
-const (
- // UnbondingOpIndexKey should be of set length: prefix + hashed chain ID + uint64
- UnbondingOpIndexKeySize = 1 + 32 + 8
-)
-
// PortKey returns the key to the port ID in the store
func PortKey() []byte {
return []byte{PortByteKey}
@@ -134,95 +131,36 @@ func InitTimeoutTimestampKey(chainID string) []byte {
// PendingCAPKey returns the key under which a pending consumer addition proposal is stored
func PendingCAPKey(timestamp time.Time, chainID string) []byte {
- timeBz := sdk.FormatTimeBytes(timestamp)
- timeBzL := len(timeBz)
- prefixL := len([]byte{PendingCAPBytePrefix})
-
- bz := make([]byte, prefixL+8+timeBzL+len(chainID))
- // copy the prefix
- copy(bz[:prefixL], []byte{PendingCAPBytePrefix})
- // copy the time length
- copy(bz[prefixL:prefixL+8], sdk.Uint64ToBigEndian(uint64(timeBzL)))
- // copy the time bytes
- copy(bz[prefixL+8:prefixL+8+timeBzL], timeBz)
- // copy the chainId
- copy(bz[prefixL+8+timeBzL:], chainID)
- return bz
+ return tsAndChainIdKey(PendingCAPBytePrefix, timestamp, chainID)
}
// ParsePendingCAPKey returns the time and chain ID for a pending consumer addition proposal key
// or an error if unparsable
func ParsePendingCAPKey(bz []byte) (time.Time, string, error) {
- expectedPrefix := []byte{PendingCAPBytePrefix}
- prefixL := len(expectedPrefix)
- if prefix := bz[:prefixL]; !bytes.Equal(prefix, expectedPrefix) {
- return time.Time{}, "", fmt.Errorf("invalid prefix; expected: %X, got: %X", expectedPrefix, prefix)
- }
-
- timeBzL := sdk.BigEndianToUint64(bz[prefixL : prefixL+8])
- timestamp, err := sdk.ParseTimeBytes(bz[prefixL+8 : prefixL+8+int(timeBzL)])
- if err != nil {
- return time.Time{}, "", err
- }
-
- chainID := string(bz[prefixL+8+int(timeBzL):])
- return timestamp, chainID, nil
+ return parseTsAndChainIdKey(PendingCAPBytePrefix, bz)
}
// PendingCRPKey returns the key under which pending consumer removal proposals are stored
func PendingCRPKey(timestamp time.Time, chainID string) []byte {
- timeBz := sdk.FormatTimeBytes(timestamp)
- timeBzL := len(timeBz)
- prefixL := len([]byte{PendingCRPBytePrefix})
-
- bz := make([]byte, prefixL+8+timeBzL+len(chainID))
- // copy the prefix
- copy(bz[:prefixL], []byte{PendingCRPBytePrefix})
- // copy the time length
- copy(bz[prefixL:prefixL+8], sdk.Uint64ToBigEndian(uint64(timeBzL)))
- // copy the time bytes
- copy(bz[prefixL+8:prefixL+8+timeBzL], timeBz)
- // copy the chainId
- copy(bz[prefixL+8+timeBzL:], chainID)
- return bz
+ return tsAndChainIdKey(PendingCRPBytePrefix, timestamp, chainID)
}
// ParsePendingCRPKey returns the time and chain ID for a pending consumer removal proposal key or an error if unparseable
func ParsePendingCRPKey(bz []byte) (time.Time, string, error) {
- expectedPrefix := []byte{PendingCRPBytePrefix}
- prefixL := len(expectedPrefix)
- if prefix := bz[:prefixL]; !bytes.Equal(prefix, expectedPrefix) {
- return time.Time{}, "", fmt.Errorf("invalid prefix; expected: %X, got: %X", expectedPrefix, prefix)
- }
-
- timeBzL := sdk.BigEndianToUint64(bz[prefixL : prefixL+8])
- timestamp, err := sdk.ParseTimeBytes(bz[prefixL+8 : prefixL+8+int(timeBzL)])
- if err != nil {
- return time.Time{}, "", err
- }
-
- chainID := string(bz[prefixL+8+int(timeBzL):])
- return timestamp, chainID, nil
+ return parseTsAndChainIdKey(PendingCRPBytePrefix, bz)
}
// UnbondingOpIndexKey returns an unbonding op index key
// Note: chainId is hashed to a fixed length sequence of bytes here to prevent
// injection attack between chainIDs.
-func UnbondingOpIndexKey(chainID string, valsetUpdateID uint64) []byte {
- return AppendMany([]byte{UnbondingOpIndexBytePrefix}, HashString(chainID),
- sdk.Uint64ToBigEndian(valsetUpdateID))
+func UnbondingOpIndexKey(chainID string, vscID uint64) []byte {
+ return chainIdAndVscIdKey(UnbondingOpIndexBytePrefix, chainID, vscID)
}
// ParseUnbondingOpIndexKey parses an unbonding op index key for VSC ID
// Removes the prefix + chainID from index key and returns only the key part.
-func ParseUnbondingOpIndexKey(key []byte) (vscID []byte, err error) {
- if len(key) != UnbondingOpIndexKeySize {
- return nil, sdkerrors.Wrapf(
- sdkerrors.ErrLogic, "key provided is incorrect: the key has incorrect length, expected %d, got %d",
- UnbondingOpIndexKeySize, len(key),
- )
- }
- return key[1+32:], nil
+func ParseUnbondingOpIndexKey(key []byte) (string, uint64, error) {
+ return parseChainIdAndVscIdKey(UnbondingOpIndexBytePrefix, key)
}
// UnbondingOpKey returns the key that stores a record of all the ids of consumer chains that
@@ -262,6 +200,18 @@ func PendingVSCsKey(chainID string) []byte {
return append([]byte{PendingVSCsBytePrefix}, []byte(chainID)...)
}
+// VscSendingTimestampKey returns the key under which the
+// sending timestamp of the VSCPacket with vsc ID is stored
+func VscSendingTimestampKey(chainID string, vscID uint64) []byte {
+ return chainIdAndVscIdKey(VscSendTimestampBytePrefix, chainID, vscID)
+}
+
+// ParseVscTimeoutTimestampKey returns chain ID and vsc ID
+// for a VscSendingTimestampKey or an error if unparsable
+func ParseVscSendingTimestampKey(bz []byte) (string, uint64, error) {
+ return parseChainIdAndVscIdKey(VscSendTimestampBytePrefix, bz)
+}
+
// LockUnbondingOnTimeoutKey returns the key that will store the consumer chain id which unbonding operations are locked
// on CCV channel timeout
func LockUnbondingOnTimeoutKey(chainID string) []byte {
@@ -276,8 +226,106 @@ func AppendMany(byteses ...[]byte) (out []byte) {
return out
}
-// HashString outputs a fixed length 32 byte hash for any string
-func HashString(x string) []byte {
- hash := sha256.Sum256([]byte(x))
- return hash[:]
+// tsAndChainIdKey returns the key with the following format:
+// bytePrefix | len(timestamp) | timestamp | chainID
+func tsAndChainIdKey(prefix byte, timestamp time.Time, chainID string) []byte {
+ timeBz := sdk.FormatTimeBytes(timestamp)
+ timeBzL := len(timeBz)
+
+ return AppendMany(
+ // Append the prefix
+ []byte{prefix},
+ // Append the time length
+ sdk.Uint64ToBigEndian(uint64(timeBzL)),
+ // Append the time bytes
+ timeBz,
+ // Append the chainId
+ []byte(chainID),
+ )
+}
+
+// parseTsAndChainIdKey returns the time and chain ID for a TsAndChainId key
+func parseTsAndChainIdKey(prefix byte, bz []byte) (time.Time, string, error) {
+ expectedPrefix := []byte{prefix}
+ prefixL := len(expectedPrefix)
+ if prefix := bz[:prefixL]; !bytes.Equal(prefix, expectedPrefix) {
+ return time.Time{}, "", fmt.Errorf("invalid prefix; expected: %X, got: %X", expectedPrefix, prefix)
+ }
+
+ timeBzL := sdk.BigEndianToUint64(bz[prefixL : prefixL+8])
+ timestamp, err := sdk.ParseTimeBytes(bz[prefixL+8 : prefixL+8+int(timeBzL)])
+ if err != nil {
+ return time.Time{}, "", err
+ }
+
+ chainID := string(bz[prefixL+8+int(timeBzL):])
+ return timestamp, chainID, nil
+}
+
+// chainIdAndTsKey returns the key with the following format:
+// bytePrefix | len(chainID) | chainID | timestamp
+func chainIdAndTsKey(prefix byte, chainID string, timestamp time.Time) []byte {
+ partialKey := ChainIdWithLenKey(prefix, chainID)
+ timeBz := sdk.FormatTimeBytes(timestamp)
+ return AppendMany(
+ // Append the partialKey
+ partialKey,
+ // Append the time bytes
+ timeBz,
+ )
+}
+
+// chainIdWithLenKey returns the key with the following format:
+// bytePrefix | len(chainID) | chainID
+func ChainIdWithLenKey(prefix byte, chainID string) []byte {
+ chainIdL := len(chainID)
+ return AppendMany(
+ // Append the prefix
+ []byte{prefix},
+ // Append the chainID length
+ sdk.Uint64ToBigEndian(uint64(chainIdL)),
+ // Append the chainID
+ []byte(chainID),
+ )
+}
+
+// parseChainIdAndTsKey returns the chain ID and time for a ChainIdAndTs key
+func parseChainIdAndTsKey(prefix byte, bz []byte) (string, time.Time, error) {
+ expectedPrefix := []byte{prefix}
+ prefixL := len(expectedPrefix)
+ if prefix := bz[:prefixL]; !bytes.Equal(prefix, expectedPrefix) {
+ return "", time.Time{}, fmt.Errorf("invalid prefix; expected: %X, got: %X", expectedPrefix, prefix)
+ }
+ chainIdL := sdk.BigEndianToUint64(bz[prefixL : prefixL+8])
+ chainID := string(bz[prefixL+8 : prefixL+8+int(chainIdL)])
+ timestamp, err := sdk.ParseTimeBytes(bz[prefixL+8+int(chainIdL):])
+ if err != nil {
+ return "", time.Time{}, err
+ }
+ return chainID, timestamp, nil
+}
+
+// chainIdAndVscIdKey returns the key with the following format:
+// bytePrefix | len(chainID) | chainID | vscID
+func chainIdAndVscIdKey(prefix byte, chainID string, vscID uint64) []byte {
+ partialKey := ChainIdWithLenKey(prefix, chainID)
+ return AppendMany(
+ // Append the partialKey
+ partialKey,
+ // Append the vscID bytes
+ sdk.Uint64ToBigEndian(vscID),
+ )
+}
+
+// parseChainIdAndVscIdKey returns the chain ID and vsc ID for a ChainIdAndVscId key
+func parseChainIdAndVscIdKey(prefix byte, bz []byte) (string, uint64, error) {
+ expectedPrefix := []byte{prefix}
+ prefixL := len(expectedPrefix)
+ if prefix := bz[:prefixL]; !bytes.Equal(prefix, expectedPrefix) {
+ return "", 0, fmt.Errorf("invalid prefix; expected: %X, got: %X", expectedPrefix, prefix)
+ }
+ chainIdL := sdk.BigEndianToUint64(bz[prefixL : prefixL+8])
+ chainID := string(bz[prefixL+8 : prefixL+8+int(chainIdL)])
+ vscID := sdk.BigEndianToUint64(bz[prefixL+8+int(chainIdL):])
+ return chainID, vscID, nil
}
diff --git a/x/ccv/provider/types/keys_test.go b/x/ccv/provider/types/keys_test.go
index d1b6525ca3..8152ecf01a 100644
--- a/x/ccv/provider/types/keys_test.go
+++ b/x/ccv/provider/types/keys_test.go
@@ -53,81 +53,85 @@ func getSingleByteKeys() [][]byte {
keys[i], i = []byte{SlashAcksBytePrefix}, i+1
keys[i], i = []byte{InitChainHeightBytePrefix}, i+1
keys[i], i = []byte{PendingVSCsBytePrefix}, i+1
+ keys[i], i = []byte{VscSendTimestampBytePrefix}, i+1
keys[i], i = []byte{LockUnbondingOnTimeoutBytePrefix}, i+1
return keys[:i]
}
-// Tests the construction and parsing of keys for storing pending consumer addition proposals
-func TestPendingCAPKeyAndParse(t *testing.T) {
+// Tests the construction and parsing of TsAndChainId keys
+func TestTsAndChainIdKeyAndParse(t *testing.T) {
tests := []struct {
+ prefix byte
timestamp time.Time
chainID string
}{
- {timestamp: time.Now(), chainID: "1"},
- {timestamp: time.Date(
+ {prefix: 0x01, timestamp: time.Now(), chainID: "1"},
+ {prefix: 0x02, timestamp: time.Date(
2003, 11, 17, 20, 34, 58, 651387237, time.UTC), chainID: "some other ID"},
- {timestamp: time.Now().Add(5000 * time.Hour), chainID: "some other other chain ID"},
+ {prefix: 0x03, timestamp: time.Now().Add(5000 * time.Hour), chainID: "some other other chain ID"},
}
for _, test := range tests {
- key := PendingCAPKey(test.timestamp, test.chainID)
+ key := tsAndChainIdKey(test.prefix, test.timestamp, test.chainID)
require.NotEmpty(t, key)
// Expected bytes = prefix + time length + time bytes + length of chainID
- expectedBytes := 1 + 8 + len(sdk.FormatTimeBytes(time.Time{})) + len(test.chainID)
- require.Equal(t, expectedBytes, len(key))
- parsedTime, parsedID, err := ParsePendingCAPKey(key)
+ expectedLen := 1 + 8 + len(sdk.FormatTimeBytes(time.Time{})) + len(test.chainID)
+ require.Equal(t, expectedLen, len(key))
+ parsedTime, parsedID, err := parseTsAndChainIdKey(test.prefix, key)
require.Equal(t, test.timestamp.UTC(), parsedTime.UTC())
require.Equal(t, test.chainID, parsedID)
require.NoError(t, err)
}
}
-// Tests the construction and parsing of keys for storing pending consumer removal proposals
-func TestPendingCRPKeyAndParse(t *testing.T) {
+// Tests the construction and parsing of ChainIdAndTs keys
+func TestChainIdAndTsKeyAndParse(t *testing.T) {
tests := []struct {
- timestamp time.Time
+ prefix byte
chainID string
+ timestamp time.Time
}{
- {timestamp: time.Now(), chainID: "5"},
- {timestamp: time.Date(
- 2003, 11, 17, 20, 34, 58, 651387237, time.UTC), chainID: "some other ID"},
- {timestamp: time.Now().Add(5000 * time.Hour), chainID: "some other other chain ID"},
+ {prefix: 0x01, chainID: "1", timestamp: time.Now()},
+ {prefix: 0x02, chainID: "some other ID", timestamp: time.Date(
+ 2003, 11, 17, 20, 34, 58, 651387237, time.UTC)},
+ {prefix: 0x03, chainID: "some other other chain ID", timestamp: time.Now().Add(5000 * time.Hour)},
}
for _, test := range tests {
- key := PendingCRPKey(test.timestamp, test.chainID)
+ key := chainIdAndTsKey(test.prefix, test.chainID, test.timestamp)
require.NotEmpty(t, key)
- // Expected bytes = prefix + time length + time bytes + length of chainID
- expectedBytes := 1 + 8 + len(sdk.FormatTimeBytes(time.Time{})) + len(test.chainID)
- require.Equal(t, expectedBytes, len(key))
- parsedTime, parsedID, err := ParsePendingCRPKey(key)
- require.Equal(t, test.timestamp.UTC(), parsedTime.UTC())
+ // Expected bytes = prefix + chainID length + chainID + time bytes
+ expectedLen := 1 + 8 + len(test.chainID) + len(sdk.FormatTimeBytes(time.Time{}))
+ require.Equal(t, expectedLen, len(key))
+ parsedID, parsedTime, err := parseChainIdAndTsKey(test.prefix, key)
require.Equal(t, test.chainID, parsedID)
+ require.Equal(t, test.timestamp.UTC(), parsedTime.UTC())
require.NoError(t, err)
}
}
-func TestUnbondingOpIndexKeyAndParse(t *testing.T) {
+// Tests the construction and parsing of ChainIdAndVscId keys
+func TestChainIdAndVscIdAndParse(t *testing.T) {
tests := []struct {
- chainID string
- valsetUpdateID uint64
+ prefix byte
+ chainID string
+ vscID uint64
}{
- {chainID: " some chain id", valsetUpdateID: 45},
- {chainID: " some chain id that is longer", valsetUpdateID: 54038},
- {chainID: " some chain id that is longer-er ", valsetUpdateID: 9999999999999999999},
- {chainID: "2", valsetUpdateID: 0},
+ {prefix: 0x01, chainID: "1", vscID: 1},
+ {prefix: 0x02, chainID: "some other ID", vscID: 2},
+ {prefix: 0x03, chainID: "some other other chain ID", vscID: 3},
}
for _, test := range tests {
- key := UnbondingOpIndexKey(test.chainID, test.valsetUpdateID)
+ key := chainIdAndVscIdKey(test.prefix, test.chainID, test.vscID)
require.NotEmpty(t, key)
- // This key should be of set length: prefix + hashed chain ID + uint64
- require.Equal(t, 1+32+8, len(key))
- parsedVSCID, err := ParseUnbondingOpIndexKey(key)
- require.NotEmpty(t, parsedVSCID)
- asUint64 := sdk.BigEndianToUint64(parsedVSCID)
- require.Equal(t, test.valsetUpdateID, asUint64)
+ // Expected bytes = prefix + chainID length + chainID + vscId bytes
+ expectedLen := 1 + 8 + len(test.chainID) + 8
+ require.Equal(t, expectedLen, len(key))
+ parsedID, parsedVscID, err := parseChainIdAndVscIdKey(test.prefix, key)
+ require.Equal(t, test.chainID, parsedID)
+ require.Equal(t, test.vscID, parsedVscID)
require.NoError(t, err)
}
}
diff --git a/x/ccv/provider/types/params.go b/x/ccv/provider/types/params.go
index 9590b46dda..be3281ee06 100644
--- a/x/ccv/provider/types/params.go
+++ b/x/ccv/provider/types/params.go
@@ -21,8 +21,11 @@ const (
// as UnbondingPeriod / TrustingPeriodFraction
DefaultTrustingPeriodFraction = 2
- // DafaultInitTimeoutPeriod defines the init timeout period
+ // DefaultInitTimeoutPeriod defines the init timeout period
DefaultInitTimeoutPeriod = 7 * 24 * time.Hour
+
+ // DefaultVscTimeoutPeriod defines the VSC timeout period
+ DefaultVscTimeoutPeriod = 5 * 7 * 24 * time.Hour
)
// Reflection based keys for params subspace
@@ -30,6 +33,7 @@ var (
KeyTemplateClient = []byte("TemplateClient")
KeyTrustingPeriodFraction = []byte("TrustingPeriodFraction")
KeyInitTimeoutPeriod = []byte("InitTimeoutPeriod")
+ KeyVscTimeoutPeriod = []byte("VscTimeoutPeriod")
)
// ParamKeyTable returns a key table with the necessary registered provider params
@@ -43,12 +47,14 @@ func NewParams(
trustingPeriodFraction int64,
ccvTimeoutPeriod time.Duration,
initTimeoutPeriod time.Duration,
+ vscTimeoutPeriod time.Duration,
) Params {
return Params{
TemplateClient: cs,
TrustingPeriodFraction: trustingPeriodFraction,
CcvTimeoutPeriod: ccvTimeoutPeriod,
InitTimeoutPeriod: initTimeoutPeriod,
+ VscTimeoutPeriod: vscTimeoutPeriod,
}
}
@@ -72,6 +78,7 @@ func DefaultParams() Params {
DefaultTrustingPeriodFraction,
ccvtypes.DefaultCCVTimeoutPeriod,
DefaultInitTimeoutPeriod,
+ DefaultVscTimeoutPeriod,
)
}
@@ -92,6 +99,9 @@ func (p Params) Validate() error {
if err := ccvtypes.ValidateDuration(p.InitTimeoutPeriod); err != nil {
return fmt.Errorf("init timeout period is invalid: %s", err)
}
+ if err := ccvtypes.ValidateDuration(p.VscTimeoutPeriod); err != nil {
+ return fmt.Errorf("vsc timeout period is invalid: %s", err)
+ }
return nil
}
@@ -102,6 +112,7 @@ func (p *Params) ParamSetPairs() paramtypes.ParamSetPairs {
paramtypes.NewParamSetPair(KeyTrustingPeriodFraction, p.TrustingPeriodFraction, ccvtypes.ValidatePositiveInt64),
paramtypes.NewParamSetPair(ccvtypes.KeyCCVTimeoutPeriod, p.CcvTimeoutPeriod, ccvtypes.ValidateDuration),
paramtypes.NewParamSetPair(KeyInitTimeoutPeriod, p.InitTimeoutPeriod, ccvtypes.ValidateDuration),
+ paramtypes.NewParamSetPair(KeyVscTimeoutPeriod, p.VscTimeoutPeriod, ccvtypes.ValidateDuration),
}
}
diff --git a/x/ccv/provider/types/params_test.go b/x/ccv/provider/types/params_test.go
index 81ee001c0c..f4a13a7bd3 100644
--- a/x/ccv/provider/types/params_test.go
+++ b/x/ccv/provider/types/params_test.go
@@ -22,22 +22,25 @@ func TestValidateParams(t *testing.T) {
{"default params", types.DefaultParams(), true},
{"custom valid params", types.NewParams(ibctmtypes.NewClientState("", ibctmtypes.DefaultTrustLevel, 0, 0,
time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
- 3, time.Hour, time.Hour), true},
+ 3, time.Hour, time.Hour, time.Hour), true},
{"custom invalid params", types.NewParams(ibctmtypes.NewClientState("", ibctmtypes.DefaultTrustLevel, 0, 0,
0, clienttypes.Height{}, nil, []string{"ibc", "upgradedIBCState"}, true, false),
- 3, time.Hour, time.Hour), false},
+ 3, time.Hour, time.Hour, time.Hour), false},
{"blank client", types.NewParams(&ibctmtypes.ClientState{},
- 3, time.Hour, time.Hour), false},
- {"nil client", types.NewParams(nil, 3, time.Hour, time.Hour), false},
+ 3, time.Hour, time.Hour, time.Hour), false},
+ {"nil client", types.NewParams(nil, 3, time.Hour, time.Hour, time.Hour), false},
{"0 trusting period fraction (denominator)", types.NewParams(ibctmtypes.NewClientState("", ibctmtypes.DefaultTrustLevel, 0, 0,
time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
- 0, time.Hour, time.Hour), false},
+ 0, time.Hour, time.Hour, time.Hour), false},
{"0 ccv timeout period", types.NewParams(ibctmtypes.NewClientState("", ibctmtypes.DefaultTrustLevel, 0, 0,
time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
- 3, 0, time.Hour), false},
+ 3, 0, time.Hour, time.Hour), false},
{"0 init timeout period", types.NewParams(ibctmtypes.NewClientState("", ibctmtypes.DefaultTrustLevel, 0, 0,
time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
- 3, time.Hour, 0), false},
+ 3, time.Hour, 0, time.Hour), false},
+ {"0 vsc timeout period", types.NewParams(ibctmtypes.NewClientState("", ibctmtypes.DefaultTrustLevel, 0, 0,
+ time.Second*40, clienttypes.Height{}, commitmenttypes.GetSDKSpecs(), []string{"ibc", "upgradedIBCState"}, true, false),
+ 3, time.Hour, time.Hour, 0), false},
}
for _, tc := range testCases {
diff --git a/x/ccv/provider/types/provider.pb.go b/x/ccv/provider/types/provider.pb.go
index f5f7575be5..b928cb9751 100644
--- a/x/ccv/provider/types/provider.pb.go
+++ b/x/ccv/provider/types/provider.pb.go
@@ -173,6 +173,11 @@ type Params struct {
CcvTimeoutPeriod time.Duration `protobuf:"bytes,3,opt,name=ccv_timeout_period,json=ccvTimeoutPeriod,proto3,stdduration" json:"ccv_timeout_period"`
// The channel initialization (IBC channel opening handshake) will timeout after this duration
InitTimeoutPeriod time.Duration `protobuf:"bytes,4,opt,name=init_timeout_period,json=initTimeoutPeriod,proto3,stdduration" json:"init_timeout_period"`
+ // The VSC packets sent by the provider will timeout after this duration.
+ // Note that unlike ccv_timeout_period which is an IBC param,
+ // the vsc_timeout_period is a provider-side param that enables the provider
+ // to timeout VSC packets even when a consumer chain is not live.
+ VscTimeoutPeriod time.Duration `protobuf:"bytes,5,opt,name=vsc_timeout_period,json=vscTimeoutPeriod,proto3,stdduration" json:"vsc_timeout_period"`
}
func (m *Params) Reset() { *m = Params{} }
@@ -236,6 +241,13 @@ func (m *Params) GetInitTimeoutPeriod() time.Duration {
return 0
}
+func (m *Params) GetVscTimeoutPeriod() time.Duration {
+ if m != nil {
+ return m.VscTimeoutPeriod
+ }
+ return 0
+}
+
type HandshakeMetadata struct {
ProviderFeePoolAddr string `protobuf:"bytes,1,opt,name=provider_fee_pool_addr,json=providerFeePoolAddr,proto3" json:"provider_fee_pool_addr,omitempty"`
Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
@@ -441,56 +453,57 @@ func init() {
}
var fileDescriptor_f22ec409a72b7b72 = []byte{
- // 782 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x41, 0x73, 0xe3, 0x34,
- 0x14, 0x8e, 0x37, 0xd9, 0x26, 0x51, 0x96, 0x85, 0xd5, 0xee, 0x14, 0xa7, 0xc3, 0x24, 0x21, 0x5c,
- 0xc2, 0x30, 0xd8, 0x93, 0xec, 0x05, 0x76, 0xe0, 0x90, 0x96, 0x59, 0xca, 0x81, 0x21, 0xb8, 0x05,
- 0x66, 0xb8, 0x78, 0x64, 0x49, 0xb5, 0x35, 0xb5, 0x2d, 0x8f, 0x24, 0x1b, 0x7a, 0xe7, 0xc0, 0x71,
- 0x8f, 0x7b, 0xdc, 0x7f, 0xc0, 0x5f, 0xe0, 0xd8, 0x63, 0x8f, 0x9c, 0x80, 0x69, 0xff, 0x08, 0x23,
- 0xc9, 0x4e, 0xda, 0x40, 0x67, 0xda, 0x03, 0x37, 0xe9, 0xbd, 0xef, 0xfb, 0xf4, 0x9e, 0xbe, 0x67,
- 0x19, 0x2c, 0x58, 0xae, 0xa8, 0xc0, 0x09, 0x62, 0x79, 0x28, 0x29, 0x2e, 0x05, 0x53, 0x67, 0x3e,
- 0xc6, 0x95, 0x5f, 0x08, 0x5e, 0x31, 0x42, 0x85, 0x5f, 0xcd, 0xd7, 0x6b, 0xaf, 0x10, 0x5c, 0x71,
- 0xf8, 0xc1, 0x7f, 0x70, 0x3c, 0x8c, 0x2b, 0x6f, 0x8d, 0xab, 0xe6, 0x7b, 0xcf, 0x62, 0x1e, 0x73,
- 0x83, 0xf7, 0xf5, 0xca, 0x52, 0xf7, 0xc6, 0x31, 0xe7, 0x71, 0x4a, 0x7d, 0xb3, 0x8b, 0xca, 0x13,
- 0x5f, 0xb1, 0x8c, 0x4a, 0x85, 0xb2, 0xa2, 0x06, 0x8c, 0xb6, 0x01, 0xa4, 0x14, 0x48, 0x31, 0x9e,
- 0x37, 0x02, 0x2c, 0xc2, 0x3e, 0xe6, 0x82, 0xfa, 0x38, 0x65, 0x34, 0x57, 0xba, 0x3c, 0xbb, 0xaa,
- 0x01, 0xbe, 0x06, 0xa4, 0x2c, 0x4e, 0x94, 0x0d, 0x4b, 0x5f, 0xd1, 0x9c, 0x50, 0x91, 0x31, 0x0b,
- 0xde, 0xec, 0x2c, 0x61, 0xfa, 0x4b, 0x1b, 0xb8, 0x07, 0x3c, 0x97, 0x65, 0x46, 0xc5, 0x92, 0x10,
- 0xa6, 0x0f, 0x5b, 0x09, 0x5e, 0x70, 0x89, 0x52, 0xf8, 0x0c, 0x3c, 0x54, 0x4c, 0xa5, 0xd4, 0x75,
- 0x26, 0xce, 0xac, 0x1f, 0xd8, 0x0d, 0x9c, 0x80, 0x01, 0xa1, 0x12, 0x0b, 0x56, 0x68, 0xb0, 0xfb,
- 0xc0, 0xe4, 0xae, 0x87, 0xe0, 0x10, 0xf4, 0xec, 0xfd, 0x30, 0xe2, 0xb6, 0x4d, 0xba, 0x6b, 0xf6,
- 0x5f, 0x11, 0xf8, 0x25, 0x78, 0xcc, 0x72, 0xa6, 0x18, 0x4a, 0xc3, 0x84, 0xea, 0x3a, 0xdd, 0xce,
- 0xc4, 0x99, 0x0d, 0x16, 0x7b, 0x1e, 0x8b, 0xb0, 0xa7, 0x5b, 0xf3, 0xea, 0x86, 0xaa, 0xb9, 0x77,
- 0x68, 0x10, 0xfb, 0x9d, 0xf3, 0x3f, 0xc7, 0xad, 0xe0, 0xad, 0x9a, 0x67, 0x83, 0xf0, 0x7d, 0xf0,
- 0x28, 0xa6, 0x39, 0x95, 0x4c, 0x86, 0x09, 0x92, 0x89, 0xfb, 0x70, 0xe2, 0xcc, 0x1e, 0x05, 0x83,
- 0x3a, 0x76, 0x88, 0x64, 0x02, 0xc7, 0x60, 0x10, 0xb1, 0x1c, 0x89, 0x33, 0x8b, 0xd8, 0x31, 0x08,
- 0x60, 0x43, 0x06, 0x70, 0x00, 0x80, 0x2c, 0xd0, 0x4f, 0x79, 0xa8, 0x7d, 0x70, 0xbb, 0x75, 0x21,
- 0xd6, 0x03, 0xaf, 0xf1, 0xc0, 0x3b, 0x6e, 0x4c, 0xda, 0xef, 0xe9, 0x42, 0x5e, 0xfd, 0x35, 0x76,
- 0x82, 0xbe, 0xe1, 0xe9, 0x0c, 0xfc, 0x14, 0x0c, 0x53, 0x8e, 0x4f, 0xc3, 0x32, 0x8f, 0x78, 0x4e,
- 0x58, 0x1e, 0x87, 0xdc, 0x0a, 0xf2, 0x52, 0xb9, 0xbd, 0x89, 0x33, 0xeb, 0x05, 0xbb, 0x1a, 0xf0,
- 0x5d, 0x93, 0xff, 0xc6, 0xf0, 0x78, 0xa9, 0x5e, 0xf4, 0x7e, 0x7d, 0x33, 0x6e, 0xbd, 0x7e, 0x33,
- 0x6e, 0x4d, 0x7f, 0x73, 0xc0, 0xbb, 0x8d, 0x0d, 0x01, 0xcd, 0x78, 0x85, 0xd2, 0xff, 0xd3, 0x85,
- 0x25, 0xe8, 0x4b, 0xc5, 0x0b, 0xdb, 0x77, 0xe7, 0x1e, 0x7d, 0xf7, 0x34, 0x4d, 0x27, 0xa6, 0xbf,
- 0x3f, 0x00, 0x3b, 0x2b, 0x24, 0x50, 0x26, 0xe1, 0x31, 0x78, 0x5b, 0xd1, 0xac, 0x48, 0x91, 0xa2,
- 0xa1, 0x35, 0xcf, 0x94, 0x3a, 0x58, 0x7c, 0x64, 0x4c, 0xbd, 0x3e, 0x8e, 0xde, 0xb5, 0x01, 0xac,
- 0xe6, 0xde, 0x81, 0x89, 0x1e, 0x29, 0xa4, 0x68, 0xf0, 0xb8, 0xd1, 0xb0, 0x41, 0xf8, 0x09, 0x70,
- 0x95, 0x28, 0xa5, 0xd2, 0x37, 0x5a, 0x50, 0xc1, 0x38, 0x09, 0x4f, 0x04, 0xc2, 0xeb, 0x6e, 0xdb,
- 0xc1, 0x6e, 0x93, 0x5f, 0x99, 0xf4, 0xcb, 0x3a, 0x0b, 0xbf, 0x05, 0x10, 0xe3, 0xaa, 0xf1, 0xa0,
- 0x26, 0x9b, 0x2b, 0x18, 0x2c, 0x86, 0xff, 0x6a, 0xf3, 0x8b, 0xfa, 0x13, 0xb3, 0x5d, 0xbe, 0xd6,
- 0x5d, 0xbe, 0x83, 0x71, 0x55, 0x7b, 0x64, 0xa5, 0xe1, 0x11, 0x78, 0xaa, 0xc7, 0x6f, 0x5b, 0xb3,
- 0x73, 0x77, 0xcd, 0x27, 0x9a, 0x7f, 0x43, 0x74, 0x1a, 0x81, 0x27, 0x87, 0x28, 0x27, 0x32, 0x41,
- 0xa7, 0xf4, 0x6b, 0xaa, 0x10, 0x41, 0x0a, 0xc1, 0xe7, 0x60, 0xb7, 0x79, 0x48, 0xc2, 0x13, 0x4a,
- 0xc3, 0x82, 0xf3, 0x34, 0x44, 0x84, 0x88, 0xda, 0xfe, 0xa7, 0x4d, 0xf6, 0x25, 0xa5, 0x2b, 0xce,
- 0xd3, 0x25, 0x21, 0x02, 0xba, 0xa0, 0x5b, 0x51, 0x21, 0x37, 0x83, 0xd0, 0x6c, 0xa7, 0x1f, 0x82,
- 0xfe, 0x51, 0x8a, 0x64, 0xb2, 0xc4, 0xa7, 0x12, 0xbe, 0x07, 0xfa, 0x5a, 0x89, 0x4a, 0x49, 0xa5,
- 0xeb, 0x4c, 0xda, 0xb3, 0x7e, 0xb0, 0x09, 0x4c, 0x15, 0x18, 0xde, 0xf6, 0x12, 0x48, 0xf8, 0x03,
- 0xe8, 0x16, 0xd4, 0x8c, 0xaf, 0x21, 0x0e, 0x16, 0x9f, 0x7b, 0x77, 0x78, 0x07, 0xbd, 0xdb, 0x04,
- 0x83, 0x46, 0x6d, 0x2a, 0x36, 0xef, 0xcf, 0xd6, 0xe0, 0x4b, 0xf8, 0xfd, 0xf6, 0xa1, 0x9f, 0xdd,
- 0xeb, 0xd0, 0x2d, 0xbd, 0xf5, 0x99, 0xfb, 0xc7, 0xe7, 0x97, 0x23, 0xe7, 0xe2, 0x72, 0xe4, 0xfc,
- 0x7d, 0x39, 0x72, 0x5e, 0x5d, 0x8d, 0x5a, 0x17, 0x57, 0xa3, 0xd6, 0x1f, 0x57, 0xa3, 0xd6, 0x8f,
- 0x2f, 0x62, 0xa6, 0x92, 0x32, 0xf2, 0x30, 0xcf, 0x7c, 0xcc, 0x65, 0xc6, 0xa5, 0xbf, 0x39, 0xf1,
- 0xe3, 0xf5, 0x2f, 0xe2, 0xe7, 0x9b, 0x3f, 0x09, 0x75, 0x56, 0x50, 0x19, 0xed, 0x18, 0xfb, 0x9f,
- 0xff, 0x13, 0x00, 0x00, 0xff, 0xff, 0xc2, 0x4e, 0xda, 0x4e, 0x55, 0x06, 0x00, 0x00,
+ // 797 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x41, 0x6f, 0xe3, 0x44,
+ 0x14, 0x8e, 0x49, 0xda, 0x24, 0x93, 0x65, 0x61, 0x67, 0x57, 0xc5, 0xa9, 0x50, 0x12, 0xc2, 0x25,
+ 0x08, 0x61, 0x2b, 0xd9, 0x0b, 0xac, 0xe0, 0x90, 0x16, 0x2d, 0xe5, 0x80, 0xc8, 0xba, 0x05, 0x24,
+ 0x2e, 0xd6, 0x78, 0x66, 0x6a, 0x8f, 0x6a, 0x7b, 0xac, 0x99, 0xb1, 0xa1, 0x77, 0x0e, 0x1c, 0x57,
+ 0xe2, 0xb2, 0xc7, 0xfd, 0x07, 0xfc, 0x8d, 0x3d, 0xf6, 0xc8, 0x09, 0x50, 0xfb, 0x47, 0xd0, 0xcc,
+ 0xd8, 0x49, 0x1b, 0xa8, 0xd4, 0x1e, 0xb8, 0x79, 0xde, 0xfb, 0xbe, 0x6f, 0xde, 0x9b, 0xef, 0xcd,
+ 0x18, 0x2c, 0x58, 0xae, 0xa8, 0xc0, 0x09, 0x62, 0x79, 0x28, 0x29, 0x2e, 0x05, 0x53, 0xe7, 0x3e,
+ 0xc6, 0x95, 0x5f, 0x08, 0x5e, 0x31, 0x42, 0x85, 0x5f, 0xcd, 0xd7, 0xdf, 0x5e, 0x21, 0xb8, 0xe2,
+ 0xf0, 0xc3, 0xff, 0xe0, 0x78, 0x18, 0x57, 0xde, 0x1a, 0x57, 0xcd, 0xf7, 0x9f, 0xc4, 0x3c, 0xe6,
+ 0x06, 0xef, 0xeb, 0x2f, 0x4b, 0xdd, 0x1f, 0xc7, 0x9c, 0xc7, 0x29, 0xf5, 0xcd, 0x2a, 0x2a, 0x4f,
+ 0x7d, 0xc5, 0x32, 0x2a, 0x15, 0xca, 0x8a, 0x1a, 0x30, 0xda, 0x06, 0x90, 0x52, 0x20, 0xc5, 0x78,
+ 0xde, 0x08, 0xb0, 0x08, 0xfb, 0x98, 0x0b, 0xea, 0xe3, 0x94, 0xd1, 0x5c, 0xe9, 0xf2, 0xec, 0x57,
+ 0x0d, 0xf0, 0x35, 0x20, 0x65, 0x71, 0xa2, 0x6c, 0x58, 0xfa, 0x8a, 0xe6, 0x84, 0x8a, 0x8c, 0x59,
+ 0xf0, 0x66, 0x65, 0x09, 0xd3, 0x5f, 0xda, 0xc0, 0x3d, 0xe4, 0xb9, 0x2c, 0x33, 0x2a, 0x96, 0x84,
+ 0x30, 0xbd, 0xd9, 0x4a, 0xf0, 0x82, 0x4b, 0x94, 0xc2, 0x27, 0x60, 0x47, 0x31, 0x95, 0x52, 0xd7,
+ 0x99, 0x38, 0xb3, 0x7e, 0x60, 0x17, 0x70, 0x02, 0x06, 0x84, 0x4a, 0x2c, 0x58, 0xa1, 0xc1, 0xee,
+ 0x5b, 0x26, 0x77, 0x3d, 0x04, 0x87, 0xa0, 0x67, 0xcf, 0x87, 0x11, 0xb7, 0x6d, 0xd2, 0x5d, 0xb3,
+ 0xfe, 0x9a, 0xc0, 0xaf, 0xc0, 0x43, 0x96, 0x33, 0xc5, 0x50, 0x1a, 0x26, 0x54, 0xd7, 0xe9, 0x76,
+ 0x26, 0xce, 0x6c, 0xb0, 0xd8, 0xf7, 0x58, 0x84, 0x3d, 0xdd, 0x9a, 0x57, 0x37, 0x54, 0xcd, 0xbd,
+ 0x23, 0x83, 0x38, 0xe8, 0xbc, 0xf9, 0x73, 0xdc, 0x0a, 0xde, 0xae, 0x79, 0x36, 0x08, 0x3f, 0x00,
+ 0x0f, 0x62, 0x9a, 0x53, 0xc9, 0x64, 0x98, 0x20, 0x99, 0xb8, 0x3b, 0x13, 0x67, 0xf6, 0x20, 0x18,
+ 0xd4, 0xb1, 0x23, 0x24, 0x13, 0x38, 0x06, 0x83, 0x88, 0xe5, 0x48, 0x9c, 0x5b, 0xc4, 0xae, 0x41,
+ 0x00, 0x1b, 0x32, 0x80, 0x43, 0x00, 0x64, 0x81, 0x7e, 0xca, 0x43, 0xed, 0x83, 0xdb, 0xad, 0x0b,
+ 0xb1, 0x1e, 0x78, 0x8d, 0x07, 0xde, 0x49, 0x63, 0xd2, 0x41, 0x4f, 0x17, 0xf2, 0xf2, 0xaf, 0xb1,
+ 0x13, 0xf4, 0x0d, 0x4f, 0x67, 0xe0, 0x67, 0x60, 0x98, 0x72, 0x7c, 0x16, 0x96, 0x79, 0xc4, 0x73,
+ 0xc2, 0xf2, 0x38, 0xe4, 0x56, 0x90, 0x97, 0xca, 0xed, 0x4d, 0x9c, 0x59, 0x2f, 0xd8, 0xd3, 0x80,
+ 0xef, 0x9a, 0xfc, 0xb7, 0x86, 0xc7, 0x4b, 0xf5, 0xac, 0xf7, 0xeb, 0xeb, 0x71, 0xeb, 0xd5, 0xeb,
+ 0x71, 0x6b, 0xfa, 0xbb, 0x03, 0xde, 0x6b, 0x6c, 0x08, 0x68, 0xc6, 0x2b, 0x94, 0xfe, 0x9f, 0x2e,
+ 0x2c, 0x41, 0x5f, 0x2a, 0x5e, 0xd8, 0xbe, 0x3b, 0xf7, 0xe8, 0xbb, 0xa7, 0x69, 0x3a, 0x31, 0xfd,
+ 0xad, 0x0d, 0x76, 0x57, 0x48, 0xa0, 0x4c, 0xc2, 0x13, 0xf0, 0x8e, 0xa2, 0x59, 0x91, 0x22, 0x45,
+ 0x43, 0x6b, 0x9e, 0x29, 0x75, 0xb0, 0xf8, 0xd8, 0x98, 0x7a, 0x7d, 0x1c, 0xbd, 0x6b, 0x03, 0x58,
+ 0xcd, 0xbd, 0x43, 0x13, 0x3d, 0x56, 0x48, 0xd1, 0xe0, 0x61, 0xa3, 0x61, 0x83, 0xf0, 0x53, 0xe0,
+ 0x2a, 0x51, 0x4a, 0xa5, 0x4f, 0xb4, 0xa0, 0x82, 0x71, 0x12, 0x9e, 0x0a, 0x84, 0xd7, 0xdd, 0xb6,
+ 0x83, 0xbd, 0x26, 0xbf, 0x32, 0xe9, 0xe7, 0x75, 0x16, 0xbe, 0x00, 0x10, 0xe3, 0xaa, 0xf1, 0xa0,
+ 0x26, 0x9b, 0x23, 0x18, 0x2c, 0x86, 0xff, 0x6a, 0xf3, 0xcb, 0xfa, 0x8a, 0xd9, 0x2e, 0x5f, 0xe9,
+ 0x2e, 0xdf, 0xc5, 0xb8, 0xaa, 0x3d, 0xb2, 0xd2, 0xf0, 0x18, 0x3c, 0xd6, 0xe3, 0xb7, 0xad, 0xd9,
+ 0xb9, 0xbb, 0xe6, 0x23, 0xcd, 0xbf, 0x29, 0xfa, 0x02, 0xc0, 0x4a, 0xe2, 0x6d, 0xcd, 0x9d, 0x7b,
+ 0xd4, 0x59, 0x49, 0x7c, 0x43, 0x72, 0x1a, 0x81, 0x47, 0x47, 0x28, 0x27, 0x32, 0x41, 0x67, 0xf4,
+ 0x1b, 0xaa, 0x10, 0x41, 0x0a, 0xc1, 0xa7, 0x60, 0xaf, 0x79, 0x9b, 0xc2, 0x53, 0x4a, 0xc3, 0x82,
+ 0xf3, 0x34, 0x44, 0x84, 0x88, 0x7a, 0xa2, 0x1e, 0x37, 0xd9, 0xe7, 0x94, 0xae, 0x38, 0x4f, 0x97,
+ 0x84, 0x08, 0xe8, 0x82, 0x6e, 0x45, 0x85, 0xdc, 0xcc, 0x56, 0xb3, 0x9c, 0x7e, 0x04, 0xfa, 0xc7,
+ 0x29, 0x92, 0xc9, 0x12, 0x9f, 0x49, 0xf8, 0x3e, 0xe8, 0x6b, 0x25, 0x2a, 0x25, 0x95, 0xae, 0x33,
+ 0x69, 0xcf, 0xfa, 0xc1, 0x26, 0x30, 0x55, 0x60, 0x78, 0xdb, 0xe3, 0x22, 0xe1, 0x0f, 0xa0, 0x5b,
+ 0x50, 0x73, 0x23, 0x0c, 0x71, 0xb0, 0xf8, 0xc2, 0xbb, 0xc3, 0xd3, 0xea, 0xdd, 0x26, 0x18, 0x34,
+ 0x6a, 0x53, 0xb1, 0x79, 0xd2, 0xb6, 0xee, 0x92, 0x84, 0xdf, 0x6f, 0x6f, 0xfa, 0xf9, 0xbd, 0x36,
+ 0xdd, 0xd2, 0x5b, 0xef, 0x79, 0x70, 0xf2, 0xe6, 0x72, 0xe4, 0x5c, 0x5c, 0x8e, 0x9c, 0xbf, 0x2f,
+ 0x47, 0xce, 0xcb, 0xab, 0x51, 0xeb, 0xe2, 0x6a, 0xd4, 0xfa, 0xe3, 0x6a, 0xd4, 0xfa, 0xf1, 0x59,
+ 0xcc, 0x54, 0x52, 0x46, 0x1e, 0xe6, 0x99, 0x8f, 0xb9, 0xcc, 0xb8, 0xf4, 0x37, 0x3b, 0x7e, 0xb2,
+ 0xfe, 0xeb, 0xfc, 0x7c, 0xf3, 0xbf, 0xa3, 0xce, 0x0b, 0x2a, 0xa3, 0x5d, 0xe3, 0xfe, 0xd3, 0x7f,
+ 0x02, 0x00, 0x00, 0xff, 0xff, 0x97, 0x79, 0x3f, 0x95, 0xa8, 0x06, 0x00, 0x00,
}
func (m *ConsumerAdditionProposal) Marshal() (dAtA []byte, err error) {
@@ -651,21 +664,29 @@ func (m *Params) MarshalToSizedBuffer(dAtA []byte) (int, error) {
_ = i
var l int
_ = l
- n4, err4 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.InitTimeoutPeriod, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.InitTimeoutPeriod):])
+ n4, err4 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.VscTimeoutPeriod, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.VscTimeoutPeriod):])
if err4 != nil {
return 0, err4
}
i -= n4
i = encodeVarintProvider(dAtA, i, uint64(n4))
i--
- dAtA[i] = 0x22
- n5, err5 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.CcvTimeoutPeriod, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.CcvTimeoutPeriod):])
+ dAtA[i] = 0x2a
+ n5, err5 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.InitTimeoutPeriod, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.InitTimeoutPeriod):])
if err5 != nil {
return 0, err5
}
i -= n5
i = encodeVarintProvider(dAtA, i, uint64(n5))
i--
+ dAtA[i] = 0x22
+ n6, err6 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.CcvTimeoutPeriod, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.CcvTimeoutPeriod):])
+ if err6 != nil {
+ return 0, err6
+ }
+ i -= n6
+ i = encodeVarintProvider(dAtA, i, uint64(n6))
+ i--
dAtA[i] = 0x1a
if m.TrustingPeriodFraction != 0 {
i = encodeVarintProvider(dAtA, i, uint64(m.TrustingPeriodFraction))
@@ -917,6 +938,8 @@ func (m *Params) Size() (n int) {
n += 1 + l + sovProvider(uint64(l))
l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.InitTimeoutPeriod)
n += 1 + l + sovProvider(uint64(l))
+ l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.VscTimeoutPeriod)
+ n += 1 + l + sovProvider(uint64(l))
return n
}
@@ -1617,6 +1640,39 @@ func (m *Params) Unmarshal(dAtA []byte) error {
return err
}
iNdEx = postIndex
+ case 5:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field VscTimeoutPeriod", wireType)
+ }
+ var msglen int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowProvider
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ msglen |= int(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ if msglen < 0 {
+ return ErrInvalidLengthProvider
+ }
+ postIndex := iNdEx + msglen
+ if postIndex < 0 {
+ return ErrInvalidLengthProvider
+ }
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.VscTimeoutPeriod, dAtA[iNdEx:postIndex]); err != nil {
+ return err
+ }
+ iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipProvider(dAtA[iNdEx:])