x/ibc-transfer: use ibc testing package (#6641)

* x/ibc-transfer: use ibc testing package

* coin denom failure

* update handler tests

* refactor first test in relay

* update onrecvtest, failing tests expected due to existing bug

* update on acknowledgement test, failing test is expected

* refactor timeout test

* revert coin validation changed, comment out tests

* minor fix

* update coordinator.go

* update coordinator.go

* add comment to fix handler test

* add link to issue in handler test

* fix test

Co-authored-by: Colin Axner <colinaxner@berkeley.edu>
Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
This commit is contained in:
Federico Kunze 2020-07-28 10:24:13 +02:00 committed by GitHub
parent 5a7e22022c
commit 2651427ab4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 358 additions and 749 deletions

View File

@ -3,309 +3,69 @@ package transfer_test
import (
"fmt"
"testing"
"time"
"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"
"github.com/cosmos/cosmos-sdk/simapp"
sdk "github.com/cosmos/cosmos-sdk/types"
banktypes "github.com/cosmos/cosmos-sdk/x/bank/types"
transfer "github.com/cosmos/cosmos-sdk/x/ibc-transfer"
ibctransfer "github.com/cosmos/cosmos-sdk/x/ibc-transfer"
"github.com/cosmos/cosmos-sdk/x/ibc-transfer/types"
connectiontypes "github.com/cosmos/cosmos-sdk/x/ibc/03-connection/types"
channeltypes "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types"
ibctmtypes "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types"
commitmenttypes "github.com/cosmos/cosmos-sdk/x/ibc/23-commitment/types"
host "github.com/cosmos/cosmos-sdk/x/ibc/24-host"
stakingtypes "github.com/cosmos/cosmos-sdk/x/staking/types"
)
// define constants used for testing
const (
testClientIDA = "testclientida"
testClientIDB = "testclientidb"
testConnection = "testconnection"
testPort1 = "bank"
testPort2 = "testportid"
testChannel1 = "firstchannel"
testChannel2 = "secondchannel"
trustingPeriod time.Duration = time.Hour * 24 * 7 * 2
ubdPeriod time.Duration = time.Hour * 24 * 7 * 3
maxClockDrift time.Duration = time.Second * 10
)
// define variables used for testing
var (
testAddr1 = sdk.AccAddress([]byte("testaddr1"))
testAddr2 = sdk.AccAddress([]byte("testaddr2"))
testCoins, _ = sdk.ParseCoins("100atom")
testPrefixedCoins1, _ = sdk.ParseCoins(fmt.Sprintf("100%satom", types.GetDenomPrefix(testPort1, testChannel1)))
testPrefixedCoins2, _ = sdk.ParseCoins(fmt.Sprintf("100%satom", types.GetDenomPrefix(testPort2, testChannel2)))
ibctesting "github.com/cosmos/cosmos-sdk/x/ibc/testing"
)
type HandlerTestSuite struct {
suite.Suite
cdc *codec.Codec
coordinator *ibctesting.Coordinator
chainA *TestChain
chainB *TestChain
// testing chains used for convenience and readability
chainA *ibctesting.TestChain
chainB *ibctesting.TestChain
}
func (suite *HandlerTestSuite) SetupTest() {
suite.chainA = NewTestChain(testClientIDA)
suite.chainB = NewTestChain(testClientIDB)
suite.cdc = suite.chainA.App.Codec()
suite.coordinator = ibctesting.NewCoordinator(suite.T(), 2)
suite.chainA = suite.coordinator.GetChain(ibctesting.GetChainID(0))
suite.chainB = suite.coordinator.GetChain(ibctesting.GetChainID(1))
}
// constructs a send from chainA to chainB on the established channel/connection
// and sends the coins back from chainB to chainA.
// FIX: this test currently passes because source is incorrectly determined
// by the ibc-transfer module, so what actually occurs is chainA and chainB
// send coins to each other, but no coins are ever sent back. This can be
// fixed by receving and acknowledeging the send on the counterparty chain.
// https://github.com/cosmos/cosmos-sdk/issues/6827
func (suite *HandlerTestSuite) TestHandleMsgTransfer() {
handler := transfer.NewHandler(suite.chainA.App.TransferKeeper)
clientA, clientB, _, _, channelA, channelB := suite.coordinator.Setup(suite.chainA, suite.chainB)
handlerA := ibctransfer.NewHandler(suite.chainA.App.TransferKeeper)
// create channel capability from ibc scoped keeper and claim with transfer scoped keeper
capName := host.ChannelCapabilityPath(testPort1, testChannel1)
cap, err := suite.chainA.App.ScopedIBCKeeper.NewCapability(suite.chainA.GetContext(), capName)
suite.Require().Nil(err, "could not create capability")
err = suite.chainA.App.ScopedTransferKeeper.ClaimCapability(suite.chainA.GetContext(), cap, capName)
suite.Require().Nil(err, "transfer module could not claim capability")
coinToSendToB := sdk.NewCoins(sdk.NewCoin(fmt.Sprintf("%s/%s/%s", channelB.PortID, channelB.ID, sdk.DefaultBondDenom), sdk.NewInt(100)))
ctx := suite.chainA.GetContext()
msg := types.NewMsgTransfer(testPort1, testChannel1, testPrefixedCoins2, testAddr1, testAddr2.String(), 110, 0)
res, err := handler(ctx, msg)
suite.Require().Error(err)
suite.Require().Nil(res, "%+v", res) // channel does not exist
// send from chainA to chainB
msg := types.NewMsgTransfer(channelA.PortID, channelA.ID, coinToSendToB, suite.chainA.SenderAccount.GetAddress(), suite.chainB.SenderAccount.GetAddress().String(), 110, 0)
// Setup channel from A to B
suite.chainA.CreateClient(suite.chainB)
suite.chainA.createConnection(testConnection, testConnection, testClientIDB, testClientIDA, connectiontypes.OPEN)
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channeltypes.OPEN, channeltypes.ORDERED, testConnection)
res, err = handler(ctx, msg)
suite.Require().Error(err)
suite.Require().Nil(res, "%+v", res) // next send sequence not found
nextSeqSend := uint64(1)
suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceSend(ctx, testPort1, testChannel1, nextSeqSend)
res, err = handler(ctx, msg)
suite.Require().Error(err)
suite.Require().Nil(res, "%+v", res) // sender has insufficient coins
_ = suite.chainA.App.BankKeeper.SetBalances(ctx, testAddr1, testCoins)
res, err = handler(ctx, msg)
res, err := handlerA(suite.chainA.GetContext(), msg)
suite.Require().NoError(err)
suite.Require().NotNil(res, "%+v", res) // successfully executed
// test when the source is false
msg = types.NewMsgTransfer(testPort1, testChannel1, testPrefixedCoins2, testAddr1, testAddr2.String(), 110, 0)
_ = suite.chainA.App.BankKeeper.SetBalances(ctx, testAddr1, testPrefixedCoins2)
err = suite.coordinator.SendMsgs(suite.chainA, suite.chainB, clientB, msg)
suite.Require().NoError(err) // message committed
res, err = handler(ctx, msg)
suite.Require().Error(err)
suite.Require().Nil(res, "%+v", res) // incorrect denom prefix
handlerB := ibctransfer.NewHandler(suite.chainB.App.TransferKeeper)
msg = types.NewMsgTransfer(testPort1, testChannel1, testPrefixedCoins1, testAddr1, testAddr2.String(), 110, 0)
suite.chainA.App.BankKeeper.SetSupply(ctx, banktypes.NewSupply(testPrefixedCoins1))
_ = suite.chainA.App.BankKeeper.SetBalances(ctx, testAddr1, testPrefixedCoins1)
coinToSendBackToA := sdk.NewCoins(sdk.NewCoin(fmt.Sprintf("%s/%s/%s", channelA.PortID, channelA.ID, sdk.DefaultBondDenom), sdk.NewInt(100)))
res, err = handler(ctx, msg)
// send from chainB back to chainA
msg = types.NewMsgTransfer(channelA.PortID, channelA.ID, coinToSendBackToA, suite.chainB.SenderAccount.GetAddress(), suite.chainA.SenderAccount.GetAddress().String(), 110, 0)
res, err = handlerB(suite.chainB.GetContext(), msg)
suite.Require().NoError(err)
suite.Require().NotNil(res, "%+v", res) // successfully executed
err = suite.coordinator.SendMsgs(suite.chainB, suite.chainA, clientA, msg)
suite.Require().NoError(err) // message committed
}
func TestHandlerTestSuite(t *testing.T) {
suite.Run(t, new(HandlerTestSuite))
}
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: chain.Header.SignedHeader.Header.Height})
}
// 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: client.Header.SignedHeader.Header.Height, 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, client.Header.SignedHeader.Header.Height, histInfo)
// Create target ctx
ctxTarget := chain.GetContext()
// create client
clientState, err := ibctmtypes.Initialize(lite.DefaultTrustLevel, trustingPeriod, ubdPeriod, maxClockDrift, client.Header, commitmenttypes.GetSDKSpecs())
if err != nil {
return err
}
_, err = chain.App.IBCKeeper.ClientKeeper.CreateClient(ctxTarget, client.ClientID, 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,
// )
}
// nolint: unused
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: client.Header.SignedHeader.Header.Height, 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, client.Header.SignedHeader.Header.Height, histInfo)
consensusState := ibctmtypes.ConsensusState{
Height: uint64(client.Header.SignedHeader.Header.Height),
Timestamp: client.Header.Time,
Root: commitmenttypes.NewMerkleRoot(commitID.Hash),
ValidatorSet: client.Vals,
}
chain.App.IBCKeeper.ClientKeeper.SetClientConsensusState(
ctxTarget, client.ClientID, uint64(client.Header.SignedHeader.Header.Height), consensusState,
)
chain.App.IBCKeeper.ClientKeeper.SetClientState(
ctxTarget, client.ClientID, ibctmtypes.NewClientState(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.GetCompatibleEncodedVersions(),
}
ctx := chain.GetContext()
chain.App.IBCKeeper.ConnectionKeeper.SetConnection(ctx, connID, connection)
return connection
}
// nolint: unused
func (chain *TestChain) createChannel(
portID, channelID, counterpartyPortID, counterpartyChannelID string,
state channeltypes.State, order channeltypes.Order, connectionID string,
) channeltypes.Channel {
counterparty := channeltypes.NewCounterparty(counterpartyPortID, counterpartyChannelID)
channel := channeltypes.NewChannel(state, order, counterparty,
[]string{connectionID}, "1.0",
)
ctx := chain.GetContext()
chain.App.IBCKeeper.ChannelKeeper.SetChannel(ctx, portID, channelID, channel)
return channel
}
func nextHeader(chain *TestChain) ibctmtypes.Header {
return ibctmtypes.CreateTestHeader(chain.Header.SignedHeader.Header.ChainID, chain.Header.SignedHeader.Header.Height+1,
chain.Header.Time.Add(time.Minute), chain.Vals, chain.Signers)
}

View File

@ -1,89 +1,30 @@
package keeper_test
import (
"fmt"
"testing"
"time"
"github.com/stretchr/testify/suite"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto"
lite "github.com/tendermint/tendermint/lite2"
tmtypes "github.com/tendermint/tendermint/types"
"github.com/cosmos/cosmos-sdk/codec"
"github.com/cosmos/cosmos-sdk/simapp"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/cosmos/cosmos-sdk/x/ibc-transfer/types"
connectiontypes "github.com/cosmos/cosmos-sdk/x/ibc/03-connection/types"
channeltypes "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types"
ibctmtypes "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types"
commitmenttypes "github.com/cosmos/cosmos-sdk/x/ibc/23-commitment/types"
host "github.com/cosmos/cosmos-sdk/x/ibc/24-host"
stakingtypes "github.com/cosmos/cosmos-sdk/x/staking/types"
)
// define constants used for testing
const (
testClientIDA = "testclientIDA"
testClientIDB = "testClientIDb"
testConnection = "testconnectionatob"
testPort1 = "bank"
testPort2 = "testportid"
testChannel1 = "firstchannel"
testChannel2 = "secondchannel"
trustingPeriod time.Duration = time.Hour * 24 * 7 * 2
ubdPeriod time.Duration = time.Hour * 24 * 7 * 3
maxClockDrift time.Duration = time.Second * 10
)
// define variables used for testing
var (
testAddr1, _ = sdk.AccAddressFromBech32("cosmos1scqhwpgsmr6vmztaa7suurfl52my6nd2kmrudl")
testAddr2, _ = sdk.AccAddressFromBech32("cosmos1scqhwpgsmr6vmztaa7suurfl52my6nd2kmrujl")
testCoins, _ = sdk.ParseCoins("100atom")
prefixCoins = sdk.NewCoins(sdk.NewCoin("bank/firstchannel/atom", sdk.NewInt(100)))
prefixCoins2 = sdk.NewCoins(sdk.NewCoin("testportid/secondchannel/atom", sdk.NewInt(100)))
ibctesting "github.com/cosmos/cosmos-sdk/x/ibc/testing"
)
type KeeperTestSuite struct {
suite.Suite
cdc *codec.Codec
coordinator *ibctesting.Coordinator
chainA *TestChain
chainB *TestChain
// testing chains used for convenience and readability
chainA *ibctesting.TestChain
chainB *ibctesting.TestChain
}
func (suite *KeeperTestSuite) SetupTest() {
suite.chainA = NewTestChain(testClientIDA)
suite.chainB = NewTestChain(testClientIDB)
// reset prefixCoins at each setup
prefixCoins = sdk.NewCoins(sdk.NewCoin("bank/firstchannel/atom", sdk.NewInt(100)))
prefixCoins2 = sdk.NewCoins(sdk.NewCoin("testportid/secondchannel/atom", sdk.NewInt(100)))
suite.cdc = suite.chainA.App.Codec()
}
// nolint: unused
func (suite *KeeperTestSuite) queryProof(key []byte) (proof commitmenttypes.MerkleProof, height int64) {
res := suite.chainA.App.Query(abci.RequestQuery{
Path: fmt.Sprintf("store/%s/key", host.StoreKey),
Data: key,
Prove: true,
})
height = res.Height
proof = commitmenttypes.MerkleProof{
Proof: res.Proof,
}
return
suite.coordinator = ibctesting.NewCoordinator(suite.T(), 2)
suite.chainA = suite.coordinator.GetChain(ibctesting.GetChainID(0))
suite.chainB = suite.coordinator.GetChain(ibctesting.GetChainID(1))
}
func (suite *KeeperTestSuite) TestGetTransferAccount() {
@ -91,195 +32,11 @@ func (suite *KeeperTestSuite) TestGetTransferAccount() {
macc := suite.chainA.App.TransferKeeper.GetTransferAccount(suite.chainA.GetContext())
suite.NotNil(macc)
suite.Equal(types.ModuleName, macc.GetName())
suite.Equal(expectedMaccAddr, macc.GetAddress())
suite.Require().NotNil(macc)
suite.Require().Equal(types.ModuleName, macc.GetName())
suite.Require().Equal(expectedMaccAddr, macc.GetAddress())
}
func TestKeeperTestSuite(t *testing.T) {
suite.Run(t, new(KeeperTestSuite))
}
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: chain.Header.SignedHeader.Header.Height})
}
// 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: client.Header.SignedHeader.Header.Height, 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, client.Header.SignedHeader.Header.Height, histInfo)
// Create target ctx
ctxTarget := chain.GetContext()
// create client
clientState, err := ibctmtypes.Initialize(lite.DefaultTrustLevel, trustingPeriod, ubdPeriod, maxClockDrift, client.Header, commitmenttypes.GetSDKSpecs())
if err != nil {
return err
}
_, err = chain.App.IBCKeeper.ClientKeeper.CreateClient(ctxTarget, client.ClientID, 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,
// )
}
// nolint: unused
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: client.Header.SignedHeader.Header.Height, 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, client.Header.SignedHeader.Header.Height, 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, client.ClientID, ibctmtypes.NewClientState(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.GetCompatibleEncodedVersions(),
}
ctx := chain.GetContext()
chain.App.IBCKeeper.ConnectionKeeper.SetConnection(ctx, connID, connection)
return connection
}
func (chain *TestChain) createChannel(
portID, channelID, counterpartyPortID, counterpartyChannelID string,
state channeltypes.State, order channeltypes.Order, connectionID string,
) channeltypes.Channel {
counterparty := channeltypes.NewCounterparty(counterpartyPortID, counterpartyChannelID)
channel := channeltypes.NewChannel(state, order, counterparty,
[]string{connectionID}, "1.0",
)
ctx := chain.GetContext()
chain.App.IBCKeeper.ChannelKeeper.SetChannel(ctx, portID, channelID, channel)
return channel
}
func nextHeader(chain *TestChain) ibctmtypes.Header {
return ibctmtypes.CreateTestHeader(chain.Header.SignedHeader.Header.ChainID, chain.Header.SignedHeader.Header.Height+1,
chain.Header.Time.Add(time.Minute), chain.Vals, chain.Signers)
}

View File

@ -1,177 +1,260 @@
package keeper_test
// TODO: these tests are blocked on multiple bugs in the existing ibc-transfer
// code
// - https://github.com/cosmos/cosmos-sdk/issues/6649
// - https://github.com/cosmos/cosmos-sdk/issues/6827
/*
import (
"fmt"
sdk "github.com/cosmos/cosmos-sdk/types"
banktypes "github.com/cosmos/cosmos-sdk/x/bank/types"
"github.com/cosmos/cosmos-sdk/x/ibc-transfer/types"
connectiontypes "github.com/cosmos/cosmos-sdk/x/ibc/03-connection/types"
clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
channeltypes "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types"
host "github.com/cosmos/cosmos-sdk/x/ibc/24-host"
ibctesting "github.com/cosmos/cosmos-sdk/x/ibc/testing"
)
// test sending from chainA to chainB using both coins that orignate on
// chainA and coins that orignate on chainB
func (suite *KeeperTestSuite) TestSendTransfer() {
testCoins2 := sdk.NewCoins(sdk.NewCoin("testportid/secondchannel/atom", sdk.NewInt(100)))
capName := host.ChannelCapabilityPath(testPort1, testChannel1)
var (
amount sdk.Coins
channelA, channelB ibctesting.TestChannel
err error
)
testCases := []struct {
msg string
amount sdk.Coins
malleate func()
isSourceChain bool
expPass bool
msg string
malleate func()
source bool
expPass bool
}{
{"successful transfer from source chain", testCoins2,
{"successful transfer from source chain",
func() {
suite.chainA.App.BankKeeper.AddCoins(suite.chainA.GetContext(), testAddr1, testCoins)
suite.chainA.CreateClient(suite.chainB)
suite.chainA.createConnection(testConnection, testConnection, testClientIDB, testClientIDA, connectiontypes.OPEN)
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channeltypes.OPEN, channeltypes.ORDERED, testConnection)
suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceSend(suite.chainA.GetContext(), testPort1, testChannel1, 1)
_, _, _, _, channelA, channelB = suite.coordinator.Setup(suite.chainA, suite.chainB)
amount = ibctesting.NewTransferCoins(channelB, sdk.DefaultBondDenom, 100)
}, true, true},
{"successful transfer from external chain", prefixCoins,
{"successful transfer with coins from counterparty chain",
func() {
suite.chainA.App.BankKeeper.SetSupply(suite.chainA.GetContext(), banktypes.NewSupply(prefixCoins))
_, err := suite.chainA.App.BankKeeper.AddCoins(suite.chainA.GetContext(), testAddr1, prefixCoins)
suite.Require().NoError(err)
suite.chainA.CreateClient(suite.chainB)
suite.chainA.createConnection(testConnection, testConnection, testClientIDB, testClientIDA, connectiontypes.OPEN)
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channeltypes.OPEN, channeltypes.ORDERED, testConnection)
suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceSend(suite.chainA.GetContext(), testPort1, testChannel1, 1)
// send coins from chainA back to chainB
_, _, _, _, channelA, channelB = suite.coordinator.Setup(suite.chainA, suite.chainB)
amount = ibctesting.NewTransferCoins(channelA, sdk.DefaultBondDenom, 100)
}, false, true},
{"source channel not found", testCoins,
func() {}, true, false},
{"next seq send not found", testCoins,
{"source channel not found",
func() {
suite.chainA.CreateClient(suite.chainB)
suite.chainA.createConnection(testConnection, testConnection, testClientIDB, testClientIDA, connectiontypes.OPEN)
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channeltypes.OPEN, channeltypes.ORDERED, testConnection)
// channel references wrong ID
_, _, _, _, channelA, channelB = suite.coordinator.Setup(suite.chainA, suite.chainB)
channelA.ID = ibctesting.InvalidID
amount = ibctesting.NewTransferCoins(channelB, sdk.DefaultBondDenom, 100)
}, true, false},
{"next seq send not found",
func() {
_, _, connA, connB := suite.coordinator.SetupClientConnections(suite.chainA, suite.chainB, clientexported.Tendermint)
channelA = connA.NextTestChannel()
channelB = connB.NextTestChannel()
// manually create channel so next seq send is never set
suite.chainA.App.IBCKeeper.ChannelKeeper.SetChannel(
suite.chainA.GetContext(),
channelA.PortID, channelA.ID,
channeltypes.NewChannel(channeltypes.OPEN, channeltypes.ORDERED, channeltypes.NewCounterparty(channelB.PortID, channelB.ID), []string{connA.ID}, ibctesting.ChannelVersion),
)
suite.chainA.CreateChannelCapability(channelA.PortID, channelA.ID)
amount = ibctesting.NewTransferCoins(channelB, sdk.DefaultBondDenom, 100)
}, true, false},
// createOutgoingPacket tests
// - source chain
{"send coins failed", testCoins,
{"send coins failed",
func() {
suite.chainA.CreateClient(suite.chainB)
suite.chainA.createConnection(testConnection, testConnection, testClientIDB, testClientIDA, connectiontypes.OPEN)
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channeltypes.OPEN, channeltypes.ORDERED, testConnection)
suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceSend(suite.chainA.GetContext(), testPort1, testChannel1, 1)
_, _, _, _, channelA, channelB = suite.coordinator.Setup(suite.chainA, suite.chainB)
amount = ibctesting.NewTransferCoins(channelB, "randomdenom", 100)
}, true, false},
// - receiving chain
{"send from module account failed", testCoins,
{"send from module account failed",
func() {
suite.chainA.CreateClient(suite.chainB)
suite.chainA.createConnection(testConnection, testConnection, testClientIDB, testClientIDA, connectiontypes.OPEN)
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channeltypes.OPEN, channeltypes.ORDERED, testConnection)
suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceSend(suite.chainA.GetContext(), testPort1, testChannel1, 1)
_, _, _, _, channelA, channelB = suite.coordinator.Setup(suite.chainA, suite.chainB)
amount = ibctesting.NewTransferCoins(channelA, "randomdenom", 100)
}, false, false},
{"channel capability not found", testCoins,
{"channel capability not found",
func() {
suite.chainA.App.BankKeeper.AddCoins(suite.chainA.GetContext(), testAddr1, testCoins)
suite.chainA.CreateClient(suite.chainB)
suite.chainA.createConnection(testConnection, testConnection, testClientIDB, testClientIDA, connectiontypes.OPEN)
suite.chainA.createChannel(testPort1, testChannel1, testPort2, testChannel2, channeltypes.OPEN, channeltypes.ORDERED, testConnection)
suite.chainA.App.IBCKeeper.ChannelKeeper.SetNextSequenceSend(suite.chainA.GetContext(), testPort1, testChannel1, 1)
_, _, _, _, channelA, channelB = suite.coordinator.Setup(suite.chainA, suite.chainB)
cap := suite.chainA.GetChannelCapability(channelA.PortID, channelA.ID)
// Release channel capability
cap, _ := suite.chainA.App.ScopedTransferKeeper.GetCapability(suite.chainA.GetContext(), capName)
suite.chainA.App.ScopedTransferKeeper.ReleaseCapability(suite.chainA.GetContext(), cap)
}, true, false},
}
for i, tc := range testCases {
for _, tc := range testCases {
tc := tc
i := i
suite.Run(fmt.Sprintf("Case %s", tc.msg), func() {
suite.SetupTest() // reset
// create channel capability from ibc scoped keeper and claim with transfer scoped keeper
cap, err := suite.chainA.App.ScopedIBCKeeper.NewCapability(suite.chainA.GetContext(), capName)
suite.Require().Nil(err, "could not create capability")
err = suite.chainA.App.ScopedTransferKeeper.ClaimCapability(suite.chainA.GetContext(), cap, capName)
suite.Require().Nil(err, "transfer module could not claim capability")
tc.malleate()
if !tc.source {
// send coins from chainB to chainA
coinFromBToA := ibctesting.NewTransferCoins(channelA, sdk.DefaultBondDenom, 100)
transferMsg := types.NewMsgTransfer(channelB.PortID, channelB.ID, coinFromBToA, suite.chainB.SenderAccount.GetAddress(), suite.chainA.SenderAccount.GetAddress().String(), 110, 0)
err = suite.coordinator.SendMsgs(suite.chainB, suite.chainA, channelA.ClientID, transferMsg)
suite.Require().NoError(err) // message committed
// receive coins on chainA from chainB
fungibleTokenPacket := types.NewFungibleTokenPacketData(coinFromBToA, suite.chainB.SenderAccount.GetAddress().String(), suite.chainA.SenderAccount.GetAddress().String())
packet := channeltypes.NewPacket(fungibleTokenPacket.GetBytes(), 1, channelB.PortID, channelB.ID, channelA.PortID, channelA.ID, 110, 0)
// get proof of packet commitment from chainB
packetKey := host.KeyPacketCommitment(packet.GetSourcePort(), packet.GetSourceChannel(), packet.GetSequence())
proof, proofHeight := suite.chainB.QueryProof(packetKey)
recvMsg := channeltypes.NewMsgRecvPacket(packet, proof, proofHeight, suite.chainA.SenderAccount.GetAddress())
err = suite.coordinator.SendMsgs(suite.chainA, suite.chainB, channelB.ClientID, recvMsg)
suite.Require().NoError(err) // message committed
}
err = suite.chainA.App.TransferKeeper.SendTransfer(
suite.chainA.GetContext(), testPort1, testChannel1, tc.amount, testAddr1, testAddr2.String(), 110, 0,
suite.chainA.GetContext(), channelA.PortID, channelA.ID, amount,
suite.chainA.SenderAccount.GetAddress(), suite.chainB.SenderAccount.GetAddress().String(), 110, 0,
)
if tc.expPass {
suite.Require().NoError(err, "valid test case %d failed: %s", i, tc.msg)
suite.Require().NoError(err)
} else {
suite.Require().Error(err, "invalid test case %d passed: %s", i, tc.msg)
suite.Require().Error(err)
}
})
}
}
// test receiving coins on chainB with coins that orignate on chainA and
// coins that orignated on chainB. Coins from source (chainA) have channelB
// as the denom prefix. The bulk of the testing occurs in the test case
// for loop since setup is intensive for all cases. The malleate function
// allows for testing invalid cases.
func (suite *KeeperTestSuite) TestOnRecvPacket() {
data := types.NewFungibleTokenPacketData(prefixCoins2, testAddr1.String(), testAddr2.String())
var (
channelA, channelB ibctesting.TestChannel
coins sdk.Coins
receiver string
)
testCases := []struct {
msg string
malleate func()
source bool
expPass bool
}{
{"success receive from source chain",
func() {}, true},
{"success receive from source chain", func() {}, true, true},
{"success receive with coins orignated on this chain", func() {}, false, true},
{"empty amount", func() {
coins = nil
}, true, false},
{"invalid receiver address", func() {
receiver = "gaia1scqhwpgsmr6vmztaa7suurfl52my6nd2kmrudl"
}, true, false},
{"no dest prefix on coin denom", func() {
coins = sdk.NewCoins(sdk.NewInt64Coin("bitcoin", 100))
}, false, false},
// onRecvPacket
// - source chain
{"no dest prefix on coin denom",
func() {
data.Amount = testCoins
}, false},
{"mint failed",
func() {
data.Amount = prefixCoins2
data.Amount[0].Amount = sdk.ZeroInt()
}, false},
// - receiving chain
{"incorrect dest prefix on coin denom",
func() {
data.Amount = prefixCoins
}, false},
{"success receive from external chain",
func() {
data.Amount = prefixCoins
escrow := types.GetEscrowAddress(testPort2, testChannel2)
_, err := suite.chainA.App.BankKeeper.AddCoins(suite.chainA.GetContext(), escrow, testCoins)
suite.Require().NoError(err)
}, true},
// - coins from source chain (chainA)
{"failure: mint zero coins", func() {
coins = ibctesting.NewTransferCoins(channelB, sdk.DefaultBondDenom, 0)
}, true, false},
// - coins being sent back to original chain (chainB)
{"tries to unescrow more tokens than allowed", func() {
coins = ibctesting.NewTransferCoins(channelA, sdk.DefaultBondDenom, 1000000)
}, false, false},
}
packet := channeltypes.NewPacket(data.GetBytes(), 1, testPort1, testChannel1, testPort2, testChannel2, 100, 0)
for i, tc := range testCases {
for _, tc := range testCases {
tc := tc
i := i
suite.Run(fmt.Sprintf("Case %s", tc.msg), func() {
suite.SetupTest() // reset
_, _, _, _, channelA, channelB = suite.coordinator.Setup(suite.chainA, suite.chainB)
receiver = suite.chainB.SenderAccount.GetAddress().String() // must be explicitly changed in malleate
seq := uint64(1)
if !tc.source {
// send coins from chainB to chainA, receive them, acknowledge them, and send back to chainB
coinFromBToA := ibctesting.NewTransferCoins(channelA, sdk.DefaultBondDenom, 100)
transferMsg := types.NewMsgTransfer(channelB.PortID, channelB.ID, coinFromBToA, suite.chainB.SenderAccount.GetAddress(), suite.chainA.SenderAccount.GetAddress().String(), 110, 0)
err := suite.coordinator.SendMsgs(suite.chainB, suite.chainA, channelA.ClientID, transferMsg)
suite.Require().NoError(err) // message committed
// receive coins on chainA from chainB
fungibleTokenPacket := types.NewFungibleTokenPacketData(coinFromBToA, suite.chainB.SenderAccount.GetAddress().String(), suite.chainA.SenderAccount.GetAddress().String())
packet := channeltypes.NewPacket(fungibleTokenPacket.GetBytes(), 1, channelB.PortID, channelB.ID, channelA.PortID, channelA.ID, 110, 0)
// get proof of packet commitment from chainB
packetKey := host.KeyPacketCommitment(packet.GetSourcePort(), packet.GetSourceChannel(), packet.GetSequence())
proof, proofHeight := suite.chainB.QueryProof(packetKey)
recvMsg := channeltypes.NewMsgRecvPacket(packet, proof, proofHeight, suite.chainA.SenderAccount.GetAddress())
err = suite.coordinator.SendMsgs(suite.chainA, suite.chainB, channelB.ClientID, recvMsg)
suite.Require().NoError(err) // message committed
// get proof of acknowledgement on chainA
packetKey = host.KeyPacketAcknowledgement(packet.GetDestPort(), packet.GetDestChannel(), packet.GetSequence())
proof, proofHeight = suite.chainA.QueryProof(packetKey)
// acknowledge on chainB the receive that happened on chainA
ack := types.FungibleTokenPacketAcknowledgement{true, ""}
ackMsg := channeltypes.NewMsgAcknowledgement(packet, ack.GetBytes(), proof, proofHeight, suite.chainB.SenderAccount.GetAddress())
err = suite.coordinator.SendMsgs(suite.chainB, suite.chainA, channelA.ClientID, ackMsg)
suite.Require().NoError(err) // message committed
seq++
// NOTE: coins must be explicitly changed in malleate to test invalid cases
coins = ibctesting.NewTransferCoins(channelA, sdk.DefaultBondDenom, 100)
} else {
coins = ibctesting.NewTransferCoins(channelB, sdk.DefaultBondDenom, 100)
}
// send coins from chainA to chainB
transferMsg := types.NewMsgTransfer(channelA.PortID, channelA.ID, coins, suite.chainA.SenderAccount.GetAddress(), receiver, 110, 0)
err := suite.coordinator.SendMsgs(suite.chainA, suite.chainB, channelB.ClientID, transferMsg)
suite.Require().NoError(err) // message committed
tc.malleate()
err := suite.chainA.App.TransferKeeper.OnRecvPacket(suite.chainA.GetContext(), packet, data)
data := types.NewFungibleTokenPacketData(coins, suite.chainA.SenderAccount.GetAddress().String(), receiver)
packet := channeltypes.NewPacket(data.GetBytes(), seq, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, 100, 0)
err = suite.chainB.App.TransferKeeper.OnRecvPacket(suite.chainB.GetContext(), packet, data)
if tc.expPass {
suite.Require().NoError(err, "valid test case %d failed: %s", i, tc.msg)
suite.Require().NoError(err)
} else {
suite.Require().Error(err, "invalid test case %d passed: %s", i, tc.msg)
suite.Require().Error(err)
}
})
}
}
// TestOnAcknowledgementPacket tests that successful acknowledgement is a no-op
// and failure acknowledment leads to refund
// and failure acknowledment leads to refund when attempting to send from chainA
// to chainB.
func (suite *KeeperTestSuite) TestOnAcknowledgementPacket() {
data := types.NewFungibleTokenPacketData(prefixCoins2, testAddr1.String(), testAddr2.String())
var (
successAck = types.FungibleTokenPacketAcknowledgement{
Success: true,
}
failedAck = types.FungibleTokenPacketAcknowledgement{
Success: false,
Error: "failed packet transfer",
}
successAck := types.FungibleTokenPacketAcknowledgement{
Success: true,
}
failedAck := types.FungibleTokenPacketAcknowledgement{
Success: false,
Error: "failed packet transfer",
}
channelA, channelB ibctesting.TestChannel
coins sdk.Coins
)
testCases := []struct {
msg string
@ -180,59 +263,69 @@ func (suite *KeeperTestSuite) TestOnAcknowledgementPacket() {
source bool
success bool // success of ack
}{
{"success ack causes no-op", successAck,
func() {}, true, true},
{"success ack causes no-op", successAck, func() {
coins = ibctesting.NewTransferCoins(channelB, sdk.DefaultBondDenom, 100)
}, true, true},
{"successful refund from source chain", failedAck,
func() {
escrow := types.GetEscrowAddress(testPort1, testChannel1)
_, err := suite.chainA.App.BankKeeper.AddCoins(suite.chainA.GetContext(), escrow, sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(100))))
escrow := types.GetEscrowAddress(channelA.PortID, channelA.ID)
_, err := suite.chainA.App.BankKeeper.AddCoins(suite.chainA.GetContext(), escrow, sdk.NewCoins(sdk.NewCoin(sdk.DefaultBondDenom, sdk.NewInt(100))))
suite.Require().NoError(err)
coins = ibctesting.NewTransferCoins(channelB, sdk.DefaultBondDenom, 100)
}, true, false},
{"successful refund from external chain", failedAck,
func() {
data.Amount = prefixCoins
coins = ibctesting.NewTransferCoins(channelA, sdk.DefaultBondDenom, 100)
}, false, false},
}
packet := channeltypes.NewPacket(data.GetBytes(), 1, testPort1, testChannel1, testPort2, testChannel2, 100, 0)
for i, tc := range testCases {
for _, tc := range testCases {
tc := tc
i := i
suite.Run(fmt.Sprintf("Case %s", tc.msg), func() {
suite.SetupTest() // reset
_, _, _, _, channelA, channelB = suite.coordinator.Setup(suite.chainA, suite.chainB)
tc.malleate()
data := types.NewFungibleTokenPacketData(coins, suite.chainA.SenderAccount.GetAddress().String(), suite.chainB.SenderAccount.GetAddress().String())
packet := channeltypes.NewPacket(data.GetBytes(), 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, 100, 0)
var denom string
if tc.source {
prefix := types.GetDenomPrefix(packet.GetDestPort(), packet.GetDestChannel())
denom = prefixCoins2[0].Denom[len(prefix):]
// peel off ibc denom if returning back to original chain
denom = sdk.DefaultBondDenom
} else {
denom = data.Amount[0].Denom
denom = coins[0].Denom
}
preCoin := suite.chainA.App.BankKeeper.GetBalance(suite.chainA.GetContext(), testAddr1, denom)
preCoin := suite.chainA.App.BankKeeper.GetBalance(suite.chainA.GetContext(), suite.chainA.SenderAccount.GetAddress(), denom)
err := suite.chainA.App.TransferKeeper.OnAcknowledgementPacket(suite.chainA.GetContext(), packet, data, tc.ack)
suite.Require().NoError(err, "valid test case %d failed: %s", i, tc.msg)
suite.Require().NoError(err)
postCoin := suite.chainA.App.BankKeeper.GetBalance(suite.chainA.GetContext(), testAddr1, denom)
postCoin := suite.chainA.App.BankKeeper.GetBalance(suite.chainA.GetContext(), suite.chainA.SenderAccount.GetAddress(), denom)
deltaAmount := postCoin.Amount.Sub(preCoin.Amount)
if tc.success {
suite.Require().Equal(sdk.ZeroInt(), deltaAmount, "successful ack changed balance")
suite.Require().Equal(int64(0), deltaAmount.Int64(), "successful ack changed balance")
} else {
suite.Require().Equal(prefixCoins2[0].Amount, deltaAmount, "failed ack did not trigger refund")
suite.Require().Equal(coins[0].Amount, deltaAmount, "failed ack did not trigger refund")
}
})
}
}
// TestOnTimeoutPacket test private refundPacket function since it is a simple wrapper over it
// TestOnTimeoutPacket test private refundPacket function since it is a simple
// wrapper over it. The actual timeout does not matter since IBC core logic
// is not being tested. The test is timing out a send from chainA to chainB
// so the refunds are occurring on chainA.
func (suite *KeeperTestSuite) TestOnTimeoutPacket() {
data := types.NewFungibleTokenPacketData(prefixCoins2, testAddr1.String(), testAddr2.String())
testCoins2 := sdk.NewCoins(sdk.NewCoin("bank/firstchannel/atom", sdk.NewInt(100)))
var (
channelA, channelB ibctesting.TestChannel
coins sdk.Coins
)
testCases := []struct {
msg string
@ -242,59 +335,63 @@ func (suite *KeeperTestSuite) TestOnTimeoutPacket() {
}{
{"successful timeout from source chain",
func() {
escrow := types.GetEscrowAddress(testPort1, testChannel1)
_, err := suite.chainA.App.BankKeeper.AddCoins(suite.chainA.GetContext(), escrow, sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(100))))
escrow := types.GetEscrowAddress(channelA.PortID, channelA.ID)
_, err := suite.chainA.App.BankKeeper.AddCoins(suite.chainA.GetContext(), escrow, sdk.NewCoins(sdk.NewCoin(sdk.DefaultBondDenom, sdk.NewInt(100))))
suite.Require().NoError(err)
coins = ibctesting.NewTransferCoins(channelB, sdk.DefaultBondDenom, 100)
}, true, true},
{"successful timeout from external chain",
func() {
data.Amount = testCoins2
coins = ibctesting.NewTransferCoins(channelA, sdk.DefaultBondDenom, 100)
}, false, true},
{"no source prefix on coin denom",
func() {
data.Amount = prefixCoins2
coins = sdk.NewCoins(sdk.NewCoin("bitcoin", sdk.NewInt(100)))
}, false, false},
{"unescrow failed",
func() {
coins = ibctesting.NewTransferCoins(channelB, sdk.DefaultBondDenom, 100)
}, true, false},
{"mint failed",
func() {
data.Amount[0].Denom = prefixCoins2[0].Denom
data.Amount[0].Amount = sdk.ZeroInt()
coins = ibctesting.NewTransferCoins(channelA, sdk.DefaultBondDenom, 0)
}, true, false},
}
packet := channeltypes.NewPacket(data.GetBytes(), 1, testPort1, testChannel1, testPort2, testChannel2, 100, 0)
for i, tc := range testCases {
for _, tc := range testCases {
tc := tc
i := i
suite.Run(fmt.Sprintf("Case %s", tc.msg), func() {
suite.SetupTest() // reset
_, _, _, _, channelA, channelB = suite.coordinator.Setup(suite.chainA, suite.chainB)
tc.malleate()
data := types.NewFungibleTokenPacketData(coins, suite.chainA.SenderAccount.GetAddress().String(), suite.chainB.SenderAccount.GetAddress().String())
packet := channeltypes.NewPacket(data.GetBytes(), 1, channelA.PortID, channelA.ID, channelB.PortID, channelB.ID, 100, 0)
var denom string
if tc.source {
prefix := types.GetDenomPrefix(packet.GetDestPort(), packet.GetDestChannel())
denom = prefixCoins2[0].Denom[len(prefix):]
denom = sdk.DefaultBondDenom
} else {
denom = data.Amount[0].Denom
denom = coins[0].Denom
}
preCoin := suite.chainA.App.BankKeeper.GetBalance(suite.chainA.GetContext(), testAddr1, denom)
preCoin := suite.chainA.App.BankKeeper.GetBalance(suite.chainA.GetContext(), suite.chainA.SenderAccount.GetAddress(), denom)
err := suite.chainA.App.TransferKeeper.OnTimeoutPacket(suite.chainA.GetContext(), packet, data)
postCoin := suite.chainA.App.BankKeeper.GetBalance(suite.chainA.GetContext(), testAddr1, denom)
postCoin := suite.chainA.App.BankKeeper.GetBalance(suite.chainA.GetContext(), suite.chainA.SenderAccount.GetAddress(), denom)
deltaAmount := postCoin.Amount.Sub(preCoin.Amount)
if tc.expPass {
suite.Require().NoError(err, "valid test case %d failed: %s", i, tc.msg)
suite.Require().Equal(prefixCoins2[0].Amount.Int64(), deltaAmount.Int64(), "successful timeout did not trigger refund")
suite.Require().NoError(err)
suite.Require().Equal(coins[0].Amount.Int64(), deltaAmount.Int64(), "successful timeout did not trigger refund")
} else {
suite.Require().Error(err, "invalid test case %d passed: %s", i, tc.msg)
suite.Require().Error(err)
}
})
}
}
*/

View File

@ -45,3 +45,12 @@ func GetEscrowAddress(portID, channelID string) sdk.AccAddress {
func GetDenomPrefix(portID, channelID string) string {
return fmt.Sprintf("%s/%s/", portID, channelID)
}
// GetPrefixedCoins creates a copy of the given coins with the denom updated with the prefix.
func GetPrefixedCoins(portID, channelID string, coins ...sdk.Coin) sdk.Coins {
prefixedCoins := make(sdk.Coins, len(coins))
for i := range coins {
prefixedCoins[i] = sdk.NewCoin(GetDenomPrefix(portID, channelID)+coins[i].Denom, coins[i].Amount)
}
return prefixedCoins
}

View File

@ -20,6 +20,7 @@ func RegisterCodec(cdc *codec.Codec) {
cdc.RegisterConcrete(&MsgChannelOpenConfirm{}, "ibc/channel/MsgChannelOpenConfirm", nil)
cdc.RegisterConcrete(&MsgChannelCloseInit{}, "ibc/channel/MsgChannelCloseInit", nil)
cdc.RegisterConcrete(&MsgChannelCloseConfirm{}, "ibc/channel/MsgChannelCloseConfirm", nil)
cdc.RegisterConcrete(&MsgRecvPacket{}, "ibc/channel/MsgRecvPacket", nil)
cdc.RegisterConcrete(&MsgAcknowledgement{}, "ibc/channel/MsgAcknowledgement", nil)
cdc.RegisterConcrete(&MsgTimeout{}, "ibc/channel/MsgTimeout", nil)
}

View File

@ -416,7 +416,7 @@ func (suite *MsgTestSuite) TestMsgRecvPacketGetSignBytes() {
res := msg.GetSignBytes()
expected := fmt.Sprintf(
`{"packet":{"data":%s,"destination_channel":"testcpchannel","destination_port":"testcpport","sequence":"1","source_channel":"testchannel","source_port":"testportid","timeout_height":"100","timeout_timestamp":"100"},"proof":"Co0BCi4KCmljczIzOmlhdmwSA0tFWRobChkKA0tFWRIFVkFMVUUaCwgBGAEgASoDAAICClsKDGljczIzOnNpbXBsZRIMaWF2bFN0b3JlS2V5Gj0KOwoMaWF2bFN0b3JlS2V5EiAcIiDXSHQRSvh/Wa07MYpTK0B4XtbaXtzxBED76xk0WhoJCAEYASABKgEA","proof_height":"1","signer":"cosmos1w3jhxarpv3j8yvg4ufs4x"}`,
`{"type":"ibc/channel/MsgRecvPacket","value":{"packet":{"data":%s,"destination_channel":"testcpchannel","destination_port":"testcpport","sequence":"1","source_channel":"testchannel","source_port":"testportid","timeout_height":"100","timeout_timestamp":"100"},"proof":"Co0BCi4KCmljczIzOmlhdmwSA0tFWRobChkKA0tFWRIFVkFMVUUaCwgBGAEgASoDAAICClsKDGljczIzOnNpbXBsZRIMaWF2bFN0b3JlS2V5Gj0KOwoMaWF2bFN0b3JlS2V5EiAcIiDXSHQRSvh/Wa07MYpTK0B4XtbaXtzxBED76xk0WhoJCAEYASABKgEA","proof_height":"1","signer":"cosmos1w3jhxarpv3j8yvg4ufs4x"}}`,
string(msg.GetDataSignBytes()),
)
suite.Equal(expected, string(res))

View File

@ -36,8 +36,8 @@ import (
"github.com/cosmos/cosmos-sdk/x/ibc/types"
)
// Default params constants used to create a TM client
const (
// Default params used to create a TM client
TrustingPeriod time.Duration = time.Hour * 24 * 7 * 2
UnbondingPeriod time.Duration = time.Hour * 24 * 7 * 3
MaxClockDrift time.Duration = time.Second * 10
@ -50,6 +50,7 @@ const (
maxInt = int(^uint(0) >> 1)
)
// Default params variables used to create a TM client
var (
DefaultTrustLevel tmmath.Fraction = lite.DefaultTrustLevel
TestHash = []byte("TESTING HASH")
@ -207,15 +208,15 @@ func (chain *TestChain) NextBlock() {
}
// SendMsg delivers a transaction through the application. It updates the senders sequence
// SendMsgs delivers a transaction through the application. It updates the senders sequence
// number and updates the TestChain's headers.
func (chain *TestChain) SendMsg(msg sdk.Msg) error {
func (chain *TestChain) SendMsgs(msgs ...sdk.Msg) error {
_, _, err := simapp.SignCheckDeliver(
chain.t,
chain.TxConfig,
chain.App.BaseApp,
chain.GetContext().BlockHeader(),
[]sdk.Msg{msg},
msgs,
[]uint64{chain.SenderAccount.GetAccountNumber()},
[]uint64{chain.SenderAccount.GetSequence()},
true, true, chain.senderPrivKey,
@ -304,7 +305,7 @@ func (chain *TestChain) ConstructNextTestConnection(clientID, counterpartyClient
}
}
// FirstTestConnection returns the first test connection for a given clientID.
// GetFirstTestConnection returns the first test connection for a given clientID.
// The connection may or may not exist in the chain state.
func (chain *TestChain) GetFirstTestConnection(clientID, counterpartyClientID string) *TestConnection {
if len(chain.Connections) > 0 {
@ -324,7 +325,7 @@ func (chain *TestChain) CreateTMClient(counterparty *TestChain, clientID string)
commitmenttypes.GetSDKSpecs(), chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// UpdateTMClient will construct and execute a 07-tendermint MsgUpdateClient. The counterparty
@ -335,7 +336,7 @@ func (chain *TestChain) UpdateTMClient(counterparty *TestChain, clientID string)
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// CreateTMClientHeader creates a TM header to update the TM client.
@ -377,7 +378,7 @@ func (chain *TestChain) CreateTMClientHeader() ibctmtypes.Header {
}
}
// Copied unimported test functions from tmtypes to use them here
// MakeBlockID 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,
@ -399,7 +400,7 @@ func (chain *TestChain) ConnectionOpenInit(
counterparty.GetPrefix(),
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// ConnectionOpenTry will construct and execute a MsgConnectionOpenTry.
@ -420,7 +421,7 @@ func (chain *TestChain) ConnectionOpenTry(
proofHeight, consensusHeight,
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// ConnectionOpenAck will construct and execute a MsgConnectionOpenAck.
@ -440,7 +441,7 @@ func (chain *TestChain) ConnectionOpenAck(
ConnectionVersion,
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// ConnectionOpenConfirm will construct and execute a MsgConnectionOpenConfirm.
@ -456,7 +457,7 @@ func (chain *TestChain) ConnectionOpenConfirm(
proof, height,
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// CreatePortCapability binds and claims a capability for the given portID if it does not
@ -524,7 +525,7 @@ func (chain *TestChain) ChanOpenInit(
counterparty.PortID, counterparty.ID,
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// ChanOpenTry will construct and execute a MsgChannelOpenTry.
@ -544,7 +545,7 @@ func (chain *TestChain) ChanOpenTry(
proof, height,
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// ChanOpenAck will construct and execute a MsgChannelOpenAck.
@ -560,7 +561,7 @@ func (chain *TestChain) ChanOpenAck(
proof, height,
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// ChanOpenConfirm will construct and execute a MsgChannelOpenConfirm.
@ -575,7 +576,7 @@ func (chain *TestChain) ChanOpenConfirm(
proof, height,
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// ChanCloseInit will construct and execute a MsgChannelCloseInit.
@ -589,7 +590,7 @@ func (chain *TestChain) ChanCloseInit(
channel.PortID, channel.ID,
chain.SenderAccount.GetAddress(),
)
return chain.SendMsg(msg)
return chain.SendMsgs(msg)
}
// GetPacketData returns a ibc-transfer marshalled packet to be used for

View File

@ -9,6 +9,7 @@ import (
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
sdk "github.com/cosmos/cosmos-sdk/types"
clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
channelexported "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/exported"
channeltypes "github.com/cosmos/cosmos-sdk/x/ibc/04-channel/types"
@ -199,14 +200,10 @@ func (coord *Coordinator) SendPacket(
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
return coord.UpdateClient(
counterparty, source,
counterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
)
}
// PacketExecuted receives a packet through the channel keeper on the source chain and updates the
@ -222,14 +219,10 @@ func (coord *Coordinator) PacketExecuted(
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
return coord.UpdateClient(
counterparty, source,
counterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
)
}
// AcknowledgementExecuted deletes the packet commitment with the given
@ -245,14 +238,10 @@ func (coord *Coordinator) AcknowledgementExecuted(
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
return 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
@ -266,6 +255,22 @@ func (coord *Coordinator) IncrementTime() {
}
}
// SendMsgs delivers the provided messages to the chain. The counterparty
// client is updated with the new source consensus state.
func (coord *Coordinator) SendMsgs(source, counterparty *TestChain, counterpartyClientID string, msgs ...sdk.Msg) error {
if err := source.SendMsgs(msgs...); err != nil {
return err
}
coord.IncrementTime()
// update source client on counterparty connection
return coord.UpdateClient(
counterparty, source,
counterpartyClientID, clientexported.Tendermint,
)
}
// GetChain returns the TestChain using the given chainID and returns an error if it does
// not exist.
func (coord *Coordinator) GetChain(chainID string) *TestChain {
@ -342,14 +347,10 @@ func (coord *Coordinator) ConnOpenTry(
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
return coord.UpdateClient(
counterparty, source,
counterpartyConnection.ClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
)
}
// ConnOpenAck initializes a connection on the source chain with the state OPEN
@ -365,14 +366,10 @@ func (coord *Coordinator) ConnOpenAck(
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
return coord.UpdateClient(
counterparty, source,
counterpartyConnection.ClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
)
}
// ConnOpenConfirm initializes a connection on the source chain with the state OPEN
@ -387,14 +384,10 @@ func (coord *Coordinator) ConnOpenConfirm(
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
return coord.UpdateClient(
counterparty, source,
counterpartyConnection.ClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
)
}
// ChanOpenInit initializes a channel on the source chain with the state INIT
@ -447,14 +440,10 @@ func (coord *Coordinator) ChanOpenTry(
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
return coord.UpdateClient(
counterparty, source,
connection.CounterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
)
}
// ChanOpenAck initializes a channel on the source chain with the state OPEN
@ -470,14 +459,10 @@ func (coord *Coordinator) ChanOpenAck(
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
return coord.UpdateClient(
counterparty, source,
sourceChannel.CounterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
)
}
// ChanOpenConfirm initializes a channel on the source chain with the state OPEN
@ -493,14 +478,10 @@ func (coord *Coordinator) ChanOpenConfirm(
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
return coord.UpdateClient(
counterparty, source,
sourceChannel.CounterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
)
}
// ChanCloseInit closes a channel on the source chain resulting in the channels state
@ -518,14 +499,10 @@ func (coord *Coordinator) ChanCloseInit(
coord.IncrementTime()
// update source client on counterparty connection
if err := coord.UpdateClient(
return coord.UpdateClient(
counterparty, source,
channel.CounterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
)
}
// SetChannelClosed sets a channel state to CLOSED.
@ -541,12 +518,8 @@ func (coord *Coordinator) SetChannelClosed(
coord.CommitBlock(source)
// update source client on counterparty connection
if err := coord.UpdateClient(
return coord.UpdateClient(
counterparty, source,
testChannel.CounterpartyClientID, clientexported.Tendermint,
); err != nil {
return err
}
return nil
)
}

11
x/ibc/testing/transfer.go Normal file
View File

@ -0,0 +1,11 @@
package testing
import (
"fmt"
sdk "github.com/cosmos/cosmos-sdk/types"
)
func NewTransferCoins(dst TestChannel, denom string, amount int64) sdk.Coins {
return sdk.NewCoins(sdk.NewCoin(fmt.Sprintf("%s/%s/%s", dst.PortID, dst.ID, denom), sdk.NewInt(amount)))
}

View File

@ -4,7 +4,7 @@ import (
"fmt"
)
// TestConnections is a testing helper struct to keep track of the connectionID, source clientID,
// TestConnection is a testing helper struct to keep track of the connectionID, source clientID,
// and counterparty clientID used in creating and interacting with a connection.
type TestConnection struct {
ID string
@ -31,7 +31,7 @@ func (conn *TestConnection) AddTestChannel() TestChannel {
// 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))
channelID := fmt.Sprintf("%s%d", conn.ID, len(conn.Channels))
return TestChannel{
PortID: portID,
ID: channelID,