Modular IBC Client (#7028)

* start modular client work

* fix panic

* reuse keeper marshal methods

* readd TODO

* add nil checks for misbehaviour

* address reviews

* address rest of reviews and fix builds

* fixed tests

* address rest of reviews

* fix expired blocks bug

* fix expired bug
This commit is contained in:
Aditya 2020-08-14 04:47:13 -04:00 committed by GitHub
parent 4c762db64e
commit 3735b182bc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 400 additions and 373 deletions

View File

@ -35,7 +35,7 @@ var DefaultConsensusParams = &abci.ConsensusParams{
},
Evidence: &abci.EvidenceParams{
MaxAgeNumBlocks: 302400,
MaxAgeDuration: 1814400,
MaxAgeDuration: 504 * time.Hour, // 3 weeks is the max duration
},
Validator: &abci.ValidatorParams{
PubKeyTypes: []string{

View File

@ -3,29 +3,24 @@ package client_test
import (
"testing"
ibctesting "github.com/cosmos/cosmos-sdk/x/ibc/testing"
"github.com/stretchr/testify/suite"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/cosmos/cosmos-sdk/codec"
"github.com/cosmos/cosmos-sdk/simapp"
sdk "github.com/cosmos/cosmos-sdk/types"
)
type ClientTestSuite struct {
suite.Suite
cdc *codec.LegacyAmino
ctx sdk.Context
app *simapp.SimApp
coordinator *ibctesting.Coordinator
chainA *ibctesting.TestChain
chainB *ibctesting.TestChain
}
func (suite *ClientTestSuite) SetupTest() {
isCheckTx := false
suite.app = simapp.Setup(isCheckTx)
suite.cdc = suite.app.LegacyAmino()
suite.ctx = suite.app.BaseApp.NewContext(isCheckTx, abci.Header{Height: 0, ChainID: "localhost_chain"})
suite.coordinator = ibctesting.NewCoordinator(suite.T(), 2)
suite.chainA = suite.coordinator.GetChain(ibctesting.GetChainID(0))
suite.chainB = suite.coordinator.GetChain(ibctesting.GetChainID(1))
}
func TestClientTestSuite(t *testing.T) {

View File

@ -25,6 +25,11 @@ type ClientState interface {
Validate() error
GetProofSpecs() []*ics23.ProofSpec
// Update and Misbehaviour functions
CheckHeaderAndUpdateState(sdk.Context, codec.BinaryMarshaler, sdk.KVStore, Header) (ClientState, ConsensusState, error)
CheckMisbehaviourAndUpdateState(sdk.Context, codec.BinaryMarshaler, sdk.KVStore, Misbehaviour) (ClientState, error)
// State verification functions
VerifyClientConsensusState(
@ -138,6 +143,7 @@ type MsgCreateClient interface {
GetClientID() string
GetClientType() string
GetConsensusState() ConsensusState
InitializeClientState() ClientState
}
// MsgUpdateClient defines the msg interface that the

View File

@ -10,34 +10,28 @@ import (
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/keeper"
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/types"
ibctmtypes "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types"
localhosttypes "github.com/cosmos/cosmos-sdk/x/ibc/09-localhost/types"
)
// HandleMsgCreateClient defines the sdk.Handler for MsgCreateClient
func HandleMsgCreateClient(ctx sdk.Context, k keeper.Keeper, msg exported.MsgCreateClient) (*sdk.Result, error) {
clientType := exported.ClientTypeFromString(msg.GetClientType())
var (
clientState exported.ClientState
consensusHeight uint64
clientState exported.ClientState
)
switch clientType {
case exported.Tendermint:
tmMsg, ok := msg.(*ibctmtypes.MsgCreateClient)
if !ok {
return nil, sdkerrors.Wrapf(types.ErrInvalidClientType, "got %T, expected %T", msg, &ibctmtypes.MsgCreateClient{})
}
clientState = ibctmtypes.InitializeFromMsg(tmMsg)
consensusHeight = msg.GetConsensusState().GetHeight()
case exported.Localhost:
// msg client id is always "localhost"
switch msg.(type) {
// localhost is a special case that must initialize client state
// from context and not from msg
case *localhosttypes.MsgCreateClient:
clientState = localhosttypes.NewClientState(ctx.ChainID(), ctx.BlockHeight())
// Localhost consensus height is chain's blockheight
consensusHeight = uint64(ctx.BlockHeight())
default:
return nil, sdkerrors.Wrapf(types.ErrInvalidClientType, "unsupported client type (%s)", msg.GetClientType())
clientState = msg.InitializeClientState()
if consState := msg.GetConsensusState(); consState != nil {
consensusHeight = consState.GetHeight()
}
}
_, err := k.CreateClient(

View File

@ -0,0 +1,53 @@
package client_test
import (
client "github.com/cosmos/cosmos-sdk/x/ibc/02-client"
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
localhosttypes "github.com/cosmos/cosmos-sdk/x/ibc/09-localhost/types"
)
func (suite *ClientTestSuite) TestHandleCreateClientLocalHost() {
cases := []struct {
name string
clientID string
msg exported.MsgCreateClient
expPass bool
}{
{
"localhost",
exported.ClientTypeLocalHost,
&localhosttypes.MsgCreateClient{suite.chainA.SenderAccount.GetAddress()},
true,
},
{
"tendermint client",
"gaiamainnet",
suite.chainA.ConstructMsgCreateClient(suite.chainB, "gaiamainnet"),
true,
},
{
"client already exists",
exported.ClientTypeLocalHost,
&localhosttypes.MsgCreateClient{suite.chainA.SenderAccount.GetAddress()},
false,
},
}
for _, tc := range cases {
_, err := client.HandleMsgCreateClient(
suite.chainA.GetContext(),
suite.chainA.App.IBCKeeper.ClientKeeper,
tc.msg,
)
if tc.expPass {
suite.Require().NoError(err, "expected test case %s to pass, got error %v", tc.name, err)
clientState, ok := suite.chainA.App.IBCKeeper.ClientKeeper.GetClientState(suite.chainA.GetContext(), tc.clientID)
suite.Require().True(ok, "could not retrieve clientState")
suite.Require().NotNil(clientState, "clientstate is nil")
} else {
suite.Require().Error(err, "invalid test case %s passed", tc.name)
}
}
}

View File

@ -7,9 +7,6 @@ import (
sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/types"
tendermint "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint"
ibctmtypes "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types"
localhosttypes "github.com/cosmos/cosmos-sdk/x/ibc/09-localhost/types"
)
// CreateClient creates a new client state and populates it with a given consensus
@ -69,34 +66,7 @@ func (k Keeper) UpdateClient(ctx sdk.Context, clientID string, header exported.H
err error
)
switch clientType {
case exported.Tendermint:
tmHeader, ok := header.(ibctmtypes.Header)
if !ok {
err = sdkerrors.Wrapf(types.ErrInvalidHeader, "expected tendermint header: %T, got header type: %T", ibctmtypes.Header{}, header)
break
}
// Get the consensus state at the trusted height of header
trustedConsState, found := k.GetClientConsensusState(ctx, clientID, tmHeader.TrustedHeight)
if !found {
return nil, sdkerrors.Wrapf(types.ErrConsensusStateNotFound, "could not find consensus state for trusted header height: %d to verify header against for clientID: %s", tmHeader.TrustedHeight, clientID)
}
clientState, consensusState, err = tendermint.CheckValidityAndUpdateState(
clientState, trustedConsState, header, ctx.BlockTime(),
)
if err != nil {
err = sdkerrors.Wrapf(err, "failed to update client using trusted consensus state height %d", trustedConsState.GetHeight())
}
case exported.Localhost:
// override client state and update the block height
clientState = localhosttypes.NewClientState(
ctx.ChainID(), // use the chain ID from context since the client is from the running chain (i.e self).
ctx.BlockHeight(),
)
consensusHeight = uint64(ctx.BlockHeight())
default:
err = types.ErrInvalidClientType
}
clientState, consensusState, err = clientState.CheckHeaderAndUpdateState(ctx, k.cdc, k.ClientStore(ctx, clientID), header)
if err != nil {
return nil, sdkerrors.Wrapf(err, "cannot update client with ID %s", clientID)
@ -136,30 +106,7 @@ func (k Keeper) CheckMisbehaviourAndUpdateState(ctx sdk.Context, misbehaviour ex
return sdkerrors.Wrap(err, "IBC misbehaviour failed validate basic")
}
var err error
switch e := misbehaviour.(type) {
case ibctmtypes.Evidence:
// Get consensus states at TrustedHeight for each header
consensusState1, found := k.GetClientConsensusState(ctx, misbehaviour.GetClientID(), e.Header1.TrustedHeight)
if !found {
return sdkerrors.Wrapf(types.ErrConsensusStateNotFound, "could not find ConsensusState for clientID %s at TrustedHeight (%d) for first header",
misbehaviour.GetClientID(), e.Header1.TrustedHeight)
}
consensusState2, found := k.GetClientConsensusState(ctx, misbehaviour.GetClientID(), e.Header2.TrustedHeight)
if !found {
return sdkerrors.Wrapf(types.ErrConsensusStateNotFound, "could not find ConsensusState for clientID %s at TrustedHeight (%d) for second header",
misbehaviour.GetClientID(), e.Header2.TrustedHeight)
}
// TODO: Retrieve consensusparams from client and not context
// Issue #6516: https://github.com/cosmos/cosmos-sdk/issues/6516
clientState, err = tendermint.CheckMisbehaviourAndUpdateState(
clientState, consensusState1, consensusState2, misbehaviour, ctx.BlockTime(), ctx.ConsensusParams(),
)
default:
err = sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "unrecognized IBC client evidence type: %T", e)
}
clientState, err := clientState.CheckMisbehaviourAndUpdateState(ctx, k.cdc, k.ClientStore(ctx, misbehaviour.GetClientID()), misbehaviour)
if err != nil {
return err

View File

@ -1,90 +1,30 @@
package keeper
import (
"fmt"
"github.com/cosmos/cosmos-sdk/codec"
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/types"
)
// MustUnmarshalClientState attempts to decode and return an ClientState object from
// raw encoded bytes. It panics on error.
func (k Keeper) MustUnmarshalClientState(bz []byte) exported.ClientState {
clientState, err := k.UnmarshalClientState(bz)
if err != nil {
panic(fmt.Errorf("failed to decode client state: %w", err))
}
return clientState
}
// MustMarshalClientState attempts to encode an ClientState object and returns the
// raw encoded bytes. It panics on error.
func (k Keeper) MustMarshalClientState(clientState exported.ClientState) []byte {
bz, err := k.MarshalClientState(clientState)
if err != nil {
panic(fmt.Errorf("failed to encode client state: %w", err))
}
return bz
}
// MarshalClientState marshals an ClientState interface. If the given type implements
// the Marshaler interface, it is treated as a Proto-defined message and
// serialized that way.
func (k Keeper) MarshalClientState(clientStateI exported.ClientState) ([]byte, error) {
return codec.MarshalAny(k.cdc, clientStateI)
}
// UnmarshalClientState returns an ClientState interface from raw encoded clientState
// bytes of a Proto-based ClientState type. An error is returned upon decoding
// failure.
func (k Keeper) UnmarshalClientState(bz []byte) (exported.ClientState, error) {
var clientState exported.ClientState
if err := codec.UnmarshalAny(k.cdc, &clientState, bz); err != nil {
return nil, err
}
return clientState, nil
return types.MustUnmarshalClientState(k.cdc, bz)
}
// MustUnmarshalConsensusState attempts to decode and return an ConsensusState object from
// raw encoded bytes. It panics on error.
func (k Keeper) MustUnmarshalConsensusState(bz []byte) exported.ConsensusState {
consensusState, err := k.UnmarshalConsensusState(bz)
if err != nil {
panic(fmt.Errorf("failed to decode consensus state: %w", err))
}
return types.MustUnmarshalConsensusState(k.cdc, bz)
}
return consensusState
// MustMarshalClientState attempts to encode an ClientState object and returns the
// raw encoded bytes. It panics on error.
func (k Keeper) MustMarshalClientState(clientState exported.ClientState) []byte {
return types.MustMarshalClientState(k.cdc, clientState)
}
// MustMarshalConsensusState attempts to encode an ConsensusState object and returns the
// raw encoded bytes. It panics on error.
func (k Keeper) MustMarshalConsensusState(consensusState exported.ConsensusState) []byte {
bz, err := k.MarshalConsensusState(consensusState)
if err != nil {
panic(fmt.Errorf("failed to encode consensus state: %w", err))
}
return bz
}
// MarshalConsensusState marshals an ConsensusState interface. If the given type implements
// the Marshaler interface, it is treated as a Proto-defined message and
// serialized that way.
func (k Keeper) MarshalConsensusState(consensusStateI exported.ConsensusState) ([]byte, error) {
return codec.MarshalAny(k.cdc, consensusStateI)
}
// UnmarshalConsensusState returns an ConsensusState interface from raw encoded clientState
// bytes of a Proto-based ConsensusState type. An error is returned upon decoding
// failure.
func (k Keeper) UnmarshalConsensusState(bz []byte) (exported.ConsensusState, error) {
var consensusState exported.ConsensusState
if err := codec.UnmarshalAny(k.cdc, &consensusState, bz); err != nil {
return nil, err
}
return consensusState, nil
return types.MustMarshalConsensusState(k.cdc, consensusState)
}

View File

@ -6,9 +6,12 @@ import (
"github.com/cosmos/cosmos-sdk/types/kv"
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/keeper"
host "github.com/cosmos/cosmos-sdk/x/ibc/24-host"
)
var _ ClientUnmarshaler = (*keeper.Keeper)(nil)
// ClientUnmarshaler defines an interface for unmarshaling ICS02 interfaces.
type ClientUnmarshaler interface {
MustUnmarshalClientState([]byte) exported.ClientState

View File

@ -0,0 +1,90 @@
package types
import (
"fmt"
"github.com/cosmos/cosmos-sdk/codec"
"github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
)
// MustUnmarshalClientState attempts to decode and return an ClientState object from
// raw encoded bytes. It panics on error.
func MustUnmarshalClientState(cdc codec.BinaryMarshaler, bz []byte) exported.ClientState {
clientState, err := UnmarshalClientState(cdc, bz)
if err != nil {
panic(fmt.Errorf("failed to decode client state: %w", err))
}
return clientState
}
// MustMarshalClientState attempts to encode an ClientState object and returns the
// raw encoded bytes. It panics on error.
func MustMarshalClientState(cdc codec.BinaryMarshaler, clientState exported.ClientState) []byte {
bz, err := MarshalClientState(cdc, clientState)
if err != nil {
panic(fmt.Errorf("failed to encode client state: %w", err))
}
return bz
}
// MarshalClientState marshals an ClientState interface. If the given type implements
// the Marshaler interface, it is treated as a Proto-defined message and
// serialized that way.
func MarshalClientState(cdc codec.BinaryMarshaler, clientStateI exported.ClientState) ([]byte, error) {
return codec.MarshalAny(cdc, clientStateI)
}
// UnmarshalClientState returns an ClientState interface from raw encoded clientState
// bytes of a Proto-based ClientState type. An error is returned upon decoding
// failure.
func UnmarshalClientState(cdc codec.BinaryMarshaler, bz []byte) (exported.ClientState, error) {
var clientState exported.ClientState
if err := codec.UnmarshalAny(cdc, &clientState, bz); err != nil {
return nil, err
}
return clientState, nil
}
// MustUnmarshalConsensusState attempts to decode and return an ConsensusState object from
// raw encoded bytes. It panics on error.
func MustUnmarshalConsensusState(cdc codec.BinaryMarshaler, bz []byte) exported.ConsensusState {
consensusState, err := UnmarshalConsensusState(cdc, bz)
if err != nil {
panic(fmt.Errorf("failed to decode consensus state: %w", err))
}
return consensusState
}
// MustMarshalConsensusState attempts to encode an ConsensusState object and returns the
// raw encoded bytes. It panics on error.
func MustMarshalConsensusState(cdc codec.BinaryMarshaler, consensusState exported.ConsensusState) []byte {
bz, err := MarshalConsensusState(cdc, consensusState)
if err != nil {
panic(fmt.Errorf("failed to encode consensus state: %w", err))
}
return bz
}
// MarshalConsensusState marshals an ConsensusState interface. If the given type implements
// the Marshaler interface, it is treated as a Proto-defined message and
// serialized that way.
func MarshalConsensusState(cdc codec.BinaryMarshaler, consensusStateI exported.ConsensusState) ([]byte, error) {
return codec.MarshalAny(cdc, consensusStateI)
}
// UnmarshalConsensusState returns an ConsensusState interface from raw encoded clientState
// bytes of a Proto-based ConsensusState type. An error is returned upon decoding
// failure.
func UnmarshalConsensusState(cdc codec.BinaryMarshaler, bz []byte) (exported.ConsensusState, error) {
var consensusState exported.ConsensusState
if err := codec.UnmarshalAny(cdc, &consensusState, bz); err != nil {
return nil, err
}
return consensusState, nil
}

View File

@ -1,70 +0,0 @@
package tendermint_test
import (
"testing"
"time"
"github.com/stretchr/testify/suite"
tmbytes "github.com/tendermint/tendermint/libs/bytes"
tmtypes "github.com/tendermint/tendermint/types"
"github.com/cosmos/cosmos-sdk/codec"
cryptocodec "github.com/cosmos/cosmos-sdk/crypto/codec"
ibctmtypes "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types"
commitmenttypes "github.com/cosmos/cosmos-sdk/x/ibc/23-commitment/types"
)
const (
chainID = "gaia"
height = 4
trustingPeriod time.Duration = time.Hour * 24 * 7 * 2
ubdPeriod time.Duration = time.Hour * 24 * 7 * 3
maxClockDrift time.Duration = time.Second * 10
)
type TendermintTestSuite struct {
suite.Suite
cdc *codec.LegacyAmino
signers []tmtypes.PrivValidator
privVal tmtypes.PrivValidator
valSet *tmtypes.ValidatorSet
valsHash tmbytes.HexBytes
header ibctmtypes.Header
now time.Time
clientTime time.Time
headerTime time.Time
}
func (suite *TendermintTestSuite) SetupTest() {
suite.cdc = codec.New()
cryptocodec.RegisterCrypto(suite.cdc)
ibctmtypes.RegisterCodec(suite.cdc)
commitmenttypes.RegisterCodec(suite.cdc)
// now is the time of the current chain, must be after the updating header
// mocks ctx.BlockTime()
suite.now = time.Date(2020, 1, 3, 0, 0, 0, 0, time.UTC)
suite.clientTime = time.Date(2020, 1, 1, 0, 0, 0, 0, time.UTC)
// Header time is intended to be time for any new header used for updates
suite.headerTime = time.Date(2020, 1, 2, 0, 0, 0, 0, time.UTC)
suite.privVal = tmtypes.NewMockPV()
suite.signers = []tmtypes.PrivValidator{suite.privVal}
pubKey, err := suite.privVal.GetPubKey()
suite.Require().NoError(err)
val := tmtypes.NewValidator(pubKey, 10)
suite.valSet = tmtypes.NewValidatorSet([]*tmtypes.Validator{val})
suite.valsHash = suite.valSet.Hash()
// Suite header is intended to be header passed in for initial ClientState
// Thus it should have same height and time as ClientState
// Note: default header has the same validator set suite.valSet as next validators set
suite.header = ibctmtypes.CreateTestHeader(chainID, height, height-1, suite.clientTime, suite.valSet, suite.valSet, suite.signers)
}
func TestTendermintTestSuite(t *testing.T) {
suite.Run(t, new(TendermintTestSuite))
}

View File

@ -23,14 +23,6 @@ import (
var _ clientexported.ClientState = (*ClientState)(nil)
// InitializeFromMsg creates a tendermint client state from a CreateClientMsg
func InitializeFromMsg(msg *MsgCreateClient) *ClientState {
return NewClientState(msg.Header.ChainID, msg.TrustLevel,
msg.TrustingPeriod, msg.UnbondingPeriod, msg.MaxClockDrift,
uint64(msg.Header.Height), msg.ProofSpecs,
)
}
// NewClientState creates a new ClientState instance
func NewClientState(
chainID string, trustLevel Fraction,

View File

@ -1,14 +1,14 @@
package tendermint
package types
import (
"time"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/cosmos/cosmos-sdk/codec"
sdk "github.com/cosmos/cosmos-sdk/types"
sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
clienttypes "github.com/cosmos/cosmos-sdk/x/ibc/02-client/types"
"github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types"
host "github.com/cosmos/cosmos-sdk/x/ibc/24-host"
)
// CheckMisbehaviourAndUpdateState determines whether or not two conflicting
@ -18,53 +18,64 @@ import (
// of misbehaviour.Header1
// Similarly, consensusState2 is the trusted consensus state that corresponds
// to misbehaviour.Header2
func CheckMisbehaviourAndUpdateState(
clientState clientexported.ClientState,
consensusState1, consensusState2 clientexported.ConsensusState,
func (cs ClientState) CheckMisbehaviourAndUpdateState(
ctx sdk.Context,
cdc codec.BinaryMarshaler,
clientStore sdk.KVStore,
misbehaviour clientexported.Misbehaviour,
currentTimestamp time.Time,
consensusParams *abci.ConsensusParams,
) (clientexported.ClientState, error) {
// cast the interface to specific types before checking for misbehaviour
tmClientState, ok := clientState.(*types.ClientState)
if !ok {
return nil, sdkerrors.Wrapf(clienttypes.ErrInvalidClientType, "expected type %T, got %T", &types.ClientState{}, clientState)
}
// If client is already frozen at earlier height than evidence, return with error
if tmClientState.IsFrozen() && tmClientState.FrozenHeight <= uint64(misbehaviour.GetHeight()) {
if cs.IsFrozen() && cs.FrozenHeight <= uint64(misbehaviour.GetHeight()) {
return nil, sdkerrors.Wrapf(clienttypes.ErrInvalidEvidence,
"client is already frozen at earlier height %d than misbehaviour height %d", tmClientState.FrozenHeight, misbehaviour.GetHeight())
"client is already frozen at earlier height %d than misbehaviour height %d", cs.FrozenHeight, misbehaviour.GetHeight())
}
tmConsensusState1, ok := consensusState1.(*types.ConsensusState)
tmEvidence, ok := misbehaviour.(Evidence)
if !ok {
return nil, sdkerrors.Wrapf(clienttypes.ErrInvalidClientType, "invalid consensus state type for first header: expected type %T, got %T", &types.ConsensusState{}, consensusState1)
}
tmConsensusState2, ok := consensusState2.(*types.ConsensusState)
if !ok {
return nil, sdkerrors.Wrapf(clienttypes.ErrInvalidClientType, "invalid consensus state for second header: expected type %T, got %T", &types.ConsensusState{}, consensusState2)
return nil, sdkerrors.Wrapf(clienttypes.ErrInvalidClientType, "expected type %T, got %T", misbehaviour, Evidence{})
}
tmEvidence, ok := misbehaviour.(types.Evidence)
// Retrieve trusted consensus states for each Header in misbehaviour
// and unmarshal from clientStore
// Get consensus bytes from clientStore
consBytes1 := clientStore.Get(host.KeyConsensusState(tmEvidence.Header1.TrustedHeight))
if consBytes1 == nil {
return nil, sdkerrors.Wrapf(clienttypes.ErrConsensusStateNotFound,
"could not find trusted consensus state at height %d", tmEvidence.Header1.TrustedHeight)
}
// Unmarshal consensus bytes into clientexported.ConensusState
consensusState1 := clienttypes.MustUnmarshalConsensusState(cdc, consBytes1)
// Cast to tendermint-specific type
tmConsensusState1, ok := consensusState1.(*ConsensusState)
if !ok {
return nil, sdkerrors.Wrapf(clienttypes.ErrInvalidClientType, "expected type %T, got %T", misbehaviour, types.Evidence{})
return nil, sdkerrors.Wrapf(clienttypes.ErrInvalidClientType, "invalid consensus state type for first header: expected type %T, got %T", &ConsensusState{}, consensusState1)
}
// use earliest height of trusted consensus states to verify ageBlocks
var height uint64
if tmConsensusState1.Height < tmConsensusState2.Height {
height = tmConsensusState1.Height
} else {
height = tmConsensusState2.Height
// Get consensus bytes from clientStore
consBytes2 := clientStore.Get(host.KeyConsensusState(tmEvidence.Header2.TrustedHeight))
if consBytes2 == nil {
return nil, sdkerrors.Wrapf(clienttypes.ErrConsensusStateNotFound,
"could not find trusted consensus state at height %d", tmEvidence.Header2.TrustedHeight)
}
// Unmarshal consensus bytes into clientexported.ConensusState
consensusState2 := clienttypes.MustUnmarshalConsensusState(cdc, consBytes2)
// Cast to tendermint-specific type
tmConsensusState2, ok := consensusState2.(*ConsensusState)
if !ok {
return nil, sdkerrors.Wrapf(clienttypes.ErrInvalidClientType, "invalid consensus state for second header: expected type %T, got %T", &ConsensusState{}, consensusState2)
}
// calculate the age of the misbehaviour evidence
infractionHeight := tmEvidence.GetHeight()
infractionTime := tmEvidence.GetTime()
ageDuration := currentTimestamp.Sub(infractionTime)
ageBlocks := uint64(infractionHeight) - height
ageDuration := ctx.BlockTime().Sub(infractionTime)
ageBlocks := int64(cs.LatestHeight) - infractionHeight
// TODO: Retrieve consensusparams from client state and not context
// Issue #6516: https://github.com/cosmos/cosmos-sdk/issues/6516
consensusParams := ctx.ConsensusParams()
// Reject misbehaviour if the age is too old. Evidence is considered stale
// if the difference in time and number of blocks is greater than the allowed
@ -76,8 +87,8 @@ func CheckMisbehaviourAndUpdateState(
// use the default values.
if consensusParams != nil &&
consensusParams.Evidence != nil &&
ageDuration > consensusParams.Evidence.MaxAgeDuration &&
ageBlocks > uint64(consensusParams.Evidence.MaxAgeNumBlocks) {
(ageDuration > consensusParams.Evidence.MaxAgeDuration ||
ageBlocks > consensusParams.Evidence.MaxAgeNumBlocks) {
return nil, sdkerrors.Wrapf(clienttypes.ErrInvalidEvidence,
"age duration (%s) and age blocks (%d) are greater than max consensus params for duration (%s) and block (%d)",
ageDuration, ageBlocks, consensusParams.Evidence.MaxAgeDuration, consensusParams.Evidence.MaxAgeNumBlocks,
@ -90,24 +101,24 @@ func CheckMisbehaviourAndUpdateState(
// evidence.ValidateBasic by the client keeper and msg.ValidateBasic
// by the base application.
if err := checkMisbehaviourHeader(
tmClientState, tmConsensusState1, tmEvidence.Header1, currentTimestamp,
&cs, tmConsensusState1, tmEvidence.Header1, ctx.BlockTime(),
); err != nil {
return nil, sdkerrors.Wrap(err, "verifying Header1 in Evidence failed")
}
if err := checkMisbehaviourHeader(
tmClientState, tmConsensusState2, tmEvidence.Header2, currentTimestamp,
&cs, tmConsensusState2, tmEvidence.Header2, ctx.BlockTime(),
); err != nil {
return nil, sdkerrors.Wrap(err, "verifying Header2 in Evidence failed")
}
tmClientState.FrozenHeight = uint64(tmEvidence.GetHeight())
return tmClientState, nil
cs.FrozenHeight = uint64(tmEvidence.GetHeight())
return &cs, nil
}
// checkMisbehaviourHeader checks that a Header in Misbehaviour is valid evidence given
// a trusted ConsensusState
func checkMisbehaviourHeader(
clientState *types.ClientState, consState *types.ConsensusState, header types.Header, currentTimestamp time.Time,
clientState *ClientState, consState *ConsensusState, header Header, currentTimestamp time.Time,
) error {
// check the trusted fields for the header against ConsensusState
if err := checkTrustedHeader(header, consState); err != nil {
@ -117,7 +128,7 @@ func checkMisbehaviourHeader(
// assert that the timestamp is not from more than an unbonding period ago
if currentTimestamp.Sub(consState.Timestamp) >= clientState.UnbondingPeriod {
return sdkerrors.Wrapf(
types.ErrUnbondingPeriodExpired,
ErrUnbondingPeriodExpired,
"current timestamp minus the latest consensus state timestamp is greater than or equal to the unbonding period (%s >= %s)",
currentTimestamp.Sub(consState.Timestamp), clientState.UnbondingPeriod,
)

View File

@ -1,16 +1,15 @@
package tendermint_test
package types_test
import (
"bytes"
fmt "fmt"
"time"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/tmhash"
tmtypes "github.com/tendermint/tendermint/types"
"github.com/cosmos/cosmos-sdk/simapp"
clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
tendermint "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint"
"github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types"
commitmenttypes "github.com/cosmos/cosmos-sdk/x/ibc/23-commitment/types"
)
@ -48,7 +47,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
consensusState1 clientexported.ConsensusState
consensusState2 clientexported.ConsensusState
evidence clientexported.Misbehaviour
consensusParams *abci.ConsensusParams
timestamp time.Time
expPass bool
}{
@ -63,7 +61,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
true,
},
@ -78,7 +75,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
true,
},
@ -93,7 +89,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
true,
},
@ -108,7 +103,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
true,
},
@ -123,7 +117,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
false,
},
@ -138,22 +131,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
false,
},
{
"invalid tendermint client state",
nil,
types.NewConsensusState(suite.now, commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), height, bothValsHash),
types.NewConsensusState(suite.now, commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), height, bothValsHash),
types.Evidence{
Header1: types.CreateTestHeader(chainID, height, height, suite.now, bothValSet, bothValSet, bothSigners),
Header2: types.CreateTestHeader(chainID, height, height, suite.now.Add(time.Minute), bothValSet, altValSet, bothSigners),
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
false,
},
@ -168,22 +145,20 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
false,
},
{
"invalid tendermint consensus state",
"trusted consensus state does not exist",
types.NewClientState(chainID, types.DefaultTrustLevel, trustingPeriod, ubdPeriod, maxClockDrift, height, commitmenttypes.GetSDKSpecs()),
nil,
nil, // consensus state for trusted height - 1 does not exist in store
types.NewConsensusState(suite.now, commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), height, bothValsHash),
types.Evidence{
Header1: types.CreateTestHeader(chainID, height, height, suite.now, bothValSet, bothValSet, bothSigners),
Header1: types.CreateTestHeader(chainID, height, height-1, suite.now, bothValSet, bothValSet, bothSigners),
Header2: types.CreateTestHeader(chainID, height, height, suite.now.Add(time.Minute), bothValSet, bothValSet, bothSigners),
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
false,
},
@ -193,27 +168,37 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
types.NewConsensusState(suite.now, commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), height, bothValsHash),
types.NewConsensusState(suite.now, commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), height, bothValsHash),
nil,
simapp.DefaultConsensusParams,
suite.now,
false,
},
{
"rejected misbehaviour due to expired age",
"rejected misbehaviour due to expired age duration",
types.NewClientState(chainID, types.DefaultTrustLevel, trustingPeriod, ubdPeriod, maxClockDrift, height, commitmenttypes.GetSDKSpecs()),
types.NewConsensusState(suite.now, commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), height, bothValsHash),
types.NewConsensusState(suite.now, commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), height, bothValsHash),
types.Evidence{
Header1: types.CreateTestHeader(chainID, int64(2*height+uint64(simapp.DefaultConsensusParams.Evidence.MaxAgeNumBlocks)), height,
suite.now, bothValSet, bothValSet, bothSigners),
Header2: types.CreateTestHeader(chainID, int64(2*height+uint64(simapp.DefaultConsensusParams.Evidence.MaxAgeNumBlocks)), height,
suite.now.Add(time.Minute), bothValSet, bothValSet, bothSigners),
Header1: types.CreateTestHeader(chainID, height, height, suite.now, bothValSet, bothValSet, bothSigners),
Header2: types.CreateTestHeader(chainID, height, height, suite.now.Add(time.Minute), bothValSet, bothValSet, bothSigners),
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now.Add(2 * time.Minute).Add(simapp.DefaultConsensusParams.Evidence.MaxAgeDuration),
false,
},
{
"rejected misbehaviour due to expired block duration",
types.NewClientState(chainID, types.DefaultTrustLevel, trustingPeriod, ubdPeriod, maxClockDrift, uint64(height+simapp.DefaultConsensusParams.Evidence.MaxAgeNumBlocks+1), commitmenttypes.GetSDKSpecs()),
types.NewConsensusState(suite.now, commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), height, bothValsHash),
types.NewConsensusState(suite.now, commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), height, bothValsHash),
types.Evidence{
Header1: types.CreateTestHeader(chainID, height, height, suite.now, bothValSet, bothValSet, bothSigners),
Header2: types.CreateTestHeader(chainID, height, height, suite.now.Add(time.Minute), bothValSet, bothValSet, bothSigners),
ChainID: chainID,
ClientID: chainID,
},
suite.now.Add(time.Hour),
false,
},
{
"provided height > header height",
types.NewClientState(chainID, types.DefaultTrustLevel, trustingPeriod, ubdPeriod, maxClockDrift, height, commitmenttypes.GetSDKSpecs()),
@ -225,14 +210,13 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
false,
},
{
"unbonding period expired",
types.NewClientState(chainID, types.DefaultTrustLevel, trustingPeriod, ubdPeriod, maxClockDrift, height, commitmenttypes.GetSDKSpecs()),
types.ConsensusState{Timestamp: time.Time{}, Root: commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), NextValidatorsHash: bothValsHash},
&types.ConsensusState{Timestamp: time.Time{}, Root: commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), NextValidatorsHash: bothValsHash},
types.NewConsensusState(suite.now, commitmenttypes.NewMerkleRoot(tmhash.Sum([]byte("app_hash"))), height, bothValsHash),
types.Evidence{
Header1: types.CreateTestHeader(chainID, height, height, suite.now, bothValSet, bothValSet, bothSigners),
@ -240,8 +224,7 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
suite.now.Add(ubdPeriod),
false,
},
{
@ -255,7 +238,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
false,
},
@ -270,7 +252,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
false,
},
@ -285,7 +266,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
false,
},
@ -300,7 +280,6 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
ChainID: chainID,
ClientID: chainID,
},
simapp.DefaultConsensusParams,
suite.now,
false,
},
@ -308,18 +287,45 @@ func (suite *TendermintTestSuite) TestCheckMisbehaviourAndUpdateState() {
for i, tc := range testCases {
tc := tc
suite.Run(fmt.Sprintf("Case: %s", tc.name), func() {
clientState, err := tendermint.CheckMisbehaviourAndUpdateState(tc.clientState, tc.consensusState1, tc.consensusState2, tc.evidence, tc.timestamp, tc.consensusParams)
if i != 10 {
return
}
if tc.expPass {
suite.Require().NoError(err, "valid test case %d failed: %s", i, tc.name)
suite.Require().NotNil(clientState, "valid test case %d failed: %s", i, tc.name)
suite.Require().True(clientState.IsFrozen(), "valid test case %d failed: %s", i, tc.name)
suite.Require().Equal(uint64(tc.evidence.GetHeight()), clientState.GetFrozenHeight(),
"valid test case %d failed: %s. Expected FrozenHeight %d got %d", tc.evidence.GetHeight(), clientState.GetFrozenHeight())
} else {
suite.Require().Error(err, "invalid test case %d passed: %s", i, tc.name)
suite.Require().Nil(clientState, "invalid test case %d passed: %s", i, tc.name)
}
// reset suite to create fresh application state
suite.SetupTest()
// Set current timestamp in context
ctx := suite.chainA.GetContext().WithBlockTime(tc.timestamp)
ctx = ctx.WithConsensusParams(simapp.DefaultConsensusParams)
// Set trusted consensus states in client store
if tc.consensusState1 != nil {
suite.chainA.App.IBCKeeper.ClientKeeper.SetClientConsensusState(ctx, clientID, tc.consensusState1.GetHeight(), tc.consensusState1)
}
if tc.consensusState2 != nil {
suite.chainA.App.IBCKeeper.ClientKeeper.SetClientConsensusState(ctx, clientID, tc.consensusState2.GetHeight(), tc.consensusState2)
}
clientState, err := tc.clientState.CheckMisbehaviourAndUpdateState(
ctx,
suite.cdc,
suite.chainA.App.IBCKeeper.ClientKeeper.ClientStore(ctx, clientID), // pass in clientID prefixed clientStore
tc.evidence,
)
if tc.expPass {
suite.Require().NoError(err, "valid test case %d failed: %s", i, tc.name)
suite.Require().NotNil(clientState, "valid test case %d failed: %s", i, tc.name)
suite.Require().True(clientState.IsFrozen(), "valid test case %d failed: %s", i, tc.name)
suite.Require().Equal(uint64(tc.evidence.GetHeight()), clientState.GetFrozenHeight(),
"valid test case %d failed: %s. Expected FrozenHeight %d got %d", tc.evidence.GetHeight(), clientState.GetFrozenHeight())
} else {
suite.Require().Error(err, "invalid test case %d passed: %s", i, tc.name)
suite.Require().Nil(clientState, "invalid test case %d passed: %s", i, tc.name)
}
})
}
}

View File

@ -148,6 +148,14 @@ func (msg MsgCreateClient) GetConsensusState() clientexported.ConsensusState {
}
}
// InitializeFromMsg creates a tendermint client state from a CreateClientMsg
func (msg MsgCreateClient) InitializeClientState() clientexported.ClientState {
return NewClientState(msg.Header.ChainID, msg.TrustLevel,
msg.TrustingPeriod, msg.UnbondingPeriod, msg.MaxClockDrift,
uint64(msg.Header.Height), msg.ProofSpecs,
)
}
// MsgUpdateClient defines a message to update an IBC client
type MsgUpdateClient struct {
ClientID string `json:"client_id" yaml:"client_id"`

View File

@ -36,14 +36,16 @@ type TendermintTestSuite struct {
chainB *ibctesting.TestChain
// TODO: deprecate usage in favor of testing package
ctx sdk.Context
aminoCdc *codec.LegacyAmino
cdc codec.Marshaler
privVal tmtypes.PrivValidator
valSet *tmtypes.ValidatorSet
valsHash tmbytes.HexBytes
header ibctmtypes.Header
now time.Time
ctx sdk.Context
aminoCdc *codec.LegacyAmino
cdc codec.Marshaler
privVal tmtypes.PrivValidator
valSet *tmtypes.ValidatorSet
valsHash tmbytes.HexBytes
header ibctmtypes.Header
now time.Time
headerTime time.Time
clientTime time.Time
}
func (suite *TendermintTestSuite) SetupTest() {
@ -58,7 +60,13 @@ func (suite *TendermintTestSuite) SetupTest() {
suite.aminoCdc = app.LegacyAmino()
suite.cdc = app.AppCodec()
// now is the time of the current chain, must be after the updating header
// mocks ctx.BlockTime()
suite.now = time.Date(2020, 1, 2, 0, 0, 0, 0, time.UTC)
suite.clientTime = time.Date(2020, 1, 1, 0, 0, 0, 0, time.UTC)
// Header time is intended to be time for any new header used for updates
suite.headerTime = time.Date(2020, 1, 2, 0, 0, 0, 0, time.UTC)
suite.privVal = tmtypes.NewMockPV()
pubKey, err := suite.privVal.GetPubKey()

View File

@ -1,4 +1,4 @@
package tendermint
package types
import (
"bytes"
@ -7,14 +7,16 @@ import (
lite "github.com/tendermint/tendermint/lite2"
tmtypes "github.com/tendermint/tendermint/types"
"github.com/cosmos/cosmos-sdk/codec"
sdk "github.com/cosmos/cosmos-sdk/types"
sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
clientexported "github.com/cosmos/cosmos-sdk/x/ibc/02-client/exported"
clienttypes "github.com/cosmos/cosmos-sdk/x/ibc/02-client/types"
"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"
)
// CheckValidityAndUpdateState checks if the provided header is valid, and if valid it will:
// CheckHeaderAndUpdateState checks if the provided header is valid, and if valid it will:
// create the consensus state for the header.Height
// and update the client state if the header height is greater than the latest client state height
// It returns an error if:
@ -33,44 +35,47 @@ import (
// the new latest height
// Tendermint client validity checking uses the bisection algorithm described
// in the [Tendermint spec](https://github.com/tendermint/spec/blob/master/spec/consensus/light-client.md).
func CheckValidityAndUpdateState(
clientState clientexported.ClientState, consState clientexported.ConsensusState,
header clientexported.Header, currentTimestamp time.Time,
func (cs ClientState) CheckHeaderAndUpdateState(
ctx sdk.Context, cdc codec.BinaryMarshaler, clientStore sdk.KVStore,
header clientexported.Header,
) (clientexported.ClientState, clientexported.ConsensusState, error) {
tmClientState, ok := clientState.(*types.ClientState)
tmHeader, ok := header.(Header)
if !ok {
return nil, nil, sdkerrors.Wrapf(
clienttypes.ErrInvalidClientType, "expected type %T, got %T", types.ClientState{}, clientState,
clienttypes.ErrInvalidHeader, "expected type %T, got %T", Header{}, header,
)
}
tmConsState, ok := consState.(*types.ConsensusState)
// Get consensus bytes from clientStore
consBytes := clientStore.Get(host.KeyConsensusState(tmHeader.TrustedHeight))
if consBytes == nil {
return nil, nil, sdkerrors.Wrapf(
clienttypes.ErrConsensusStateNotFound, "consensus state not found for trusted height %d", tmHeader.TrustedHeight,
)
}
// Unmarshal consensus bytes into clientexported.ConensusState
consState := clienttypes.MustUnmarshalConsensusState(cdc, consBytes)
// Cast to tendermint-specific type
tmConsState, ok := consState.(*ConsensusState)
if !ok {
return nil, nil, sdkerrors.Wrapf(
clienttypes.ErrInvalidConsensus, "expected type %T, got %T", types.ConsensusState{}, consState,
clienttypes.ErrInvalidConsensus, "expected type %T, got %T", ConsensusState{}, consState,
)
}
tmHeader, ok := header.(types.Header)
if !ok {
return nil, nil, sdkerrors.Wrapf(
clienttypes.ErrInvalidHeader, "expected type %T, got %T", types.Header{}, header,
)
}
if err := checkValidity(tmClientState, tmConsState, tmHeader, currentTimestamp); err != nil {
if err := checkValidity(&cs, tmConsState, tmHeader, ctx.BlockTime()); err != nil {
return nil, nil, err
}
tmClientState, consensusState := update(tmClientState, tmHeader)
return tmClientState, consensusState, nil
newClientState, consensusState := update(&cs, tmHeader)
return newClientState, consensusState, nil
}
// checkTrustedHeader checks that consensus state matches trusted fields of Header
func checkTrustedHeader(header types.Header, consState *types.ConsensusState) error {
func checkTrustedHeader(header Header, consState *ConsensusState) error {
if header.TrustedHeight != consState.Height {
return sdkerrors.Wrapf(
types.ErrInvalidHeaderHeight,
ErrInvalidHeaderHeight,
"trusted header height %d does not match consensus state height %d",
header.TrustedHeight, consState.Height,
)
@ -80,7 +85,7 @@ func checkTrustedHeader(header types.Header, consState *types.ConsensusState) er
tvalHash := header.TrustedValidators.Hash()
if !bytes.Equal(consState.NextValidatorsHash, tvalHash) {
return sdkerrors.Wrapf(
types.ErrInvalidValidatorSet,
ErrInvalidValidatorSet,
"trusted validators %s, does not hash to latest trusted validators. Expected: %X, got: %X",
header.TrustedValidators, consState.NextValidatorsHash, tvalHash,
)
@ -91,8 +96,8 @@ func checkTrustedHeader(header types.Header, consState *types.ConsensusState) er
// checkValidity checks if the Tendermint header is valid.
// CONTRACT: consState.Height == header.TrustedHeight
func checkValidity(
clientState *types.ClientState, consState *types.ConsensusState,
header types.Header, currentTimestamp time.Time,
clientState *ClientState, consState *ConsensusState,
header Header, currentTimestamp time.Time,
) error {
if err := checkTrustedHeader(header, consState); err != nil {
return err
@ -134,11 +139,11 @@ func checkValidity(
}
// update the consensus state from a new header
func update(clientState *types.ClientState, header types.Header) (*types.ClientState, *types.ConsensusState) {
func update(clientState *ClientState, header Header) (*ClientState, *ConsensusState) {
if uint64(header.Height) > clientState.LatestHeight {
clientState.LatestHeight = uint64(header.Height)
}
consensusState := &types.ConsensusState{
consensusState := &ConsensusState{
Height: uint64(header.Height),
Timestamp: header.Time,
Root: commitmenttypes.NewMerkleRoot(header.AppHash),

View File

@ -1,4 +1,4 @@
package tendermint_test
package types_test
import (
"bytes"
@ -6,12 +6,11 @@ import (
tmtypes "github.com/tendermint/tendermint/types"
tendermint "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint"
types "github.com/cosmos/cosmos-sdk/x/ibc/07-tendermint/types"
commitmenttypes "github.com/cosmos/cosmos-sdk/x/ibc/23-commitment/types"
)
func (suite *TendermintTestSuite) TestCheckValidity() {
func (suite *TendermintTestSuite) TestCheckHeaderAndUpdateState() {
var (
clientState *types.ClientState
consensusState *types.ConsensusState
@ -192,6 +191,12 @@ func (suite *TendermintTestSuite) TestCheckValidity() {
// setup test
tc.setup()
// Set current timestamp in context
ctx := suite.chainA.GetContext().WithBlockTime(currentTime)
// Set trusted consensus state in client store
suite.chainA.App.IBCKeeper.ClientKeeper.SetClientConsensusState(ctx, clientID, consensusState.Height, consensusState)
expectedConsensus := &types.ConsensusState{
Height: uint64(newHeader.Height),
Timestamp: newHeader.Time,
@ -199,7 +204,12 @@ func (suite *TendermintTestSuite) TestCheckValidity() {
NextValidatorsHash: newHeader.NextValidatorsHash,
}
newClientState, consensusState, err := tendermint.CheckValidityAndUpdateState(clientState, consensusState, newHeader, currentTime)
newClientState, consensusState, err := clientState.CheckHeaderAndUpdateState(
ctx,
suite.cdc,
suite.chainA.App.IBCKeeper.ClientKeeper.ClientStore(suite.chainA.GetContext(), clientID), // pass in clientID prefixed clientStore
newHeader,
)
if tc.expPass {
suite.Require().NoError(err, "valid test case %d failed: %s", i, tc.name)

View File

@ -72,6 +72,25 @@ func (cs ClientState) GetProofSpecs() []*ics23.ProofSpec {
return nil
}
// CheckHeaderAndUpdateState updates the localhost client. It only needs access to the context
func (cs ClientState) CheckHeaderAndUpdateState(
ctx sdk.Context, _ codec.BinaryMarshaler, _ sdk.KVStore, _ clientexported.Header,
) (clientexported.ClientState, clientexported.ConsensusState, error) {
return NewClientState(
ctx.ChainID(), // use the chain ID from context since the client is from the running chain (i.e self).
ctx.BlockHeight(),
), nil, nil
}
// CheckMisbehaviourAndUpdateState implements ClientState
// Since localhost is the client of the running chain, misbehaviour cannot be submitted to it
// Thus, CheckMisbehaviourAndUpdateState returns an error for localhost
func (cs ClientState) CheckMisbehaviourAndUpdateState(
_ sdk.Context, _ codec.BinaryMarshaler, _ sdk.KVStore, _ clientexported.Misbehaviour,
) (clientexported.ClientState, error) {
return nil, sdkerrors.Wrap(clienttypes.ErrInvalidEvidence, "cannot submit misbehaviour to localhost client")
}
// VerifyClientConsensusState returns an error since a local host client does not store consensus
// states.
func (cs ClientState) VerifyClientConsensusState(

View File

@ -65,3 +65,10 @@ func (msg MsgCreateClient) GetClientType() string {
func (msg MsgCreateClient) GetConsensusState() clientexported.ConsensusState {
return nil
}
// InitializeClientState implements clientexported.MsgCreateClient
// localhost is a special case that require the running chain's context to initialize
// the client state, thus this function is a no-op
func (msg MsgCreateClient) InitializeClientState() clientexported.ClientState {
return nil
}

View File

@ -333,16 +333,19 @@ func (chain *TestChain) GetFirstTestConnection(clientID, counterpartyClientID st
return chain.ConstructNextTestConnection(clientID, counterpartyClientID)
}
// CreateTMClient will construct and execute a 07-tendermint MsgCreateClient. A counterparty
// client will be created on the (target) chain.
func (chain *TestChain) CreateTMClient(counterparty *TestChain, clientID string) error {
// construct MsgCreateClient using counterparty
msg := ibctmtypes.NewMsgCreateClient(
func (chain *TestChain) ConstructMsgCreateClient(counterparty *TestChain, clientID string) clientexported.MsgCreateClient {
return ibctmtypes.NewMsgCreateClient(
clientID, counterparty.LastHeader,
DefaultTrustLevel, TrustingPeriod, UnbondingPeriod, MaxClockDrift,
commitmenttypes.GetSDKSpecs(), chain.SenderAccount.GetAddress(),
)
}
// CreateTMClient will construct and execute a 07-tendermint MsgCreateClient. A counterparty
// client will be created on the (target) chain.
func (chain *TestChain) CreateTMClient(counterparty *TestChain, clientID string) error {
// construct MsgCreateClient using counterparty
msg := chain.ConstructMsgCreateClient(counterparty, clientID)
return chain.SendMsgs(msg)
}