diff --git a/tests/integration/double_vote.go b/tests/integration/double_vote.go index 2af7162d77..af05c745df 100644 --- a/tests/integration/double_vote.go +++ b/tests/integration/double_vote.go @@ -37,6 +37,14 @@ func (s *CCVTestSuite) TestHandleConsumerDoubleVoting() { blockID1 := testutil.MakeBlockID([]byte("blockhash"), 1000, []byte("partshash")) blockID2 := testutil.MakeBlockID([]byte("blockhash2"), 1000, []byte("partshash")) + // Set the equivocation evidence min height to the previous block height + equivocationEvidenceMinHeight := uint64(s.consumerCtx().BlockHeight() - 1) + s.providerApp.GetProviderKeeper().SetEquivocationEvidenceMinHeight( + s.providerCtx(), + s.consumerChain.ChainID, + equivocationEvidenceMinHeight, + ) + // Note that votes are signed along with the chain ID // see VoteSignBytes in https://github.com/cometbft/cometbft/blob/v0.37.2/types/vote.go#L93 @@ -78,6 +86,17 @@ func (s *CCVTestSuite) TestHandleConsumerDoubleVoting() { s.consumerChain.ChainID, ) + // create a vote using the consumer validator key + // with block height that is smaller than the equivocation evidence min height + consuVoteOld := testutil.MakeAndSignVote( + blockID1, + int64(equivocationEvidenceMinHeight-1), + s.consumerCtx().BlockTime(), + consuValSet, + consuSigner, + s.consumerChain.ChainID, + ) + testCases := []struct { name string ev *tmtypes.DuplicateVoteEvidence @@ -86,7 +105,7 @@ func (s *CCVTestSuite) TestHandleConsumerDoubleVoting() { expPass bool }{ { - "invalid consumer chain id - shouldn't pass", + "cannot find consumer chain for the given chain ID - shouldn't pass", &tmtypes.DuplicateVoteEvidence{ VoteA: consuVote, VoteB: consuBadVote, @@ -98,6 +117,32 @@ func (s *CCVTestSuite) TestHandleConsumerDoubleVoting() { consuVal.PubKey, false, }, + { + "evidence is older than equivocation evidence min height - shouldn't pass", + &tmtypes.DuplicateVoteEvidence{ + VoteA: consuVoteOld, + VoteB: consuBadVote, + ValidatorPower: consuVal.VotingPower, + TotalVotingPower: consuVal.VotingPower, + Timestamp: s.consumerCtx().BlockTime(), + }, + s.consumerChain.ChainID, + consuVal.PubKey, + false, + }, + { + "the votes in the evidence are for different height - shouldn't pass", + &tmtypes.DuplicateVoteEvidence{ + VoteA: consuVote, + VoteB: consuVoteOld, + ValidatorPower: consuVal.VotingPower, + TotalVotingPower: consuVal.VotingPower, + Timestamp: s.consumerCtx().BlockTime(), + }, + s.consumerChain.ChainID, + consuVal.PubKey, + false, + }, { "wrong public key - shouldn't pass", &tmtypes.DuplicateVoteEvidence{ @@ -343,7 +388,7 @@ func (s *CCVTestSuite) TestHandleConsumerDoubleVotingSlashesUndelegationsAndRele currentBalance := sdk.NewDecFromInt(unb.Balance) s.Require().True(initialBalance.Sub(initialBalance.Mul(slashFraction)).Equal(currentBalance)) } -// check that redelegations are slashed + // check that redelegations are slashed delegations := s.providerApp.GetTestStakingKeeper().Delegation(s.providerCtx(), delAddr, validator2.GetOperator()) s.Require().Equal(delegations.GetShares(), delShares.Add(redelShares).Sub(redelShares.Mul(slashFraction))) }) diff --git a/tests/integration/misbehaviour.go b/tests/integration/misbehaviour.go index 97aef5f2aa..c0f2cf10b9 100644 --- a/tests/integration/misbehaviour.go +++ b/tests/integration/misbehaviour.go @@ -428,6 +428,14 @@ func (s *CCVTestSuite) TestCheckMisbehaviour() { altSigners2, ) + // Set the equivocation evidence min height to the previous block height + equivocationEvidenceMinHeight := clientHeight.RevisionHeight + 1 + s.providerApp.GetProviderKeeper().SetEquivocationEvidenceMinHeight( + s.providerCtx(), + s.consumerChain.ChainID, + equivocationEvidenceMinHeight, + ) + testCases := []struct { name string misbehaviour *ibctmtypes.Misbehaviour @@ -487,6 +495,24 @@ func (s *CCVTestSuite) TestCheckMisbehaviour() { }, false, }, + { + "invalid misbehaviour older than the min equivocation evidence height - shouldn't pass", + &ibctmtypes.Misbehaviour{ + ClientId: s.path.EndpointA.ClientID, + Header1: s.consumerChain.CreateTMClientHeader( + s.consumerChain.ChainID, + int64(equivocationEvidenceMinHeight-1), + clientHeight, + headerTs, + altValset, + altValset, + clientTMValset, + altSigners, + ), + Header2: clientHeader, + }, + false, + }, { "one header of the misbehaviour has insufficient voting power - shouldn't pass", &ibctmtypes.Misbehaviour{ diff --git a/x/ccv/provider/keeper/consumer_equivocation.go b/x/ccv/provider/keeper/consumer_equivocation.go new file mode 100644 index 0000000000..53232b4205 --- /dev/null +++ b/x/ccv/provider/keeper/consumer_equivocation.go @@ -0,0 +1,480 @@ +package keeper + +import ( + "bytes" + "encoding/binary" + "fmt" + + ibcclienttypes "github.com/cosmos/ibc-go/v7/modules/core/02-client/types" + ibctmtypes "github.com/cosmos/ibc-go/v7/modules/light-clients/07-tendermint" + + errorsmod "cosmossdk.io/errors" + "cosmossdk.io/math" + + cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" + sdk "github.com/cosmos/cosmos-sdk/types" + evidencetypes "github.com/cosmos/cosmos-sdk/x/evidence/types" + slashingtypes "github.com/cosmos/cosmos-sdk/x/slashing/types" + stakingtypes "github.com/cosmos/cosmos-sdk/x/staking/types" + + tmtypes "github.com/cometbft/cometbft/types" + + "github.com/cosmos/interchain-security/v3/x/ccv/provider/types" + ccvtypes "github.com/cosmos/interchain-security/v3/x/ccv/types" +) + +// +// Double Voting section +// + +// HandleConsumerDoubleVoting verifies a double voting evidence for a given a consumer chain ID +// and a public key and, if successful, executes the slashing, jailing, and tombstoning of the malicious validator. +func (k Keeper) HandleConsumerDoubleVoting( + ctx sdk.Context, + evidence *tmtypes.DuplicateVoteEvidence, + chainID string, + pubkey cryptotypes.PubKey, +) error { + // verifies the double voting evidence using the consumer chain public key + if err := k.VerifyDoubleVotingEvidence(*evidence, chainID, pubkey); err != nil { + return err + } + + // get the validator's consensus address on the provider + providerAddr := k.GetProviderAddrFromConsumerAddr( + ctx, + chainID, + types.NewConsumerConsAddress(sdk.ConsAddress(evidence.VoteA.ValidatorAddress.Bytes())), + ) + + if err := k.SlashValidator(ctx, providerAddr); err != nil { + return err + } + if err := k.JailAndTombstoneValidator(ctx, providerAddr); err != nil { + return err + } + + k.Logger(ctx).Info( + "confirmed equivocation", + "byzantine validator address", providerAddr.String(), + ) + + return nil +} + +// VerifyDoubleVotingEvidence verifies a double voting evidence +// for a given chain id and a validator public key +func (k Keeper) VerifyDoubleVotingEvidence( + evidence tmtypes.DuplicateVoteEvidence, + chainID string, + pubkey cryptotypes.PubKey, +) error { + if pubkey == nil { + return fmt.Errorf("validator public key cannot be empty") + } + + // check that the validator address in the evidence is derived from the provided public key + if !bytes.Equal(pubkey.Address(), evidence.VoteA.ValidatorAddress) { + return errorsmod.Wrapf( + ccvtypes.ErrInvalidDoubleVotingEvidence, + "public key %s doesn't correspond to the validator address %s in double vote evidence", + pubkey.String(), evidence.VoteA.ValidatorAddress.String(), + ) + } + + // Note the age of the evidence isn't checked. + + // height/round/type must be the same + if evidence.VoteA.Height != evidence.VoteB.Height || + evidence.VoteA.Round != evidence.VoteB.Round || + evidence.VoteA.Type != evidence.VoteB.Type { + return errorsmod.Wrapf( + ccvtypes.ErrInvalidDoubleVotingEvidence, + "height/round/type are not the same: %d/%d/%v vs %d/%d/%v", + evidence.VoteA.Height, evidence.VoteA.Round, evidence.VoteA.Type, + evidence.VoteB.Height, evidence.VoteB.Round, evidence.VoteB.Type) + } + + // Addresses must be the same + if !bytes.Equal(evidence.VoteA.ValidatorAddress, evidence.VoteB.ValidatorAddress) { + return errorsmod.Wrapf( + ccvtypes.ErrInvalidDoubleVotingEvidence, + "validator addresses do not match: %X vs %X", + evidence.VoteA.ValidatorAddress, + evidence.VoteB.ValidatorAddress, + ) + } + + // BlockIDs must be different + if evidence.VoteA.BlockID.Equals(evidence.VoteB.BlockID) { + return errorsmod.Wrapf( + ccvtypes.ErrInvalidDoubleVotingEvidence, + "block IDs are the same (%v) - not a real duplicate vote", + evidence.VoteA.BlockID, + ) + } + + va := evidence.VoteA.ToProto() + vb := evidence.VoteB.ToProto() + + // signatures must be valid + if !pubkey.VerifySignature(tmtypes.VoteSignBytes(chainID, va), evidence.VoteA.Signature) { + return fmt.Errorf("verifying VoteA: %w", tmtypes.ErrVoteInvalidSignature) + } + if !pubkey.VerifySignature(tmtypes.VoteSignBytes(chainID, vb), evidence.VoteB.Signature) { + return fmt.Errorf("verifying VoteB: %w", tmtypes.ErrVoteInvalidSignature) + } + + return nil +} + +// +// Light Client Attack (IBC misbehavior) section +// + +// HandleConsumerMisbehaviour checks if the given IBC misbehaviour corresponds to an equivocation light client attack, +// and in this case, slashes, jails, and tombstones +func (k Keeper) HandleConsumerMisbehaviour(ctx sdk.Context, misbehaviour ibctmtypes.Misbehaviour) error { + logger := k.Logger(ctx) + + // Check that the misbehaviour is valid and that the client consensus states at trusted heights are within trusting period + if err := k.CheckMisbehaviour(ctx, misbehaviour); err != nil { + logger.Info("Misbehaviour rejected", err.Error()) + + return err + } + + // Since the misbehaviour packet was received within the trusting period + // w.r.t to the trusted consensus states the infraction age + // isn't too old. see ibc-go/modules/light-clients/07-tendermint/types/misbehaviour_handle.go + + // Get Byzantine validators from the conflicting headers + byzantineValidators, err := k.GetByzantineValidators(ctx, misbehaviour) + if err != nil { + return err + } + + provAddrs := make([]types.ProviderConsAddress, len(byzantineValidators)) + + // slash, jail, and tombstone the Byzantine validators + for _, v := range byzantineValidators { + providerAddr := k.GetProviderAddrFromConsumerAddr( + ctx, + misbehaviour.Header1.Header.ChainID, + types.NewConsumerConsAddress(sdk.ConsAddress(v.Address.Bytes())), + ) + err := k.SlashValidator(ctx, providerAddr) + if err != nil { + logger.Error("failed to slash validator: %s", err) + continue + } + err = k.JailAndTombstoneValidator(ctx, providerAddr) + // JailAndTombstoneValidator should never return an error if + // SlashValidator succeeded because both methods fail if the malicious + // validator is either or both !found, unbonded and tombstoned. + if err != nil { + panic(err) + } + + provAddrs = append(provAddrs, providerAddr) + } + + // Return an error if no validators were punished + if len(provAddrs) == 0 { + return fmt.Errorf("failed to slash, jail, or tombstone all validators: %v", byzantineValidators) + } + + logger.Info( + "confirmed equivocation light client attack", + "byzantine validators slashed, jailed and tombstoned", provAddrs, + ) + + return nil +} + +// GetByzantineValidators returns the validators that signed both headers. +// If the misbehavior is an equivocation light client attack, then these +// validators are the Byzantine validators. +func (k Keeper) GetByzantineValidators(ctx sdk.Context, misbehaviour ibctmtypes.Misbehaviour) (validators []*tmtypes.Validator, err error) { + // construct the trusted and conflicted light blocks + lightBlock1, err := headerToLightBlock(*misbehaviour.Header1) + if err != nil { + return validators, err + } + lightBlock2, err := headerToLightBlock(*misbehaviour.Header2) + if err != nil { + return validators, err + } + + // Check if the misbehaviour corresponds to an Amnesia attack, + // meaning that the conflicting headers have both valid state transitions + // and different commit rounds. In this case, we return no validators as + // we can't identify the byzantine validators. + // + // Note that we cannot differentiate which of the headers is trusted or malicious, + if !headersStateTransitionsAreConflicting(*lightBlock1.Header, *lightBlock2.Header) && lightBlock1.Commit.Round != lightBlock2.Commit.Round { + return validators, nil + } + + // compare the signatures of the headers + // and return the intersection of validators who signed both + + // create a map with the validators' address that signed header1 + header1Signers := map[string]int{} + for idx, sign := range lightBlock1.Commit.Signatures { + if sign.Absent() { + continue + } + header1Signers[sign.ValidatorAddress.String()] = idx + } + + // iterate over the header2 signers and check if they signed header1 + for sigIdxHeader2, sign := range lightBlock2.Commit.Signatures { + if sign.Absent() { + continue + } + if sigIdxHeader1, ok := header1Signers[sign.ValidatorAddress.String()]; ok { + if err := verifyLightBlockCommitSig(*lightBlock1, sigIdxHeader1); err != nil { + return nil, err + } + + if err := verifyLightBlockCommitSig(*lightBlock2, sigIdxHeader2); err != nil { + return nil, err + } + + _, val := lightBlock1.ValidatorSet.GetByAddress(sign.ValidatorAddress) + validators = append(validators, val) + } + } + + return validators, nil +} + +// headerToLightBlock returns a CometBFT light block from the given IBC header +func headerToLightBlock(h ibctmtypes.Header) (*tmtypes.LightBlock, error) { + sh, err := tmtypes.SignedHeaderFromProto(h.SignedHeader) + if err != nil { + return nil, err + } + + vs, err := tmtypes.ValidatorSetFromProto(h.ValidatorSet) + if err != nil { + return nil, err + } + + return &tmtypes.LightBlock{ + SignedHeader: sh, + ValidatorSet: vs, + }, nil +} + +// CheckMisbehaviour checks that headers in the given misbehaviour forms +// a valid light client attack on a light client that tracks an ICS consumer chain +func (k Keeper) CheckMisbehaviour(ctx sdk.Context, misbehaviour ibctmtypes.Misbehaviour) error { + // check that the misbehaviour is for an ICS consumer chain + clientId, found := k.GetConsumerClientId(ctx, misbehaviour.Header1.Header.ChainID) + if !found { + return fmt.Errorf("incorrect misbehaviour with conflicting headers from a non-existent consumer chain: %s", misbehaviour.Header1.Header.ChainID) + } else if misbehaviour.ClientId != clientId { + return fmt.Errorf("incorrect misbehaviour: expected client ID for consumer chain %s is %s got %s", + misbehaviour.Header1.Header.ChainID, + clientId, + misbehaviour.ClientId, + ) + } + + clientState, found := k.clientKeeper.GetClientState(ctx, clientId) + if !found { + return errorsmod.Wrapf(ibcclienttypes.ErrClientNotFound, "cannot find client state for client with ID %s", clientId) + } + + clientStore := k.clientKeeper.ClientStore(ctx, clientId) + + // Check that the headers are at the same height to ensure that + // the misbehaviour is for a light client attack and not a time violation, + // see CheckForMisbehaviour in ibc-go/blob/v7.3.0/modules/light-clients/07-tendermint/misbehaviour_handle.go#L73 + if !misbehaviour.Header1.GetHeight().EQ(misbehaviour.Header2.GetHeight()) { + return errorsmod.Wrap(ibcclienttypes.ErrInvalidMisbehaviour, "headers are not at same height") + } + + // CheckForMisbehaviour verifies that the headers have different blockID hashes + ok := clientState.CheckForMisbehaviour(ctx, k.cdc, clientStore, &misbehaviour) + if !ok { + return errorsmod.Wrapf(ibcclienttypes.ErrInvalidMisbehaviour, "invalid misbehaviour for client-id: %s", misbehaviour.ClientId) + } + + // VerifyClientMessage calls verifyMisbehaviour which verifies that the headers in the misbehaviour + // are valid against their respective trusted consensus states and that at least a TrustLevel of the validator set signed their commit, + // see checkMisbehaviourHeader in ibc-go/blob/v7.3.0/modules/light-clients/07-tendermint/misbehaviour_handle.go#L126 + if err := clientState.VerifyClientMessage(ctx, k.cdc, clientStore, &misbehaviour); err != nil { + return err + } + + return nil +} + +// Check if the given block headers have conflicting state transitions. +// Note that this method was copied from ConflictingHeaderIsInvalid in CometBFT, +// see https://github.com/cometbft/cometbft/blob/v0.34.27/types/evidence.go#L285 +func headersStateTransitionsAreConflicting(h1, h2 tmtypes.Header) bool { + return !bytes.Equal(h1.ValidatorsHash, h2.ValidatorsHash) || + !bytes.Equal(h1.NextValidatorsHash, h2.NextValidatorsHash) || + !bytes.Equal(h1.ConsensusHash, h2.ConsensusHash) || + !bytes.Equal(h1.AppHash, h2.AppHash) || + !bytes.Equal(h1.LastResultsHash, h2.LastResultsHash) +} + +func verifyLightBlockCommitSig(lightBlock tmtypes.LightBlock, sigIdx int) error { + // get signature + sig := lightBlock.Commit.Signatures[sigIdx] + + // get validator + idx, val := lightBlock.ValidatorSet.GetByAddress(sig.ValidatorAddress) + if idx == -1 { + return fmt.Errorf("incorrect signature: validator address %s isn't part of the validator set", sig.ValidatorAddress.String()) + } + + // verify validator pubkey corresponds to signature validator address + if !bytes.Equal(val.PubKey.Address(), sig.ValidatorAddress) { + return fmt.Errorf("validator public key doesn't correspond to signature validator address: %s!= %s", val.PubKey.Address(), sig.ValidatorAddress) + } + + // validate signature + voteSignBytes := lightBlock.Commit.VoteSignBytes(lightBlock.ChainID, int32(sigIdx)) + if !val.PubKey.VerifySignature(voteSignBytes, sig.Signature) { + return fmt.Errorf("wrong signature (#%d): %X", sigIdx, sig.Signature) + } + + return nil +} + +// +// Punish Validator section +// + +// JailAndTombstoneValidator jails and tombstones the validator with the given provider consensus address +func (k Keeper) JailAndTombstoneValidator(ctx sdk.Context, providerAddr types.ProviderConsAddress) error { + validator, found := k.stakingKeeper.GetValidatorByConsAddr(ctx, providerAddr.ToSdkConsAddr()) + if !found { + return errorsmod.Wrapf(slashingtypes.ErrNoValidatorForAddress, "provider consensus address: %s", providerAddr.String()) + } + + if validator.IsUnbonded() { + return fmt.Errorf("validator is unbonded. provider consensus address: %s", providerAddr.String()) + } + + if k.slashingKeeper.IsTombstoned(ctx, providerAddr.ToSdkConsAddr()) { + return fmt.Errorf("validator is tombstoned. provider consensus address: %s", providerAddr.String()) + } + + // jail validator if not already + if !validator.IsJailed() { + k.stakingKeeper.Jail(ctx, providerAddr.ToSdkConsAddr()) + } + + k.slashingKeeper.JailUntil(ctx, providerAddr.ToSdkConsAddr(), evidencetypes.DoubleSignJailEndTime) + + // Tombstone the validator so that we cannot slash the validator more than once + // Note that we cannot simply use the fact that a validator is jailed to avoid slashing more than once + // because then a validator could i) perform an equivocation, ii) get jailed (e.g., through downtime) + // and in such a case the validator would not get slashed when we call `SlashValidator`. + k.slashingKeeper.Tombstone(ctx, providerAddr.ToSdkConsAddr()) + + return nil +} + +// ComputePowerToSlash computes the power to be slashed based on the tokens in non-matured `undelegations` and +// `redelegations`, as well as the current `power` of the validator. +// Note that this method does not perform any slashing. +func (k Keeper) ComputePowerToSlash(ctx sdk.Context, validator stakingtypes.Validator, undelegations []stakingtypes.UnbondingDelegation, + redelegations []stakingtypes.Redelegation, power int64, powerReduction math.Int, +) int64 { + // compute the total numbers of tokens currently being undelegated + undelegationsInTokens := sdk.NewInt(0) + + // Note that we use a **cached** context to avoid any actual slashing of undelegations or redelegations. + cachedCtx, _ := ctx.CacheContext() + for _, u := range undelegations { + amountSlashed := k.stakingKeeper.SlashUnbondingDelegation(cachedCtx, u, 0, sdk.NewDec(1)) + undelegationsInTokens = undelegationsInTokens.Add(amountSlashed) + } + + // compute the total numbers of tokens currently being redelegated + redelegationsInTokens := sdk.NewInt(0) + for _, r := range redelegations { + amountSlashed := k.stakingKeeper.SlashRedelegation(cachedCtx, validator, r, 0, sdk.NewDec(1)) + redelegationsInTokens = redelegationsInTokens.Add(amountSlashed) + } + + // The power we pass to staking's keeper `Slash` method is the current power of the validator together with the total + // power of all the currently undelegated and redelegated tokens (see docs/docs/adrs/adr-013-equivocation-slashing.md). + undelegationsAndRedelegationsInPower := sdk.TokensToConsensusPower( + undelegationsInTokens.Add(redelegationsInTokens), powerReduction) + + return power + undelegationsAndRedelegationsInPower +} + +// SlashValidator slashes validator with given provider Address +func (k Keeper) SlashValidator(ctx sdk.Context, providerAddr types.ProviderConsAddress) error { + validator, found := k.stakingKeeper.GetValidatorByConsAddr(ctx, providerAddr.ToSdkConsAddr()) + if !found { + return errorsmod.Wrapf(slashingtypes.ErrNoValidatorForAddress, "provider consensus address: %s", providerAddr.String()) + } + + if validator.IsUnbonded() { + return fmt.Errorf("validator is unbonded. provider consensus address: %s", providerAddr.String()) + } + + if k.slashingKeeper.IsTombstoned(ctx, providerAddr.ToSdkConsAddr()) { + return fmt.Errorf("validator is tombstoned. provider consensus address: %s", providerAddr.String()) + } + + undelegations := k.stakingKeeper.GetUnbondingDelegationsFromValidator(ctx, validator.GetOperator()) + redelegations := k.stakingKeeper.GetRedelegationsFromSrcValidator(ctx, validator.GetOperator()) + lastPower := k.stakingKeeper.GetLastValidatorPower(ctx, validator.GetOperator()) + powerReduction := k.stakingKeeper.PowerReduction(ctx) + totalPower := k.ComputePowerToSlash(ctx, validator, undelegations, redelegations, lastPower, powerReduction) + slashFraction := k.slashingKeeper.SlashFractionDoubleSign(ctx) + + consAdrr, err := validator.GetConsAddr() + if err != nil { + panic(err) + } + + k.stakingKeeper.SlashWithInfractionReason(ctx, consAdrr, 0, totalPower, slashFraction, stakingtypes.Infraction_INFRACTION_DOUBLE_SIGN) + + return nil +} + +// +// CRUD section +// + +// SetEquivocationEvidenceMinHeight sets the the minimum height +// of a valid consumer equivocation evidence for a given consumer chain ID +func (k Keeper) SetEquivocationEvidenceMinHeight(ctx sdk.Context, chainID string, height uint64) { + store := ctx.KVStore(k.storeKey) + heightBytes := make([]byte, 8) + binary.BigEndian.PutUint64(heightBytes, height) + + store.Set(types.EquivocationEvidenceMinHeightKey(chainID), heightBytes) +} + +// GetEquivocationEvidenceMinHeight returns the the minimum height +// of a valid consumer equivocation evidence for a given consumer chain ID +func (k Keeper) GetEquivocationEvidenceMinHeight(ctx sdk.Context, chainID string) uint64 { + store := ctx.KVStore(k.storeKey) + bz := store.Get(types.EquivocationEvidenceMinHeightKey(chainID)) + if bz == nil { + return 0 + } + + return binary.BigEndian.Uint64(bz) +} + +// DeleteEquivocationEvidenceMinHeight deletes the the minimum height +// of a valid consumer equivocation evidence for a given consumer chain ID +func (k Keeper) DeleteEquivocationEvidenceMinHeight(ctx sdk.Context, chainID string) { + store := ctx.KVStore(k.storeKey) + store.Delete(types.EquivocationEvidenceMinHeightKey(chainID)) +} diff --git a/x/ccv/provider/keeper/punish_validator_test.go b/x/ccv/provider/keeper/consumer_equivocation_test.go similarity index 72% rename from x/ccv/provider/keeper/punish_validator_test.go rename to x/ccv/provider/keeper/consumer_equivocation_test.go index 63e7ab9e86..f7a736dff1 100644 --- a/x/ccv/provider/keeper/punish_validator_test.go +++ b/x/ccv/provider/keeper/consumer_equivocation_test.go @@ -11,6 +11,7 @@ import ( "cosmossdk.io/math" cryptocodec "github.com/cosmos/cosmos-sdk/crypto/codec" + cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" sdk "github.com/cosmos/cosmos-sdk/types" evidencetypes "github.com/cosmos/cosmos-sdk/x/evidence/types" stakingtypes "github.com/cosmos/cosmos-sdk/x/staking/types" @@ -22,6 +23,302 @@ import ( "github.com/cosmos/interchain-security/v3/x/ccv/provider/types" ) +func TestVerifyDoubleVotingEvidence(t *testing.T) { + keeper, ctx, ctrl, _ := testkeeper.GetProviderKeeperAndCtx(t, testkeeper.NewInMemKeeperParams(t)) + defer ctrl.Finish() + + chainID := "consumer" + + signer1 := tmtypes.NewMockPV() + signer2 := tmtypes.NewMockPV() + + val1 := tmtypes.NewValidator(signer1.PrivKey.PubKey(), 1) + val2 := tmtypes.NewValidator(signer2.PrivKey.PubKey(), 1) + + valSet := tmtypes.NewValidatorSet([]*tmtypes.Validator{val1, val2}) + + blockID1 := cryptotestutil.MakeBlockID([]byte("blockhash"), 1000, []byte("partshash")) + blockID2 := cryptotestutil.MakeBlockID([]byte("blockhash2"), 1000, []byte("partshash")) + + ctx = ctx.WithBlockTime(time.Now()) + + valPubkey1, err := cryptocodec.FromTmPubKeyInterface(val1.PubKey) + require.NoError(t, err) + + valPubkey2, err := cryptocodec.FromTmPubKeyInterface(val2.PubKey) + require.NoError(t, err) + + testCases := []struct { + name string + votes []*tmtypes.Vote + chainID string + pubkey cryptotypes.PubKey + expPass bool + }{ + { + "invalid verifying public key - shouldn't pass", + []*tmtypes.Vote{ + cryptotestutil.MakeAndSignVote( + blockID1, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + cryptotestutil.MakeAndSignVote( + blockID2, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + }, + chainID, + nil, + false, + }, + { + "verifying public key doesn't correspond to validator address", + []*tmtypes.Vote{ + cryptotestutil.MakeAndSignVoteWithForgedValAddress( + blockID1, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + signer2, + chainID, + ), + cryptotestutil.MakeAndSignVoteWithForgedValAddress( + blockID2, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + signer2, + chainID, + ), + }, + chainID, + valPubkey1, + false, + }, + { + "evidence has votes with different block height - shouldn't pass", + []*tmtypes.Vote{ + cryptotestutil.MakeAndSignVote( + blockID1, + ctx.BlockHeight()+1, + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + cryptotestutil.MakeAndSignVote( + blockID2, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + }, + chainID, + valPubkey1, + false, + }, + { + "evidence has votes with different validator address - shouldn't pass", + []*tmtypes.Vote{ + cryptotestutil.MakeAndSignVote( + blockID1, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + cryptotestutil.MakeAndSignVote( + blockID2, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer2, + chainID, + ), + }, + chainID, + valPubkey1, + false, + }, + { + "evidence has votes with same block IDs - shouldn't pass", + []*tmtypes.Vote{ + cryptotestutil.MakeAndSignVote( + blockID1, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + cryptotestutil.MakeAndSignVote( + blockID1, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + }, + chainID, + valPubkey1, + false, + }, + { + "given chain ID isn't the same as the one used to sign the votes - shouldn't pass", + []*tmtypes.Vote{ + cryptotestutil.MakeAndSignVote( + blockID1, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + cryptotestutil.MakeAndSignVote( + blockID2, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + }, + "WrongChainID", + valPubkey1, + false, + }, + { + "voteA is signed using the wrong chain ID - shouldn't pass", + []*tmtypes.Vote{ + cryptotestutil.MakeAndSignVote( + blockID1, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + "WrongChainID", + ), + cryptotestutil.MakeAndSignVote( + blockID2, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + }, + chainID, + valPubkey1, + false, + }, + { + "voteB is signed using the wrong chain ID - shouldn't pass", + []*tmtypes.Vote{ + cryptotestutil.MakeAndSignVote( + blockID1, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + cryptotestutil.MakeAndSignVote( + blockID2, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + "WrongChainID", + ), + }, + chainID, + valPubkey1, + false, + }, + { + "wrong public key - shouldn't pass", + []*tmtypes.Vote{ + cryptotestutil.MakeAndSignVote( + blockID1, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + cryptotestutil.MakeAndSignVote( + blockID2, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + }, + chainID, + valPubkey2, + false, + }, + { + "valid double voting evidence should pass", + []*tmtypes.Vote{ + cryptotestutil.MakeAndSignVote( + blockID1, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + cryptotestutil.MakeAndSignVote( + blockID2, + ctx.BlockHeight(), + ctx.BlockTime(), + valSet, + signer1, + chainID, + ), + }, + chainID, + valPubkey1, + true, + }, + } + + for _, tc := range testCases { + err = keeper.VerifyDoubleVotingEvidence( + tmtypes.DuplicateVoteEvidence{ + VoteA: tc.votes[0], + VoteB: tc.votes[1], + ValidatorPower: val1.VotingPower, + TotalVotingPower: val1.VotingPower, + Timestamp: tc.votes[0].Timestamp, + }, + tc.chainID, + tc.pubkey, + ) + if tc.expPass { + require.NoError(t, err) + } else { + require.Error(t, err) + } + } +} + // TestJailAndTombstoneValidator tests that the jailing of a validator is only executed // under the conditions that the validator is neither unbonded, nor jailed, nor tombstoned. func TestJailAndTombstoneValidator(t *testing.T) { @@ -473,3 +770,21 @@ func TestSlashValidatorDoesNotSlashIfValidatorIsUnbonded(t *testing.T) { gomock.InOrder(expectedCalls...) keeper.SlashValidator(ctx, providerAddr) } + +func TestEquivocationEvidenceMinHeightCRUD(t *testing.T) { + chainID := consumer + expMinHeight := uint64(12) + keeper, ctx, ctrl, _ := testkeeper.GetProviderKeeperAndCtx(t, testkeeper.NewInMemKeeperParams(t)) + defer ctrl.Finish() + + height := keeper.GetEquivocationEvidenceMinHeight(ctx, chainID) + require.Zero(t, height, "equivocation evidence min height should be 0") + + keeper.SetEquivocationEvidenceMinHeight(ctx, chainID, expMinHeight) + height = keeper.GetEquivocationEvidenceMinHeight(ctx, chainID) + require.Equal(t, height, expMinHeight) + + keeper.DeleteEquivocationEvidenceMinHeight(ctx, chainID) + height = keeper.GetEquivocationEvidenceMinHeight(ctx, chainID) + require.Zero(t, height, "equivocation evidence min height should be 0") +} diff --git a/x/ccv/provider/keeper/double_vote.go b/x/ccv/provider/keeper/double_vote.go deleted file mode 100644 index d2ea4aba46..0000000000 --- a/x/ccv/provider/keeper/double_vote.go +++ /dev/null @@ -1,117 +0,0 @@ -package keeper - -import ( - "bytes" - "fmt" - - errorsmod "cosmossdk.io/errors" - - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - sdk "github.com/cosmos/cosmos-sdk/types" - - tmtypes "github.com/cometbft/cometbft/types" - - "github.com/cosmos/interchain-security/v3/x/ccv/provider/types" - ccvtypes "github.com/cosmos/interchain-security/v3/x/ccv/types" -) - -// HandleConsumerDoubleVoting verifies a double voting evidence for a given a consumer chain ID -// and a public key and, if successful, executes the slashing, jailing, and tombstoning of the malicious validator. -func (k Keeper) HandleConsumerDoubleVoting( - ctx sdk.Context, - evidence *tmtypes.DuplicateVoteEvidence, - chainID string, - pubkey cryptotypes.PubKey, -) error { - // verifies the double voting evidence using the consumer chain public key - if err := k.VerifyDoubleVotingEvidence(*evidence, chainID, pubkey); err != nil { - return err - } - - // get the validator's consensus address on the provider - providerAddr := k.GetProviderAddrFromConsumerAddr( - ctx, - chainID, - types.NewConsumerConsAddress(sdk.ConsAddress(evidence.VoteA.ValidatorAddress.Bytes())), - ) - - if err := k.SlashValidator(ctx, providerAddr); err != nil { - return err - } - if err := k.JailAndTombstoneValidator(ctx, providerAddr); err != nil { - return err - } - - k.Logger(ctx).Info( - "confirmed equivocation", - "byzantine validator address", providerAddr.String(), - ) - - return nil -} - -// VerifyDoubleVotingEvidence verifies a double voting evidence -// for a given chain id and a validator public key -func (k Keeper) VerifyDoubleVotingEvidence( - evidence tmtypes.DuplicateVoteEvidence, - chainID string, - pubkey cryptotypes.PubKey, -) error { - if pubkey == nil { - return fmt.Errorf("validator public key cannot be empty") - } - - // check that the validator address in the evidence is derived from the provided public key - if !bytes.Equal(pubkey.Address(), evidence.VoteA.ValidatorAddress) { - return errorsmod.Wrapf( - ccvtypes.ErrInvalidDoubleVotingEvidence, - "public key %s doesn't correspond to the validator address %s in double vote evidence", - pubkey.String(), evidence.VoteA.ValidatorAddress.String(), - ) - } - - // Note the age of the evidence isn't checked. - - // height/round/type must be the same - if evidence.VoteA.Height != evidence.VoteB.Height || - evidence.VoteA.Round != evidence.VoteB.Round || - evidence.VoteA.Type != evidence.VoteB.Type { - return errorsmod.Wrapf( - ccvtypes.ErrInvalidDoubleVotingEvidence, - "height/round/type are not the same: %d/%d/%v vs %d/%d/%v", - evidence.VoteA.Height, evidence.VoteA.Round, evidence.VoteA.Type, - evidence.VoteB.Height, evidence.VoteB.Round, evidence.VoteB.Type) - } - - // Addresses must be the same - if !bytes.Equal(evidence.VoteA.ValidatorAddress, evidence.VoteB.ValidatorAddress) { - return errorsmod.Wrapf( - ccvtypes.ErrInvalidDoubleVotingEvidence, - "validator addresses do not match: %X vs %X", - evidence.VoteA.ValidatorAddress, - evidence.VoteB.ValidatorAddress, - ) - } - - // BlockIDs must be different - if evidence.VoteA.BlockID.Equals(evidence.VoteB.BlockID) { - return errorsmod.Wrapf( - ccvtypes.ErrInvalidDoubleVotingEvidence, - "block IDs are the same (%v) - not a real duplicate vote", - evidence.VoteA.BlockID, - ) - } - - va := evidence.VoteA.ToProto() - vb := evidence.VoteB.ToProto() - - // signatures must be valid - if !pubkey.VerifySignature(tmtypes.VoteSignBytes(chainID, va), evidence.VoteA.Signature) { - return fmt.Errorf("verifying VoteA: %w", tmtypes.ErrVoteInvalidSignature) - } - if !pubkey.VerifySignature(tmtypes.VoteSignBytes(chainID, vb), evidence.VoteB.Signature) { - return fmt.Errorf("verifying VoteB: %w", tmtypes.ErrVoteInvalidSignature) - } - - return nil -} diff --git a/x/ccv/provider/keeper/double_vote_test.go b/x/ccv/provider/keeper/double_vote_test.go deleted file mode 100644 index 05ab238ccc..0000000000 --- a/x/ccv/provider/keeper/double_vote_test.go +++ /dev/null @@ -1,312 +0,0 @@ -package keeper_test - -import ( - "testing" - "time" - - "github.com/stretchr/testify/require" - - cryptocodec "github.com/cosmos/cosmos-sdk/crypto/codec" - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - - tmtypes "github.com/cometbft/cometbft/types" - - testutil "github.com/cosmos/interchain-security/v3/testutil/crypto" - testkeeper "github.com/cosmos/interchain-security/v3/testutil/keeper" -) - -func TestVerifyDoubleVotingEvidence(t *testing.T) { - keeper, ctx, ctrl, _ := testkeeper.GetProviderKeeperAndCtx(t, testkeeper.NewInMemKeeperParams(t)) - defer ctrl.Finish() - - chainID := "consumer" - - signer1 := tmtypes.NewMockPV() - signer2 := tmtypes.NewMockPV() - - val1 := tmtypes.NewValidator(signer1.PrivKey.PubKey(), 1) - val2 := tmtypes.NewValidator(signer2.PrivKey.PubKey(), 1) - - valSet := tmtypes.NewValidatorSet([]*tmtypes.Validator{val1, val2}) - - blockID1 := testutil.MakeBlockID([]byte("blockhash"), 1000, []byte("partshash")) - blockID2 := testutil.MakeBlockID([]byte("blockhash2"), 1000, []byte("partshash")) - - ctx = ctx.WithBlockTime(time.Now()) - - valPubkey1, err := cryptocodec.FromTmPubKeyInterface(val1.PubKey) - require.NoError(t, err) - - valPubkey2, err := cryptocodec.FromTmPubKeyInterface(val2.PubKey) - require.NoError(t, err) - - testCases := []struct { - name string - votes []*tmtypes.Vote - chainID string - pubkey cryptotypes.PubKey - expPass bool - }{ - { - "invalid verifying public key - shouldn't pass", - []*tmtypes.Vote{ - testutil.MakeAndSignVote( - blockID1, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - testutil.MakeAndSignVote( - blockID2, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - }, - chainID, - nil, - false, - }, - { - "verifying public key doesn't correspond to validator address", - []*tmtypes.Vote{ - testutil.MakeAndSignVoteWithForgedValAddress( - blockID1, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - signer2, - chainID, - ), - testutil.MakeAndSignVoteWithForgedValAddress( - blockID2, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - signer2, - chainID, - ), - }, - chainID, - valPubkey1, - false, - }, - { - "evidence has votes with different block height - shouldn't pass", - []*tmtypes.Vote{ - testutil.MakeAndSignVote( - blockID1, - ctx.BlockHeight()+1, - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - testutil.MakeAndSignVote( - blockID2, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - }, - chainID, - valPubkey1, - false, - }, - { - "evidence has votes with different validator address - shouldn't pass", - []*tmtypes.Vote{ - testutil.MakeAndSignVote( - blockID1, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - testutil.MakeAndSignVote( - blockID2, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer2, - chainID, - ), - }, - chainID, - valPubkey1, - false, - }, - { - "evidence has votes with same block IDs - shouldn't pass", - []*tmtypes.Vote{ - testutil.MakeAndSignVote( - blockID1, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - testutil.MakeAndSignVote( - blockID1, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - }, - chainID, - valPubkey1, - false, - }, - { - "given chain ID isn't the same as the one used to sign the votes - shouldn't pass", - []*tmtypes.Vote{ - testutil.MakeAndSignVote( - blockID1, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - testutil.MakeAndSignVote( - blockID2, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - }, - "WrongChainID", - valPubkey1, - false, - }, - { - "voteA is signed using the wrong chain ID - shouldn't pass", - []*tmtypes.Vote{ - testutil.MakeAndSignVote( - blockID1, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - "WrongChainID", - ), - testutil.MakeAndSignVote( - blockID2, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - }, - chainID, - valPubkey1, - false, - }, - { - "voteB is signed using the wrong chain ID - shouldn't pass", - []*tmtypes.Vote{ - testutil.MakeAndSignVote( - blockID1, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - testutil.MakeAndSignVote( - blockID2, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - "WrongChainID", - ), - }, - chainID, - valPubkey1, - false, - }, - { - "wrong public key - shouldn't pass", - []*tmtypes.Vote{ - testutil.MakeAndSignVote( - blockID1, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - testutil.MakeAndSignVote( - blockID2, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - }, - chainID, - valPubkey2, - false, - }, - { - "valid double voting evidence should pass", - []*tmtypes.Vote{ - testutil.MakeAndSignVote( - blockID1, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - testutil.MakeAndSignVote( - blockID2, - ctx.BlockHeight(), - ctx.BlockTime(), - valSet, - signer1, - chainID, - ), - }, - chainID, - valPubkey1, - true, - }, - } - - for _, tc := range testCases { - err = keeper.VerifyDoubleVotingEvidence( - tmtypes.DuplicateVoteEvidence{ - VoteA: tc.votes[0], - VoteB: tc.votes[1], - ValidatorPower: val1.VotingPower, - TotalVotingPower: val1.VotingPower, - Timestamp: tc.votes[0].Timestamp, - }, - tc.chainID, - tc.pubkey, - ) - if tc.expPass { - require.NoError(t, err) - } else { - require.Error(t, err) - } - } -} diff --git a/x/ccv/provider/keeper/misbehaviour.go b/x/ccv/provider/keeper/misbehaviour.go deleted file mode 100644 index 9a09f0682d..0000000000 --- a/x/ccv/provider/keeper/misbehaviour.go +++ /dev/null @@ -1,233 +0,0 @@ -package keeper - -import ( - "bytes" - "fmt" - - ibcclienttypes "github.com/cosmos/ibc-go/v7/modules/core/02-client/types" - ibctmtypes "github.com/cosmos/ibc-go/v7/modules/light-clients/07-tendermint" - - errorsmod "cosmossdk.io/errors" - - sdk "github.com/cosmos/cosmos-sdk/types" - - tmtypes "github.com/cometbft/cometbft/types" - - "github.com/cosmos/interchain-security/v3/x/ccv/provider/types" -) - -// HandleConsumerMisbehaviour checks if the given IBC misbehaviour corresponds to an equivocation light client attack, -// and in this case, slashes, jails, and tombstones -func (k Keeper) HandleConsumerMisbehaviour(ctx sdk.Context, misbehaviour ibctmtypes.Misbehaviour) error { - logger := k.Logger(ctx) - - // Check that the misbehaviour is valid and that the client consensus states at trusted heights are within trusting period - if err := k.CheckMisbehaviour(ctx, misbehaviour); err != nil { - logger.Info("Misbehaviour rejected", err.Error()) - - return err - } - - // Since the misbehaviour packet was received within the trusting period - // w.r.t to the trusted consensus states the infraction age - // isn't too old. see ibc-go/modules/light-clients/07-tendermint/types/misbehaviour_handle.go - - // Get Byzantine validators from the conflicting headers - byzantineValidators, err := k.GetByzantineValidators(ctx, misbehaviour) - if err != nil { - return err - } - - provAddrs := make([]types.ProviderConsAddress, len(byzantineValidators)) - - // slash, jail, and tombstone the Byzantine validators - for _, v := range byzantineValidators { - providerAddr := k.GetProviderAddrFromConsumerAddr( - ctx, - misbehaviour.Header1.Header.ChainID, - types.NewConsumerConsAddress(sdk.ConsAddress(v.Address.Bytes())), - ) - err := k.SlashValidator(ctx, providerAddr) - if err != nil { - logger.Error("failed to slash validator: %s", err) - continue - } - err = k.JailAndTombstoneValidator(ctx, providerAddr) - // JailAndTombstoneValidator should never return an error if - // SlashValidator succeeded because both methods fail if the malicious - // validator is either or both !found, unbonded and tombstoned. - if err != nil { - panic(err) - } - - provAddrs = append(provAddrs, providerAddr) - } - - // Return an error if no validators were punished - if len(provAddrs) == 0 { - return fmt.Errorf("failed to slash, jail, or tombstone all validators: %v", byzantineValidators) - } - - logger.Info( - "confirmed equivocation light client attack", - "byzantine validators slashed, jailed and tombstoned", provAddrs, - ) - - return nil -} - -// GetByzantineValidators returns the validators that signed both headers. -// If the misbehavior is an equivocation light client attack, then these -// validators are the Byzantine validators. -func (k Keeper) GetByzantineValidators(ctx sdk.Context, misbehaviour ibctmtypes.Misbehaviour) (validators []*tmtypes.Validator, err error) { - // construct the trusted and conflicted light blocks - lightBlock1, err := headerToLightBlock(*misbehaviour.Header1) - if err != nil { - return validators, err - } - lightBlock2, err := headerToLightBlock(*misbehaviour.Header2) - if err != nil { - return validators, err - } - - // Check if the misbehaviour corresponds to an Amnesia attack, - // meaning that the conflicting headers have both valid state transitions - // and different commit rounds. In this case, we return no validators as - // we can't identify the byzantine validators. - // - // Note that we cannot differentiate which of the headers is trusted or malicious, - if !headersStateTransitionsAreConflicting(*lightBlock1.Header, *lightBlock2.Header) && lightBlock1.Commit.Round != lightBlock2.Commit.Round { - return validators, nil - } - - // compare the signatures of the headers - // and return the intersection of validators who signed both - - // create a map with the validators' address that signed header1 - header1Signers := map[string]int{} - for idx, sign := range lightBlock1.Commit.Signatures { - if sign.Absent() { - continue - } - header1Signers[sign.ValidatorAddress.String()] = idx - } - - // iterate over the header2 signers and check if they signed header1 - for sigIdxHeader2, sign := range lightBlock2.Commit.Signatures { - if sign.Absent() { - continue - } - if sigIdxHeader1, ok := header1Signers[sign.ValidatorAddress.String()]; ok { - if err := verifyLightBlockCommitSig(*lightBlock1, sigIdxHeader1); err != nil { - return nil, err - } - - if err := verifyLightBlockCommitSig(*lightBlock2, sigIdxHeader2); err != nil { - return nil, err - } - - _, val := lightBlock1.ValidatorSet.GetByAddress(sign.ValidatorAddress) - validators = append(validators, val) - } - } - - return validators, nil -} - -// headerToLightBlock returns a CometBFT light block from the given IBC header -func headerToLightBlock(h ibctmtypes.Header) (*tmtypes.LightBlock, error) { - sh, err := tmtypes.SignedHeaderFromProto(h.SignedHeader) - if err != nil { - return nil, err - } - - vs, err := tmtypes.ValidatorSetFromProto(h.ValidatorSet) - if err != nil { - return nil, err - } - - return &tmtypes.LightBlock{ - SignedHeader: sh, - ValidatorSet: vs, - }, nil -} - -// CheckMisbehaviour checks that headers in the given misbehaviour forms -// a valid light client attack on a light client that tracks an ICS consumer chain -func (k Keeper) CheckMisbehaviour(ctx sdk.Context, misbehaviour ibctmtypes.Misbehaviour) error { - // check that the misbehaviour is for an ICS consumer chain - clientId, found := k.GetConsumerClientId(ctx, misbehaviour.Header1.Header.ChainID) - if !found { - return fmt.Errorf("incorrect misbehaviour with conflicting headers from a non-existent consumer chain: %s", misbehaviour.Header1.Header.ChainID) - } else if misbehaviour.ClientId != clientId { - return fmt.Errorf("incorrect misbehaviour: expected client ID for consumer chain %s is %s got %s", - misbehaviour.Header1.Header.ChainID, - clientId, - misbehaviour.ClientId, - ) - } - - clientState, found := k.clientKeeper.GetClientState(ctx, clientId) - if !found { - return errorsmod.Wrapf(ibcclienttypes.ErrClientNotFound, "cannot find client state for client with ID %s", clientId) - } - - clientStore := k.clientKeeper.ClientStore(ctx, clientId) - - // Check that the headers are at the same height to ensure that - // the misbehaviour is for a light client attack and not a time violation, - // see CheckForMisbehaviour in ibc-go/blob/v7.3.0/modules/light-clients/07-tendermint/misbehaviour_handle.go#L73 - if !misbehaviour.Header1.GetHeight().EQ(misbehaviour.Header2.GetHeight()) { - return errorsmod.Wrap(ibcclienttypes.ErrInvalidMisbehaviour, "headers are not at same height") - } - - // CheckForMisbehaviour verifies that the headers have different blockID hashes - ok := clientState.CheckForMisbehaviour(ctx, k.cdc, clientStore, &misbehaviour) - if !ok { - return errorsmod.Wrapf(ibcclienttypes.ErrInvalidMisbehaviour, "invalid misbehaviour for client-id: %s", misbehaviour.ClientId) - } - - // VerifyClientMessage calls verifyMisbehaviour which verifies that the headers in the misbehaviour - // are valid against their respective trusted consensus states and that at least a TrustLevel of the validator set signed their commit, - // see checkMisbehaviourHeader in ibc-go/blob/v7.3.0/modules/light-clients/07-tendermint/misbehaviour_handle.go#L126 - if err := clientState.VerifyClientMessage(ctx, k.cdc, clientStore, &misbehaviour); err != nil { - return err - } - - return nil -} - -// Check if the given block headers have conflicting state transitions. -// Note that this method was copied from ConflictingHeaderIsInvalid in CometBFT, -// see https://github.com/cometbft/cometbft/blob/v0.34.27/types/evidence.go#L285 -func headersStateTransitionsAreConflicting(h1, h2 tmtypes.Header) bool { - return !bytes.Equal(h1.ValidatorsHash, h2.ValidatorsHash) || - !bytes.Equal(h1.NextValidatorsHash, h2.NextValidatorsHash) || - !bytes.Equal(h1.ConsensusHash, h2.ConsensusHash) || - !bytes.Equal(h1.AppHash, h2.AppHash) || - !bytes.Equal(h1.LastResultsHash, h2.LastResultsHash) -} - -func verifyLightBlockCommitSig(lightBlock tmtypes.LightBlock, sigIdx int) error { - // get signature - sig := lightBlock.Commit.Signatures[sigIdx] - - // get validator - idx, val := lightBlock.ValidatorSet.GetByAddress(sig.ValidatorAddress) - if idx == -1 { - return fmt.Errorf("incorrect signature: validator address %s isn't part of the validator set", sig.ValidatorAddress.String()) - } - - // verify validator pubkey corresponds to signature validator address - if !bytes.Equal(val.PubKey.Address(), sig.ValidatorAddress) { - return fmt.Errorf("validator public key doesn't correspond to signature validator address: %s!= %s", val.PubKey.Address(), sig.ValidatorAddress) - } - - // validate signature - voteSignBytes := lightBlock.Commit.VoteSignBytes(lightBlock.ChainID, int32(sigIdx)) - if !val.PubKey.VerifySignature(voteSignBytes, sig.Signature) { - return fmt.Errorf("wrong signature (#%d): %X", sigIdx, sig.Signature) - } - - return nil -} diff --git a/x/ccv/provider/keeper/punish_validator.go b/x/ccv/provider/keeper/punish_validator.go deleted file mode 100644 index f991c7cc5f..0000000000 --- a/x/ccv/provider/keeper/punish_validator.go +++ /dev/null @@ -1,109 +0,0 @@ -package keeper - -import ( - "fmt" - - errorsmod "cosmossdk.io/errors" - "cosmossdk.io/math" - - sdk "github.com/cosmos/cosmos-sdk/types" - evidencetypes "github.com/cosmos/cosmos-sdk/x/evidence/types" - slashingtypes "github.com/cosmos/cosmos-sdk/x/slashing/types" - stakingtypes "github.com/cosmos/cosmos-sdk/x/staking/types" - - "github.com/cosmos/interchain-security/v3/x/ccv/provider/types" -) - -// JailAndTombstoneValidator jails and tombstones the validator with the given provider consensus address -func (k Keeper) JailAndTombstoneValidator(ctx sdk.Context, providerAddr types.ProviderConsAddress) error { - validator, found := k.stakingKeeper.GetValidatorByConsAddr(ctx, providerAddr.ToSdkConsAddr()) - if !found { - return errorsmod.Wrapf(slashingtypes.ErrNoValidatorForAddress, "provider consensus address: %s", providerAddr.String()) - } - - if validator.IsUnbonded() { - return fmt.Errorf("validator is unbonded. provider consensus address: %s", providerAddr.String()) - } - - if k.slashingKeeper.IsTombstoned(ctx, providerAddr.ToSdkConsAddr()) { - return fmt.Errorf("validator is tombstoned. provider consensus address: %s", providerAddr.String()) - } - - // jail validator if not already - if !validator.IsJailed() { - k.stakingKeeper.Jail(ctx, providerAddr.ToSdkConsAddr()) - } - - k.slashingKeeper.JailUntil(ctx, providerAddr.ToSdkConsAddr(), evidencetypes.DoubleSignJailEndTime) - - // Tombstone the validator so that we cannot slash the validator more than once - // Note that we cannot simply use the fact that a validator is jailed to avoid slashing more than once - // because then a validator could i) perform an equivocation, ii) get jailed (e.g., through downtime) - // and in such a case the validator would not get slashed when we call `SlashValidator`. - k.slashingKeeper.Tombstone(ctx, providerAddr.ToSdkConsAddr()) - - return nil -} - -// ComputePowerToSlash computes the power to be slashed based on the tokens in non-matured `undelegations` and -// `redelegations`, as well as the current `power` of the validator. -// Note that this method does not perform any slashing. -func (k Keeper) ComputePowerToSlash(ctx sdk.Context, validator stakingtypes.Validator, undelegations []stakingtypes.UnbondingDelegation, - redelegations []stakingtypes.Redelegation, power int64, powerReduction math.Int, -) int64 { - // compute the total numbers of tokens currently being undelegated - undelegationsInTokens := sdk.NewInt(0) - - // Note that we use a **cached** context to avoid any actual slashing of undelegations or redelegations. - cachedCtx, _ := ctx.CacheContext() - for _, u := range undelegations { - amountSlashed := k.stakingKeeper.SlashUnbondingDelegation(cachedCtx, u, 0, sdk.NewDec(1)) - undelegationsInTokens = undelegationsInTokens.Add(amountSlashed) - } - - // compute the total numbers of tokens currently being redelegated - redelegationsInTokens := sdk.NewInt(0) - for _, r := range redelegations { - amountSlashed := k.stakingKeeper.SlashRedelegation(cachedCtx, validator, r, 0, sdk.NewDec(1)) - redelegationsInTokens = redelegationsInTokens.Add(amountSlashed) - } - - // The power we pass to staking's keeper `Slash` method is the current power of the validator together with the total - // power of all the currently undelegated and redelegated tokens (see docs/docs/adrs/adr-013-equivocation-slashing.md). - undelegationsAndRedelegationsInPower := sdk.TokensToConsensusPower( - undelegationsInTokens.Add(redelegationsInTokens), powerReduction) - - return power + undelegationsAndRedelegationsInPower -} - -// SlashValidator slashes validator with given provider Address -func (k Keeper) SlashValidator(ctx sdk.Context, providerAddr types.ProviderConsAddress) error { - validator, found := k.stakingKeeper.GetValidatorByConsAddr(ctx, providerAddr.ToSdkConsAddr()) - if !found { - return errorsmod.Wrapf(slashingtypes.ErrNoValidatorForAddress, "provider consensus address: %s", providerAddr.String()) - } - - if validator.IsUnbonded() { - return fmt.Errorf("validator is unbonded. provider consensus address: %s", providerAddr.String()) - } - - if k.slashingKeeper.IsTombstoned(ctx, providerAddr.ToSdkConsAddr()) { - return fmt.Errorf("validator is tombstoned. provider consensus address: %s", providerAddr.String()) - } - - undelegations := k.stakingKeeper.GetUnbondingDelegationsFromValidator(ctx, validator.GetOperator()) - redelegations := k.stakingKeeper.GetRedelegationsFromSrcValidator(ctx, validator.GetOperator()) - lastPower := k.stakingKeeper.GetLastValidatorPower(ctx, validator.GetOperator()) - powerReduction := k.stakingKeeper.PowerReduction(ctx) - totalPower := k.ComputePowerToSlash(ctx, validator, undelegations, redelegations, lastPower, powerReduction) - slashFraction := k.slashingKeeper.SlashFractionDoubleSign(ctx) - - consAdrr, err := validator.GetConsAddr() - if err != nil { - panic(err) - } - - k.stakingKeeper.SlashWithInfractionReason(ctx, consAdrr, 0, totalPower, slashFraction, stakingtypes.Infraction_INFRACTION_DOUBLE_SIGN) - - return nil -} diff --git a/x/ccv/provider/types/keys.go b/x/ccv/provider/types/keys.go index ee4c11015b..95b5b4b820 100644 --- a/x/ccv/provider/types/keys.go +++ b/x/ccv/provider/types/keys.go @@ -138,6 +138,10 @@ const ( // handled in the current block VSCMaturedHandledThisBlockBytePrefix + // EquivocationEvidenceMinHeightBytePrefix is the byte prefix storing the mapping from consumer chain IDs + // to the minimum height of a valid consumer equivocation evidence + EquivocationEvidenceMinHeightBytePrefix + // NOTE: DO NOT ADD NEW BYTE PREFIXES HERE WITHOUT ADDING THEM TO getAllKeyPrefixes() IN keys_test.go ) @@ -377,6 +381,12 @@ func ConsumerRewardDenomsKey(denom string) []byte { return append([]byte{ConsumerRewardDenomsBytePrefix}, []byte(denom)...) } +// EquivocationEvidenceMinHeightKey returns the key storing the minimum height +// of a valid consumer equivocation evidence for a given consumer chain ID +func EquivocationEvidenceMinHeightKey(consumerChainID string) []byte { + return append([]byte{EquivocationEvidenceMinHeightBytePrefix}, []byte(consumerChainID)...) +} + // NOTE: DO NOT ADD FULLY DEFINED KEY FUNCTIONS WITHOUT ADDING THEM TO getAllFullyDefinedKeys() IN keys_test.go // diff --git a/x/ccv/provider/types/keys_test.go b/x/ccv/provider/types/keys_test.go index 9f470f4a82..6b263a4fdb 100644 --- a/x/ccv/provider/types/keys_test.go +++ b/x/ccv/provider/types/keys_test.go @@ -54,6 +54,7 @@ func getAllKeyPrefixes() []byte { providertypes.ConsumerAddrsToPruneBytePrefix, providertypes.SlashLogBytePrefix, providertypes.VSCMaturedHandledThisBlockBytePrefix, + providertypes.EquivocationEvidenceMinHeightBytePrefix, } } @@ -98,6 +99,7 @@ func getAllFullyDefinedKeys() [][]byte { providertypes.ConsumerAddrsToPruneKey("chainID", 88), providertypes.SlashLogKey(providertypes.NewProviderConsAddress([]byte{0x05})), providertypes.VSCMaturedHandledThisBlockKey(), + providertypes.EquivocationEvidenceMinHeightKey("chainID"), } }