Refactor 04-channel/keeper tests to ibc testing pkg (#6400)

* update testing pkg and first keeper test

* fix version bug

* add more helper testing funcs

* move create header to testing pkg

* fix connection state bug

* add staking genesis state

* update simapp with setting validator helper func

* update simapp with valset helper

* fix app hash issue

* update to query from correct iavl proof height

* first keeper test passing

* second test passing 🎉

* fix build

* update tests in all keeper_test

* fix lint

* begin refactoring querier test

* update first querier test and update testing helpers

* finish updating rest of querier tests

* rename ChannelID in TestChannel to ID

* remove usage of chain id for calling helper funcs

* update openinit and opentry tests

* finish opening channel handshake tests

* finish handshake tests

* general testing pkg cleanup

* finish packetsend refactor

* update recvpacket

* packet executed refactored

* finish packet test 🎉

* all tests passing

* cleanup and increase code cov

* remove todos in favor of opened issue #6509

* bump invalid id to meet validation requirements

* bubble up proof height + 1

* Apply suggestions from code review

Co-authored-by: Aditya <adityasripal@gmail.com>
Co-authored-by: Federico Kunze <31522760+fedekunze@users.noreply.github.com>

* fix uninit conn test

* fix compile and address various pr review issues

* Update x/ibc/04-channel/keeper/handshake_test.go

Co-authored-by: Aditya <adityasripal@gmail.com>

* Update x/ibc/04-channel/keeper/handshake_test.go

Co-authored-by: Aditya <adityasripal@gmail.com>

* address @AdityaSripal comments and increase cov

Co-authored-by: Aditya <adityasripal@gmail.com>
Co-authored-by: Federico Kunze <31522760+fedekunze@users.noreply.github.com>
This commit is contained in:
colin axner 2020-06-26 18:36:04 +02:00 committed by GitHub
parent 9bf3ff75f5
commit 43837b16e7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 2536 additions and 1936 deletions

View File

@ -6,6 +6,7 @@ import (
"fmt" "fmt"
"strconv" "strconv"
"testing" "testing"
"time"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
@ -21,6 +22,7 @@ import (
sdk "github.com/cosmos/cosmos-sdk/types" sdk "github.com/cosmos/cosmos-sdk/types"
authtypes "github.com/cosmos/cosmos-sdk/x/auth/types" authtypes "github.com/cosmos/cosmos-sdk/x/auth/types"
banktypes "github.com/cosmos/cosmos-sdk/x/bank/types" banktypes "github.com/cosmos/cosmos-sdk/x/bank/types"
stakingtypes "github.com/cosmos/cosmos-sdk/x/staking/types"
) )
// DefaultConsensusParams defines the default Tendermint consensus params used in // DefaultConsensusParams defines the default Tendermint consensus params used in
@ -67,6 +69,77 @@ func Setup(isCheckTx bool) *SimApp {
return app return app
} }
// SetupWithGenesisValSet initializes a new SimApp with a validator set and genesis accounts
// that also act as delegators. For simplicity, each validator is bonded with a delegation
// of one consensus engine unit (10^6) in the default token of the simapp from first genesis
// account. A Nop logger is set in SimApp.
func SetupWithGenesisValSet(t *testing.T, valSet *tmtypes.ValidatorSet, genAccs []authtypes.GenesisAccount, balances ...banktypes.Balance) *SimApp {
db := dbm.NewMemDB()
app := NewSimApp(log.NewNopLogger(), db, nil, true, map[int64]bool{}, DefaultNodeHome, 5)
genesisState := NewDefaultGenesisState()
// set genesis accounts
authGenesis := authtypes.NewGenesisState(authtypes.DefaultParams(), genAccs)
genesisState[authtypes.ModuleName] = app.Codec().MustMarshalJSON(authGenesis)
validators := make([]stakingtypes.Validator, 0, len(valSet.Validators))
delegations := make([]stakingtypes.Delegation, 0, len(valSet.Validators))
bondAmt := sdk.NewInt(1000000)
for _, val := range valSet.Validators {
validator := stakingtypes.Validator{
OperatorAddress: val.Address.Bytes(),
ConsensusPubkey: sdk.MustBech32ifyPubKey(sdk.Bech32PubKeyTypeConsPub, val.PubKey),
Jailed: false,
Status: sdk.Bonded,
Tokens: bondAmt,
DelegatorShares: sdk.OneDec(),
Description: stakingtypes.Description{},
UnbondingHeight: int64(0),
UnbondingTime: time.Unix(0, 0).UTC(),
Commission: stakingtypes.NewCommission(sdk.ZeroDec(), sdk.ZeroDec(), sdk.ZeroDec()),
MinSelfDelegation: sdk.ZeroInt(),
}
validators = append(validators, validator)
delegations = append(delegations, stakingtypes.NewDelegation(genAccs[0].GetAddress(), val.Address.Bytes(), sdk.OneDec()))
}
// set validators and delegations
stakingGenesis := stakingtypes.NewGenesisState(stakingtypes.DefaultParams(), validators, delegations)
genesisState[stakingtypes.ModuleName] = app.Codec().MustMarshalJSON(stakingGenesis)
totalSupply := sdk.NewCoins()
for _, b := range balances {
// add genesis acc tokens and delegated tokens to total supply
totalSupply = totalSupply.Add(b.Coins.Add(sdk.NewCoin(sdk.DefaultBondDenom, bondAmt))...)
}
// update total supply
bankGenesis := banktypes.NewGenesisState(banktypes.DefaultGenesisState().SendEnabled, balances, totalSupply)
genesisState[banktypes.ModuleName] = app.Codec().MustMarshalJSON(bankGenesis)
stateBytes, err := codec.MarshalJSONIndent(app.Codec(), genesisState)
require.NoError(t, err)
// init chain will set the validator set and initialize the genesis accounts
app.InitChain(
abci.RequestInitChain{
Validators: []abci.ValidatorUpdate{},
ConsensusParams: DefaultConsensusParams,
AppStateBytes: stateBytes,
},
)
// commit genesis changes
app.Commit()
app.BeginBlock(abci.RequestBeginBlock{Header: abci.Header{Height: app.LastBlockHeight() + 1, AppHash: app.LastCommitID().Hash}})
return app
}
// SetupWithGenesisAccounts initializes a new SimApp with the provided genesis // SetupWithGenesisAccounts initializes a new SimApp with the provided genesis
// accounts and possible balances. // accounts and possible balances.
func SetupWithGenesisAccounts(genAccs []authtypes.GenesisAccount, balances ...banktypes.Balance) *SimApp { func SetupWithGenesisAccounts(genAccs []authtypes.GenesisAccount, balances ...banktypes.Balance) *SimApp {

File diff suppressed because it is too large Load Diff

View File

@ -1,234 +1,250 @@
package keeper_test package keeper_test
import ( import (
"fmt"
"testing" "testing"
"time"
"github.com/stretchr/testify/suite" "github.com/stretchr/testify/suite"
abci "github.com/tendermint/tendermint/abci/types"
lite "github.com/tendermint/tendermint/lite2"
tmtypes "github.com/tendermint/tendermint/types"
"github.com/cosmos/cosmos-sdk/codec" clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
"github.com/cosmos/cosmos-sdk/simapp"
sdk "github.com/cosmos/cosmos-sdk/types"
connectiontypes "github.com/cosmos/cosmos-sdk/x/ibc/03-connection/types"
"github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types" "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types"
ibctmtypes "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types" ibctesting "github.com/cosmos/cosmos-sdk/x/ibc/testing"
commitmenttypes "github.com/cosmos/cosmos-sdk/x/ibc/23-commitment/types"
host "github.com/cosmos/cosmos-sdk/x/ibc/24-host"
ibckeeper "github.com/cosmos/cosmos-sdk/x/ibc/keeper"
stakingtypes "github.com/cosmos/cosmos-sdk/x/staking/types"
)
// define constants used for testing
const (
testClientIDA = "testclientida"
testConnectionIDA = "connectionidatob"
testClientIDB = "testclientidb"
testConnectionIDB = "connectionidbtoa"
testPort1 = "firstport"
testPort2 = "secondport"
testPort3 = "thirdport"
testChannel1 = "firstchannel"
testChannel2 = "secondchannel"
testChannel3 = "thirdchannel"
testChannelOrder = types.ORDERED
testChannelVersion = "1.0"
trustingPeriod time.Duration = time.Hour * 24 * 7 * 2
ubdPeriod time.Duration = time.Hour * 24 * 7 * 3
maxClockDrift time.Duration = time.Second * 10
timeoutHeight = 100
timeoutTimestamp = 100
disabledTimeoutTimestamp = 0
disabledTimeoutHeight = 0
)
var (
testPacketCommitment = []byte("packet commitment")
testAcknowledgement = []byte("acknowledgement")
) )
// KeeperTestSuite is a testing suite to test keeper functions.
type KeeperTestSuite struct { type KeeperTestSuite struct {
suite.Suite suite.Suite
cdc *codec.Codec coordinator *ibctesting.Coordinator
querier sdk.Querier
chainA *TestChain // testing chains used for convience and readability
chainB *TestChain chainA *ibctesting.TestChain
chainB *ibctesting.TestChain
} }
// TestKeeperTestSuite runs all the tests within this package.
func TestKeeperTestSuite(t *testing.T) {
suite.Run(t, new(KeeperTestSuite))
}
// SetupTest creates a coordinator with 2 test chains.
func (suite *KeeperTestSuite) SetupTest() { func (suite *KeeperTestSuite) SetupTest() {
suite.chainA = NewTestChain(testClientIDA) suite.coordinator = ibctesting.NewCoordinator(suite.T(), 2)
suite.chainB = NewTestChain(testClientIDB) suite.chainA = suite.coordinator.GetChain(ibctesting.GetChainID(0))
suite.chainB = suite.coordinator.GetChain(ibctesting.GetChainID(1))
suite.cdc = suite.chainA.App.Codec()
suite.querier = ibckeeper.NewQuerier(*suite.chainA.App.IBCKeeper)
} }
// TestSetChannel create clients and connections on both chains. It tests for the non-existence
// and existence of a channel in INIT on chainA.
func (suite *KeeperTestSuite) TestSetChannel() { func (suite *KeeperTestSuite) TestSetChannel() {
ctx := suite.chainB.GetContext() // create client and connections on both chains
_, found := suite.chainB.App.IBCKeeper.ChannelKeeper.GetChannel(ctx, testPort1, testChannel1) _, _, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
// check for channel to be created on chainB
channelA := connA.NextTestChannel()
_, found := suite.chainA.App.IBCKeeper.ChannelKeeper.GetChannel(suite.chainA.GetContext(), channelA.PortID, channelA.ID)
suite.False(found) suite.False(found)
counterparty2 := types.NewCounterparty(testPort2, testChannel2) // init channel
channel := types.NewChannel( channelA, channelB, err := suite.coordinator.ChanOpenInit(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
types.INIT, testChannelOrder, suite.NoError(err)
counterparty2, []string{testConnectionIDA}, testChannelVersion,
) storedChannel, found := suite.chainA.App.IBCKeeper.ChannelKeeper.GetChannel(suite.chainA.GetContext(), channelA.PortID, channelA.ID)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetChannel(ctx, testPort1, testChannel1, channel) expectedCounterparty := types.NewCounterparty(channelB.PortID, channelB.ID)
storedChannel, found := suite.chainB.App.IBCKeeper.ChannelKeeper.GetChannel(ctx, testPort1, testChannel1)
suite.True(found) suite.True(found)
suite.Equal(channel, storedChannel) suite.Equal(types.INIT, storedChannel.State)
suite.Equal(types.ORDERED, storedChannel.Ordering)
suite.Equal(expectedCounterparty, storedChannel.Counterparty)
} }
// TestGetAllChannels creates multiple channels on chain A through various connections
// and tests their retrieval. 2 channels are on connA0 and 1 channel is on connA1
func (suite KeeperTestSuite) TestGetAllChannels() { func (suite KeeperTestSuite) TestGetAllChannels() {
// Channel (Counterparty): A(C) -> C(B) -> B(A) clientA, clientB, connA0, connB0, testchannel0, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
counterparty1 := types.NewCounterparty(testPort1, testChannel1) // channel0 on first connection on chainA
counterparty2 := types.NewCounterparty(testPort2, testChannel2) counterparty0 := types.Counterparty{
counterparty3 := types.NewCounterparty(testPort3, testChannel3) PortID: connB0.Channels[0].PortID,
ChannelID: connB0.Channels[0].ID,
}
// channel1 is second channel on first connection on chainA
testchannel1, _ := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA0, connB0, types.ORDERED)
counterparty1 := types.Counterparty{
PortID: connB0.Channels[1].PortID,
ChannelID: connB0.Channels[1].ID,
}
connA1, connB1 := suite.coordinator.CreateConnection(suite.chainA, suite.chainB, clientA, clientB)
// channel2 is on a second connection on chainA
testchannel2, _, err := suite.coordinator.ChanOpenInit(suite.chainA, suite.chainB, connA1, connB1, types.UNORDERED)
suite.Require().NoError(err)
counterparty2 := types.Counterparty{
PortID: connB1.Channels[0].PortID,
ChannelID: connB1.Channels[0].ID,
}
channel0 := types.NewChannel(
types.OPEN, types.UNORDERED,
counterparty0, []string{connB0.ID}, ibctesting.ChannelVersion,
)
channel1 := types.NewChannel( channel1 := types.NewChannel(
types.INIT, testChannelOrder, types.OPEN, types.ORDERED,
counterparty3, []string{testConnectionIDA}, testChannelVersion, counterparty1, []string{connB0.ID}, ibctesting.ChannelVersion,
) )
channel2 := types.NewChannel( channel2 := types.NewChannel(
types.INIT, testChannelOrder, types.INIT, types.UNORDERED,
counterparty1, []string{testConnectionIDA}, testChannelVersion, counterparty2, []string{connB1.ID}, ibctesting.ChannelVersion,
)
channel3 := types.NewChannel(
types.CLOSED, testChannelOrder,
counterparty2, []string{testConnectionIDA}, testChannelVersion,
) )
expChannels := []types.IdentifiedChannel{ expChannels := []types.IdentifiedChannel{
types.NewIdentifiedChannel(testPort1, testChannel1, channel1), types.NewIdentifiedChannel(testchannel0.PortID, testchannel0.ID, channel0),
types.NewIdentifiedChannel(testPort2, testChannel2, channel2), types.NewIdentifiedChannel(testchannel1.PortID, testchannel1.ID, channel1),
types.NewIdentifiedChannel(testPort3, testChannel3, channel3), types.NewIdentifiedChannel(testchannel2.PortID, testchannel2.ID, channel2),
} }
ctx := suite.chainB.GetContext() ctxA := suite.chainA.GetContext()
suite.chainB.App.IBCKeeper.ChannelKeeper.SetChannel(ctx, testPort1, testChannel1, channel1) channels := suite.chainA.App.IBCKeeper.ChannelKeeper.GetAllChannels(ctxA)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetChannel(ctx, testPort2, testChannel2, channel2)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetChannel(ctx, testPort3, testChannel3, channel3)
channels := suite.chainB.App.IBCKeeper.ChannelKeeper.GetAllChannels(ctx)
suite.Require().Len(channels, len(expChannels)) suite.Require().Len(channels, len(expChannels))
suite.Require().Equal(expChannels, channels) suite.Require().Equal(expChannels, channels)
} }
// TestGetAllSequences sets all packet sequences for two different channels on chain A and
// tests their retrieval.
func (suite KeeperTestSuite) TestGetAllSequences() { func (suite KeeperTestSuite) TestGetAllSequences() {
seq1 := types.NewPacketSequence(testPort1, testChannel1, 1) _, _, connA, connB, channelA0, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
seq2 := types.NewPacketSequence(testPort2, testChannel2, 2) channelA1, _ := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.UNORDERED)
expSeqs := []types.PacketSequence{seq1, seq2} seq1 := types.NewPacketSequence(channelA0.PortID, channelA0.ID, 1)
seq2 := types.NewPacketSequence(channelA0.PortID, channelA0.ID, 2)
seq3 := types.NewPacketSequence(channelA1.PortID, channelA1.ID, 3)
ctx := suite.chainB.GetContext() // seq1 should be overwritten by seq2
expSeqs := []types.PacketSequence{seq2, seq3}
for _, seq := range expSeqs { ctxA := suite.chainA.GetContext()
suite.chainB.App.IBCKeeper.ChannelKeeper.SetNextSequenceSend(ctx, seq.PortID, seq.ChannelID, seq.Sequence)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetNextSequenceRecv(ctx, seq.PortID, seq.ChannelID, seq.Sequence) for _, seq := range []types.PacketSequence{seq1, seq2, seq3} {
suite.chainB.App.IBCKeeper.ChannelKeeper.SetNextSequenceAck(ctx, seq.PortID, seq.ChannelID, seq.Sequence) suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceSend(ctxA, seq.PortID, seq.ChannelID, seq.Sequence)
suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceRecv(ctxA, seq.PortID, seq.ChannelID, seq.Sequence)
suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceAck(ctxA, seq.PortID, seq.ChannelID, seq.Sequence)
} }
sendSeqs := suite.chainB.App.IBCKeeper.ChannelKeeper.GetAllPacketSendSeqs(ctx) sendSeqs := suite.chainA.App.IBCKeeper.ChannelKeeper.GetAllPacketSendSeqs(ctxA)
recvSeqs := suite.chainB.App.IBCKeeper.ChannelKeeper.GetAllPacketRecvSeqs(ctx) recvSeqs := suite.chainA.App.IBCKeeper.ChannelKeeper.GetAllPacketRecvSeqs(ctxA)
ackSeqs := suite.chainB.App.IBCKeeper.ChannelKeeper.GetAllPacketAckSeqs(ctx) ackSeqs := suite.chainA.App.IBCKeeper.ChannelKeeper.GetAllPacketAckSeqs(ctxA)
suite.Require().Len(sendSeqs, 2) suite.Len(sendSeqs, 2)
suite.Require().Len(recvSeqs, 2) suite.Len(recvSeqs, 2)
suite.Require().Len(ackSeqs, 2) suite.Len(ackSeqs, 2)
suite.Require().Equal(expSeqs, sendSeqs) suite.Equal(expSeqs, sendSeqs)
suite.Require().Equal(expSeqs, recvSeqs) suite.Equal(expSeqs, recvSeqs)
suite.Require().Equal(expSeqs, ackSeqs) suite.Equal(expSeqs, ackSeqs)
} }
// TestGetAllCommitmentsAcks creates a set of acks and packet commitments on two different
// channels on chain A and tests their retrieval.
func (suite KeeperTestSuite) TestGetAllCommitmentsAcks() { func (suite KeeperTestSuite) TestGetAllCommitmentsAcks() {
ack1 := types.NewPacketAckCommitment(testPort1, testChannel1, 1, []byte("ack")) _, _, connA, connB, channelA0, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
ack2 := types.NewPacketAckCommitment(testPort1, testChannel1, 2, []byte("ack")) channelA1, _ := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.UNORDERED)
comm1 := types.NewPacketAckCommitment(testPort1, testChannel1, 1, []byte("hash"))
comm2 := types.NewPacketAckCommitment(testPort1, testChannel1, 2, []byte("hash"))
expAcks := []types.PacketAckCommitment{ack1, ack2} // channel 0 acks
expCommitments := []types.PacketAckCommitment{comm1, comm2} ack1 := types.NewPacketAckCommitment(channelA0.PortID, channelA0.ID, 1, []byte("ack"))
ack2 := types.NewPacketAckCommitment(channelA0.PortID, channelA0.ID, 2, []byte("ack"))
ctx := suite.chainB.GetContext() // duplicate ack
ack2dup := types.NewPacketAckCommitment(channelA0.PortID, channelA0.ID, 2, []byte("ack"))
for i := 0; i < 2; i++ { // channel 1 acks
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketAcknowledgement(ctx, expAcks[i].PortID, expAcks[i].ChannelID, expAcks[i].Sequence, expAcks[i].Hash) ack3 := types.NewPacketAckCommitment(channelA1.PortID, channelA1.ID, 1, []byte("ack"))
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(ctx, expCommitments[i].PortID, expCommitments[i].ChannelID, expCommitments[i].Sequence, expCommitments[i].Hash)
// channel 0 packet commitments
comm1 := types.NewPacketAckCommitment(channelA0.PortID, channelA0.ID, 1, []byte("hash"))
comm2 := types.NewPacketAckCommitment(channelA0.PortID, channelA0.ID, 2, []byte("hash"))
// channel 1 packet commitments
comm3 := types.NewPacketAckCommitment(channelA1.PortID, channelA1.ID, 1, []byte("hash"))
comm4 := types.NewPacketAckCommitment(channelA1.PortID, channelA1.ID, 2, []byte("hash"))
expAcks := []types.PacketAckCommitment{ack1, ack2, ack3}
expCommitments := []types.PacketAckCommitment{comm1, comm2, comm3, comm4}
ctxA := suite.chainA.GetContext()
// set acknowledgements
for _, ack := range []types.PacketAckCommitment{ack1, ack2, ack2dup, ack3} {
suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketAcknowledgement(ctxA, ack.PortID, ack.ChannelID, ack.Sequence, ack.Hash)
} }
acks := suite.chainB.App.IBCKeeper.ChannelKeeper.GetAllPacketAcks(ctx) // set packet commitments
commitments := suite.chainB.App.IBCKeeper.ChannelKeeper.GetAllPacketCommitments(ctx) for _, comm := range expCommitments {
suite.Require().Len(acks, 2) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(ctxA, comm.PortID, comm.ChannelID, comm.Sequence, comm.Hash)
suite.Require().Len(commitments, 2) }
acks := suite.chainA.App.IBCKeeper.ChannelKeeper.GetAllPacketAcks(ctxA)
commitments := suite.chainA.App.IBCKeeper.ChannelKeeper.GetAllPacketCommitments(ctxA)
suite.Require().Len(acks, len(expAcks))
suite.Require().Len(commitments, len(expCommitments))
suite.Require().Equal(expAcks, acks) suite.Require().Equal(expAcks, acks)
suite.Require().Equal(expCommitments, commitments) suite.Require().Equal(expCommitments, commitments)
} }
// TestSetSequence verifies that the keeper correctly sets the sequence counters.
func (suite *KeeperTestSuite) TestSetSequence() { func (suite *KeeperTestSuite) TestSetSequence() {
ctx := suite.chainB.GetContext() _, _, _, _, channelA, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
_, found := suite.chainB.App.IBCKeeper.ChannelKeeper.GetNextSequenceSend(ctx, testPort1, testChannel1)
suite.False(found)
_, found = suite.chainB.App.IBCKeeper.ChannelKeeper.GetNextSequenceRecv(ctx, testPort1, testChannel1) ctxA := suite.chainA.GetContext()
suite.False(found) one := uint64(1)
_, found = suite.chainB.App.IBCKeeper.ChannelKeeper.GetNextSequenceAck(ctx, testPort1, testChannel1) // initialized channel has next send seq of 1
suite.False(found) seq, found := suite.chainA.App.IBCKeeper.ChannelKeeper.GetNextSequenceSend(ctxA, channelA.PortID, channelA.ID)
suite.True(found)
suite.Equal(one, seq)
// initialized channel has next seq recv of 1
seq, found = suite.chainA.App.IBCKeeper.ChannelKeeper.GetNextSequenceRecv(ctxA, channelA.PortID, channelA.ID)
suite.True(found)
suite.Equal(one, seq)
// initialized channel has next seq ack of
seq, found = suite.chainA.App.IBCKeeper.ChannelKeeper.GetNextSequenceAck(ctxA, channelA.PortID, channelA.ID)
suite.True(found)
suite.Equal(one, seq)
nextSeqSend, nextSeqRecv, nextSeqAck := uint64(10), uint64(10), uint64(10) nextSeqSend, nextSeqRecv, nextSeqAck := uint64(10), uint64(10), uint64(10)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetNextSequenceSend(ctx, testPort1, testChannel1, nextSeqSend) suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceSend(ctxA, channelA.PortID, channelA.ID, nextSeqSend)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetNextSequenceRecv(ctx, testPort1, testChannel1, nextSeqRecv) suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceRecv(ctxA, channelA.PortID, channelA.ID, nextSeqRecv)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetNextSequenceAck(ctx, testPort1, testChannel1, nextSeqAck) suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceAck(ctxA, channelA.PortID, channelA.ID, nextSeqAck)
storedNextSeqSend, found := suite.chainB.App.IBCKeeper.ChannelKeeper.GetNextSequenceSend(ctx, testPort1, testChannel1) storedNextSeqSend, found := suite.chainA.App.IBCKeeper.ChannelKeeper.GetNextSequenceSend(ctxA, channelA.PortID, channelA.ID)
suite.True(found) suite.True(found)
suite.Equal(nextSeqSend, storedNextSeqSend) suite.Equal(nextSeqSend, storedNextSeqSend)
storedNextSeqRecv, found := suite.chainB.App.IBCKeeper.ChannelKeeper.GetNextSequenceSend(ctx, testPort1, testChannel1) storedNextSeqRecv, found := suite.chainA.App.IBCKeeper.ChannelKeeper.GetNextSequenceSend(ctxA, channelA.PortID, channelA.ID)
suite.True(found) suite.True(found)
suite.Equal(nextSeqRecv, storedNextSeqRecv) suite.Equal(nextSeqRecv, storedNextSeqRecv)
storedNextSeqAck, found := suite.chainB.App.IBCKeeper.ChannelKeeper.GetNextSequenceAck(ctx, testPort1, testChannel1) storedNextSeqAck, found := suite.chainA.App.IBCKeeper.ChannelKeeper.GetNextSequenceAck(ctxA, channelA.PortID, channelA.ID)
suite.True(found) suite.True(found)
suite.Equal(nextSeqAck, storedNextSeqAck) suite.Equal(nextSeqAck, storedNextSeqAck)
} }
// TestPacketCommitment does basic verification of setting and getting of packet commitments within // TestGetAllPacketCommitmentsAtChannel verifies that the keeper returns all stored packet
// the Channel Keeper. // commitments for a specific channel. The test will store consecutive commitments up to the
func (suite *KeeperTestSuite) TestPacketCommitment() { // value of "seq" and then add non-consecutive up to the value of "maxSeq". A final commitment
ctx := suite.chainB.GetContext() // with the value maxSeq + 1 is set on a different channel.
seq := uint64(10)
storedCommitment := suite.chainB.App.IBCKeeper.ChannelKeeper.GetPacketCommitment(ctx, testPort1, testChannel1, seq)
suite.Nil(storedCommitment)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(ctx, testPort1, testChannel1, seq, testPacketCommitment)
storedCommitment = suite.chainB.App.IBCKeeper.ChannelKeeper.GetPacketCommitment(ctx, testPort1, testChannel1, seq)
suite.Equal(testPacketCommitment, storedCommitment)
}
// TestGetAllPacketCommitmentsAtChannel verifies that iterator returns all stored packet commitments
// for a specific channel.
func (suite *KeeperTestSuite) TestGetAllPacketCommitmentsAtChannel() { func (suite *KeeperTestSuite) TestGetAllPacketCommitmentsAtChannel() {
// setup _, _, connA, connB, channelA, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
ctx := suite.chainB.GetContext()
// create second channel
channelA1, _ := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.UNORDERED)
ctxA := suite.chainA.GetContext()
expectedSeqs := make(map[uint64]bool) expectedSeqs := make(map[uint64]bool)
hash := []byte("commitment")
seq := uint64(15) seq := uint64(15)
maxSeq := uint64(25) maxSeq := uint64(25)
@ -236,289 +252,53 @@ func (suite *KeeperTestSuite) TestGetAllPacketCommitmentsAtChannel() {
// create consecutive commitments // create consecutive commitments
for i := uint64(1); i < seq; i++ { for i := uint64(1); i < seq; i++ {
suite.chainB.storePacketCommitment(ctx, testPort1, testChannel1, i) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(ctxA, channelA.PortID, channelA.ID, i, hash)
expectedSeqs[i] = true expectedSeqs[i] = true
} }
// add non-consecutive commitments // add non-consecutive commitments
for i := seq; i < maxSeq; i += 2 { for i := seq; i < maxSeq; i += 2 {
suite.chainB.storePacketCommitment(ctx, testPort1, testChannel1, i) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(ctxA, channelA.PortID, channelA.ID, i, hash)
expectedSeqs[i] = true expectedSeqs[i] = true
} }
// add sequence on different channel/port // add sequence on different channel/port
suite.chainB.storePacketCommitment(ctx, testPort2, testChannel2, maxSeq+1) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(ctxA, channelA1.PortID, channelA1.ID, maxSeq+1, hash)
commitments := suite.chainB.App.IBCKeeper.ChannelKeeper.GetAllPacketCommitmentsAtChannel(ctx, testPort1, testChannel1) commitments := suite.chainA.App.IBCKeeper.ChannelKeeper.GetAllPacketCommitmentsAtChannel(ctxA, channelA.PortID, channelA.ID)
suite.Equal(len(expectedSeqs), len(commitments)) suite.Equal(len(expectedSeqs), len(commitments))
// ensure above for loops occurred
suite.NotEqual(0, len(commitments)) suite.NotEqual(0, len(commitments))
// verify that all the packet commitments were stored // verify that all the packet commitments were stored
for _, packet := range commitments { for _, packet := range commitments {
suite.True(expectedSeqs[packet.Sequence]) suite.True(expectedSeqs[packet.Sequence])
suite.Equal(testPort1, packet.PortID) suite.Equal(channelA.PortID, packet.PortID)
suite.Equal(testChannel1, packet.ChannelID) suite.Equal(channelA.ID, packet.ChannelID)
suite.Equal(hash, packet.Hash)
// prevent duplicates from passing checks // prevent duplicates from passing checks
expectedSeqs[packet.Sequence] = false expectedSeqs[packet.Sequence] = false
} }
} }
// TestSetPacketAcknowledgement verifies that packet acknowledgements are correctly
// set in the keeper.
func (suite *KeeperTestSuite) TestSetPacketAcknowledgement() { func (suite *KeeperTestSuite) TestSetPacketAcknowledgement() {
ctx := suite.chainB.GetContext() _, _, _, _, channelA, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
ctxA := suite.chainA.GetContext()
seq := uint64(10) seq := uint64(10)
storedAckHash, found := suite.chainB.App.IBCKeeper.ChannelKeeper.GetPacketAcknowledgement(ctx, testPort1, testChannel1, seq) storedAckHash, found := suite.chainA.App.IBCKeeper.ChannelKeeper.GetPacketAcknowledgement(ctxA, channelA.PortID, channelA.ID, seq)
suite.False(found) suite.False(found)
suite.Nil(storedAckHash) suite.Nil(storedAckHash)
ackHash := []byte("ackhash") ackHash := []byte("ackhash")
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketAcknowledgement(ctx, testPort1, testChannel1, seq, ackHash) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketAcknowledgement(ctxA, channelA.PortID, channelA.ID, seq, ackHash)
storedAckHash, found = suite.chainB.App.IBCKeeper.ChannelKeeper.GetPacketAcknowledgement(ctx, testPort1, testChannel1, seq) storedAckHash, found = suite.chainA.App.IBCKeeper.ChannelKeeper.GetPacketAcknowledgement(ctxA, channelA.PortID, channelA.ID, seq)
suite.True(found) suite.True(found)
suite.Equal(ackHash, storedAckHash) suite.Equal(ackHash, storedAckHash)
} }
func TestKeeperTestSuite(t *testing.T) {
suite.Run(t, new(KeeperTestSuite))
}
func commitNBlocks(chain *TestChain, n int) {
for i := 0; i < n; i++ {
chain.App.Commit()
chain.App.BeginBlock(abci.RequestBeginBlock{Header: abci.Header{Height: chain.App.LastBlockHeight() + 1}})
}
}
// commit current block and start the next block with the provided time
func commitBlockWithNewTimestamp(chain *TestChain, timestamp int64) {
chain.App.Commit()
chain.App.BeginBlock(abci.RequestBeginBlock{Header: abci.Header{Height: chain.App.LastBlockHeight() + 1, Time: time.Unix(timestamp, 0)}})
}
// nolint: unused
func queryProof(chain *TestChain, key []byte) ([]byte, uint64) {
res := chain.App.Query(abci.RequestQuery{
Path: fmt.Sprintf("store/%s/key", host.StoreKey),
Height: chain.App.LastBlockHeight(),
Data: key,
Prove: true,
})
merkleProof := commitmenttypes.MerkleProof{
Proof: res.Proof,
}
proof, _ := chain.App.AppCodec().MarshalBinaryBare(&merkleProof)
return proof, uint64(res.Height)
}
type TestChain struct {
ClientID string
App *simapp.SimApp
Header ibctmtypes.Header
Vals *tmtypes.ValidatorSet
Signers []tmtypes.PrivValidator
}
func NewTestChain(clientID string) *TestChain {
privVal := tmtypes.NewMockPV()
pubKey, err := privVal.GetPubKey()
if err != nil {
panic(err)
}
validator := tmtypes.NewValidator(pubKey, 1)
valSet := tmtypes.NewValidatorSet([]*tmtypes.Validator{validator})
signers := []tmtypes.PrivValidator{privVal}
now := time.Date(2020, 1, 2, 0, 0, 0, 0, time.UTC)
header := ibctmtypes.CreateTestHeader(clientID, 1, now, valSet, signers)
return &TestChain{
ClientID: clientID,
App: simapp.Setup(false),
Header: header,
Vals: valSet,
Signers: signers,
}
}
// Creates simple context for testing purposes
func (chain *TestChain) GetContext() sdk.Context {
return chain.App.BaseApp.NewContext(false, abci.Header{ChainID: chain.Header.SignedHeader.Header.ChainID, Height: int64(chain.Header.GetHeight())})
}
// createClient will create a client for clientChain on targetChain
func (chain *TestChain) CreateClient(client *TestChain) error {
client.Header = nextHeader(client)
// Commit and create a new block on appTarget to get a fresh CommitID
client.App.Commit()
commitID := client.App.LastCommitID()
client.App.BeginBlock(abci.RequestBeginBlock{Header: abci.Header{Height: int64(client.Header.GetHeight()), Time: client.Header.Time}})
// Set HistoricalInfo on client chain after Commit
ctxClient := client.GetContext()
validator := stakingtypes.NewValidator(
sdk.ValAddress(client.Vals.Validators[0].Address), client.Vals.Validators[0].PubKey, stakingtypes.Description{},
)
validator.Status = sdk.Bonded
validator.Tokens = sdk.NewInt(1000000) // get one voting power
validators := []stakingtypes.Validator{validator}
histInfo := stakingtypes.HistoricalInfo{
Header: abci.Header{
AppHash: commitID.Hash,
},
Valset: validators,
}
client.App.StakingKeeper.SetHistoricalInfo(ctxClient, int64(client.Header.GetHeight()), histInfo)
// Create target ctx
ctxTarget := chain.GetContext()
// create client
clientState, err := ibctmtypes.Initialize(client.ClientID, lite.DefaultTrustLevel, trustingPeriod, ubdPeriod, maxClockDrift, client.Header, commitmenttypes.GetSDKSpecs())
if err != nil {
return err
}
_, err = chain.App.IBCKeeper.ClientKeeper.CreateClient(ctxTarget, clientState, client.Header.ConsensusState())
if err != nil {
return err
}
return nil
// _, _, err := simapp.SignCheckDeliver(
// suite.T(),
// suite.cdc,
// suite.app.BaseApp,
// ctx.BlockHeader(),
// []sdk.Msg{clienttypes.NewMsgCreateClient(clientID, clientexported.ClientTypeTendermint, consState, accountAddress)},
// []uint64{baseAccount.GetAccountNumber()},
// []uint64{baseAccount.GetSequence()},
// true, true, accountPrivKey,
// )
}
func (chain *TestChain) updateClient(client *TestChain) {
// Create target ctx
ctxTarget := chain.GetContext()
// if clientState does not already exist, return without updating
_, found := chain.App.IBCKeeper.ClientKeeper.GetClientState(
ctxTarget, client.ClientID,
)
if !found {
return
}
// always commit when updateClient and begin a new block
client.App.Commit()
commitID := client.App.LastCommitID()
client.Header = nextHeader(client)
client.App.BeginBlock(abci.RequestBeginBlock{Header: abci.Header{Height: int64(client.Header.GetHeight()), Time: client.Header.Time}})
// Set HistoricalInfo on client chain after Commit
ctxClient := client.GetContext()
validator := stakingtypes.NewValidator(
sdk.ValAddress(client.Vals.Validators[0].Address), client.Vals.Validators[0].PubKey, stakingtypes.Description{},
)
validator.Status = sdk.Bonded
validator.Tokens = sdk.NewInt(1000000)
validators := []stakingtypes.Validator{validator}
histInfo := stakingtypes.HistoricalInfo{
Header: abci.Header{
AppHash: commitID.Hash,
},
Valset: validators,
}
client.App.StakingKeeper.SetHistoricalInfo(ctxClient, int64(client.Header.GetHeight()), histInfo)
consensusState := ibctmtypes.ConsensusState{
Height: client.Header.GetHeight(),
Timestamp: client.Header.Time,
Root: commitmenttypes.NewMerkleRoot(commitID.Hash),
ValidatorSet: client.Vals,
}
chain.App.IBCKeeper.ClientKeeper.SetClientConsensusState(
ctxTarget, client.ClientID, client.Header.GetHeight(), consensusState,
)
chain.App.IBCKeeper.ClientKeeper.SetClientState(
ctxTarget, ibctmtypes.NewClientState(client.ClientID, lite.DefaultTrustLevel, trustingPeriod, ubdPeriod, maxClockDrift, client.Header, commitmenttypes.GetSDKSpecs()),
)
// _, _, err := simapp.SignCheckDeliver(
// suite.T(),
// suite.cdc,
// suite.app.BaseApp,
// ctx.BlockHeader(),
// []sdk.Msg{clienttypes.NewMsgUpdateClient(clientID, suite.header, accountAddress)},
// []uint64{baseAccount.GetAccountNumber()},
// []uint64{baseAccount.GetSequence()},
// true, true, accountPrivKey,
// )
// suite.Require().NoError(err)
}
func (chain *TestChain) createConnection(
connID, counterpartyConnID, clientID, counterpartyClientID string,
state connectiontypes.State,
) connectiontypes.ConnectionEnd {
counterparty := connectiontypes.NewCounterparty(counterpartyClientID, counterpartyConnID, commitmenttypes.NewMerklePrefix(chain.App.IBCKeeper.ConnectionKeeper.GetCommitmentPrefix().Bytes()))
connection := connectiontypes.ConnectionEnd{
State: state,
ClientID: clientID,
Counterparty: counterparty,
Versions: connectiontypes.GetCompatibleVersions(),
}
ctx := chain.GetContext()
chain.App.IBCKeeper.ConnectionKeeper.SetConnection(ctx, connID, connection)
return connection
}
func (chain *TestChain) createChannel(
portID, channelID, counterpartyPortID, counterpartyChannelID string,
state types.State, order types.Order, connectionID string,
) types.Channel {
counterparty := types.NewCounterparty(counterpartyPortID, counterpartyChannelID)
// sets channel with given state
channel := types.NewChannel(state, order, counterparty,
[]string{connectionID}, testChannelVersion,
)
ctx := chain.GetContext()
chain.App.IBCKeeper.ChannelKeeper.SetChannel(ctx, portID, channelID, channel)
return channel
}
// storePacketCommitment is a helper function that sets a packet commitment in the Channel Keeper.
func (chain *TestChain) storePacketCommitment(ctx sdk.Context, portID, channelID string, sequence uint64) {
chain.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(ctx, portID, channelID, sequence, testPacketCommitment)
}
// storeAcknowledgement is a helper function that sets a packet commitment in the Channel Keeper.
func (chain *TestChain) storeAcknowledgement(ctx sdk.Context, portID, channelID string, sequence uint64) {
chain.App.IBCKeeper.ChannelKeeper.SetPacketAcknowledgement(ctx, portID, channelID, sequence, testAcknowledgement)
}
func nextHeader(chain *TestChain) ibctmtypes.Header {
return ibctmtypes.CreateTestHeader(chain.Header.SignedHeader.Header.ChainID, int64(chain.Header.GetHeight())+1,
chain.Header.Time.Add(time.Minute), chain.Vals, chain.Signers)
}
// Mocked types
type mockSuccessPacket struct{}
// GetBytes returns the serialised packet data
func (mp mockSuccessPacket) GetBytes() []byte { return []byte("THIS IS A SUCCESS PACKET") }
type mockFailPacket struct{}
// GetBytes returns the serialised packet data (without timeout)
func (mp mockFailPacket) GetBytes() []byte { return []byte("THIS IS A FAILURE PACKET") }

File diff suppressed because it is too large Load Diff

View File

@ -4,7 +4,8 @@ import (
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
"github.com/cosmos/cosmos-sdk/codec" "github.com/cosmos/cosmos-sdk/codec"
connection "github.com/cosmos/cosmos-sdk/x/ibc/03-connection" clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
connectiontypes "github.com/cosmos/cosmos-sdk/x/ibc/03-connection/types"
"github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types" "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types"
) )
@ -18,7 +19,7 @@ func (suite *KeeperTestSuite) TestQueryChannels() {
) )
params := types.NewQueryAllChannelsParams(1, 100) params := types.NewQueryAllChannelsParams(1, 100)
data, err := suite.cdc.MarshalJSON(params) data, err := suite.chainA.App.AppCodec().MarshalJSON(params)
suite.Require().NoError(err) suite.Require().NoError(err)
query := abci.RequestQuery{ query := abci.RequestQuery{
@ -33,88 +34,83 @@ func (suite *KeeperTestSuite) TestQueryChannels() {
{ {
"success with different connection channels", "success with different connection channels",
func() { func() {
suite.SetupTest()
channels := make([]types.IdentifiedChannel, 0, 2) channels := make([]types.IdentifiedChannel, 0, 2)
// create channels on different connections // create first connection/channel
suite.chainA.createConnection( clientA, clientB, _, _, channelA0, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
testConnectionIDA, testConnectionIDB,
testClientIDA, testClientIDB,
connection.OPEN,
)
channels = append(channels, channels = append(channels,
types.NewIdentifiedChannel(testPort1, testChannel1, types.NewIdentifiedChannel(
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channelA0.PortID,
types.OPEN, types.ORDERED, testConnectionIDA, channelA0.ID,
), suite.chainA.GetChannel(channelA0),
), ),
) )
suite.chainA.createConnection( // create second connection
testConnectionIDB, testConnectionIDA, connA1, connB1 := suite.coordinator.CreateConnection(suite.chainA, suite.chainB, clientA, clientB)
testClientIDB, testClientIDA,
connection.OPEN, // create second channel on second connection
) channelA1, _ := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA1, connB1, types.ORDERED)
channels = append(channels, channels = append(channels,
types.NewIdentifiedChannel(testPort2, testChannel2, types.NewIdentifiedChannel(
suite.chainA.createChannel(testPort2, testChannel2, testPort1, testChannel1, channelA1.PortID,
types.OPEN, types.ORDERED, testConnectionIDB, channelA1.ID,
), suite.chainA.GetChannel(channelA1),
), ),
) )
// set expected result // set expected result
expRes, err = codec.MarshalJSONIndent(suite.cdc, channels) expRes, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), channels)
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
{ {
"success with singular connection channels", "success with singular connection channels",
func() { func() {
suite.SetupTest()
channels := make([]types.IdentifiedChannel, 0, 2) channels := make([]types.IdentifiedChannel, 0, 2)
// create channels on singular connections // create first connection/channel
suite.chainA.createConnection( _, _, connA, connB, channelA0, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
testConnectionIDA, testConnectionIDB,
testClientIDA, testClientIDB,
connection.OPEN,
)
channels = append(channels, channels = append(channels,
types.NewIdentifiedChannel(testPort1, testChannel1, types.NewIdentifiedChannel(
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channelA0.PortID,
types.OPEN, types.ORDERED, testConnectionIDA, channelA0.ID,
), suite.chainA.GetChannel(channelA0),
), ),
) )
// create second channel on the same connection
channelA1, _ := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.UNORDERED)
channels = append(channels, channels = append(channels,
types.NewIdentifiedChannel(testPort2, testChannel2, types.NewIdentifiedChannel(
suite.chainA.createChannel(testPort2, testChannel2, testPort1, testChannel1, channelA1.PortID,
types.OPEN, types.UNORDERED, testConnectionIDA, channelA1.ID,
), suite.chainA.GetChannel(channelA1),
), ),
) )
// set expected result // set expected result
expRes, err = codec.MarshalJSONIndent(suite.cdc, channels) expRes, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), channels)
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
{ {
"success no channels", "success no channels",
func() { func() {
suite.SetupTest() expRes, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), []types.IdentifiedChannel{})
expRes, err = codec.MarshalJSONIndent(suite.cdc, []types.IdentifiedChannel{})
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
} }
for i, tc := range testCases { for i, tc := range testCases {
suite.SetupTest() // reset
tc.setup() tc.setup()
bz, err := suite.querier(suite.chainA.GetContext(), path, query) bz, err := suite.chainA.Querier(suite.chainA.GetContext(), path, query)
suite.Require().NoError(err, "test case %d failed: %s", i, tc.name) suite.Require().NoError(err, "test case %d failed: %s", i, tc.name)
suite.Require().Equal(expRes, bz, "test case %d failed: %s", i, tc.name) suite.Require().Equal(expRes, bz, "test case %d failed: %s", i, tc.name)
@ -127,18 +123,10 @@ func (suite *KeeperTestSuite) TestQueryConnectionChannels() {
var ( var (
expRes []byte expRes []byte
params types.QueryConnectionChannelsParams
err error err error
) )
params := types.NewQueryConnectionChannelsParams(testConnectionIDA, 1, 100)
data, err := suite.cdc.MarshalJSON(params)
suite.Require().NoError(err)
query := abci.RequestQuery{
Path: "",
Data: data,
}
testCases := []struct { testCases := []struct {
name string name string
setup func() setup func()
@ -146,101 +134,106 @@ func (suite *KeeperTestSuite) TestQueryConnectionChannels() {
{ {
"success with singular connection channels", "success with singular connection channels",
func() { func() {
suite.SetupTest()
channels := make([]types.IdentifiedChannel, 0, 2) channels := make([]types.IdentifiedChannel, 0, 2)
// create channels on singular connections // create first connection/channel
suite.chainA.createConnection( _, _, connA, connB, channelA0, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
testConnectionIDA, testConnectionIDB,
testClientIDA, testClientIDB,
connection.OPEN,
)
channels = append(channels, channels = append(channels,
types.NewIdentifiedChannel(testPort1, testChannel1, types.NewIdentifiedChannel(
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channelA0.PortID,
types.OPEN, types.ORDERED, testConnectionIDA, channelA0.ID,
), suite.chainA.GetChannel(channelA0),
), ),
) )
// create second channel on the same connection
channelA1, _ := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
channels = append(channels, channels = append(channels,
types.NewIdentifiedChannel(testPort2, testChannel2, types.NewIdentifiedChannel(
suite.chainA.createChannel(testPort2, testChannel2, testPort1, testChannel1, channelA1.PortID,
types.OPEN, types.UNORDERED, testConnectionIDA, channelA1.ID,
), suite.chainA.GetChannel(channelA1),
), ),
) )
params = types.NewQueryConnectionChannelsParams(connA.ID, 1, 100)
// set expected result // set expected result
expRes, err = codec.MarshalJSONIndent(suite.cdc, channels) expRes, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), channels)
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
{ {
"success multiple connection channels", "success multiple connection channels",
func() { func() {
suite.SetupTest()
channels := make([]types.IdentifiedChannel, 0, 1) channels := make([]types.IdentifiedChannel, 0, 1)
// create channels on different connections // create first connection/channel
suite.chainA.createConnection( clientA, clientB, connA, _, channelA0, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
testConnectionIDA, testConnectionIDB,
testClientIDA, testClientIDB,
connection.OPEN,
)
channels = append(channels, channels = append(channels,
types.NewIdentifiedChannel(testPort1, testChannel1, types.NewIdentifiedChannel(
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channelA0.PortID,
types.OPEN, types.ORDERED, testConnectionIDA, channelA0.ID,
), suite.chainA.GetChannel(channelA0),
), ),
) )
suite.chainA.createConnection( // create second connection
testConnectionIDB, testConnectionIDA, connA1, connB1 := suite.coordinator.CreateConnection(suite.chainA, suite.chainB, clientA, clientB)
testClientIDB, testClientIDA,
connection.OPEN, // create second channel on second connection
) suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA1, connB1, types.ORDERED)
suite.chainA.createChannel(
testPort2, testChannel2, testPort1, testChannel1, params = types.NewQueryConnectionChannelsParams(connA.ID, 1, 100)
types.OPEN, types.ORDERED, testConnectionIDB,
)
// set expected result // set expected result
expRes, err = codec.MarshalJSONIndent(suite.cdc, channels) expRes, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), channels)
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
{ {
"success no channels", "success no channels",
func() { func() {
suite.SetupTest() // create connection but no channels
expRes, err = codec.MarshalJSONIndent(suite.cdc, []types.IdentifiedChannel{}) _, _, connA, _ := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
params = types.NewQueryConnectionChannelsParams(connA.ID, 1, 100)
expRes, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), []types.IdentifiedChannel{})
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
} }
for i, tc := range testCases { for i, tc := range testCases {
suite.SetupTest() // reset
tc.setup() tc.setup()
bz, err := suite.querier(suite.chainA.GetContext(), path, query) data, err := suite.chainA.App.AppCodec().MarshalJSON(params)
suite.Require().NoError(err)
query := abci.RequestQuery{
Path: "",
Data: data,
}
bz, err := suite.chainA.Querier(suite.chainA.GetContext(), path, query)
suite.Require().NoError(err, "test case %d failed: %s", i, tc.name) suite.Require().NoError(err, "test case %d failed: %s", i, tc.name)
suite.Require().Equal(expRes, bz, "test case %d failed: %s", i, tc.name) suite.Require().Equal(expRes, bz, "test case %d failed: %s", i, tc.name)
} }
} }
// TestQuerierChannelClientState verifies correct querying of client state associated
// with a channel end.
func (suite *KeeperTestSuite) TestQuerierChannelClientState() { func (suite *KeeperTestSuite) TestQuerierChannelClientState() {
path := []string{types.SubModuleName, types.QueryChannelClientState} path := []string{types.SubModuleName, types.QueryChannelClientState}
params := types.NewQueryChannelClientStateParams(testPort1, testChannel1)
data, err := suite.cdc.MarshalJSON(params)
suite.Require().NoError(err)
query := abci.RequestQuery{ var (
Path: "", clientID string
Data: data, params types.QueryChannelClientStateParams
} )
testCases := []struct { testCases := []struct {
name string name string
@ -249,77 +242,81 @@ func (suite *KeeperTestSuite) TestQuerierChannelClientState() {
}{ }{
{ {
"channel not found", "channel not found",
func() {}, func() {
clientA, err := suite.coordinator.CreateClient(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.Require().NoError(err)
clientID = clientA
params = types.NewQueryChannelClientStateParams("doesnotexist", "doesnotexist")
},
false, false,
}, },
{ {
"connection for channel not found", "connection for channel not found",
func() { func() {
_ = suite.chainA.createChannel( // connection for channel is deleted from state
testPort1, testChannel1, testPort2, testChannel2, clientA, _, _, _, channelA, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
types.OPEN, types.ORDERED, testConnectionIDA,
) channel := suite.chainA.GetChannel(channelA)
channel.ConnectionHops[0] = "doesnotexist"
// set connection hops to wrong connection ID
suite.chainA.App.IBCKeeper.ChannelKeeper.SetChannel(suite.chainA.GetContext(), channelA.PortID, channelA.ID, channel)
clientID = clientA
params = types.NewQueryChannelClientStateParams(channelA.PortID, channelA.ID)
}, },
false, false,
}, },
{ {
"client state for channel's connection not found", "client state for channel's connection not found",
func() { func() {
_ = suite.chainA.createConnection( clientA, _, connA, _, channelA, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
testConnectionIDA, testConnectionIDB,
testClientIDA, testClientIDB, // setting connection to empty results in wrong clientID used
connection.OPEN, suite.chainA.App.IBCKeeper.ConnectionKeeper.SetConnection(suite.chainA.GetContext(), connA.ID, connectiontypes.ConnectionEnd{})
)
_ = suite.chainA.createChannel( clientID = clientA
testPort1, testChannel1, testPort2, testChannel2, params = types.NewQueryChannelClientStateParams(channelA.PortID, channelA.ID)
types.OPEN, types.ORDERED, testConnectionIDA,
)
}, },
false, false,
}, },
{ {
"success", "success",
func() { func() {
err = suite.chainA.CreateClient(suite.chainB) clientA, _, _, _, channelA, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
suite.Require().NoError(err)
err = suite.chainB.CreateClient(suite.chainA) clientID = clientA
suite.Require().NoError(err) params = types.NewQueryChannelClientStateParams(channelA.PortID, channelA.ID)
suite.chainA.createConnection(
testConnectionIDB, testConnectionIDA, testClientIDB, testClientIDA,
connection.OPEN,
)
suite.chainB.createConnection(
testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB,
connection.OPEN,
)
suite.chainA.createChannel(
testPort1, testChannel1, testPort2, testChannel2, types.INIT,
types.ORDERED, testConnectionIDB,
)
suite.chainB.createChannel(
testPort2, testChannel2, testPort1, testChannel1, types.TRYOPEN,
types.ORDERED, testConnectionIDA,
)
}, },
true, true,
}, },
} }
for i, tc := range testCases { for i, tc := range testCases {
suite.SetupTest() // reset
tc.setup() tc.setup()
clientState, found := suite.chainA.App.IBCKeeper.ClientKeeper.GetClientState(suite.chainA.GetContext(), testClientIDB) data, err := suite.chainA.App.AppCodec().MarshalJSON(params)
bz, err := suite.querier(suite.chainA.GetContext(), path, query) suite.Require().NoError(err)
query := abci.RequestQuery{
Path: "",
Data: data,
}
clientState, found := suite.chainA.App.IBCKeeper.ClientKeeper.GetClientState(suite.chainA.GetContext(), clientID)
bz, err := suite.chainA.Querier(suite.chainA.GetContext(), path, query)
if tc.expPass { if tc.expPass {
// set expected result // set expected result
expRes, merr := codec.MarshalJSONIndent(suite.cdc, clientState) expRes, merr := codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), clientState)
suite.Require().NoError(merr) suite.Require().NoError(merr)
suite.Require().True(found) suite.Require().True(found, "test case %d failed: %s", i, tc.name)
suite.Require().NoError(err, "test case %d failed: %s", i, tc.name) suite.Require().NoError(err, "test case %d failed: %s", i, tc.name)
suite.Require().Equal(string(expRes), string(bz), "test case %d failed: %s", i, tc.name) suite.Require().Equal(string(expRes), string(bz), "test case %d failed: %s", i, tc.name)
} else { } else {
suite.Require().False(found)
suite.Require().Error(err, "test case %d passed: %s", i, tc.name) suite.Require().Error(err, "test case %d passed: %s", i, tc.name)
suite.Require().Nil(bz, "test case %d passed: %s", i, tc.name) suite.Require().Nil(bz, "test case %d passed: %s", i, tc.name)
} }
@ -332,17 +329,10 @@ func (suite *KeeperTestSuite) TestQueryPacketCommitments() {
var ( var (
expRes []byte expRes []byte
params types.QueryPacketCommitmentsParams
err error
) )
params := types.NewQueryPacketCommitmentsParams(testPort1, testChannel1, 1, 100)
data, err := suite.cdc.MarshalJSON(params)
suite.Require().NoError(err)
query := abci.RequestQuery{
Path: "",
Data: data,
}
testCases := []struct { testCases := []struct {
name string name string
setup func() setup func()
@ -350,58 +340,77 @@ func (suite *KeeperTestSuite) TestQueryPacketCommitments() {
{ {
"success", "success",
func() { func() {
suite.SetupTest() _, _, _, _, channelA, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
ctx := suite.chainA.GetContext()
seq := uint64(1) seq := uint64(1)
commitments := []uint64{} commitments := []uint64{}
// create several commitments on the same channel and port // create several commitments on the same channel and port
for i := seq; i < 10; i++ { for i := seq; i < 10; i++ {
suite.chainA.storePacketCommitment(ctx, testPort1, testChannel1, i) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainA.GetContext(), channelA.PortID, channelA.ID, i, []byte("ack"))
commitments = append(commitments, i) commitments = append(commitments, i)
} }
expRes, err = codec.MarshalJSONIndent(suite.cdc, commitments) params = types.NewQueryPacketCommitmentsParams(channelA.PortID, channelA.ID, 1, 100)
expRes, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), commitments)
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
{ {
"success with multiple channels", "success with multiple channels",
func() { func() {
suite.SetupTest() _, _, connA, connB, channelA0, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
ctx := suite.chainA.GetContext()
seq := uint64(1) seq := uint64(1)
commitments := []uint64{} commitments := []uint64{}
// create several commitments on the same channel and port // create several commitments on the same channel and port
for i := seq; i < 10; i++ { for i := seq; i < 10; i++ {
suite.chainA.storePacketCommitment(ctx, testPort1, testChannel1, i) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainA.GetContext(), channelA0.PortID, channelA0.ID, i, []byte("ack"))
commitments = append(commitments, i) commitments = append(commitments, i)
} }
// create second channel
channelA1, _ := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
// create several commitments on a different channel and port // create several commitments on a different channel and port
for i := seq; i < 10; i++ { for i := seq; i < 10; i++ {
suite.chainA.storePacketCommitment(ctx, testPort2, testChannel2, i) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainA.GetContext(), channelA1.PortID, channelA1.ID, i, []byte("ack"))
} }
expRes, err = codec.MarshalJSONIndent(suite.cdc, commitments) params = types.NewQueryPacketCommitmentsParams(channelA0.PortID, channelA1.ID, 1, 100)
expRes, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), commitments)
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
{ {
"success no packet commitments", "success no packet commitments",
func() { func() {
suite.SetupTest() _, _, _, _, channelA, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
expRes, err = codec.MarshalJSONIndent(suite.cdc, []uint64{})
params = types.NewQueryPacketCommitmentsParams(channelA.PortID, channelA.ID, 1, 100)
expRes, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), []uint64{})
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
} }
for i, tc := range testCases { for i, tc := range testCases {
suite.SetupTest() // reset
tc.setup() tc.setup()
bz, err := suite.querier(suite.chainA.GetContext(), path, query) data, err := suite.chainA.App.AppCodec().MarshalJSON(params)
suite.Require().NoError(err)
query := abci.RequestQuery{
Path: "",
Data: data,
}
bz, err := suite.chainA.Querier(suite.chainA.GetContext(), path, query)
suite.Require().NoError(err, "test case %d failed: %s", i, tc.name) suite.Require().NoError(err, "test case %d failed: %s", i, tc.name)
suite.Require().Equal(expRes, bz, "test case %d failed: %s", i, tc.name) suite.Require().Equal(expRes, bz, "test case %d failed: %s", i, tc.name)
@ -419,17 +428,10 @@ func (suite *KeeperTestSuite) TestQueryUnrelayedAcks() {
var ( var (
expResAck []byte expResAck []byte
expResSend []byte expResSend []byte
params types.QueryUnrelayedPacketsParams
err error
) )
params := types.NewQueryUnrelayedPacketsParams(testPort1, testChannel1, sequences, 1, 100)
data, err := suite.cdc.MarshalJSON(params)
suite.Require().NoError(err)
query := abci.RequestQuery{
Path: "",
Data: data,
}
testCases := []struct { testCases := []struct {
name string name string
setup func() setup func()
@ -437,25 +439,27 @@ func (suite *KeeperTestSuite) TestQueryUnrelayedAcks() {
{ {
"success", "success",
func() { func() {
suite.SetupTest() _, _, _, _, channelA, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
ctx := suite.chainA.GetContext()
unrelayedAcks := []uint64{} unrelayedAcks := []uint64{}
unrelayedSends := []uint64{} unrelayedSends := []uint64{}
// create acknowledgements for first 3 sequences // create acknowledgements for first 3 sequences
for _, seq := range sequences { for _, seq := range sequences {
if seq < 4 { if seq < 4 {
suite.chainA.storeAcknowledgement(ctx, testPort1, testChannel1, seq) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketAcknowledgement(suite.chainA.GetContext(), channelA.PortID, channelA.ID, seq, []byte("ack"))
unrelayedAcks = append(unrelayedAcks, seq) unrelayedAcks = append(unrelayedAcks, seq)
} else { } else {
unrelayedSends = append(unrelayedSends, seq) unrelayedSends = append(unrelayedSends, seq)
} }
} }
expResAck, err = codec.MarshalJSONIndent(suite.cdc, unrelayedAcks) params = types.NewQueryUnrelayedPacketsParams(channelA.PortID, channelA.ID, sequences, 1, 100)
expResAck, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), unrelayedAcks)
suite.Require().NoError(err) suite.Require().NoError(err)
expResSend, err = codec.MarshalJSONIndent(suite.cdc, unrelayedSends) expResSend, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), unrelayedSends)
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
@ -463,64 +467,80 @@ func (suite *KeeperTestSuite) TestQueryUnrelayedAcks() {
{ {
"success with multiple channels", "success with multiple channels",
func() { func() {
suite.SetupTest() _, _, connA, connB, channelA0, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
ctx := suite.chainA.GetContext() ctxA := suite.chainA.GetContext()
unrelayedAcks := []uint64{} unrelayedAcks := []uint64{}
unrelayedSends := []uint64{} unrelayedSends := []uint64{}
// create acknowledgements for first 3 sequences // create acknowledgements for first 3 sequences
for _, seq := range sequences { for _, seq := range sequences {
if seq < 4 { if seq < 4 {
suite.chainA.storeAcknowledgement(ctx, testPort1, testChannel1, seq) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketAcknowledgement(ctxA, channelA0.PortID, channelA0.ID, seq, []byte("ack"))
unrelayedAcks = append(unrelayedAcks, seq) unrelayedAcks = append(unrelayedAcks, seq)
} else { } else {
unrelayedSends = append(unrelayedSends, seq) unrelayedSends = append(unrelayedSends, seq)
} }
} }
// create second channel
channelA1, _ := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.UNORDERED)
// create acknowledgements for other sequences on different channel/port // create acknowledgements for other sequences on different channel/port
for _, seq := range sequences { for _, seq := range sequences {
if seq >= 4 { if seq >= 4 {
suite.chainA.storeAcknowledgement(ctx, testPort2, testChannel2, seq) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketAcknowledgement(ctxA, channelA1.PortID, channelA1.ID, seq, []byte("ack"))
} }
} }
expResAck, err = codec.MarshalJSONIndent(suite.cdc, unrelayedAcks) params = types.NewQueryUnrelayedPacketsParams(channelA0.PortID, channelA0.ID, sequences, 1, 100)
expResAck, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), unrelayedAcks)
suite.Require().NoError(err) suite.Require().NoError(err)
expResSend, err = codec.MarshalJSONIndent(suite.cdc, unrelayedSends) expResSend, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), unrelayedSends)
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
{ {
"success no unrelayed acks", "success no unrelayed acks",
func() { func() {
suite.SetupTest() _, _, _, _, channelA, _ := suite.coordinator.Setup(suite.chainA, suite.chainB)
ctx := suite.chainA.GetContext()
// create acknowledgements for all sequences // create acknowledgements for all sequences
for _, seq := range sequences { for _, seq := range sequences {
suite.chainA.storeAcknowledgement(ctx, testPort1, testChannel1, seq) suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketAcknowledgement(suite.chainA.GetContext(), channelA.PortID, channelA.ID, seq, []byte("ack"))
} }
expResSend, err = codec.MarshalJSONIndent(suite.cdc, []uint64{}) params = types.NewQueryUnrelayedPacketsParams(channelA.PortID, channelA.ID, sequences, 1, 100)
expResSend, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), []uint64{})
suite.Require().NoError(err) suite.Require().NoError(err)
expResAck, err = codec.MarshalJSONIndent(suite.cdc, sequences) expResAck, err = codec.MarshalJSONIndent(suite.chainA.App.AppCodec(), sequences)
suite.Require().NoError(err) suite.Require().NoError(err)
}, },
}, },
} }
for i, tc := range testCases { for i, tc := range testCases {
suite.SetupTest() // reset
tc.setup() tc.setup()
bz, err := suite.querier(suite.chainA.GetContext(), pathAck, query) data, err := suite.chainA.App.AppCodec().MarshalJSON(params)
suite.Require().NoError(err)
query := abci.RequestQuery{
Path: "",
Data: data,
}
bz, err := suite.chainA.Querier(suite.chainA.GetContext(), pathAck, query)
suite.Require().NoError(err, "test case %d failed: %s", i, tc.name) suite.Require().NoError(err, "test case %d failed: %s", i, tc.name)
suite.Require().Equal(expResAck, bz, "test case %d failed: %s", i, tc.name) suite.Require().Equal(expResAck, bz, "test case %d failed: %s", i, tc.name)
bz, err = suite.querier(suite.chainA.GetContext(), pathSend, query) bz, err = suite.chainA.Querier(suite.chainA.GetContext(), pathSend, query)
suite.Require().NoError(err, "test case %d failed: %s", i, tc.name) suite.Require().NoError(err, "test case %d failed: %s", i, tc.name)
suite.Require().Equal(expResSend, bz, "test case %d failed: %s", i, tc.name) suite.Require().Equal(expResSend, bz, "test case %d failed: %s", i, tc.name)

View File

@ -4,111 +4,154 @@ import (
"fmt" "fmt"
capabilitytypes "github.com/cosmos/cosmos-sdk/x/capability/types" capabilitytypes "github.com/cosmos/cosmos-sdk/x/capability/types"
connection "github.com/cosmos/cosmos-sdk/x/ibc/03-connection" clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
"github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types" "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types"
host "github.com/cosmos/cosmos-sdk/x/ibc/24-host" host "github.com/cosmos/cosmos-sdk/x/ibc/24-host"
ibctesting "github.com/cosmos/cosmos-sdk/x/ibc/testing"
) )
// TestTimeoutPacket test the TimeoutPacket call on chainA by ensuring the timeout has passed
// on chainB, but that no ack has been written yet. Test cases expected to reach proof
// verification must specify which proof to use using the ordered bool.
func (suite *KeeperTestSuite) TestTimeoutPacket() { func (suite *KeeperTestSuite) TestTimeoutPacket() {
counterparty := types.NewCounterparty(testPort2, testChannel2)
packetKey := host.KeyPacketAcknowledgement(testPort2, testChannel2, 2)
var ( var (
packet types.Packet packet types.Packet
nextSeqRecv uint64 nextSeqRecv uint64
ordered bool
) )
testCases := []testCase{ testCases := []testCase{
{"success", func() { {"success: ORDERED", func() {
nextSeqRecv = 1 ordered = true
packet = types.NewPacket(newMockTimeoutPacket().GetBytes(), 2, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), 1, disabledTimeoutTimestamp)
suite.chainB.CreateClient(suite.chainA) clientA, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.chainA.CreateClient(suite.chainB) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
suite.chainA.createConnection(testConnectionIDA, testConnectionIDB, testClientIDB, testClientIDA, connection.OPEN) suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.UNORDERED, testConnectionIDA) // need to update chainA's client representing chainB to prove missing ack
suite.chainA.createChannel(testPort2, testChannel2, testPort1, testChannel1, types.OPEN, types.UNORDERED, testConnectionIDB) suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainB.GetContext(), testPort1, testChannel1, 2, types.CommitPacket(packet))
}, true}, }, true},
{"channel not found", func() {}, false}, {"success: UNORDERED", func() {
ordered = false
clientA, clientB, _, _, channelA, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), disabledTimeoutTimestamp)
suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
// need to update chainA's client representing chainB to prove missing ack
suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
}, true},
{"channel not found", func() {
// use wrong channel naming
_, _, _, _, _, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
packet = types.NewPacket(validPacketData, 1, ibctesting.InvalidID, ibctesting.InvalidID, channelB.PortID, channelB.ID, timeoutHeight, disabledTimeoutTimestamp)
}, false},
{"channel not open", func() { {"channel not open", func() {
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 1, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) _, _, _, _, channelA, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.CLOSED, types.ORDERED, testConnectionIDA) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, timeoutHeight, disabledTimeoutTimestamp)
err := suite.coordinator.SetChannelClosed(suite.chainA, suite.chainB, channelA)
suite.Require().NoError(err)
}, false}, }, false},
{"packet source port ≠ channel counterparty port", func() { {"packet destination port ≠ channel counterparty port", func() {
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 1, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) _, _, _, _, channelA, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
suite.chainB.createChannel(testPort1, testChannel1, testPort3, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) // use wrong port for dest
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, ibctesting.InvalidID, channelB.ID, timeoutHeight, disabledTimeoutTimestamp)
}, false}, }, false},
{"packet source channel ID ≠ channel counterparty channel ID", func() { {"packet destination channel ID ≠ channel counterparty channel ID", func() {
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 1, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) _, _, _, _, channelA, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel3, types.OPEN, types.ORDERED, testConnectionIDA) // use wrong channel for dest
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, ibctesting.InvalidID, timeoutHeight, disabledTimeoutTimestamp)
}, false}, }, false},
{"connection not found", func() { {"connection not found", func() {
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 1, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) channelA := ibctesting.TestChannel{PortID: portID, ID: channelIDA}
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) channelB := ibctesting.TestChannel{PortID: portID, ID: channelIDB}
// pass channel check
suite.chainA.App.IBCKeeper.ChannelKeeper.SetChannel(
suite.chainA.GetContext(),
channelA.PortID, channelA.ID,
types.NewChannel(types.OPEN, types.ORDERED, types.NewCounterparty(channelB.PortID, channelB.ID), []string{connIDA}, ibctesting.ChannelVersion),
)
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, timeoutHeight, disabledTimeoutTimestamp)
}, false}, }, false},
{"timeout", func() { {"timeout", func() {
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 10, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) clientA, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, timeoutHeight, disabledTimeoutTimestamp)
suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
}, false}, }, false},
{"packet already received ", func() { {"packet already received ", func() {
ordered = true
nextSeqRecv = 2 nextSeqRecv = 2
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 1, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp)
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN) clientA, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, timeoutHeight, uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
}, false}, }, false},
{"packet hasn't been sent", func() { {"packet hasn't been sent", func() {
nextSeqRecv = 1 clientA, _, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 2, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, timeoutHeight, uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
}, false}, }, false},
{"next seq receive verification failed", func() { {"next seq receive verification failed", func() {
nextSeqRecv = 1 // set ordered to false resulting in wrong proof provided
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 2, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) ordered = false
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) clientA, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainB.GetContext(), testPort1, testChannel1, 2, types.CommitPacket(packet)) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), disabledTimeoutTimestamp)
suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
}, false}, }, false},
{"packet ack verification failed", func() { {"packet ack verification failed", func() {
nextSeqRecv = 1 // set ordered to true resulting in wrong proof provided
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 2, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) ordered = true
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.UNORDERED, testConnectionIDA) clientA, clientB, _, _, channelA, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainB.GetContext(), testPort1, testChannel1, 2, types.CommitPacket(packet)) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), disabledTimeoutTimestamp)
suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
}, false}, }, false},
} }
for i, tc := range testCases { for i, tc := range testCases {
tc := tc tc := tc
suite.Run(fmt.Sprintf("Case %s, %d/%d tests", tc.msg, i, len(testCases)), func() { suite.Run(fmt.Sprintf("Case %s, %d/%d tests", tc.msg, i, len(testCases)), func() {
var (
proof []byte
proofHeight uint64
)
suite.SetupTest() // reset suite.SetupTest() // reset
nextSeqRecv = 1 // must be explicitly changed
tc.malleate() tc.malleate()
ctx := suite.chainB.GetContext() orderedPacketKey := host.KeyNextSequenceRecv(packet.GetDestPort(), packet.GetDestChannel())
unorderedPacketKey := host.KeyPacketAcknowledgement(packet.GetDestPort(), packet.GetDestChannel(), packet.GetSequence())
suite.chainB.updateClient(suite.chainA) if ordered {
suite.chainA.updateClient(suite.chainB) proof, proofHeight = suite.chainB.QueryProof(orderedPacketKey)
proof, proofHeight := queryProof(suite.chainA, packetKey) } else {
proof, proofHeight = suite.chainB.QueryProof(unorderedPacketKey)
}
_, err := suite.chainA.App.IBCKeeper.ChannelKeeper.TimeoutPacket(suite.chainA.GetContext(), packet, proof, proofHeight, nextSeqRecv)
if tc.expPass { if tc.expPass {
packetOut, err := suite.chainB.App.IBCKeeper.ChannelKeeper.TimeoutPacket(ctx, packet, proof, proofHeight+1, nextSeqRecv)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.Require().NotNil(packetOut)
} else { } else {
packetOut, err := suite.chainB.App.IBCKeeper.ChannelKeeper.TimeoutPacket(ctx, packet, proof, proofHeight, nextSeqRecv)
suite.Require().Error(err) suite.Require().Error(err)
suite.Require().Nil(packetOut)
} }
}) })
} }
} }
// TestTimeoutExectued verifies that packet commitments are deleted after // TestTimeoutExectued verifies that packet commitments are deleted on chainA after the
// capabilities are verified. // channel capabilities are verified.
func (suite *KeeperTestSuite) TestTimeoutExecuted() { func (suite *KeeperTestSuite) TestTimeoutExecuted() {
sequence := uint64(1)
var ( var (
packet types.Packet packet types.Packet
chanCap *capabilitytypes.Capability chanCap *capabilitytypes.Capability
@ -116,14 +159,24 @@ func (suite *KeeperTestSuite) TestTimeoutExecuted() {
testCases := []testCase{ testCases := []testCase{
{"success ORDERED", func() { {"success ORDERED", func() {
packet = types.NewPacket(newMockTimeoutPacket().GetBytes(), 1, testPort1, testChannel1, testPort2, testChannel2, timeoutHeight, disabledTimeoutTimestamp) _, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
suite.chainA.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainA.GetContext(), packet.GetSourcePort(), packet.GetSourceChannel(), sequence, types.CommitPacket(packet)) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
}, true}, }, true},
{"channel not found", func() {}, false}, {"channel not found", func() {
// use wrong channel naming
_, _, _, _, _, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
packet = types.NewPacket(validPacketData, 1, ibctesting.InvalidID, ibctesting.InvalidID, channelB.PortID, channelB.ID, timeoutHeight, disabledTimeoutTimestamp)
}, false},
{"incorrect capability", func() { {"incorrect capability", func() {
packet = types.NewPacket(newMockTimeoutPacket().GetBytes(), 1, testPort1, testChannel1, testPort2, testChannel2, timeoutHeight, disabledTimeoutTimestamp) _, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
chanCap = capabilitytypes.NewCapability(100) chanCap = capabilitytypes.NewCapability(100)
}, false}, }, false},
} }
@ -133,16 +186,10 @@ func (suite *KeeperTestSuite) TestTimeoutExecuted() {
suite.Run(fmt.Sprintf("Case %s, %d/%d tests", tc.msg, i, len(testCases)), func() { suite.Run(fmt.Sprintf("Case %s, %d/%d tests", tc.msg, i, len(testCases)), func() {
suite.SetupTest() // reset suite.SetupTest() // reset
var err error
chanCap, err = suite.chainA.App.ScopedIBCKeeper.NewCapability(
suite.chainA.GetContext(), host.ChannelCapabilityPath(testPort1, testChannel1),
)
suite.Require().NoError(err, "could not create capability")
tc.malleate() tc.malleate()
err = suite.chainA.App.IBCKeeper.ChannelKeeper.TimeoutExecuted(suite.chainA.GetContext(), chanCap, packet) err := suite.chainA.App.IBCKeeper.ChannelKeeper.TimeoutExecuted(suite.chainA.GetContext(), chanCap, packet)
pc := suite.chainA.App.IBCKeeper.ChannelKeeper.GetPacketCommitment(suite.chainA.GetContext(), packet.GetSourcePort(), packet.GetSourceChannel(), sequence) pc := suite.chainA.App.IBCKeeper.ChannelKeeper.GetPacketCommitment(suite.chainA.GetContext(), packet.GetSourcePort(), packet.GetSourceChannel(), packet.GetSequence())
if tc.expPass { if tc.expPass {
suite.NoError(err) suite.NoError(err)
@ -154,87 +201,131 @@ func (suite *KeeperTestSuite) TestTimeoutExecuted() {
} }
} }
// TestTimeoutOnClose tests the call TimeoutOnClose on chainA by closing the corresponding
// channel on chainB after the packet commitment has been created.
func (suite *KeeperTestSuite) TestTimeoutOnClose() { func (suite *KeeperTestSuite) TestTimeoutOnClose() {
channelKey := host.KeyChannel(testPort2, testChannel2)
unorderedPacketKey := host.KeyPacketAcknowledgement(testPort2, testChannel2, 2)
orderedPacketKey := host.KeyNextSequenceRecv(testPort2, testChannel2)
counterparty := types.NewCounterparty(testPort2, testChannel2)
var ( var (
packet types.Packet packet types.Packet
chanCap *capabilitytypes.Capability
nextSeqRecv uint64 nextSeqRecv uint64
ordered bool ordered bool
) )
testCases := []testCase{ testCases := []testCase{
{"success on ordered channel", func() { {"success: ORDERED", func() {
ordered = true ordered = true
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 2, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) clientA, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.chainB.CreateClient(suite.chainA) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
suite.chainA.CreateClient(suite.chainB) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN) suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.chainA.createConnection(testConnectionIDB, testConnectionIDA, testClientIDB, testClientIDA, connection.OPEN) suite.coordinator.SetChannelClosed(suite.chainB, suite.chainA, channelB)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) // need to update chainA's client representing chainB to prove missing ack
suite.chainA.createChannel(testPort2, testChannel2, testPort1, testChannel1, types.CLOSED, types.ORDERED, testConnectionIDB) // channel on chainA is closed suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainB.GetContext(), testPort1, testChannel1, 2, types.CommitPacket(packet))
suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceRecv(suite.chainA.GetContext(), testPort2, testChannel2, nextSeqRecv) chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
}, true}, }, true},
{"success on unordered channel", func() { {"success: UNORDERED", func() {
ordered = false ordered = false
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 2, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) clientA, clientB, _, _, channelA, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
suite.chainB.CreateClient(suite.chainA) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), disabledTimeoutTimestamp)
suite.chainA.CreateClient(suite.chainB) suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN) suite.coordinator.SetChannelClosed(suite.chainB, suite.chainA, channelB)
suite.chainA.createConnection(testConnectionIDB, testConnectionIDA, testClientIDB, testClientIDA, connection.OPEN) // need to update chainA's client representing chainB to prove missing ack
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.UNORDERED, testConnectionIDA) suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
suite.chainA.createChannel(testPort2, testChannel2, testPort1, testChannel1, types.CLOSED, types.UNORDERED, testConnectionIDB) // channel on chainA is closed
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainB.GetContext(), testPort1, testChannel1, 2, types.CommitPacket(packet)) chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
}, true}, }, true},
{"channel not found", func() {}, false}, {"channel not found", func() {
// use wrong channel naming
_, _, _, _, _, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
packet = types.NewPacket(validPacketData, 1, ibctesting.InvalidID, ibctesting.InvalidID, channelB.PortID, channelB.ID, timeoutHeight, disabledTimeoutTimestamp)
}, false},
{"packet dest port ≠ channel counterparty port", func() { {"packet dest port ≠ channel counterparty port", func() {
ordered = true _, _, _, _, channelA, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 1, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) // use wrong port for dest
suite.chainB.createChannel(testPort1, testChannel1, testPort3, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, ibctesting.InvalidID, channelB.ID, timeoutHeight, disabledTimeoutTimestamp)
chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
}, false}, }, false},
{"packet dest channel ID ≠ channel counterparty channel ID", func() { {"packet dest channel ID ≠ channel counterparty channel ID", func() {
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 1, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) _, _, _, _, channelA, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel3, types.OPEN, types.ORDERED, testConnectionIDA) // use wrong channel for dest
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, ibctesting.InvalidID, timeoutHeight, disabledTimeoutTimestamp)
chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
}, false}, }, false},
{"connection not found", func() { {"connection not found", func() {
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 1, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) channelA := ibctesting.TestChannel{PortID: portID, ID: channelIDA}
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) channelB := ibctesting.TestChannel{PortID: portID, ID: channelIDB}
// pass channel check
suite.chainA.App.IBCKeeper.ChannelKeeper.SetChannel(
suite.chainA.GetContext(),
channelA.PortID, channelA.ID,
types.NewChannel(types.OPEN, types.ORDERED, types.NewCounterparty(channelB.PortID, channelB.ID), []string{connIDA}, ibctesting.ChannelVersion),
)
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, timeoutHeight, disabledTimeoutTimestamp)
// create chancap
suite.chainA.CreateChannelCapability(channelA.PortID, channelA.ID)
chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
}, false}, }, false},
{"packet hasn't been sent", func() { {"packet hasn't been sent", func() {
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 2, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) _, _, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
}, false},
{"packet already received", func() {
nextSeqRecv = 2
ordered = true
clientA, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.coordinator.SetChannelClosed(suite.chainB, suite.chainA, channelB)
// need to update chainA's client representing chainB to prove missing ack
suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
}, false}, }, false},
{"channel verification failed", func() { {"channel verification failed", func() {
ordered = false ordered = true
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 2, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) _, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.chainB.CreateClient(suite.chainA) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.UNORDERED, testConnectionIDA) suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainB.GetContext(), testPort1, testChannel1, 2, types.CommitPacket(packet)) chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetNextSequenceRecv(suite.chainB.GetContext(), testPort1, testChannel1, nextSeqRecv)
}, false}, }, false},
{"next seq receive verification failed", func() { {"next seq receive verification failed", func() {
ordered = true // set ordered to false providing the wrong proof for ORDERED case
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 2, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) ordered = false
suite.chainB.CreateClient(suite.chainA) clientA, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN) channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.ORDERED, testConnectionIDA) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainB.GetContext(), testPort1, testChannel1, 2, types.CommitPacket(packet)) suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetNextSequenceRecv(suite.chainB.GetContext(), testPort1, testChannel1, nextSeqRecv) suite.coordinator.SetChannelClosed(suite.chainB, suite.chainA, channelB)
suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
}, false}, }, false},
{"packet ack verification failed", func() { {"packet ack verification failed", func() {
ordered = false // set ordered to true providing the wrong proof for UNORDERED case
packet = types.NewPacket(mockSuccessPacket{}.GetBytes(), 2, testPort1, testChannel1, counterparty.GetPortID(), counterparty.GetChannelID(), timeoutHeight, disabledTimeoutTimestamp) ordered = true
suite.chainB.CreateClient(suite.chainA) clientA, clientB, _, _, channelA, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
suite.chainB.createConnection(testConnectionIDA, testConnectionIDB, testClientIDA, testClientIDB, connection.OPEN) packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), disabledTimeoutTimestamp)
suite.chainB.createChannel(testPort1, testChannel1, testPort2, testChannel2, types.OPEN, types.UNORDERED, testConnectionIDA) suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetPacketCommitment(suite.chainB.GetContext(), testPort1, testChannel1, 2, types.CommitPacket(packet)) suite.coordinator.SetChannelClosed(suite.chainB, suite.chainA, channelB)
suite.chainB.App.IBCKeeper.ChannelKeeper.SetNextSequenceRecv(suite.chainB.GetContext(), testPort1, testChannel1, nextSeqRecv) suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
chanCap = suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
}, false},
{"channel capability not found", func() {
ordered = true
clientA, clientB, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
channelA, channelB := suite.coordinator.CreateChannel(suite.chainA, suite.chainB, connA, connB, types.ORDERED)
packet = types.NewPacket(validPacketData, 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, uint64(suite.chainB.GetContext().BlockHeight()), uint64(suite.chainB.GetContext().BlockTime().UnixNano()))
suite.coordinator.SendPacket(suite.chainA, suite.chainB, packet, clientB)
suite.coordinator.SetChannelClosed(suite.chainB, suite.chainA, channelB)
// need to update chainA's client representing chainB to prove missing ack
suite.coordinator.UpdateClient(suite.chainA, suite.chainB, clientA, clientexported.Tendermint)
chanCap = capabilitytypes.NewCapability(100)
}, false}, }, false},
} }
@ -244,42 +335,29 @@ func (suite *KeeperTestSuite) TestTimeoutOnClose() {
var proof []byte var proof []byte
suite.SetupTest() // reset suite.SetupTest() // reset
nextSeqRecv = 1 // must be explicitly changed
tc.malleate() tc.malleate()
suite.chainB.updateClient(suite.chainA) channelKey := host.KeyChannel(packet.GetDestPort(), packet.GetDestChannel())
suite.chainA.updateClient(suite.chainB) unorderedPacketKey := host.KeyPacketAcknowledgement(packet.GetDestPort(), packet.GetDestChannel(), packet.GetSequence())
proofClosed, proofHeight := queryProof(suite.chainA, channelKey) orderedPacketKey := host.KeyNextSequenceRecv(packet.GetDestPort(), packet.GetDestChannel())
proofClosed, proofHeight := suite.chainB.QueryProof(channelKey)
if ordered { if ordered {
proof, _ = queryProof(suite.chainA, orderedPacketKey) proof, _ = suite.chainB.QueryProof(orderedPacketKey)
} else { } else {
proof, _ = queryProof(suite.chainA, unorderedPacketKey) proof, _ = suite.chainB.QueryProof(unorderedPacketKey)
} }
ctx := suite.chainB.GetContext() _, err := suite.chainA.App.IBCKeeper.ChannelKeeper.TimeoutOnClose(suite.chainA.GetContext(), chanCap, packet, proof, proofClosed, proofHeight, nextSeqRecv)
cap, err := suite.chainB.App.ScopedIBCKeeper.NewCapability(ctx, host.ChannelCapabilityPath(testPort1, testChannel1))
suite.Require().NoError(err)
if tc.expPass { if tc.expPass {
packetOut, err := suite.chainB.App.IBCKeeper.ChannelKeeper.TimeoutOnClose(ctx, cap, packet, proof, proofClosed, proofHeight+1, nextSeqRecv)
suite.Require().NoError(err) suite.Require().NoError(err)
suite.Require().NotNil(packetOut)
} else { } else {
// switch the proofs to invalidate them
packetOut, err := suite.chainB.App.IBCKeeper.ChannelKeeper.TimeoutOnClose(ctx, cap, packet, proofClosed, proof, proofHeight+1, nextSeqRecv)
suite.Require().Error(err) suite.Require().Error(err)
suite.Require().Nil(packetOut)
} }
}) })
} }
} }
type mockTimeoutPacket struct{}
func newMockTimeoutPacket() mockTimeoutPacket {
return mockTimeoutPacket{}
}
// GetBytes returns the serialised packet data (without timeout)
func (mp mockTimeoutPacket) GetBytes() []byte { return []byte("THIS IS A TIMEOUT PACKET") }

View File

@ -12,7 +12,7 @@ import (
var _ sdk.Msg = &MsgChannelOpenInit{} var _ sdk.Msg = &MsgChannelOpenInit{}
// NewMsgChannelOpenInit creates a new MsgChannelCloseInit MsgChannelOpenInit // NewMsgChannelOpenInit creates a new MsgChannelOpenInit
func NewMsgChannelOpenInit( func NewMsgChannelOpenInit(
portID, channelID string, version string, channelOrder Order, connectionHops []string, portID, channelID string, version string, channelOrder Order, connectionHops []string,
counterpartyPortID, counterpartyChannelID string, signer sdk.AccAddress, counterpartyPortID, counterpartyChannelID string, signer sdk.AccAddress,

View File

@ -2,6 +2,7 @@ package testing
import ( import (
"fmt" "fmt"
"strconv"
"testing" "testing"
"time" "time"
@ -9,14 +10,20 @@ import (
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/secp256k1" "github.com/tendermint/tendermint/crypto/secp256k1"
"github.com/tendermint/tendermint/crypto/tmhash"
tmmath "github.com/tendermint/tendermint/libs/math" tmmath "github.com/tendermint/tendermint/libs/math"
lite "github.com/tendermint/tendermint/lite2" lite "github.com/tendermint/tendermint/lite2"
tmtypes "github.com/tendermint/tendermint/types" tmtypes "github.com/tendermint/tendermint/types"
"github.com/tendermint/tendermint/version"
"github.com/cosmos/cosmos-sdk/simapp" "github.com/cosmos/cosmos-sdk/simapp"
sdk "github.com/cosmos/cosmos-sdk/types" sdk "github.com/cosmos/cosmos-sdk/types"
authtypes "github.com/cosmos/cosmos-sdk/x/auth/types" authtypes "github.com/cosmos/cosmos-sdk/x/auth/types"
banktypes "github.com/cosmos/cosmos-sdk/x/bank/types"
capabilitytypes "github.com/cosmos/cosmos-sdk/x/capability/types"
clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
connectiontypes "github.com/cosmos/cosmos-sdk/x/ibc/03-connection/types" connectiontypes "github.com/cosmos/cosmos-sdk/x/ibc/03-connection/types"
channelexported "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/exported"
channeltypes "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types" channeltypes "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types"
ibctmtypes "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types" ibctmtypes "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types"
commitmenttypes "github.com/cosmos/cosmos-sdk/x/ibc/23-commitment/types" commitmenttypes "github.com/cosmos/cosmos-sdk/x/ibc/23-commitment/types"
@ -30,17 +37,18 @@ const (
UnbondingPeriod time.Duration = time.Hour * 24 * 7 * 3 UnbondingPeriod time.Duration = time.Hour * 24 * 7 * 3
MaxClockDrift time.Duration = time.Second * 10 MaxClockDrift time.Duration = time.Second * 10
ConnectionVersion = "1.0" ConnectionVersion = "1.0.0"
ChannelVersion = "1.0" ChannelVersion = "ics20-1"
InvalidID = "IDisInvalid"
ClientIDPrefix = "clientFor"
ConnectionIDPrefix = "connectionid" ConnectionIDPrefix = "connectionid"
ChannelIDPrefix = "channelid"
PortIDPrefix = "portid" maxInt = int(^uint(0) >> 1)
) )
var ( var (
DefaultTrustLevel tmmath.Fraction = lite.DefaultTrustLevel DefaultTrustLevel tmmath.Fraction = lite.DefaultTrustLevel
TestHash = []byte("TESTING HASH")
) )
// TestChain is a testing struct that wraps a simapp with the last TM Header, the current ABCI // TestChain is a testing struct that wraps a simapp with the last TM Header, the current ABCI
@ -64,9 +72,8 @@ type TestChain struct {
SenderAccount authtypes.AccountI SenderAccount authtypes.AccountI
// IBC specific helpers // IBC specific helpers
ClientIDs []string // ClientID's used on this chain ClientIDs []string // ClientID's used on this chain
Connections []TestConnection // track connectionID's created for this chain Connections []*TestConnection // track connectionID's created for this chain
Channels []TestChannel // track portID/channelID's created for this chain
} }
// NewTestChain initializes a new TestChain instance with a single validator set using a // NewTestChain initializes a new TestChain instance with a single validator set using a
@ -88,43 +95,40 @@ func NewTestChain(t *testing.T, chainID string) *TestChain {
valSet := tmtypes.NewValidatorSet([]*tmtypes.Validator{validator}) valSet := tmtypes.NewValidatorSet([]*tmtypes.Validator{validator})
signers := []tmtypes.PrivValidator{privVal} signers := []tmtypes.PrivValidator{privVal}
app := simapp.Setup(false) // generate genesis account
ctx := app.BaseApp.NewContext(false,
abci.Header{
Height: 1,
Time: globalStartTime,
},
)
// generate and set SenderAccount
senderPrivKey := secp256k1.GenPrivKey() senderPrivKey := secp256k1.GenPrivKey()
simapp.AddTestAddrsFromPubKeys(app, ctx, []crypto.PubKey{senderPrivKey.PubKey()}, sdk.NewInt(10000000000)) acc := authtypes.NewBaseAccount(senderPrivKey.PubKey().Address().Bytes(), senderPrivKey.PubKey(), 0, 0)
acc := app.AccountKeeper.GetAccount(ctx, sdk.AccAddress(senderPrivKey.PubKey().Address())) balance := banktypes.Balance{
Address: acc.GetAddress(),
Coins: sdk.NewCoins(sdk.NewCoin(sdk.DefaultBondDenom, sdk.NewInt(100000000000000))),
}
app := simapp.SetupWithGenesisValSet(t, valSet, []authtypes.GenesisAccount{acc}, balance)
// commit init chain changes so create client can be called by a counterparty chain
app.Commit()
// create current header and call begin block // create current header and call begin block
header := abci.Header{ header := abci.Header{
Height: 2, Height: 1,
Time: globalStartTime.Add(timeIncrement), Time: globalStartTime,
} }
app.BeginBlock(abci.RequestBeginBlock{Header: header})
lastHeader := ibctmtypes.CreateTestHeader(chainID, 1, globalStartTime, valSet, signers)
// create an account to send transactions from // create an account to send transactions from
return &TestChain{ chain := &TestChain{
t: t, t: t,
ChainID: chainID, ChainID: chainID,
App: app, App: app,
LastHeader: lastHeader,
CurrentHeader: header, CurrentHeader: header,
Querier: keeper.NewQuerier(*app.IBCKeeper), Querier: keeper.NewQuerier(*app.IBCKeeper),
Vals: valSet, Vals: valSet,
Signers: signers, Signers: signers,
senderPrivKey: senderPrivKey, senderPrivKey: senderPrivKey,
SenderAccount: acc, SenderAccount: acc,
ClientIDs: make([]string, 0),
Connections: make([]*TestConnection, 0),
} }
chain.NextBlock()
return chain
} }
// GetContext returns the current context for the application. // GetContext returns the current context for the application.
@ -133,11 +137,11 @@ func (chain *TestChain) GetContext() sdk.Context {
} }
// QueryProof performs an abci query with the given key and returns the proto encoded merkle proof // QueryProof performs an abci query with the given key and returns the proto encoded merkle proof
// for the query and the height at which the query was performed. // for the query and the height at which the proof will succeed on a tendermint verifier.
func (chain *TestChain) QueryProof(key []byte) ([]byte, uint64) { func (chain *TestChain) QueryProof(key []byte) ([]byte, uint64) {
res := chain.App.Query(abci.RequestQuery{ res := chain.App.Query(abci.RequestQuery{
Path: fmt.Sprintf("store/%s/key", host.StoreKey), Path: fmt.Sprintf("store/%s/key", host.StoreKey),
Height: chain.App.LastBlockHeight(), Height: chain.App.LastBlockHeight() - 1,
Data: key, Data: key,
Prove: true, Prove: true,
}) })
@ -149,7 +153,10 @@ func (chain *TestChain) QueryProof(key []byte) ([]byte, uint64) {
proof, err := chain.App.AppCodec().MarshalBinaryBare(&merkleProof) proof, err := chain.App.AppCodec().MarshalBinaryBare(&merkleProof)
require.NoError(chain.t, err) require.NoError(chain.t, err)
return proof, uint64(res.Height) // proof height + 1 is returned as the proof created corresponds to the height the proof
// was created in the IAVL tree. Tendermint and subsequently the clients that rely on it
// have heights 1 above the IAVL tree. Thus we return proof height + 1
return proof, uint64(res.Height) + 1
} }
// NextBlock sets the last header to the current header and increments the current header to be // NextBlock sets the last header to the current header and increments the current header to be
@ -158,18 +165,19 @@ func (chain *TestChain) QueryProof(key []byte) ([]byte, uint64) {
// CONTRACT: this function must only be called after app.Commit() occurs // CONTRACT: this function must only be called after app.Commit() occurs
func (chain *TestChain) NextBlock() { func (chain *TestChain) NextBlock() {
// set the last header to the current header // set the last header to the current header
chain.LastHeader = ibctmtypes.CreateTestHeader( chain.LastHeader = chain.CreateTMClientHeader()
chain.CurrentHeader.ChainID,
chain.CurrentHeader.Height,
chain.CurrentHeader.Time,
chain.Vals, chain.Signers,
)
// increment the current header // increment the current header
chain.CurrentHeader = abci.Header{ chain.CurrentHeader = abci.Header{
Height: chain.CurrentHeader.Height + 1, Height: chain.App.LastBlockHeight() + 1,
Time: chain.CurrentHeader.Time, AppHash: chain.App.LastCommitID().Hash,
// NOTE: the time is increased by the coordinator to maintain time synchrony amongst
// chains.
Time: chain.CurrentHeader.Time,
} }
chain.App.BeginBlock(abci.RequestBeginBlock{Header: chain.CurrentHeader})
} }
// SendMsg delivers a transaction through the application. It updates the senders sequence // SendMsg delivers a transaction through the application. It updates the senders sequence
@ -198,11 +206,51 @@ func (chain *TestChain) SendMsg(msg sdk.Msg) error {
return nil return nil
} }
// GetClientState retreives the client state for the provided clientID. The client is
// expected to exist otherwise testing will fail.
func (chain *TestChain) GetClientState(clientID string) clientexported.ClientState {
clientState, found := chain.App.IBCKeeper.ClientKeeper.GetClientState(chain.GetContext(), clientID)
require.True(chain.t, found)
return clientState
}
// GetConnection retreives an IBC Connection for the provided TestConnection. The
// connection is expected to exist otherwise testing will fail.
func (chain *TestChain) GetConnection(testConnection *TestConnection) connectiontypes.ConnectionEnd {
connection, found := chain.App.IBCKeeper.ConnectionKeeper.GetConnection(chain.GetContext(), testConnection.ID)
require.True(chain.t, found)
return connection
}
// GetChannel retreives an IBC Channel for the provided TestChannel. The channel
// is expected to exist otherwise testing will fail.
func (chain *TestChain) GetChannel(testChannel TestChannel) channeltypes.Channel {
channel, found := chain.App.IBCKeeper.ChannelKeeper.GetChannel(chain.GetContext(), testChannel.PortID, testChannel.ID)
require.True(chain.t, found)
return channel
}
// GetAcknowledgement retreives an acknowledgement for the provided packet. If the
// acknowledgement does not exist then testing will fail.
func (chain *TestChain) GetAcknowledgement(packet channelexported.PacketI) []byte {
ack, found := chain.App.IBCKeeper.ChannelKeeper.GetPacketAcknowledgement(chain.GetContext(), packet.GetDestPort(), packet.GetDestChannel(), packet.GetSequence())
require.True(chain.t, found)
return ack
}
// GetPrefix returns the prefix for used by a chain in connection creation
func (chain *TestChain) GetPrefix() commitmenttypes.MerklePrefix {
return commitmenttypes.NewMerklePrefix(chain.App.IBCKeeper.ConnectionKeeper.GetCommitmentPrefix().Bytes())
}
// NewClientID appends a new clientID string in the format: // NewClientID appends a new clientID string in the format:
// ClientFor<counterparty-chain-id><index> // ClientFor<counterparty-chain-id><index>
func (chain *TestChain) NewClientID(counterpartyChainID string) string { func (chain *TestChain) NewClientID(counterpartyChainID string) string {
clientID := ClientIDPrefix + counterpartyChainID + string(len(chain.ClientIDs)) clientID := "client" + strconv.Itoa(len(chain.ClientIDs)) + "For" + counterpartyChainID
chain.ClientIDs = append(chain.ClientIDs, clientID) chain.ClientIDs = append(chain.ClientIDs, clientID)
return clientID return clientID
} }
@ -210,9 +258,9 @@ func (chain *TestChain) NewClientID(counterpartyChainID string) string {
// NewConnection appends a new TestConnection which contains references to the connection id, // NewConnection appends a new TestConnection which contains references to the connection id,
// client id and counterparty client id. The connection id format: // client id and counterparty client id. The connection id format:
// connectionid<index> // connectionid<index>
func (chain *TestChain) NewTestConnection(clientID, counterpartyClientID string) TestConnection { func (chain *TestChain) NewTestConnection(clientID, counterpartyClientID string) *TestConnection {
connectionID := ConnectionIDPrefix + string(len(chain.Connections)) connectionID := ConnectionIDPrefix + strconv.Itoa(len(chain.Connections))
conn := TestConnection{ conn := &TestConnection{
ID: connectionID, ID: connectionID,
ClientID: clientID, ClientID: clientID,
CounterpartyClientID: counterpartyClientID, CounterpartyClientID: counterpartyClientID,
@ -222,23 +270,6 @@ func (chain *TestChain) NewTestConnection(clientID, counterpartyClientID string)
return conn return conn
} }
// NewTestChannel appends a new TestChannel which contains references to the port and channel ID
// used for channel creation and interaction. The channel id and port id format:
// channelid<index>
// portid<index>
func (chain *TestChain) NewTestChannel() TestChannel {
portID := PortIDPrefix + string(len(chain.Channels))
channelID := ChannelIDPrefix + string(len(chain.Channels))
channel := TestChannel{
PortID: portID,
ChannelID: channelID,
}
chain.Channels = append(chain.Channels, channel)
return channel
}
// CreateTMClient will construct and execute a 07-tendermint MsgCreateClient. A counterparty // CreateTMClient will construct and execute a 07-tendermint MsgCreateClient. A counterparty
// client will be created on the (target) chain. // client will be created on the (target) chain.
func (chain *TestChain) CreateTMClient(counterparty *TestChain, clientID string) error { func (chain *TestChain) CreateTMClient(counterparty *TestChain, clientID string) error {
@ -263,17 +294,65 @@ func (chain *TestChain) UpdateTMClient(counterparty *TestChain, clientID string)
return chain.SendMsg(msg) return chain.SendMsg(msg)
} }
// CreateTMClientHeader creates a TM header to update the TM client.
func (chain *TestChain) CreateTMClientHeader() ibctmtypes.Header {
vsetHash := chain.Vals.Hash()
tmHeader := tmtypes.Header{
Version: version.Consensus{Block: 2, App: 2},
ChainID: chain.ChainID,
Height: chain.CurrentHeader.Height,
Time: chain.CurrentHeader.Time,
LastBlockID: MakeBlockID(make([]byte, tmhash.Size), maxInt, make([]byte, tmhash.Size)),
LastCommitHash: chain.App.LastCommitID().Hash,
DataHash: tmhash.Sum([]byte("data_hash")),
ValidatorsHash: vsetHash,
NextValidatorsHash: vsetHash,
ConsensusHash: tmhash.Sum([]byte("consensus_hash")),
AppHash: chain.CurrentHeader.AppHash,
LastResultsHash: tmhash.Sum([]byte("last_results_hash")),
EvidenceHash: tmhash.Sum([]byte("evidence_hash")),
ProposerAddress: chain.Vals.Proposer.Address,
}
hhash := tmHeader.Hash()
blockID := MakeBlockID(hhash, 3, tmhash.Sum([]byte("part_set")))
voteSet := tmtypes.NewVoteSet(chain.ChainID, chain.CurrentHeader.Height, 1, tmtypes.PrecommitType, chain.Vals)
commit, err := tmtypes.MakeCommit(blockID, chain.CurrentHeader.Height, 1, voteSet, chain.Signers, chain.CurrentHeader.Time)
require.NoError(chain.t, err)
signedHeader := tmtypes.SignedHeader{
Header: &tmHeader,
Commit: commit,
}
return ibctmtypes.Header{
SignedHeader: signedHeader,
ValidatorSet: chain.Vals,
}
}
// Copied unimported test functions from tmtypes to use them here
func MakeBlockID(hash []byte, partSetSize int, partSetHash []byte) tmtypes.BlockID {
return tmtypes.BlockID{
Hash: hash,
PartsHeader: tmtypes.PartSetHeader{
Total: partSetSize,
Hash: partSetHash,
},
}
}
// ConnectionOpenInit will construct and execute a MsgConnectionOpenInit. // ConnectionOpenInit will construct and execute a MsgConnectionOpenInit.
func (chain *TestChain) ConnectionOpenInit( func (chain *TestChain) ConnectionOpenInit(
counterparty *TestChain, counterparty *TestChain,
connection, counterpartyConnection TestConnection, connection, counterpartyConnection *TestConnection,
) error { ) error {
prefix := commitmenttypes.NewMerklePrefix(counterparty.App.IBCKeeper.ConnectionKeeper.GetCommitmentPrefix().Bytes())
msg := connectiontypes.NewMsgConnectionOpenInit( msg := connectiontypes.NewMsgConnectionOpenInit(
connection.ID, connection.ClientID, connection.ID, connection.ClientID,
counterpartyConnection.ID, connection.CounterpartyClientID, counterpartyConnection.ID, connection.CounterpartyClientID,
prefix, counterparty.GetPrefix(),
chain.SenderAccount.GetAddress(), chain.SenderAccount.GetAddress(),
) )
return chain.SendMsg(msg) return chain.SendMsg(msg)
@ -282,21 +361,23 @@ func (chain *TestChain) ConnectionOpenInit(
// ConnectionOpenTry will construct and execute a MsgConnectionOpenTry. // ConnectionOpenTry will construct and execute a MsgConnectionOpenTry.
func (chain *TestChain) ConnectionOpenTry( func (chain *TestChain) ConnectionOpenTry(
counterparty *TestChain, counterparty *TestChain,
connection, counterpartyConnection TestConnection, connection, counterpartyConnection *TestConnection,
) error { ) error {
prefix := commitmenttypes.NewMerklePrefix(counterparty.App.IBCKeeper.ConnectionKeeper.GetCommitmentPrefix().Bytes())
connectionKey := host.KeyConnection(counterpartyConnection.ID) connectionKey := host.KeyConnection(counterpartyConnection.ID)
proofInit, proofHeight := counterparty.QueryProof(connectionKey) proofInit, proofHeight := counterparty.QueryProof(connectionKey)
consensusHeight := uint64(counterparty.App.LastBlockHeight()) // retrieve consensus state to provide proof for
consensusKey := prefixedClientKey(connection.ClientID, host.KeyConsensusState(consensusHeight)) consState, found := counterparty.App.IBCKeeper.ClientKeeper.GetLatestClientConsensusState(counterparty.GetContext(), counterpartyConnection.ClientID)
require.True(chain.t, found)
consensusHeight := consState.GetHeight()
consensusKey := prefixedClientKey(counterpartyConnection.ClientID, host.KeyConsensusState(consensusHeight))
proofConsensus, _ := counterparty.QueryProof(consensusKey) proofConsensus, _ := counterparty.QueryProof(consensusKey)
msg := connectiontypes.NewMsgConnectionOpenTry( msg := connectiontypes.NewMsgConnectionOpenTry(
connection.ID, connection.ClientID, connection.ID, connection.ClientID,
counterpartyConnection.ID, connection.CounterpartyClientID, counterpartyConnection.ID, counterpartyConnection.ClientID,
prefix, []string{ConnectionVersion}, counterparty.GetPrefix(), []string{ConnectionVersion},
proofInit, proofConsensus, proofInit, proofConsensus,
proofHeight, consensusHeight, proofHeight, consensusHeight,
chain.SenderAccount.GetAddress(), chain.SenderAccount.GetAddress(),
@ -307,13 +388,17 @@ func (chain *TestChain) ConnectionOpenTry(
// ConnectionOpenAck will construct and execute a MsgConnectionOpenAck. // ConnectionOpenAck will construct and execute a MsgConnectionOpenAck.
func (chain *TestChain) ConnectionOpenAck( func (chain *TestChain) ConnectionOpenAck(
counterparty *TestChain, counterparty *TestChain,
connection, counterpartyConnection TestConnection, connection, counterpartyConnection *TestConnection,
) error { ) error {
connectionKey := host.KeyConnection(counterpartyConnection.ID) connectionKey := host.KeyConnection(counterpartyConnection.ID)
proofTry, proofHeight := counterparty.QueryProof(connectionKey) proofTry, proofHeight := counterparty.QueryProof(connectionKey)
consensusHeight := uint64(counterparty.App.LastBlockHeight()) // retrieve consensus state to provide proof for
consensusKey := prefixedClientKey(connection.ClientID, host.KeyConsensusState(consensusHeight)) consState, found := counterparty.App.IBCKeeper.ClientKeeper.GetLatestClientConsensusState(counterparty.GetContext(), counterpartyConnection.ClientID)
require.True(chain.t, found)
consensusHeight := consState.GetHeight()
consensusKey := prefixedClientKey(counterpartyConnection.ClientID, host.KeyConsensusState(consensusHeight))
proofConsensus, _ := counterparty.QueryProof(consensusKey) proofConsensus, _ := counterparty.QueryProof(consensusKey)
msg := connectiontypes.NewMsgConnectionOpenAck( msg := connectiontypes.NewMsgConnectionOpenAck(
@ -329,7 +414,7 @@ func (chain *TestChain) ConnectionOpenAck(
// ConnectionOpenConfirm will construct and execute a MsgConnectionOpenConfirm. // ConnectionOpenConfirm will construct and execute a MsgConnectionOpenConfirm.
func (chain *TestChain) ConnectionOpenConfirm( func (chain *TestChain) ConnectionOpenConfirm(
counterparty *TestChain, counterparty *TestChain,
connection, counterpartyConnection TestConnection, connection, counterpartyConnection *TestConnection,
) error { ) error {
connectionKey := host.KeyConnection(counterpartyConnection.ID) connectionKey := host.KeyConnection(counterpartyConnection.ID)
proof, height := counterparty.QueryProof(connectionKey) proof, height := counterparty.QueryProof(connectionKey)
@ -342,33 +427,87 @@ func (chain *TestChain) ConnectionOpenConfirm(
return chain.SendMsg(msg) return chain.SendMsg(msg)
} }
// ChannelOpenInit will construct and execute a MsgChannelOpenInit. // CreatePortCapability binds and claims a capability for the given portID if it does not
func (chain *TestChain) ChannelOpenInit( // already exist. This function will fail testing on any resulting error.
func (chain *TestChain) CreatePortCapability(portID string) {
// check if the portId is already binded, if not bind it
_, ok := chain.App.ScopedIBCKeeper.GetCapability(chain.GetContext(), host.PortPath(portID))
if !ok {
cap, err := chain.App.ScopedIBCKeeper.NewCapability(chain.GetContext(), host.PortPath(portID))
require.NoError(chain.t, err)
err = chain.App.ScopedTransferKeeper.ClaimCapability(chain.GetContext(), cap, host.PortPath(portID))
require.NoError(chain.t, err)
}
chain.App.Commit()
chain.NextBlock()
}
// GetPortCapability returns the port capability for the given portID. The capability must
// exist, otherwise testing will fail.
func (chain *TestChain) GetPortCapability(portID string) *capabilitytypes.Capability {
cap, ok := chain.App.ScopedIBCKeeper.GetCapability(chain.GetContext(), host.PortPath(portID))
require.True(chain.t, ok)
return cap
}
// CreateChannelCapability binds and claims a capability for the given portID and channelID
// if it does not already exist. This function will fail testing on any resulting error.
func (chain *TestChain) CreateChannelCapability(portID, channelID string) {
capName := host.ChannelCapabilityPath(portID, channelID)
// check if the portId is already binded, if not bind it
_, ok := chain.App.ScopedIBCKeeper.GetCapability(chain.GetContext(), capName)
if !ok {
cap, err := chain.App.ScopedIBCKeeper.NewCapability(chain.GetContext(), capName)
require.NoError(chain.t, err)
err = chain.App.ScopedTransferKeeper.ClaimCapability(chain.GetContext(), cap, capName)
require.NoError(chain.t, err)
}
chain.App.Commit()
chain.NextBlock()
}
// GetChannelCapability returns the channel capability for the given portID and channelID.
// The capability must exist, otherwise testing will fail.
func (chain *TestChain) GetChannelCapability(portID, channelID string) *capabilitytypes.Capability {
cap, ok := chain.App.ScopedIBCKeeper.GetCapability(chain.GetContext(), host.ChannelCapabilityPath(portID, channelID))
require.True(chain.t, ok)
return cap
}
// ChanOpenInit will construct and execute a MsgChannelOpenInit.
func (chain *TestChain) ChanOpenInit(
ch, counterparty TestChannel, ch, counterparty TestChannel,
order channeltypes.Order, order channeltypes.Order,
connectionID string, connectionID string,
) error { ) error {
msg := channeltypes.NewMsgChannelOpenInit( msg := channeltypes.NewMsgChannelOpenInit(
ch.PortID, ch.ChannelID, ch.PortID, ch.ID,
ChannelVersion, order, []string{connectionID}, ChannelVersion, order, []string{connectionID},
counterparty.PortID, counterparty.ChannelID, counterparty.PortID, counterparty.ID,
chain.SenderAccount.GetAddress(), chain.SenderAccount.GetAddress(),
) )
return chain.SendMsg(msg) return chain.SendMsg(msg)
} }
// ChannelOpenTry will construct and execute a MsgChannelOpenTry. // ChanOpenTry will construct and execute a MsgChannelOpenTry.
func (chain *TestChain) ChannelOpenTry( func (chain *TestChain) ChanOpenTry(
ch, counterparty TestChannel, counterparty *TestChain,
ch, counterpartyCh TestChannel,
order channeltypes.Order, order channeltypes.Order,
connectionID string, connectionID string,
) error { ) error {
proof, height := chain.QueryProof(host.KeyConnection(connectionID)) proof, height := counterparty.QueryProof(host.KeyChannel(counterpartyCh.PortID, counterpartyCh.ID))
msg := channeltypes.NewMsgChannelOpenTry( msg := channeltypes.NewMsgChannelOpenTry(
ch.PortID, ch.ChannelID, ch.PortID, ch.ID,
ChannelVersion, order, []string{connectionID}, ChannelVersion, order, []string{connectionID},
counterparty.PortID, counterparty.ChannelID, counterpartyCh.PortID, counterpartyCh.ID,
ChannelVersion, ChannelVersion,
proof, height, proof, height,
chain.SenderAccount.GetAddress(), chain.SenderAccount.GetAddress(),
@ -376,15 +515,15 @@ func (chain *TestChain) ChannelOpenTry(
return chain.SendMsg(msg) return chain.SendMsg(msg)
} }
// ChannelOpenAck will construct and execute a MsgChannelOpenAck. // ChanOpenAck will construct and execute a MsgChannelOpenAck.
func (chain *TestChain) ChannelOpenAck( func (chain *TestChain) ChanOpenAck(
ch, counterparty TestChannel, counterparty *TestChain,
connectionID string, ch, counterpartyCh TestChannel,
) error { ) error {
proof, height := chain.QueryProof(host.KeyConnection(connectionID)) proof, height := counterparty.QueryProof(host.KeyChannel(counterpartyCh.PortID, counterpartyCh.ID))
msg := channeltypes.NewMsgChannelOpenAck( msg := channeltypes.NewMsgChannelOpenAck(
ch.PortID, ch.ChannelID, ch.PortID, ch.ID,
ChannelVersion, ChannelVersion,
proof, height, proof, height,
chain.SenderAccount.GetAddress(), chain.SenderAccount.GetAddress(),
@ -392,17 +531,70 @@ func (chain *TestChain) ChannelOpenAck(
return chain.SendMsg(msg) return chain.SendMsg(msg)
} }
// ChannelOpenConfirm will construct and execute a MsgChannelOpenConfirm. // ChanOpenConfirm will construct and execute a MsgChannelOpenConfirm.
func (chain *TestChain) ChannelOpenConfirm( func (chain *TestChain) ChanOpenConfirm(
ch, counterparty TestChannel, counterparty *TestChain,
connectionID string, ch, counterpartyCh TestChannel,
) error { ) error {
proof, height := chain.QueryProof(host.KeyConnection(connectionID)) proof, height := counterparty.QueryProof(host.KeyChannel(counterpartyCh.PortID, counterpartyCh.ID))
msg := channeltypes.NewMsgChannelOpenConfirm( msg := channeltypes.NewMsgChannelOpenConfirm(
ch.PortID, ch.ChannelID, ch.PortID, ch.ID,
proof, height, proof, height,
chain.SenderAccount.GetAddress(), chain.SenderAccount.GetAddress(),
) )
return chain.SendMsg(msg) return chain.SendMsg(msg)
} }
// ChanCloseInit will construct and execute a MsgChannelCloseInit.
//
// NOTE: does not work with ibc-transfer module
func (chain *TestChain) ChanCloseInit(
counterparty *TestChain,
channel TestChannel,
) error {
msg := channeltypes.NewMsgChannelCloseInit(
channel.PortID, channel.ID,
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
}
// SendPacket simulates sending a packet through the channel keeper. No message needs to be
// passed since this call is made from a module.
func (chain *TestChain) SendPacket(
packet channelexported.PacketI,
) error {
channelCap := chain.GetChannelCapability(packet.GetSourcePort(), packet.GetSourceChannel())
// no need to send message, acting as a module
err := chain.App.IBCKeeper.ChannelKeeper.SendPacket(chain.GetContext(), channelCap, packet)
if err != nil {
return err
}
// commit changes
chain.App.Commit()
chain.NextBlock()
return nil
}
// PacketExecuted simulates receiving and wiritng an acknowledgement to the chain.
func (chain *TestChain) PacketExecuted(
packet channelexported.PacketI,
) error {
channelCap := chain.GetChannelCapability(packet.GetSourcePort(), packet.GetSourceChannel())
// no need to send message, acting as a handler
err := chain.App.IBCKeeper.ChannelKeeper.PacketExecuted(chain.GetContext(), channelCap, packet, TestHash)
if err != nil {
return err
}
// commit changes
chain.App.Commit()
chain.NextBlock()
return nil
}

View File

@ -2,6 +2,7 @@ package testing
import ( import (
"fmt" "fmt"
"strconv"
"testing" "testing"
"time" "time"
@ -9,7 +10,7 @@ import (
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported" clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
connectiontypes "github.com/cosmos/cosmos-sdk/x/ibc/03-connection/types" channelexported "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/exported"
channeltypes "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types" channeltypes "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types"
) )
@ -32,7 +33,7 @@ func NewCoordinator(t *testing.T, n int) *Coordinator {
chains := make(map[string]*TestChain) chains := make(map[string]*TestChain)
for i := 0; i < n; i++ { for i := 0; i < n; i++ {
chainID := ChainIDPrefix + string(i) chainID := GetChainID(i)
chains[chainID] = NewTestChain(t, chainID) chains[chainID] = NewTestChain(t, chainID)
} }
return &Coordinator{ return &Coordinator{
@ -41,60 +42,57 @@ func NewCoordinator(t *testing.T, n int) *Coordinator {
} }
} }
// IncrementTime iterates through all the TestChain's and increments their current header time // Setup constructs a TM client, connection, and channel on both chains provided. It will
// by 5 seconds. // fails if any error occurs. The clientID's, TestConnections, and TestChannels are returned
// // for both chains.
// CONTRACT: this function must be called after every commit on any TestChain. func (coord *Coordinator) Setup(
func (coord *Coordinator) IncrementTime() { chainA, chainB *TestChain,
for _, chain := range coord.Chains { ) (string, string, *TestConnection, *TestConnection, TestChannel, TestChannel) {
chain.CurrentHeader = abci.Header{ clientA, clientB, connA, connB := coord.SetupClientConnections(chainA, chainB, clientexported.Tendermint)
Height: chain.CurrentHeader.Height,
Time: chain.CurrentHeader.Time.Add((timeIncrement)), // channels can also be referenced through the returned connections
} channelA, channelB := coord.CreateChannel(chainA, chainB, connA, connB, channeltypes.UNORDERED)
}
return clientA, clientB, connA, connB, channelA, channelB
} }
// GetChain returns the TestChain using the given chainID and returns an error if it does // SetupClients is a helper function to create clients on both chains. It assumes the
// not exist. // caller does not anticipate any errors.
func (coord *Coordinator) GetChain(chainID string) *TestChain { func (coord *Coordinator) SetupClients(
chain, found := coord.Chains[chainID] chainA, chainB *TestChain,
require.True(coord.t, found, fmt.Sprintf("%s chain does not exist", chainID)) clientType clientexported.ClientType,
return chain ) (string, string) {
clientA, err := coord.CreateClient(chainA, chainB, clientType)
require.NoError(coord.t, err)
clientB, err := coord.CreateClient(chainB, chainA, clientType)
require.NoError(coord.t, err)
return clientA, clientB
} }
// CommitBlock commits a block on the provided indexes and then increments the global time. // SetupClientConnections is a helper function to create clients and the appropriate
// // connections on both the source and counterparty chain. It assumes the caller does not
// CONTRACT: the passed in list of indexes must not contain duplicates // anticipate any errors.
func (coord *Coordinator) CommitBlock(chains ...string) { func (coord *Coordinator) SetupClientConnections(
for _, chainID := range chains { chainA, chainB *TestChain,
chain := coord.GetChain(chainID) clientType clientexported.ClientType,
chain.App.Commit() ) (string, string, *TestConnection, *TestConnection) {
chain.NextBlock()
}
coord.IncrementTime()
}
// CommitNBlocks commits n blocks to state and updates the block height by 1 for each commit. clientA, clientB := coord.SetupClients(chainA, chainB, clientType)
func (coord *Coordinator) CommitNBlocks(chainID string, n uint64) {
chain := coord.GetChain(chainID)
for i := uint64(0); i < n; i++ { connA, connB := coord.CreateConnection(chainA, chainB, clientA, clientB)
chain.App.BeginBlock(abci.RequestBeginBlock{Header: chain.CurrentHeader})
chain.App.Commit() return clientA, clientB, connA, connB
chain.NextBlock()
coord.IncrementTime()
}
} }
// CreateClient creates a counterparty client on the source chain and returns the clientID. // CreateClient creates a counterparty client on the source chain and returns the clientID.
func (coord *Coordinator) CreateClient( func (coord *Coordinator) CreateClient(
sourceID, counterpartyID string, source, counterparty *TestChain,
clientType clientexported.ClientType, clientType clientexported.ClientType,
) (clientID string, err error) { ) (clientID string, err error) {
coord.CommitBlock(sourceID, counterpartyID) coord.CommitBlock(source, counterparty)
source := coord.GetChain(sourceID)
counterparty := coord.GetChain(counterpartyID)
clientID = source.NewClientID(counterparty.ChainID) clientID = source.NewClientID(counterparty.ChainID)
@ -117,14 +115,11 @@ func (coord *Coordinator) CreateClient(
// UpdateClient updates a counterparty client on the source chain. // UpdateClient updates a counterparty client on the source chain.
func (coord *Coordinator) UpdateClient( func (coord *Coordinator) UpdateClient(
sourceID, counterpartyID, source, counterparty *TestChain,
clientID string, clientID string,
clientType clientexported.ClientType, clientType clientexported.ClientType,
) (err error) { ) (err error) {
coord.CommitBlock(sourceID, counterpartyID) coord.CommitBlock(source, counterparty)
source := coord.GetChain(sourceID)
counterparty := coord.GetChain(counterpartyID)
switch clientType { switch clientType {
case clientexported.Tendermint: case clientexported.Tendermint:
@ -144,58 +139,69 @@ func (coord *Coordinator) UpdateClient(
} }
// CreateConnection constructs and executes connection handshake messages in order to create // CreateConnection constructs and executes connection handshake messages in order to create
// OPEN channels on source and counterparty chains. The connection information of the source // OPEN channels on chainA and chainB. The connection information of for chainA and chainB
// and counterparty's are returned within a TestConnection struct. If there is a fault in // are returned within a TestConnection struct. The function expects the connections to be
// the connection handshake then an error is returned. // successfully opened otherwise testing will fail.
//
// NOTE: The counterparty testing connection will be created even if it is not created in the
// application state.
func (coord *Coordinator) CreateConnection( func (coord *Coordinator) CreateConnection(
sourceID, counterpartyID, chainA, chainB *TestChain,
clientID, counterpartyClientID string, clientA, clientB string,
state connectiontypes.State, ) (*TestConnection, *TestConnection) {
) (TestConnection, TestConnection, error) {
source := coord.GetChain(sourceID)
counterparty := coord.GetChain(counterpartyID)
sourceConnection := source.NewTestConnection(clientID, counterpartyClientID) connA, connB, err := coord.ConnOpenInit(chainA, chainB, clientA, clientB)
counterpartyConnection := counterparty.NewTestConnection(counterpartyClientID, clientID) require.NoError(coord.t, err)
if err := coord.CreateConnectionInit(source, counterparty, sourceConnection, counterpartyConnection); err != nil { err = coord.ConnOpenTry(chainB, chainA, connB, connA)
return sourceConnection, counterpartyConnection, err require.NoError(coord.t, err)
}
if err := coord.CreateConnectionOpenTry(counterparty, source, counterpartyConnection, sourceConnection); err != nil { err = coord.ConnOpenAck(chainA, chainB, connA, connB)
return sourceConnection, counterpartyConnection, err require.NoError(coord.t, err)
}
if err := coord.CreateConnectionOpenAck(source, counterparty, sourceConnection, counterpartyConnection); err != nil { err = coord.ConnOpenConfirm(chainB, chainA, connB, connA)
return sourceConnection, counterpartyConnection, err require.NoError(coord.t, err)
}
if err := coord.CreateConnectionOpenConfirm(counterparty, source, counterpartyConnection, sourceConnection); err != nil { return connA, connB
return sourceConnection, counterpartyConnection, err
}
return sourceConnection, counterpartyConnection, nil
} }
// CreateConenctionInit initializes a connection on the source chain with the state INIT // CreateChannel constructs and executes channel handshake messages in order to create
// using the OpenInit handshake call. // OPEN channels on chainA and chainB. The function expects the channels to be successfully
func (coord *Coordinator) CreateConnectionInit( // opened otherwise testing will fail.
func (coord *Coordinator) CreateChannel(
chainA, chainB *TestChain,
connA, connB *TestConnection,
order channeltypes.Order,
) (TestChannel, TestChannel) {
channelA, channelB, err := coord.ChanOpenInit(chainA, chainB, connA, connB, order)
require.NoError(coord.t, err)
err = coord.ChanOpenTry(chainB, chainA, channelB, channelA, connB, order)
require.NoError(coord.t, err)
err = coord.ChanOpenAck(chainA, chainB, channelA, channelB)
require.NoError(coord.t, err)
err = coord.ChanOpenConfirm(chainB, chainA, channelB, channelA)
require.NoError(coord.t, err)
return channelA, channelB
}
// SendPacket sends a packet through the channel keeper on the source chain and updates the
// counterparty client for the source chain.
func (coord *Coordinator) SendPacket(
source, counterparty *TestChain, source, counterparty *TestChain,
sourceConnection, counterpartyConnection TestConnection, packet channelexported.PacketI,
counterpartyClientID string,
) error { ) error {
// initialize connection on source if err := source.SendPacket(packet); err != nil {
if err := source.ConnectionOpenInit(counterparty, sourceConnection, counterpartyConnection); err != nil {
return err return err
} }
coord.IncrementTime() coord.IncrementTime()
// update source client on counterparty connection // update source client on counterparty connection
if err := coord.UpdateClient( if err := coord.UpdateClient(
counterparty.ChainID, source.ChainID, counterparty, source,
counterpartyConnection.ClientID, clientexported.Tendermint, counterpartyClientID, clientexported.Tendermint,
); err != nil { ); err != nil {
return err return err
} }
@ -203,11 +209,108 @@ func (coord *Coordinator) CreateConnectionInit(
return nil return nil
} }
// CreateConenctionOpenTry initializes a connection on the source chain with the state TRYOPEN // PacketExecuted receives a packet through the channel keeper on the source chain and updates the
// using the OpenTry handshake call. // counterparty client for the source chain.
func (coord *Coordinator) CreateConnectionOpenTry( func (coord *Coordinator) PacketExecuted(
source, counterparty *TestChain, source, counterparty *TestChain,
sourceConnection, counterpartyConnection TestConnection, packet channelexported.PacketI,
counterpartyClientID string,
) error {
if err := source.PacketExecuted(packet); err != nil {
return err
}
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
counterparty, source,
counterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
}
// IncrementTime iterates through all the TestChain's and increments their current header time
// by 5 seconds.
//
// CONTRACT: this function must be called after every commit on any TestChain.
func (coord *Coordinator) IncrementTime() {
for _, chain := range coord.Chains {
chain.CurrentHeader.Time = chain.CurrentHeader.Time.Add(timeIncrement)
chain.App.BeginBlock(abci.RequestBeginBlock{Header: chain.CurrentHeader})
}
}
// GetChain returns the TestChain using the given chainID and returns an error if it does
// not exist.
func (coord *Coordinator) GetChain(chainID string) *TestChain {
chain, found := coord.Chains[chainID]
require.True(coord.t, found, fmt.Sprintf("%s chain does not exist", chainID))
return chain
}
// GetChainID returns the chainID used for the provided index.
func GetChainID(index int) string {
return ChainIDPrefix + strconv.Itoa(index)
}
// CommitBlock commits a block on the provided indexes and then increments the global time.
//
// CONTRACT: the passed in list of indexes must not contain duplicates
func (coord *Coordinator) CommitBlock(chains ...*TestChain) {
for _, chain := range chains {
chain.App.Commit()
chain.NextBlock()
}
coord.IncrementTime()
}
// CommitNBlocks commits n blocks to state and updates the block height by 1 for each commit.
func (coord *Coordinator) CommitNBlocks(chain *TestChain, n uint64) {
for i := uint64(0); i < n; i++ {
chain.App.BeginBlock(abci.RequestBeginBlock{Header: chain.CurrentHeader})
chain.App.Commit()
chain.NextBlock()
coord.IncrementTime()
}
}
// ConnOpenInit initializes a connection on the source chain with the state INIT
// using the OpenInit handshake call.
//
// NOTE: The counterparty testing connection will be created even if it is not created in the
// application state.
func (coord *Coordinator) ConnOpenInit(
source, counterparty *TestChain,
clientID, counterpartyClientID string,
) (*TestConnection, *TestConnection, error) {
sourceConnection := source.NewTestConnection(clientID, counterpartyClientID)
counterpartyConnection := counterparty.NewTestConnection(counterpartyClientID, clientID)
// initialize connection on source
if err := source.ConnectionOpenInit(counterparty, sourceConnection, counterpartyConnection); err != nil {
return sourceConnection, counterpartyConnection, err
}
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
counterparty, source,
counterpartyClientID, clientexported.Tendermint,
); err != nil {
return sourceConnection, counterpartyConnection, err
}
return sourceConnection, counterpartyConnection, nil
}
// ConnOpenTry initializes a connection on the source chain with the state TRYOPEN
// using the OpenTry handshake call.
func (coord *Coordinator) ConnOpenTry(
source, counterparty *TestChain,
sourceConnection, counterpartyConnection *TestConnection,
) error { ) error {
// initialize TRYOPEN connection on source // initialize TRYOPEN connection on source
if err := source.ConnectionOpenTry(counterparty, sourceConnection, counterpartyConnection); err != nil { if err := source.ConnectionOpenTry(counterparty, sourceConnection, counterpartyConnection); err != nil {
@ -217,7 +320,7 @@ func (coord *Coordinator) CreateConnectionOpenTry(
// update source client on counterparty connection // update source client on counterparty connection
if err := coord.UpdateClient( if err := coord.UpdateClient(
counterparty.ChainID, source.ChainID, counterparty, source,
counterpartyConnection.ClientID, clientexported.Tendermint, counterpartyConnection.ClientID, clientexported.Tendermint,
); err != nil { ); err != nil {
return err return err
@ -226,11 +329,11 @@ func (coord *Coordinator) CreateConnectionOpenTry(
return nil return nil
} }
// CreateConnectionOpenAck initializes a connection on the source chain with the state OPEN // ConnOpenAck initializes a connection on the source chain with the state OPEN
// using the OpenAck handshake call. // using the OpenAck handshake call.
func (coord *Coordinator) CreateConnectionOpenAck( func (coord *Coordinator) ConnOpenAck(
source, counterparty *TestChain, source, counterparty *TestChain,
sourceConnection, counterpartyConnection TestConnection, sourceConnection, counterpartyConnection *TestConnection,
) error { ) error {
// set OPEN connection on source using OpenAck // set OPEN connection on source using OpenAck
if err := source.ConnectionOpenAck(counterparty, sourceConnection, counterpartyConnection); err != nil { if err := source.ConnectionOpenAck(counterparty, sourceConnection, counterpartyConnection); err != nil {
@ -240,7 +343,7 @@ func (coord *Coordinator) CreateConnectionOpenAck(
// update source client on counterparty connection // update source client on counterparty connection
if err := coord.UpdateClient( if err := coord.UpdateClient(
counterparty.ChainID, source.ChainID, counterparty, source,
counterpartyConnection.ClientID, clientexported.Tendermint, counterpartyConnection.ClientID, clientexported.Tendermint,
); err != nil { ); err != nil {
return err return err
@ -249,21 +352,21 @@ func (coord *Coordinator) CreateConnectionOpenAck(
return nil return nil
} }
// CreateConnectionOpenConfirm initializes a connection on the source chain with the state OPEN // ConnOpenConfirm initializes a connection on the source chain with the state OPEN
// using the OpenConfirm handshake call. // using the OpenConfirm handshake call.
func (coord *Coordinator) CreateConnectionOpenConfirm( func (coord *Coordinator) ConnOpenConfirm(
source, counterparty *TestChain, source, counterparty *TestChain,
sourceConnection, counterpartyConnection TestConnection, sourceConnection, counterpartyConnection *TestConnection,
) error { ) error {
if err := counterparty.ConnectionOpenConfirm(counterparty, sourceConnection, counterpartyConnection); err != nil { if err := source.ConnectionOpenConfirm(counterparty, sourceConnection, counterpartyConnection); err != nil {
return err return err
} }
coord.IncrementTime() coord.IncrementTime()
// update source client on counterparty connection // update source client on counterparty connection
if err := coord.UpdateClient( if err := coord.UpdateClient(
source.ChainID, counterparty.ChainID, counterparty, source,
sourceConnection.ClientID, clientexported.Tendermint, counterpartyConnection.ClientID, clientexported.Tendermint,
); err != nil { ); err != nil {
return err return err
} }
@ -271,87 +374,58 @@ func (coord *Coordinator) CreateConnectionOpenConfirm(
return nil return nil
} }
// CreateChannel constructs and executes channel handshake messages in order to create // ChanOpenInit initializes a channel on the source chain with the state INIT
// channels on source and counterparty chains with the passed in Channel State. The portID and // using the OpenInit handshake call.
// channelID of source and counterparty are returned.
// //
// NOTE: The counterparty testing channel will be created even if it is not created in the // NOTE: The counterparty testing channel will be created even if it is not created in the
// application state. // application state.
func (coord *Coordinator) CreateChannel( func (coord *Coordinator) ChanOpenInit(
sourceID, counterpartyID string, source, counterparty *TestChain,
connection, counterpartyConnection TestConnection, connection, counterpartyConnection *TestConnection,
order channeltypes.Order, order channeltypes.Order,
state channeltypes.State,
) (TestChannel, TestChannel, error) { ) (TestChannel, TestChannel, error) {
source := coord.GetChain(sourceID) sourceChannel := connection.AddTestChannel()
counterparty := coord.GetChain(counterpartyID) counterpartyChannel := counterpartyConnection.AddTestChannel()
sourceChannel := source.NewTestChannel() // create port capability
counterpartyChannel := counterparty.NewTestChannel() source.CreatePortCapability(sourceChannel.PortID)
coord.IncrementTime()
if err := coord.CreateChannelInit(source, counterparty, sourceChannel, counterpartyChannel, connection, order); err != nil { // initialize channel on source
if err := source.ChanOpenInit(sourceChannel, counterpartyChannel, order, connection.ID); err != nil {
return sourceChannel, counterpartyChannel, err return sourceChannel, counterpartyChannel, err
} }
coord.IncrementTime()
if err := coord.CreateChannelOpenTry(counterparty, source, counterpartyChannel, sourceChannel, counterpartyConnection, order); err != nil { // update source client on counterparty connection
return sourceChannel, counterpartyChannel, err if err := coord.UpdateClient(
} counterparty, source,
counterpartyConnection.ClientID, clientexported.Tendermint,
if err := coord.CreateChannelOpenAck(source, counterparty, sourceChannel, counterpartyChannel, connection); err != nil { ); err != nil {
return sourceChannel, counterpartyChannel, err
}
if err := coord.CreateChannelOpenConfirm(counterparty, source, counterpartyChannel, sourceChannel, counterpartyConnection); err != nil {
return sourceChannel, counterpartyChannel, err return sourceChannel, counterpartyChannel, err
} }
return sourceChannel, counterpartyChannel, nil return sourceChannel, counterpartyChannel, nil
} }
// CreateChannelInit initializes a channel on the source chain with the state INIT // ChanOpenTry initializes a channel on the source chain with the state TRYOPEN
// using the OpenInit handshake call.
func (coord *Coordinator) CreateChannelInit(
source, counterparty *TestChain,
sourceChannel, counterpartyChannel TestChannel,
connection TestConnection,
order channeltypes.Order,
) error {
// initialize channel on source
if err := source.ChannelOpenInit(sourceChannel, counterpartyChannel, order, connection.ID); err != nil {
return err
}
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
counterparty.ChainID, source.ChainID,
connection.CounterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
}
// CreateChannelOpenTry initializes a channel on the source chain with the state TRYOPEN
// using the OpenTry handshake call. // using the OpenTry handshake call.
func (coord *Coordinator) CreateChannelOpenTry( func (coord *Coordinator) ChanOpenTry(
source, counterparty *TestChain, source, counterparty *TestChain,
sourceChannel, counterpartyChannel TestChannel, sourceChannel, counterpartyChannel TestChannel,
connection TestConnection, connection *TestConnection,
order channeltypes.Order, order channeltypes.Order,
) error { ) error {
// initialize channel on source // initialize channel on source
if err := source.ChannelOpenTry(sourceChannel, counterpartyChannel, order, connection.ID); err != nil { if err := source.ChanOpenTry(counterparty, sourceChannel, counterpartyChannel, order, connection.ID); err != nil {
return err return err
} }
coord.IncrementTime() coord.IncrementTime()
// update source client on counterparty connection // update source client on counterparty connection
if err := coord.UpdateClient( if err := coord.UpdateClient(
counterparty.ChainID, source.ChainID, counterparty, source,
connection.CounterpartyClientID, clientexported.Tendermint, connection.CounterpartyClientID, clientexported.Tendermint,
); err != nil { ); err != nil {
return err return err
@ -360,24 +434,22 @@ func (coord *Coordinator) CreateChannelOpenTry(
return nil return nil
} }
// CreateChannelOpenAck initializes a channel on the source chain with the state OPEN // ChanOpenAck initializes a channel on the source chain with the state OPEN
// using the OpenAck handshake call. // using the OpenAck handshake call.
func (coord *Coordinator) CreateChannelOpenAck( func (coord *Coordinator) ChanOpenAck(
source, counterparty *TestChain, source, counterparty *TestChain,
sourceChannel, counterpartyChannel TestChannel, sourceChannel, counterpartyChannel TestChannel,
connection TestConnection,
) error { ) error {
// initialize channel on source if err := source.ChanOpenAck(counterparty, sourceChannel, counterpartyChannel); err != nil {
if err := source.ChannelOpenAck(sourceChannel, counterpartyChannel, connection.ID); err != nil {
return err return err
} }
coord.IncrementTime() coord.IncrementTime()
// update source client on counterparty connection // update source client on counterparty connection
if err := coord.UpdateClient( if err := coord.UpdateClient(
counterparty.ChainID, source.ChainID, counterparty, source,
connection.CounterpartyClientID, clientexported.Tendermint, sourceChannel.CounterpartyClientID, clientexported.Tendermint,
); err != nil { ); err != nil {
return err return err
} }
@ -385,24 +457,70 @@ func (coord *Coordinator) CreateChannelOpenAck(
return nil return nil
} }
// CreateChannelOpenConfirm initializes a channel on the source chain with the state OPEN // ChanOpenConfirm initializes a channel on the source chain with the state OPEN
// using the OpenConfirm handshake call. // using the OpenConfirm handshake call.
func (coord *Coordinator) CreateChannelOpenConfirm( func (coord *Coordinator) ChanOpenConfirm(
source, counterparty *TestChain, source, counterparty *TestChain,
sourceChannel, counterpartyChannel TestChannel, sourceChannel, counterpartyChannel TestChannel,
connection TestConnection,
) error { ) error {
// initialize channel on source if err := source.ChanOpenConfirm(counterparty, sourceChannel, counterpartyChannel); err != nil {
if err := source.ChannelOpenConfirm(sourceChannel, counterpartyChannel, connection.ID); err != nil {
return err return err
} }
coord.IncrementTime() coord.IncrementTime()
// update source client on counterparty connection // update source client on counterparty connection
if err := coord.UpdateClient( if err := coord.UpdateClient(
counterparty.ChainID, source.ChainID, counterparty, source,
connection.CounterpartyClientID, clientexported.Tendermint, sourceChannel.CounterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
}
// ChanCloseInit closes a channel on the source chain resulting in the channels state
// being set to CLOSED.
//
// NOTE: does not work with ibc-transfer module
func (coord *Coordinator) ChanCloseInit(
source, counterparty *TestChain,
channel TestChannel,
) error {
if err := source.ChanCloseInit(counterparty, channel); err != nil {
return err
}
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
counterparty, source,
channel.CounterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
}
// SetChannelClosed sets a channel state to CLOSED.
func (coord *Coordinator) SetChannelClosed(
source, counterparty *TestChain,
testChannel TestChannel,
) error {
channel := source.GetChannel(testChannel)
channel.State = channeltypes.CLOSED
source.App.IBCKeeper.ChannelKeeper.SetChannel(source.GetContext(), testChannel.PortID, testChannel.ID, channel)
coord.CommitBlock(source)
// update source client on counterparty connection
if err := coord.UpdateClient(
counterparty, source,
testChannel.CounterpartyClientID, clientexported.Tendermint,
); err != nil { ); err != nil {
return err return err
} }

View File

@ -1,16 +1,62 @@
package testing package testing
import (
"fmt"
)
// TestConnections is a testing helper struct to keep track of the connectionID, source clientID, // TestConnections is a testing helper struct to keep track of the connectionID, source clientID,
// and counterparty clientID used in creating and interacting with a connection. // and counterparty clientID used in creating and interacting with a connection.
type TestConnection struct { type TestConnection struct {
ID string ID string
ClientID string ClientID string
CounterpartyClientID string CounterpartyClientID string
Channels []TestChannel
}
// AddTestChannel appends a new TestChannel which contains references to the port and channel ID
// used for channel creation and interaction.
//
// channel ID format: connectionid-<channel-index>
// the port is set to "transfer" to be compatible with the ICS-transfer module, this should
// eventually be updated as described in the issue: https://github.com/cosmos/cosmos-sdk/issues/6509
func (conn *TestConnection) AddTestChannel() TestChannel {
channel := conn.NextTestChannel()
conn.Channels = append(conn.Channels, channel)
return channel
}
// NextTestChannel returns the next test channel to be created on this connection, but does not
// add it to the list of created channels. This function is expected to be used when the caller
// has not created the associated channel in app state, but would still like to refer to the
// non-existent channel usually to test for its non-existence.
func (conn *TestConnection) NextTestChannel() TestChannel {
portID := "transfer"
channelID := fmt.Sprintf("%s-%d", conn.ID, len(conn.Channels))
return TestChannel{
PortID: portID,
ID: channelID,
ClientID: conn.ClientID,
CounterpartyClientID: conn.CounterpartyClientID,
}
}
// FirstOrNextTestChannel returns the first test channel if it exists, otherwise it
// returns the next test channel to be created. This function is expected to be used
// when the caller does not know if the channel has or has not been created in app
// state, but would still like to refer to it to test existence or non-existence.
func (conn *TestConnection) FirstOrNextTestChannel() TestChannel {
if len(conn.Channels) > 0 {
return conn.Channels[0]
}
return conn.NextTestChannel()
} }
// TestChannel is a testing helper struct to keep track of the portID and channelID // TestChannel is a testing helper struct to keep track of the portID and channelID
// used in creating and interacting with a channel. // used in creating and interacting with a channel. The clientID and counterparty
// client ID are also tracked to cut down on querying and argument passing.
type TestChannel struct { type TestChannel struct {
PortID string PortID string
ChannelID string ID string
ClientID string
CounterpartyClientID string
} }