diff --git a/tests/integration/staking/keeper/unbonding_test.go b/tests/integration/staking/keeper/unbonding_test.go
index 26309b4417c6..f0ce867d8acf 100644
--- a/tests/integration/staking/keeper/unbonding_test.go
+++ b/tests/integration/staking/keeper/unbonding_test.go
@@ -44,6 +44,7 @@ func SetupUnbondingTests(t *testing.T, f *fixture, hookCalled *bool, ubdeID *uin
 	mockStackingHooks.EXPECT().BeforeDelegationSharesModified(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
 	mockStackingHooks.EXPECT().BeforeValidatorModified(gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
 	mockStackingHooks.EXPECT().BeforeValidatorSlashed(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
+	mockStackingHooks.EXPECT().AfterConsensusPubKeyUpdate(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
 	f.stakingKeeper.SetHooks(types.NewMultiStakingHooks(mockStackingHooks))
 
 	addrDels = simtestutil.AddTestAddrsIncremental(f.bankKeeper, f.stakingKeeper, f.sdkCtx, 2, math.NewInt(10000))
diff --git a/x/distribution/keeper/hooks.go b/x/distribution/keeper/hooks.go
index 924744f9035d..46605690c26f 100644
--- a/x/distribution/keeper/hooks.go
+++ b/x/distribution/keeper/hooks.go
@@ -9,6 +9,7 @@ import (
 	"cosmossdk.io/x/distribution/types"
 	stakingtypes "cosmossdk.io/x/staking/types"
 
+	cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types"
 	sdk "github.com/cosmos/cosmos-sdk/types"
 )
 
@@ -187,3 +188,7 @@ func (h Hooks) BeforeDelegationRemoved(_ context.Context, _ sdk.AccAddress, _ sd
 func (h Hooks) AfterUnbondingInitiated(_ context.Context, _ uint64) error {
 	return nil
 }
+
+func (h Hooks) AfterConsensusPubKeyUpdate(_ context.Context, _, _ cryptotypes.PubKey, _ sdk.Coin) error {
+	return nil
+}
diff --git a/x/evidence/keeper/infraction.go b/x/evidence/keeper/infraction.go
index f8979c708750..2cffa540f7d9 100644
--- a/x/evidence/keeper/infraction.go
+++ b/x/evidence/keeper/infraction.go
@@ -40,6 +40,15 @@ func (k Keeper) handleEquivocationEvidence(ctx context.Context, evidence *types.
 	}
 
 	if len(validator.GetOperator()) != 0 {
+		// Get the consAddr from the validator read from the store and not from the evidence,
+		// because if the validator has rotated its key, the key in evidence could be outdated.
+		// (ValidatorByConsAddr can get a validator even if the key has been rotated)
+		valConsAddr, err := validator.GetConsAddr()
+		if err != nil {
+			return err
+		}
+		consAddr = valConsAddr
+
 		if _, err := k.slashingKeeper.GetPubkey(ctx, consAddr.Bytes()); err != nil {
 			// Ignore evidence that cannot be handled.
 			//
diff --git a/x/slashing/keeper/hooks.go b/x/slashing/keeper/hooks.go
index a8a9b72df987..fb44c6a30d4d 100644
--- a/x/slashing/keeper/hooks.go
+++ b/x/slashing/keeper/hooks.go
@@ -9,6 +9,7 @@ import (
 	sdkmath "cosmossdk.io/math"
 	"cosmossdk.io/x/slashing/types"
 
+	cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types"
 	sdk "github.com/cosmos/cosmos-sdk/types"
 )
 
@@ -99,3 +100,16 @@ func (h Hooks) BeforeValidatorSlashed(_ context.Context, _ sdk.ValAddress, _ sdk
 func (h Hooks) AfterUnbondingInitiated(_ context.Context, _ uint64) error {
 	return nil
 }
+
+// AfterConsensusPubKeyUpdate triggers the functions to rotate the signing-infos also sets address pubkey relation.
+func (h Hooks) AfterConsensusPubKeyUpdate(ctx context.Context, oldPubKey, newPubKey cryptotypes.PubKey, _ sdk.Coin) error {
+	if err := h.k.performConsensusPubKeyUpdate(ctx, oldPubKey, newPubKey); err != nil {
+		return err
+	}
+
+	if err := h.k.AddrPubkeyRelation.Remove(ctx, oldPubKey.Address()); err != nil {
+		return err
+	}
+
+	return nil
+}
diff --git a/x/slashing/keeper/signing_info.go b/x/slashing/keeper/signing_info.go
index 51e258fea960..7f991fcf2483 100644
--- a/x/slashing/keeper/signing_info.go
+++ b/x/slashing/keeper/signing_info.go
@@ -12,6 +12,7 @@ import (
 	errorsmod "cosmossdk.io/errors"
 	"cosmossdk.io/x/slashing/types"
 
+	cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types"
 	sdk "github.com/cosmos/cosmos-sdk/types"
 )
 
@@ -75,6 +76,21 @@ func (k Keeper) SetMissedBlockBitmapChunk(ctx context.Context, addr sdk.ConsAddr
 	return k.ValidatorMissedBlockBitmap.Set(ctx, collections.Join(addr.Bytes(), uint64(chunkIndex)), chunk)
 }
 
+// getPreviousConsKey checks if the key rotated, returns the old consKey to get the missed blocks
+// because missed blocks are still pointing to the old key
+func (k Keeper) getPreviousConsKey(ctx context.Context, addr sdk.ConsAddress) (sdk.ConsAddress, error) {
+	oldPk, err := k.sk.ValidatorIdentifier(ctx, addr)
+	if err != nil {
+		return nil, err
+	}
+
+	if oldPk != nil {
+		return oldPk, nil
+	}
+
+	return addr, nil
+}
+
 // GetMissedBlockBitmapValue returns true if a validator missed signing a block
 // at the given index and false otherwise. The index provided is assumed to be
 // the index in the range [0, SignedBlocksWindow), which represents the bitmap
@@ -82,6 +98,13 @@ func (k Keeper) SetMissedBlockBitmapChunk(ctx context.Context, addr sdk.ConsAddr
 // IndexOffset modulo SignedBlocksWindow. This index is used to fetch the chunk
 // in the bitmap and the relative bit in that chunk.
 func (k Keeper) GetMissedBlockBitmapValue(ctx context.Context, addr sdk.ConsAddress, index int64) (bool, error) {
+	// check the key rotated, if rotated use the returned consKey to get the missed blocks
+	// because missed blocks are still pointing to the old key
+	addr, err := k.getPreviousConsKey(ctx, addr)
+	if err != nil {
+		return false, err
+	}
+
 	// get the chunk or "word" in the logical bitmap
 	chunkIndex := index / types.MissedBlockBitmapChunkSize
 
@@ -111,6 +134,13 @@ func (k Keeper) GetMissedBlockBitmapValue(ctx context.Context, addr sdk.ConsAddr
 // index is used to fetch the chunk in the bitmap and the relative bit in that
 // chunk.
 func (k Keeper) SetMissedBlockBitmapValue(ctx context.Context, addr sdk.ConsAddress, index int64, missed bool) error {
+	// check the key rotated, if rotated use the returned consKey to get the missed blocks
+	// because missed blocks are still pointing to the old key
+	addr, err := k.getPreviousConsKey(ctx, addr)
+	if err != nil {
+		return err
+	}
+
 	// get the chunk or "word" in the logical bitmap
 	chunkIndex := index / types.MissedBlockBitmapChunkSize
 
@@ -144,19 +174,21 @@ func (k Keeper) SetMissedBlockBitmapValue(ctx context.Context, addr sdk.ConsAddr
 
 // DeleteMissedBlockBitmap removes a validator's missed block bitmap from state.
 func (k Keeper) DeleteMissedBlockBitmap(ctx context.Context, addr sdk.ConsAddress) error {
+	// check the key rotated, if rotated use the returned consKey to delete the missed blocks
+	// because missed blocks are still pointing to the old key
+	addr, err := k.getPreviousConsKey(ctx, addr)
+	if err != nil {
+		return err
+	}
+
 	rng := collections.NewPrefixedPairRange[[]byte, uint64](addr.Bytes())
-	err := k.ValidatorMissedBlockBitmap.Walk(ctx, rng, func(key collections.Pair[[]byte, uint64], value []byte) (bool, error) {
+	return k.ValidatorMissedBlockBitmap.Walk(ctx, rng, func(key collections.Pair[[]byte, uint64], value []byte) (bool, error) {
 		err := k.ValidatorMissedBlockBitmap.Remove(ctx, key)
 		if err != nil {
 			return true, err
 		}
 		return false, nil
 	})
-	if err != nil {
-		return err
-	}
-
-	return nil
 }
 
 // IterateMissedBlockBitmap iterates over a validator's signed blocks window
@@ -168,7 +200,7 @@ func (k Keeper) DeleteMissedBlockBitmap(ctx context.Context, addr sdk.ConsAddres
 func (k Keeper) IterateMissedBlockBitmap(ctx context.Context, addr sdk.ConsAddress, cb func(index int64, missed bool) (stop bool)) error {
 	var index int64
 	rng := collections.NewPrefixedPairRange[[]byte, uint64](addr.Bytes())
-	err := k.ValidatorMissedBlockBitmap.Walk(ctx, rng, func(key collections.Pair[[]byte, uint64], value []byte) (bool, error) {
+	return k.ValidatorMissedBlockBitmap.Walk(ctx, rng, func(key collections.Pair[[]byte, uint64], value []byte) (bool, error) {
 		bs := bitset.New(uint(types.MissedBlockBitmapChunkSize))
 
 		if err := bs.UnmarshalBinary(value); err != nil {
@@ -185,10 +217,6 @@ func (k Keeper) IterateMissedBlockBitmap(ctx context.Context, addr sdk.ConsAddre
 		}
 		return false, nil
 	})
-	if err != nil {
-		return err
-	}
-	return nil
 }
 
 // GetValidatorMissedBlocks returns array of missed blocks for given validator.
@@ -209,3 +237,24 @@ func (k Keeper) GetValidatorMissedBlocks(ctx context.Context, addr sdk.ConsAddre
 
 	return missedBlocks, err
 }
+
+// performConsensusPubKeyUpdate updates cons address to its pub key relation
+// Updates signing info, missed blocks (removes old one, and sets new one)
+func (k Keeper) performConsensusPubKeyUpdate(ctx context.Context, oldPubKey, newPubKey cryptotypes.PubKey) error {
+	// Connect new consensus address with PubKey
+	if err := k.AddrPubkeyRelation.Set(ctx, newPubKey.Address(), newPubKey); err != nil {
+		return err
+	}
+
+	// Migrate ValidatorSigningInfo from oldPubKey to newPubKey
+	signingInfo, err := k.ValidatorSigningInfo.Get(ctx, sdk.ConsAddress(oldPubKey.Address()))
+	if err != nil {
+		return types.ErrInvalidConsPubKey.Wrap("failed to get signing info for old public key")
+	}
+
+	if err := k.ValidatorSigningInfo.Set(ctx, sdk.ConsAddress(newPubKey.Address()), signingInfo); err != nil {
+		return err
+	}
+
+	return k.ValidatorSigningInfo.Remove(ctx, sdk.ConsAddress(oldPubKey.Address()))
+}
diff --git a/x/slashing/keeper/signing_info_test.go b/x/slashing/keeper/signing_info_test.go
index d91564797441..13d88ca4cf78 100644
--- a/x/slashing/keeper/signing_info_test.go
+++ b/x/slashing/keeper/signing_info_test.go
@@ -3,6 +3,8 @@ package keeper_test
 import (
 	"time"
 
+	"github.com/golang/mock/gomock"
+
 	"cosmossdk.io/x/slashing/testutil"
 	slashingtypes "cosmossdk.io/x/slashing/types"
 
@@ -65,6 +67,8 @@ func (s *KeeperTestSuite) TestValidatorMissedBlockBitmap_SmallWindow() {
 		params.SignedBlocksWindow = window
 		require.NoError(keeper.Params.Set(ctx, params))
 
+		s.stakingKeeper.EXPECT().ValidatorIdentifier(gomock.Any(), consAddr).Return(consAddr, nil).AnyTimes()
+
 		// validator misses all blocks in the window
 		var valIdxOffset int64
 		for valIdxOffset < params.SignedBlocksWindow {
@@ -97,5 +101,13 @@ func (s *KeeperTestSuite) TestValidatorMissedBlockBitmap_SmallWindow() {
 		missedBlocks, err = keeper.GetValidatorMissedBlocks(ctx, consAddr)
 		require.NoError(err)
 		require.Len(missedBlocks, int(params.SignedBlocksWindow)-1)
+
+		// if the validator rotated it's key there will be different consKeys and a mapping will be added in the state.
+		consAddr1 := sdk.ConsAddress(sdk.AccAddress([]byte("addr1_______________")))
+		s.stakingKeeper.EXPECT().ValidatorIdentifier(gomock.Any(), consAddr1).Return(consAddr, nil).AnyTimes()
+
+		missedBlocks, err = keeper.GetValidatorMissedBlocks(ctx, consAddr1)
+		require.NoError(err)
+		require.Len(missedBlocks, int(params.SignedBlocksWindow)-1)
 	}
 }
diff --git a/x/slashing/testutil/expected_keepers_mocks.go b/x/slashing/testutil/expected_keepers_mocks.go
index eab397971dc9..ee9fdfebbe94 100644
--- a/x/slashing/testutil/expected_keepers_mocks.go
+++ b/x/slashing/testutil/expected_keepers_mocks.go
@@ -371,6 +371,21 @@ func (mr *MockStakingKeeperMockRecorder) ValidatorByConsAddr(arg0, arg1 interfac
 	return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ValidatorByConsAddr", reflect.TypeOf((*MockStakingKeeper)(nil).ValidatorByConsAddr), arg0, arg1)
 }
 
+// ValidatorIdentifier mocks base method.
+func (m *MockStakingKeeper) ValidatorIdentifier(arg0 context.Context, arg1 types0.ConsAddress) (types0.ConsAddress, error) {
+	m.ctrl.T.Helper()
+	ret := m.ctrl.Call(m, "ValidatorIdentifier", arg0, arg1)
+	ret0, _ := ret[0].(types0.ConsAddress)
+	ret1, _ := ret[1].(error)
+	return ret0, ret1
+}
+
+// ValidatorIdentifier indicates an expected call of ValidatorIdentifier.
+func (mr *MockStakingKeeperMockRecorder) ValidatorIdentifier(arg0, arg1 interface{}) *gomock.Call {
+	mr.mock.ctrl.T.Helper()
+	return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ValidatorIdentifier", reflect.TypeOf((*MockStakingKeeper)(nil).ValidatorIdentifier), arg0, arg1)
+}
+
 // MockStakingHooks is a mock of StakingHooks interface.
 type MockStakingHooks struct {
 	ctrl     *gomock.Controller
diff --git a/x/slashing/types/errors.go b/x/slashing/types/errors.go
index 4c26bc79fc83..4ab40d5cd8d2 100644
--- a/x/slashing/types/errors.go
+++ b/x/slashing/types/errors.go
@@ -13,4 +13,5 @@ var (
 	ErrNoSigningInfoFound           = errors.Register(ModuleName, 8, "no validator signing info found")
 	ErrValidatorTombstoned          = errors.Register(ModuleName, 9, "validator already tombstoned")
 	ErrInvalidSigner                = errors.Register(ModuleName, 10, "expected authority account as only signer for proposal message")
+	ErrInvalidConsPubKey            = errors.Register(ModuleName, 11, "invalid consensus pubkey")
 )
diff --git a/x/slashing/types/expected_keepers.go b/x/slashing/types/expected_keepers.go
index 86db9bff2582..d8c9754be182 100644
--- a/x/slashing/types/expected_keepers.go
+++ b/x/slashing/types/expected_keepers.go
@@ -53,6 +53,10 @@ type StakingKeeper interface {
 
 	// IsValidatorJailed returns if the validator is jailed.
 	IsValidatorJailed(ctx context.Context, addr sdk.ConsAddress) (bool, error)
+
+	// ValidatorIdentifier maps the new cons key to previous cons key (which is the address before the rotation).
+	// (that is: newConsKey -> oldConsKey)
+	ValidatorIdentifier(context.Context, sdk.ConsAddress) (sdk.ConsAddress, error)
 }
 
 // StakingHooks event hooks for staking validator object (noalias)
diff --git a/x/staking/keeper/cons_pubkey.go b/x/staking/keeper/cons_pubkey.go
index 690fcedbf196..c6a9763df328 100644
--- a/x/staking/keeper/cons_pubkey.go
+++ b/x/staking/keeper/cons_pubkey.go
@@ -3,13 +3,18 @@ package keeper
 import (
 	"bytes"
 	"context"
+	"errors"
 	"time"
 
 	"cosmossdk.io/collections"
+	"cosmossdk.io/collections/indexes"
+	errorsmod "cosmossdk.io/errors"
 	"cosmossdk.io/x/staking/types"
 
 	codectypes "github.com/cosmos/cosmos-sdk/codec/types"
+	cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types"
 	sdk "github.com/cosmos/cosmos-sdk/types"
+	sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
 )
 
 // maxRotations is the value of max rotations can be made in unbonding period for a validator.
@@ -29,7 +34,7 @@ func (k Keeper) setConsPubKeyRotationHistory(
 		Height:          height,
 		Fee:             fee,
 	}
-	err := k.RotationHistory.Set(ctx, valAddr, history)
+	err := k.RotationHistory.Set(ctx, collections.Join(valAddr.Bytes(), height), history)
 	if err != nil {
 		return err
 	}
@@ -39,7 +44,7 @@ func (k Keeper) setConsPubKeyRotationHistory(
 		return err
 	}
 
-	queueTime := sdkCtx.BlockHeader().Time.Add(ubdTime)
+	queueTime := sdkCtx.HeaderInfo().Time.Add(ubdTime)
 	if err := k.ValidatorConsensusKeyRotationRecordIndexKey.Set(ctx, collections.Join(valAddr.Bytes(), queueTime)); err != nil {
 		return err
 	}
@@ -47,6 +52,85 @@ func (k Keeper) setConsPubKeyRotationHistory(
 	return k.setConsKeyQueue(ctx, queueTime, valAddr)
 }
 
+// updateToNewPubkey gets called from the `ApplyAndReturnValidatorSetUpdates` method during EndBlock.
+//
+// This method makes the relative state changes to update the keys,
+// also maintains a map with old to new conskey rotation which is needed to retrieve the old conskey.
+// And also triggers the hook to make changes required in slashing and distribution modules.
+func (k Keeper) updateToNewPubkey(ctx context.Context, val types.Validator, oldPubKey, newPubKey *codectypes.Any, fee sdk.Coin) error {
+	consAddr, err := val.GetConsAddr()
+	if err != nil {
+		return err
+	}
+
+	if err := k.ValidatorByConsensusAddress.Remove(ctx, consAddr); err != nil {
+		return err
+	}
+
+	if err := k.DeleteValidatorByPowerIndex(ctx, val); err != nil {
+		return err
+	}
+
+	val.ConsensusPubkey = newPubKey
+	if err := k.SetValidator(ctx, val); err != nil {
+		return err
+	}
+	if err := k.SetValidatorByConsAddr(ctx, val); err != nil {
+		return err
+	}
+	if err := k.SetValidatorByPowerIndex(ctx, val); err != nil {
+		return err
+	}
+
+	oldPk, ok := oldPubKey.GetCachedValue().(cryptotypes.PubKey)
+	if !ok {
+		return errorsmod.Wrapf(sdkerrors.ErrInvalidType, "Expecting cryptotypes.PubKey, got %T", oldPk)
+	}
+
+	newPk, ok := newPubKey.GetCachedValue().(cryptotypes.PubKey)
+	if !ok {
+		return errorsmod.Wrapf(sdkerrors.ErrInvalidType, "Expecting cryptotypes.PubKey, got %T", newPk)
+	}
+
+	// sets a map: oldConsKey -> newConsKey
+	if err := k.OldToNewConsKeyMap.Set(ctx, oldPk.Address(), newPk.Address()); err != nil {
+		return err
+	}
+
+	// sets a map: newConsKey -> oldConsKey
+	if err := k.setNewToOldConsKeyMap(ctx, sdk.ConsAddress(oldPk.Address()), sdk.ConsAddress(newPk.Address())); err != nil {
+		return err
+	}
+
+	return k.Hooks().AfterConsensusPubKeyUpdate(ctx, oldPk, newPk, fee)
+}
+
+// setNewToOldConsKeyMap adds an entry in the state with the current consKey to the initial consKey of the validator.
+// it tries to find the oldPk if there is a entry already present in the state
+func (k Keeper) setNewToOldConsKeyMap(ctx context.Context, oldPk, newPk sdk.ConsAddress) error {
+	pk, err := k.NewToOldConsKeyMap.Get(ctx, oldPk)
+	if err != nil && !errors.Is(err, collections.ErrNotFound) {
+		return err
+	}
+
+	if pk != nil {
+		oldPk = pk
+	}
+
+	return k.NewToOldConsKeyMap.Set(ctx, newPk, oldPk)
+}
+
+// ValidatorIdentifier maps the new cons key to previous cons key (which is the address before the rotation).
+// (that is: newConsKey -> oldConsKey)
+func (k Keeper) ValidatorIdentifier(ctx context.Context, newPk sdk.ConsAddress) (sdk.ConsAddress, error) {
+	pk, err := k.NewToOldConsKeyMap.Get(ctx, newPk)
+	if err != nil && !errors.Is(err, collections.ErrNotFound) {
+		return nil, err
+	}
+
+	return pk, nil
+}
+
 // exceedsMaxRotations returns true if the key rotations exceed the limit, currently we are limiting one rotation for unbonding period.
 func (k Keeper) exceedsMaxRotations(ctx context.Context, valAddr sdk.ValAddress) error {
 	count := 0
@@ -70,10 +154,12 @@ func (k Keeper) exceedsMaxRotations(ctx context.Context, valAddr sdk.ValAddress)
 // this is to keep track of rotations made within the unbonding period
 func (k Keeper) setConsKeyQueue(ctx context.Context, ts time.Time, valAddr sdk.ValAddress) error {
 	queueRec, err := k.ValidatorConsensusKeyRotationRecordQueue.Get(ctx, ts)
-	if err != nil {
+	// we should return if the key found here.
+	if err != nil && !errors.Is(err, collections.ErrNotFound) {
 		return err
 	}
 
+	// push the address if it is not present in the array.
 	if !bytesSliceExists(queueRec.Addresses, valAddr.Bytes()) {
 		// Address does not exist, so you can append it to the list
 		queueRec.Addresses = append(queueRec.Addresses, valAddr.Bytes())
@@ -82,6 +168,7 @@ func (k Keeper) setConsKeyQueue(ctx context.Context, ts time.Time, valAddr sdk.V
 	return k.ValidatorConsensusKeyRotationRecordQueue.Set(ctx, ts, queueRec)
 }
 
+// bytesSliceExists tries to find the duplicate entry the array.
 func bytesSliceExists(sliceList [][]byte, targetBytes []byte) bool {
 	for _, bytesSlice := range sliceList {
 		if bytes.Equal(bytesSlice, targetBytes) {
@@ -90,3 +177,79 @@ func bytesSliceExists(sliceList [][]byte, targetBytes []byte) bool {
 	}
 	return false
 }
+
+// PurgeAllMaturedConsKeyRotatedKeys deletes all the matured key rotations.
+func (k Keeper) PurgeAllMaturedConsKeyRotatedKeys(ctx sdk.Context, maturedTime time.Time) error {
+	maturedRotatedValAddrs, err := k.getAndRemoveAllMaturedRotatedKeys(ctx, maturedTime)
+	if err != nil {
+		return err
+	}
+
+	for _, valAddr := range maturedRotatedValAddrs {
+		err := k.deleteConsKeyIndexKey(ctx, valAddr, maturedTime)
+		if err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+// deleteConsKeyIndexKey deletes the keys which forms a with given validator address and time lesser than the given time.
+// eventually there should be only one occurrence since we allow only one rotation for bonding period.
+func (k Keeper) deleteConsKeyIndexKey(ctx sdk.Context, valAddr sdk.ValAddress, ts time.Time) error {
+	rng := new(collections.Range[collections.Pair[[]byte, time.Time]]).
+		StartInclusive(collections.Join(valAddr.Bytes(), time.Time{})).
+		EndInclusive(collections.Join(valAddr.Bytes(), ts))
+
+	return k.ValidatorConsensusKeyRotationRecordIndexKey.Walk(ctx, rng, func(key collections.Pair[[]byte, time.Time]) (stop bool, err error) {
+		return false, k.ValidatorConsensusKeyRotationRecordIndexKey.Remove(ctx, key)
+	})
+}
+
+// getAndRemoveAllMaturedRotatedKeys returns all matured valaddresses.
+func (k Keeper) getAndRemoveAllMaturedRotatedKeys(ctx sdk.Context, matureTime time.Time) ([][]byte, error) {
+	valAddrs := [][]byte{}
+
+	// get an iterator for all timeslices from time 0 until the current HeaderInfo time
+	rng := new(collections.Range[time.Time]).EndInclusive(matureTime)
+	err := k.ValidatorConsensusKeyRotationRecordQueue.Walk(ctx, rng, func(key time.Time, value types.ValAddrsOfRotatedConsKeys) (stop bool, err error) {
+		valAddrs = append(valAddrs, value.Addresses...)
+		return false, k.ValidatorConsensusKeyRotationRecordQueue.Remove(ctx, key)
+	})
+	if err != nil {
+		return nil, err
+	}
+
+	return valAddrs, nil
+}
+
+// GetBlockConsPubKeyRotationHistory returns the rotation history for the current height.
+func (k Keeper) GetBlockConsPubKeyRotationHistory(ctx context.Context) ([]types.ConsPubKeyRotationHistory, error) {
+	sdkCtx := sdk.UnwrapSDKContext(ctx)
+
+	iterator, err := k.RotationHistory.Indexes.Block.MatchExact(ctx, uint64(sdkCtx.BlockHeight()))
+	if err != nil {
+		return nil, err
+	}
+	defer iterator.Close()
+
+	return indexes.CollectValues(ctx, k.RotationHistory, iterator)
+}
+
+// GetValidatorConsPubKeyRotationHistory iterates over all the rotated history objects in the state with the given valAddr and returns.
+func (k Keeper) GetValidatorConsPubKeyRotationHistory(ctx sdk.Context, operatorAddress sdk.ValAddress) ([]types.ConsPubKeyRotationHistory, error) {
+	var historyObjects []types.ConsPubKeyRotationHistory
+
+	rng := collections.NewPrefixedPairRange[[]byte, uint64](operatorAddress.Bytes())
+
+	err := k.RotationHistory.Walk(ctx, rng, func(key collections.Pair[[]byte, uint64], history types.ConsPubKeyRotationHistory) (stop bool, err error) {
+		historyObjects = append(historyObjects, history)
+		return false, nil
+	})
+	if err != nil {
+		return nil, err
+	}
+
+	return historyObjects, nil
+}
diff --git a/x/staking/keeper/cons_pubkey_test.go b/x/staking/keeper/cons_pubkey_test.go
new file mode 100644
index 000000000000..1431a06ff3b2
--- /dev/null
+++ b/x/staking/keeper/cons_pubkey_test.go
@@ -0,0 +1,180 @@
+package keeper_test
+
+import (
+	"time"
+
+	"github.com/golang/mock/gomock"
+
+	"cosmossdk.io/collections"
+	"cosmossdk.io/core/header"
+	authtypes "cosmossdk.io/x/auth/types"
+	stakingkeeper "cosmossdk.io/x/staking/keeper"
+	"cosmossdk.io/x/staking/testutil"
+	"cosmossdk.io/x/staking/types"
+
+	codectypes "github.com/cosmos/cosmos-sdk/codec/types"
+	sdk "github.com/cosmos/cosmos-sdk/types"
+)
+
+func (s *KeeperTestSuite) TestConsPubKeyRotationHistory() {
+	stakingKeeper, ctx := s.stakingKeeper, s.ctx
+
+	_, addrVals := createValAddrs(2)
+
+	// create a validator with a self-delegation
+	val := testutil.NewValidator(s.T(), addrVals[0], PKs[0])
+	valTokens := stakingKeeper.TokensFromConsensusPower(ctx, 10)
+	val, issuedShares := val.AddTokensFromDel(valTokens)
+	s.Require().Equal(valTokens, issuedShares.RoundInt())
+
+	s.bankKeeper.EXPECT().SendCoinsFromModuleToModule(gomock.Any(), types.NotBondedPoolName, types.BondedPoolName, gomock.Any())
+	_ = stakingkeeper.TestingUpdateValidator(stakingKeeper, ctx, val, true)
+	val0AccAddr := sdk.AccAddress(addrVals[0].Bytes())
+	selfDelegation := types.NewDelegation(val0AccAddr.String(), addrVals[0].String(), issuedShares)
+
+	err := stakingKeeper.SetDelegation(ctx, selfDelegation)
+	s.Require().NoError(err)
+
+	validators, err := stakingKeeper.GetAllValidators(ctx)
+	s.Require().NoError(err)
+	s.Require().Len(validators, 1)
+
+	validator := validators[0]
+	valAddr, err := sdk.ValAddressFromBech32(validator.OperatorAddress)
+	s.Require().NoError(err)
+
+	historyObjects, err := stakingKeeper.GetValidatorConsPubKeyRotationHistory(ctx, valAddr)
+	s.Require().NoError(err)
+	s.Require().Len(historyObjects, 0)
+
+	newConsPub, err := codectypes.NewAnyWithValue(PKs[1])
+	s.Require().NoError(err)
+
+	newConsPub2, err := codectypes.NewAnyWithValue(PKs[2])
+	s.Require().NoError(err)
+
+	params, err := stakingKeeper.Params.Get(ctx)
+	s.Require().NoError(err)
+
+	height := uint64(ctx.BlockHeight())
+	err = stakingKeeper.RotationHistory.Set(ctx, collections.Join(valAddr.Bytes(), height), types.ConsPubKeyRotationHistory{
+		OperatorAddress: valAddr,
+		OldConsPubkey:   validator.ConsensusPubkey,
+		NewConsPubkey:   newConsPub,
+		Height:          height,
+		Fee:             params.KeyRotationFee,
+	})
+	s.Require().NoError(err)
+
+	historyObjects, err = stakingKeeper.GetValidatorConsPubKeyRotationHistory(ctx, valAddr)
+	s.Require().NoError(err)
+	s.Require().Len(historyObjects, 1)
+
+	historyObjects, err = stakingKeeper.GetBlockConsPubKeyRotationHistory(ctx)
+	s.Require().NoError(err)
+	s.Require().Len(historyObjects, 1)
+
+	err = stakingKeeper.RotationHistory.Set(ctx, collections.Join(valAddr.Bytes(), height+1), types.ConsPubKeyRotationHistory{
+		OperatorAddress: valAddr,
+		OldConsPubkey:   newConsPub,
+		NewConsPubkey:   newConsPub2,
+		Height:          height + 1,
+		Fee:             params.KeyRotationFee,
+	})
+	s.Require().NoError(err)
+
+	historyObjects1, err := stakingKeeper.GetValidatorConsPubKeyRotationHistory(ctx, valAddr)
+	s.Require().NoError(err)
+	s.Require().Len(historyObjects1, 2)
+
+	historyObjects, err = stakingKeeper.GetBlockConsPubKeyRotationHistory(ctx)
+	s.Require().NoError(err)
+
+	s.Require().Len(historyObjects, 1)
+}
+
+func (s *KeeperTestSuite) TestValidatorIdentifier() {
+	stakingKeeper, ctx, accountKeeper, bankKeeper := s.stakingKeeper, s.ctx, s.accountKeeper, s.bankKeeper
+
+	msgServer := stakingkeeper.NewMsgServerImpl(stakingKeeper)
+	s.setValidators(6)
+	validators, err := stakingKeeper.GetAllValidators(ctx)
+	s.Require().NoError(err)
+	s.Require().Len(validators, 6)
+
+	initialConsAddr, err := validators[3].GetConsAddr()
+	s.Require().NoError(err)
+
+	oldPk, err := stakingKeeper.ValidatorIdentifier(ctx, initialConsAddr)
+	s.Require().NoError(err)
+	s.Require().Nil(oldPk)
+
+	bondedPool := authtypes.NewEmptyModuleAccount(types.BondedPoolName)
+	accountKeeper.EXPECT().GetModuleAccount(gomock.Any(), types.BondedPoolName).Return(bondedPool).AnyTimes()
+	bankKeeper.EXPECT().GetBalance(gomock.Any(), bondedPool.GetAddress(), sdk.DefaultBondDenom).Return(sdk.NewInt64Coin(sdk.DefaultBondDenom, 1000000)).AnyTimes()
+
+	val, err := stakingKeeper.ValidatorAddressCodec().StringToBytes(validators[3].GetOperator())
+	s.Require().NoError(err)
+	bankKeeper.EXPECT().SendCoinsFromAccountToModule(gomock.Any(), sdk.AccAddress(val), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
+
+	req, err := types.NewMsgRotateConsPubKey(validators[3].GetOperator(), PKs[495])
+	s.Require().NoError(err)
+	_, err = msgServer.RotateConsPubKey(ctx, req)
+	s.Require().NoError(err)
+	_, err = stakingKeeper.BlockValidatorUpdates(ctx)
+	s.Require().NoError(err)
+	params, err := stakingKeeper.Params.Get(ctx)
+	s.Require().NoError(err)
+
+	oldPk1, err := stakingKeeper.ValidatorIdentifier(ctx, sdk.ConsAddress(PKs[495].Address()))
+	s.Require().NoError(err)
+	s.Require().Equal(oldPk1.Bytes(), initialConsAddr)
+
+	ctx = ctx.WithHeaderInfo(header.Info{Time: ctx.BlockTime().Add(params.UnbondingTime).Add(time.Hour)})
+	_, err = stakingKeeper.BlockValidatorUpdates(ctx)
+	s.Require().NoError(err)
+
+	req, err = types.NewMsgRotateConsPubKey(validators[3].GetOperator(), PKs[494])
+	s.Require().NoError(err)
+	_, err = msgServer.RotateConsPubKey(ctx, req)
+	s.Require().NoError(err)
+	_, err = stakingKeeper.BlockValidatorUpdates(ctx)
+	s.Require().NoError(err)
+
+	ctx = ctx.WithHeaderInfo(header.Info{Time: ctx.BlockTime().Add(params.UnbondingTime)})
+
+	oldPk2, err := stakingKeeper.ValidatorIdentifier(ctx, sdk.ConsAddress(PKs[494].Address()))
+	s.Require().NoError(err)
+	_, err = stakingKeeper.BlockValidatorUpdates(ctx)
+	s.Require().NoError(err)
+
+	s.Require().Equal(oldPk2.Bytes(), initialConsAddr)
+}
+
+func (s *KeeperTestSuite) setValidators(n int) {
+	stakingKeeper, ctx := s.stakingKeeper, s.ctx
+
+	_, addrVals := createValAddrs(n)
+
+	for i := 0; i < n; i++ {
+		val := testutil.NewValidator(s.T(), addrVals[i], PKs[i])
+		valTokens := stakingKeeper.TokensFromConsensusPower(ctx, 10)
+		val, issuedShares := val.AddTokensFromDel(valTokens)
+		s.Require().Equal(valTokens, issuedShares.RoundInt())
+
+		s.bankKeeper.EXPECT().SendCoinsFromModuleToModule(gomock.Any(), types.NotBondedPoolName, types.BondedPoolName, gomock.Any())
+		_ = stakingkeeper.TestingUpdateValidator(stakingKeeper, ctx, val, true)
+		val0AccAddr := sdk.AccAddress(addrVals[i].Bytes())
+		selfDelegation := types.NewDelegation(val0AccAddr.String(), addrVals[i].String(), issuedShares)
+		err := stakingKeeper.SetDelegation(ctx, selfDelegation)
+		s.Require().NoError(err)
+
+		err = stakingKeeper.SetValidatorByConsAddr(ctx, val)
+		s.Require().NoError(err)
+
+	}
+
+	validators, err := stakingKeeper.GetAllValidators(ctx)
+	s.Require().NoError(err)
+	s.Require().Len(validators, n)
+}
diff --git a/x/staking/keeper/hooks_test.go b/x/staking/keeper/hooks_test.go
new file mode 100644
index 000000000000..b9cedf628d65
--- /dev/null
+++ b/x/staking/keeper/hooks_test.go
@@ -0,0 +1,15 @@
+package keeper_test
+
+import (
+	sdk "github.com/cosmos/cosmos-sdk/types"
+)
+
+func (s *KeeperTestSuite) TestHookAfterConsensusPubKeyUpdate() {
+	stKeeper := s.stakingKeeper
+	ctx := s.ctx
+	require := s.Require()
+
+	rotationFee := sdk.NewInt64Coin("stake", 1000000)
+	err := stKeeper.Hooks().AfterConsensusPubKeyUpdate(ctx, PKs[0], PKs[1], rotationFee)
+	require.NoError(err)
+}
diff --git a/x/staking/keeper/keeper.go b/x/staking/keeper/keeper.go
index 89b0090a501a..2ccae4ed0dfb 100644
--- a/x/staking/keeper/keeper.go
+++ b/x/staking/keeper/keeper.go
@@ -43,11 +43,11 @@ func HistoricalInfoCodec(cdc codec.BinaryCodec) collcodec.ValueCodec[types.Histo
 }
 
 type rotationHistoryIndexes struct {
-	Block *indexes.Multi[uint64, []byte, types.ConsPubKeyRotationHistory]
+	Block *indexes.Multi[uint64, collections.Pair[[]byte, uint64], types.ConsPubKeyRotationHistory]
 }
 
-func (a rotationHistoryIndexes) IndexesList() []collections.Index[[]byte, types.ConsPubKeyRotationHistory] {
-	return []collections.Index[[]byte, types.ConsPubKeyRotationHistory]{
+func (a rotationHistoryIndexes) IndexesList() []collections.Index[collections.Pair[[]byte, uint64], types.ConsPubKeyRotationHistory] {
+	return []collections.Index[collections.Pair[[]byte, uint64], types.ConsPubKeyRotationHistory]{
 		a.Block,
 	}
 }
@@ -55,8 +55,12 @@ func (a rotationHistoryIndexes) IndexesList() []collections.Index[[]byte, types.
 func NewRotationHistoryIndexes(sb *collections.SchemaBuilder) rotationHistoryIndexes {
 	return rotationHistoryIndexes{
 		Block: indexes.NewMulti(
-			sb, types.BlockConsPubKeyRotationHistoryKey, "cons_pubkey_history_by_block", collections.Uint64Key, collections.BytesKey,
-			func(_ []byte, v types.ConsPubKeyRotationHistory) (uint64, error) {
+			sb,
+			types.BlockConsPubKeyRotationHistoryKey,
+			"cons_pubkey_history_by_block",
+			collections.Uint64Key,
+			collections.PairKeyCodec(collections.BytesKey, collections.Uint64Key),
+			func(key collections.Pair[[]byte, uint64], v types.ConsPubKeyRotationHistory) (uint64, error) {
 				return v.Height, nil
 			},
 		),
@@ -119,11 +123,13 @@ type Keeper struct {
 	ValidatorConsensusKeyRotationRecordIndexKey collections.KeySet[collections.Pair[[]byte, time.Time]]
 	// ValidatorConsensusKeyRotationRecordQueue: this key is used to set the unbonding period time on each rotation
 	ValidatorConsensusKeyRotationRecordQueue collections.Map[time.Time, types.ValAddrsOfRotatedConsKeys]
-	// RotatedConsKeyMapIndex: prefix for rotated cons address to new cons address
-	RotatedConsKeyMapIndex collections.Map[[]byte, []byte]
+	// NewToOldConsKeyMap: prefix for rotated old cons address to new cons address
+	NewToOldConsKeyMap collections.Map[[]byte, []byte]
+	// OldToNewConsKeyMap: prefix for rotated new cons address to old cons address
+	OldToNewConsKeyMap collections.Map[[]byte, []byte]
 	// ValidatorConsPubKeyRotationHistory: consPubkey rotation history by validator
 	// A index is being added with key `BlockConsPubKeyRotationHistory`: consPubkey rotation history by height
-	RotationHistory *collections.IndexedMap[[]byte, types.ConsPubKeyRotationHistory, rotationHistoryIndexes]
+	RotationHistory *collections.IndexedMap[collections.Pair[[]byte, uint64], types.ConsPubKeyRotationHistory, rotationHistoryIndexes]
 }
 
 // NewKeeper creates a new staking Keeper instance
@@ -273,10 +279,18 @@ func NewKeeper(
 			codec.CollValue[types.ValAddrsOfRotatedConsKeys](cdc),
 		),
 
-		// key format is: 105 | valAddr
-		RotatedConsKeyMapIndex: collections.NewMap(
-			sb, types.RotatedConsKeyMapIndex,
-			"cons_pubkey_map",
+		// key format is: 105 | consAddr
+		NewToOldConsKeyMap: collections.NewMap(
+			sb, types.NewToOldConsKeyMap,
+			"new_to_old_cons_key_map",
+			collections.BytesKey,
+			collections.BytesValue,
+		),
+
+		// key format is: 106 | consAddr
+		OldToNewConsKeyMap: collections.NewMap(
+			sb, types.OldToNewConsKeyMap,
+			"old_to_new_cons_key_map",
 			collections.BytesKey,
 			collections.BytesValue,
 		),
@@ -287,7 +301,7 @@ func NewKeeper(
 			sb,
 			types.ValidatorConsPubKeyRotationHistoryKey,
 			"cons_pub_rotation_history",
-			collections.BytesKey,
+			collections.PairKeyCodec(collections.BytesKey, collections.Uint64Key),
 			codec.CollValue[types.ConsPubKeyRotationHistory](cdc),
 			NewRotationHistoryIndexes(sb),
 		),
diff --git a/x/staking/keeper/msg_server.go b/x/staking/keeper/msg_server.go
index 53ecbbf1e8cf..0814e4b56bf9 100644
--- a/x/staking/keeper/msg_server.go
+++ b/x/staking/keeper/msg_server.go
@@ -617,7 +617,7 @@ func (k msgServer) RotateConsPubKey(ctx context.Context, msg *types.MsgRotateCon
 	}
 
 	// check cons key is already present in the key rotation history.
-	rotatedTo, err := k.RotatedConsKeyMapIndex.Get(ctx, pk.Address())
+	rotatedTo, err := k.NewToOldConsKeyMap.Get(ctx, pk.Address())
 	if err != nil && !errors.Is(err, collections.ErrNotFound) {
 		return nil, err
 	}
@@ -630,8 +630,8 @@ func (k msgServer) RotateConsPubKey(ctx context.Context, msg *types.MsgRotateCon
 	newConsAddr := sdk.ConsAddress(pk.Address())
 
 	// checks if NewPubKey is not duplicated on ValidatorsByConsAddr
-	validator1, _ := k.Keeper.ValidatorByConsAddr(ctx, newConsAddr)
-	if validator1 != nil {
+	_, err = k.Keeper.ValidatorByConsAddr(ctx, newConsAddr)
+	if err == nil {
 		return nil, types.ErrConsensusPubKeyAlreadyUsedForValidator
 	}
 
@@ -642,6 +642,10 @@ func (k msgServer) RotateConsPubKey(ctx context.Context, msg *types.MsgRotateCon
 
 	validator2, err := k.Keeper.GetValidator(ctx, valAddr)
 	if err != nil {
+		return nil, err
+	}
+
+	if validator2.GetOperator() == "" {
 		return nil, types.ErrNoValidatorFound
 	}
 
@@ -663,7 +667,7 @@ func (k msgServer) RotateConsPubKey(ctx context.Context, msg *types.MsgRotateCon
 		return nil, err
 	}
 
-	err = k.Keeper.bankKeeper.SendCoinsFromAccountToModule(ctx, sdk.AccAddress(valAddr), types.DistributionModuleName, sdk.NewCoins(params.KeyRotationFee))
+	err = k.Keeper.bankKeeper.SendCoinsFromAccountToModule(ctx, sdk.AccAddress(valAddr), types.PoolModuleName, sdk.NewCoins(params.KeyRotationFee))
 	if err != nil {
 		return nil, err
 	}
diff --git a/x/staking/keeper/msg_server_test.go b/x/staking/keeper/msg_server_test.go
index 9f371d3223c9..3b142ada4d03 100644
--- a/x/staking/keeper/msg_server_test.go
+++ b/x/staking/keeper/msg_server_test.go
@@ -10,7 +10,9 @@ import (
 	"cosmossdk.io/collections"
 	"cosmossdk.io/core/header"
 	"cosmossdk.io/math"
-	stakingtypes "cosmossdk.io/x/staking/types"
+	authtypes "cosmossdk.io/x/auth/types"
+	stakingkeeper "cosmossdk.io/x/staking/keeper"
+	"cosmossdk.io/x/staking/types"
 
 	"github.com/cosmos/cosmos-sdk/codec/address"
 	codectypes "github.com/cosmos/cosmos-sdk/codec/types"
@@ -28,7 +30,7 @@ var (
 
 func (s *KeeperTestSuite) execExpectCalls() {
 	s.accountKeeper.EXPECT().AddressCodec().Return(address.NewBech32Codec("cosmos")).AnyTimes()
-	s.bankKeeper.EXPECT().DelegateCoinsFromAccountToModule(gomock.Any(), Addr, stakingtypes.NotBondedPoolName, gomock.Any()).AnyTimes()
+	s.bankKeeper.EXPECT().DelegateCoinsFromAccountToModule(gomock.Any(), Addr, types.NotBondedPoolName, gomock.Any()).AnyTimes()
 }
 
 func (s *KeeperTestSuite) TestMsgCreateValidator() {
@@ -54,15 +56,15 @@ func (s *KeeperTestSuite) TestMsgCreateValidator() {
 
 	testCases := []struct {
 		name      string
-		input     *stakingtypes.MsgCreateValidator
+		input     *types.MsgCreateValidator
 		expErr    bool
 		expErrMsg string
 	}{
 		{
 			name: "empty description",
-			input: &stakingtypes.MsgCreateValidator{
-				Description: stakingtypes.Description{},
-				Commission: stakingtypes.CommissionRates{
+			input: &types.MsgCreateValidator{
+				Description: types.Description{},
+				Commission: types.CommissionRates{
 					Rate:          math.LegacyNewDecWithPrec(5, 1),
 					MaxRate:       math.LegacyNewDecWithPrec(5, 1),
 					MaxChangeRate: math.LegacyNewDec(0),
@@ -78,11 +80,11 @@ func (s *KeeperTestSuite) TestMsgCreateValidator() {
 		},
 		{
 			name: "invalid validator address",
-			input: &stakingtypes.MsgCreateValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgCreateValidator{
+				Description: types.Description{
 					Moniker: "NewValidator",
 				},
-				Commission: stakingtypes.CommissionRates{
+				Commission: types.CommissionRates{
 					Rate:          math.LegacyNewDecWithPrec(5, 1),
 					MaxRate:       math.LegacyNewDecWithPrec(5, 1),
 					MaxChangeRate: math.LegacyNewDec(0),
@@ -98,11 +100,11 @@ func (s *KeeperTestSuite) TestMsgCreateValidator() {
 		},
 		{
 			name: "empty validator pubkey",
-			input: &stakingtypes.MsgCreateValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgCreateValidator{
+				Description: types.Description{
 					Moniker: "NewValidator",
 				},
-				Commission: stakingtypes.CommissionRates{
+				Commission: types.CommissionRates{
 					Rate:          math.LegacyNewDecWithPrec(5, 1),
 					MaxRate:       math.LegacyNewDecWithPrec(5, 1),
 					MaxChangeRate: math.LegacyNewDec(0),
@@ -118,11 +120,11 @@ func (s *KeeperTestSuite) TestMsgCreateValidator() {
 		},
 		{
 			name: "validator pubkey len is invalid",
-			input: &stakingtypes.MsgCreateValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgCreateValidator{
+				Description: types.Description{
 					Moniker: "NewValidator",
 				},
-				Commission: stakingtypes.CommissionRates{
+				Commission: types.CommissionRates{
 					Rate:          math.LegacyNewDecWithPrec(5, 1),
 					MaxRate:       math.LegacyNewDecWithPrec(5, 1),
 					MaxChangeRate: math.LegacyNewDec(0),
@@ -138,11 +140,11 @@ func (s *KeeperTestSuite) TestMsgCreateValidator() {
 		},
 		{
 			name: "empty delegation amount",
-			input: &stakingtypes.MsgCreateValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgCreateValidator{
+				Description: types.Description{
 					Moniker: "NewValidator",
 				},
-				Commission: stakingtypes.CommissionRates{
+				Commission: types.CommissionRates{
 					Rate:          math.LegacyNewDecWithPrec(5, 1),
 					MaxRate:       math.LegacyNewDecWithPrec(5, 1),
 					MaxChangeRate: math.LegacyNewDec(0),
@@ -158,11 +160,11 @@ func (s *KeeperTestSuite) TestMsgCreateValidator() {
 		},
 		{
 			name: "nil delegation amount",
-			input: &stakingtypes.MsgCreateValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgCreateValidator{
+				Description: types.Description{
 					Moniker: "NewValidator",
 				},
-				Commission: stakingtypes.CommissionRates{
+				Commission: types.CommissionRates{
 					Rate:          math.LegacyNewDecWithPrec(5, 1),
 					MaxRate:       math.LegacyNewDecWithPrec(5, 1),
 					MaxChangeRate: math.LegacyNewDec(0),
@@ -178,11 +180,11 @@ func (s *KeeperTestSuite) TestMsgCreateValidator() {
 		},
 		{
 			name: "zero minimum self delegation",
-			input: &stakingtypes.MsgCreateValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgCreateValidator{
+				Description: types.Description{
 					Moniker: "NewValidator",
 				},
-				Commission: stakingtypes.CommissionRates{
+				Commission: types.CommissionRates{
 					Rate:          math.LegacyNewDecWithPrec(5, 1),
 					MaxRate:       math.LegacyNewDecWithPrec(5, 1),
 					MaxChangeRate: math.LegacyNewDec(0),
@@ -198,11 +200,11 @@ func (s *KeeperTestSuite) TestMsgCreateValidator() {
 		},
 		{
 			name: "negative minimum self delegation",
-			input: &stakingtypes.MsgCreateValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgCreateValidator{
+				Description: types.Description{
 					Moniker: "NewValidator",
 				},
-				Commission: stakingtypes.CommissionRates{
+				Commission: types.CommissionRates{
 					Rate:          math.LegacyNewDecWithPrec(5, 1),
 					MaxRate:       math.LegacyNewDecWithPrec(5, 1),
 					MaxChangeRate: math.LegacyNewDec(0),
@@ -218,11 +220,11 @@ func (s *KeeperTestSuite) TestMsgCreateValidator() {
 		},
 		{
 			name: "delegation less than minimum self delegation",
-			input: &stakingtypes.MsgCreateValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgCreateValidator{
+				Description: types.Description{
 					Moniker: "NewValidator",
 				},
-				Commission: stakingtypes.CommissionRates{
+				Commission: types.CommissionRates{
 					Rate:          math.LegacyNewDecWithPrec(5, 1),
 					MaxRate:       math.LegacyNewDecWithPrec(5, 1),
 					MaxChangeRate: math.LegacyNewDec(0),
@@ -238,15 +240,15 @@ func (s *KeeperTestSuite) TestMsgCreateValidator() {
 		},
 		{
 			name: "valid msg",
-			input: &stakingtypes.MsgCreateValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgCreateValidator{
+				Description: types.Description{
 					Moniker:         "NewValidator",
 					Identity:        "xyz",
 					Website:         "xyz.com",
 					SecurityContact: "xyz@gmail.com",
 					Details:         "details",
 				},
-				Commission: stakingtypes.CommissionRates{
+				Commission: types.CommissionRates{
 					Rate:          math.LegacyNewDecWithPrec(5, 1),
 					MaxRate:       math.LegacyNewDecWithPrec(5, 1),
 					MaxChangeRate: math.LegacyNewDec(0),
@@ -284,8 +286,8 @@ func (s *KeeperTestSuite) TestMsgEditValidator() {
 	pk := ed25519.GenPrivKey().PubKey()
 	require.NotNil(pk)
 
-	comm := stakingtypes.NewCommissionRates(math.LegacyNewDec(0), math.LegacyNewDec(0), math.LegacyNewDec(0))
-	msg, err := stakingtypes.NewMsgCreateValidator(ValAddr.String(), pk, sdk.NewCoin(sdk.DefaultBondDenom, math.NewInt(10)), stakingtypes.Description{Moniker: "NewVal"}, comm, math.OneInt())
+	comm := types.NewCommissionRates(math.LegacyNewDec(0), math.LegacyNewDec(0), math.LegacyNewDec(0))
+	msg, err := types.NewMsgCreateValidator(ValAddr.String(), pk, sdk.NewCoin(sdk.DefaultBondDenom, math.NewInt(10)), types.Description{Moniker: "NewVal"}, comm, math.OneInt())
 	require.NoError(err)
 
 	res, err := msgServer.CreateValidator(ctx, msg)
@@ -303,15 +305,15 @@ func (s *KeeperTestSuite) TestMsgEditValidator() {
 	testCases := []struct {
 		name      string
 		ctx       sdk.Context
-		input     *stakingtypes.MsgEditValidator
+		input     *types.MsgEditValidator
 		expErr    bool
 		expErrMsg string
 	}{
 		{
 			name: "invalid validator",
 			ctx:  newCtx,
-			input: &stakingtypes.MsgEditValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgEditValidator{
+				Description: types.Description{
 					Moniker: "TestValidator",
 				},
 				ValidatorAddress:  sdk.AccAddress([]byte("invalid")).String(),
@@ -324,8 +326,8 @@ func (s *KeeperTestSuite) TestMsgEditValidator() {
 		{
 			name: "empty description",
 			ctx:  newCtx,
-			input: &stakingtypes.MsgEditValidator{
-				Description:       stakingtypes.Description{},
+			input: &types.MsgEditValidator{
+				Description:       types.Description{},
 				ValidatorAddress:  ValAddr.String(),
 				CommissionRate:    &newRate,
 				MinSelfDelegation: &newSelfDel,
@@ -336,8 +338,8 @@ func (s *KeeperTestSuite) TestMsgEditValidator() {
 		{
 			name: "negative self delegation",
 			ctx:  newCtx,
-			input: &stakingtypes.MsgEditValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgEditValidator{
+				Description: types.Description{
 					Moniker: "TestValidator",
 				},
 				ValidatorAddress:  ValAddr.String(),
@@ -350,8 +352,8 @@ func (s *KeeperTestSuite) TestMsgEditValidator() {
 		{
 			name: "invalid commission rate",
 			ctx:  newCtx,
-			input: &stakingtypes.MsgEditValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgEditValidator{
+				Description: types.Description{
 					Moniker: "TestValidator",
 				},
 				ValidatorAddress:  ValAddr.String(),
@@ -364,8 +366,8 @@ func (s *KeeperTestSuite) TestMsgEditValidator() {
 		{
 			name: "validator does not exist",
 			ctx:  newCtx,
-			input: &stakingtypes.MsgEditValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgEditValidator{
+				Description: types.Description{
 					Moniker: "TestValidator",
 				},
 				ValidatorAddress:  sdk.ValAddress([]byte("val")).String(),
@@ -378,8 +380,8 @@ func (s *KeeperTestSuite) TestMsgEditValidator() {
 		{
 			name: "change commmission rate in <24hrs",
 			ctx:  ctx,
-			input: &stakingtypes.MsgEditValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgEditValidator{
+				Description: types.Description{
 					Moniker: "TestValidator",
 				},
 				ValidatorAddress:  ValAddr.String(),
@@ -392,8 +394,8 @@ func (s *KeeperTestSuite) TestMsgEditValidator() {
 		{
 			name: "minimum self delegation cannot decrease",
 			ctx:  newCtx,
-			input: &stakingtypes.MsgEditValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgEditValidator{
+				Description: types.Description{
 					Moniker: "TestValidator",
 				},
 				ValidatorAddress:  ValAddr.String(),
@@ -406,8 +408,8 @@ func (s *KeeperTestSuite) TestMsgEditValidator() {
 		{
 			name: "validator self-delegation must be greater than min self delegation",
 			ctx:  newCtx,
-			input: &stakingtypes.MsgEditValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgEditValidator{
+				Description: types.Description{
 					Moniker: "TestValidator",
 				},
 				ValidatorAddress:  ValAddr.String(),
@@ -420,8 +422,8 @@ func (s *KeeperTestSuite) TestMsgEditValidator() {
 		{
 			name: "valid msg",
 			ctx:  newCtx,
-			input: &stakingtypes.MsgEditValidator{
-				Description: stakingtypes.Description{
+			input: &types.MsgEditValidator{
+				Description: types.Description{
 					Moniker:         "TestValidator",
 					Identity:        "abc",
 					Website:         "abc.com",
@@ -457,9 +459,9 @@ func (s *KeeperTestSuite) TestMsgDelegate() {
 	pk := ed25519.GenPrivKey().PubKey()
 	require.NotNil(pk)
 
-	comm := stakingtypes.NewCommissionRates(math.LegacyNewDec(0), math.LegacyNewDec(0), math.LegacyNewDec(0))
+	comm := types.NewCommissionRates(math.LegacyNewDec(0), math.LegacyNewDec(0), math.LegacyNewDec(0))
 
-	msg, err := stakingtypes.NewMsgCreateValidator(ValAddr.String(), pk, sdk.NewCoin(sdk.DefaultBondDenom, math.NewInt(10)), stakingtypes.Description{Moniker: "NewVal"}, comm, math.OneInt())
+	msg, err := types.NewMsgCreateValidator(ValAddr.String(), pk, sdk.NewCoin(sdk.DefaultBondDenom, math.NewInt(10)), types.Description{Moniker: "NewVal"}, comm, math.OneInt())
 	require.NoError(err)
 
 	res, err := msgServer.CreateValidator(ctx, msg)
@@ -468,13 +470,13 @@ func (s *KeeperTestSuite) TestMsgDelegate() {
 
 	testCases := []struct {
 		name      string
-		input     *stakingtypes.MsgDelegate
+		input     *types.MsgDelegate
 		expErr    bool
 		expErrMsg string
 	}{
 		{
 			name: "invalid validator",
-			input: &stakingtypes.MsgDelegate{
+			input: &types.MsgDelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: sdk.AccAddress([]byte("invalid")).String(),
 				Amount:           sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: keeper.TokensFromConsensusPower(s.ctx, int64(100))},
@@ -484,7 +486,7 @@ func (s *KeeperTestSuite) TestMsgDelegate() {
 		},
 		{
 			name: "empty delegator",
-			input: &stakingtypes.MsgDelegate{
+			input: &types.MsgDelegate{
 				DelegatorAddress: "",
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: keeper.TokensFromConsensusPower(s.ctx, int64(100))},
@@ -494,7 +496,7 @@ func (s *KeeperTestSuite) TestMsgDelegate() {
 		},
 		{
 			name: "invalid delegator",
-			input: &stakingtypes.MsgDelegate{
+			input: &types.MsgDelegate{
 				DelegatorAddress: "invalid",
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: keeper.TokensFromConsensusPower(s.ctx, int64(100))},
@@ -504,7 +506,7 @@ func (s *KeeperTestSuite) TestMsgDelegate() {
 		},
 		{
 			name: "validator does not exist",
-			input: &stakingtypes.MsgDelegate{
+			input: &types.MsgDelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: sdk.ValAddress([]byte("val")).String(),
 				Amount:           sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: keeper.TokensFromConsensusPower(s.ctx, int64(100))},
@@ -514,7 +516,7 @@ func (s *KeeperTestSuite) TestMsgDelegate() {
 		},
 		{
 			name: "zero amount",
-			input: &stakingtypes.MsgDelegate{
+			input: &types.MsgDelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: keeper.TokensFromConsensusPower(s.ctx, int64(0))},
@@ -524,7 +526,7 @@ func (s *KeeperTestSuite) TestMsgDelegate() {
 		},
 		{
 			name: "negative amount",
-			input: &stakingtypes.MsgDelegate{
+			input: &types.MsgDelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: keeper.TokensFromConsensusPower(s.ctx, int64(-1))},
@@ -534,7 +536,7 @@ func (s *KeeperTestSuite) TestMsgDelegate() {
 		},
 		{
 			name: "invalid BondDenom",
-			input: &stakingtypes.MsgDelegate{
+			input: &types.MsgDelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.Coin{Denom: "test", Amount: keeper.TokensFromConsensusPower(s.ctx, int64(100))},
@@ -544,7 +546,7 @@ func (s *KeeperTestSuite) TestMsgDelegate() {
 		},
 		{
 			name: "valid msg",
-			input: &stakingtypes.MsgDelegate{
+			input: &types.MsgDelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: keeper.TokensFromConsensusPower(s.ctx, int64(100))},
@@ -581,17 +583,17 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 	dstPk := ed25519.GenPrivKey().PubKey()
 	require.NotNil(dstPk)
 
-	comm := stakingtypes.NewCommissionRates(math.LegacyNewDec(0), math.LegacyNewDec(0), math.LegacyNewDec(0))
+	comm := types.NewCommissionRates(math.LegacyNewDec(0), math.LegacyNewDec(0), math.LegacyNewDec(0))
 	amt := sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: keeper.TokensFromConsensusPower(s.ctx, int64(100))}
 
-	msg, err := stakingtypes.NewMsgCreateValidator(srcValAddr.String(), pk, amt, stakingtypes.Description{Moniker: "NewVal"}, comm, math.OneInt())
+	msg, err := types.NewMsgCreateValidator(srcValAddr.String(), pk, amt, types.Description{Moniker: "NewVal"}, comm, math.OneInt())
 	require.NoError(err)
 	res, err := msgServer.CreateValidator(ctx, msg)
 	require.NoError(err)
 	require.NotNil(res)
-	s.bankKeeper.EXPECT().DelegateCoinsFromAccountToModule(gomock.Any(), addr2, stakingtypes.NotBondedPoolName, gomock.Any()).AnyTimes()
+	s.bankKeeper.EXPECT().DelegateCoinsFromAccountToModule(gomock.Any(), addr2, types.NotBondedPoolName, gomock.Any()).AnyTimes()
 
-	msg, err = stakingtypes.NewMsgCreateValidator(dstValAddr.String(), dstPk, amt, stakingtypes.Description{Moniker: "NewVal"}, comm, math.OneInt())
+	msg, err = types.NewMsgCreateValidator(dstValAddr.String(), dstPk, amt, types.Description{Moniker: "NewVal"}, comm, math.OneInt())
 	require.NoError(err)
 
 	res, err = msgServer.CreateValidator(ctx, msg)
@@ -599,20 +601,20 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 	require.NotNil(res)
 
 	shares := math.LegacyNewDec(100)
-	del := stakingtypes.NewDelegation(Addr.String(), srcValAddr.String(), shares)
+	del := types.NewDelegation(Addr.String(), srcValAddr.String(), shares)
 	require.NoError(keeper.SetDelegation(ctx, del))
 	_, err = keeper.Delegations.Get(ctx, collections.Join(Addr, srcValAddr))
 	require.NoError(err)
 
 	testCases := []struct {
 		name      string
-		input     *stakingtypes.MsgBeginRedelegate
+		input     *types.MsgBeginRedelegate
 		expErr    bool
 		expErrMsg string
 	}{
 		{
 			name: "invalid source validator",
-			input: &stakingtypes.MsgBeginRedelegate{
+			input: &types.MsgBeginRedelegate{
 				DelegatorAddress:    Addr.String(),
 				ValidatorSrcAddress: sdk.AccAddress([]byte("invalid")).String(),
 				ValidatorDstAddress: dstValAddr.String(),
@@ -623,7 +625,7 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 		},
 		{
 			name: "empty delegator",
-			input: &stakingtypes.MsgBeginRedelegate{
+			input: &types.MsgBeginRedelegate{
 				DelegatorAddress:    "",
 				ValidatorSrcAddress: srcValAddr.String(),
 				ValidatorDstAddress: dstValAddr.String(),
@@ -634,7 +636,7 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 		},
 		{
 			name: "invalid delegator",
-			input: &stakingtypes.MsgBeginRedelegate{
+			input: &types.MsgBeginRedelegate{
 				DelegatorAddress:    "invalid",
 				ValidatorSrcAddress: srcValAddr.String(),
 				ValidatorDstAddress: dstValAddr.String(),
@@ -645,7 +647,7 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 		},
 		{
 			name: "invalid destination validator",
-			input: &stakingtypes.MsgBeginRedelegate{
+			input: &types.MsgBeginRedelegate{
 				DelegatorAddress:    Addr.String(),
 				ValidatorSrcAddress: srcValAddr.String(),
 				ValidatorDstAddress: sdk.AccAddress([]byte("invalid")).String(),
@@ -656,7 +658,7 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 		},
 		{
 			name: "validator does not exist",
-			input: &stakingtypes.MsgBeginRedelegate{
+			input: &types.MsgBeginRedelegate{
 				DelegatorAddress:    Addr.String(),
 				ValidatorSrcAddress: sdk.ValAddress([]byte("invalid")).String(),
 				ValidatorDstAddress: dstValAddr.String(),
@@ -667,7 +669,7 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 		},
 		{
 			name: "self redelegation",
-			input: &stakingtypes.MsgBeginRedelegate{
+			input: &types.MsgBeginRedelegate{
 				DelegatorAddress:    Addr.String(),
 				ValidatorSrcAddress: srcValAddr.String(),
 				ValidatorDstAddress: srcValAddr.String(),
@@ -678,7 +680,7 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 		},
 		{
 			name: "amount greater than delegated shares amount",
-			input: &stakingtypes.MsgBeginRedelegate{
+			input: &types.MsgBeginRedelegate{
 				DelegatorAddress:    Addr.String(),
 				ValidatorSrcAddress: srcValAddr.String(),
 				ValidatorDstAddress: dstValAddr.String(),
@@ -689,7 +691,7 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 		},
 		{
 			name: "zero amount",
-			input: &stakingtypes.MsgBeginRedelegate{
+			input: &types.MsgBeginRedelegate{
 				DelegatorAddress:    Addr.String(),
 				ValidatorSrcAddress: srcValAddr.String(),
 				ValidatorDstAddress: dstValAddr.String(),
@@ -700,7 +702,7 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 		},
 		{
 			name: "invalid coin denom",
-			input: &stakingtypes.MsgBeginRedelegate{
+			input: &types.MsgBeginRedelegate{
 				DelegatorAddress:    Addr.String(),
 				ValidatorSrcAddress: srcValAddr.String(),
 				ValidatorDstAddress: dstValAddr.String(),
@@ -711,7 +713,7 @@ func (s *KeeperTestSuite) TestMsgBeginRedelegate() {
 		},
 		{
 			name: "valid msg",
-			input: &stakingtypes.MsgBeginRedelegate{
+			input: &types.MsgBeginRedelegate{
 				DelegatorAddress:    Addr.String(),
 				ValidatorSrcAddress: srcValAddr.String(),
 				ValidatorDstAddress: dstValAddr.String(),
@@ -743,30 +745,30 @@ func (s *KeeperTestSuite) TestMsgUndelegate() {
 	pk := ed25519.GenPrivKey().PubKey()
 	require.NotNil(pk)
 
-	comm := stakingtypes.NewCommissionRates(math.LegacyNewDec(0), math.LegacyNewDec(0), math.LegacyNewDec(0))
+	comm := types.NewCommissionRates(math.LegacyNewDec(0), math.LegacyNewDec(0), math.LegacyNewDec(0))
 	amt := sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: keeper.TokensFromConsensusPower(s.ctx, int64(100))}
 
-	msg, err := stakingtypes.NewMsgCreateValidator(ValAddr.String(), pk, amt, stakingtypes.Description{Moniker: "NewVal"}, comm, math.OneInt())
+	msg, err := types.NewMsgCreateValidator(ValAddr.String(), pk, amt, types.Description{Moniker: "NewVal"}, comm, math.OneInt())
 	require.NoError(err)
 	res, err := msgServer.CreateValidator(ctx, msg)
 	require.NoError(err)
 	require.NotNil(res)
 
 	shares := math.LegacyNewDec(100)
-	del := stakingtypes.NewDelegation(Addr.String(), ValAddr.String(), shares)
+	del := types.NewDelegation(Addr.String(), ValAddr.String(), shares)
 	require.NoError(keeper.SetDelegation(ctx, del))
 	_, err = keeper.Delegations.Get(ctx, collections.Join(Addr, ValAddr))
 	require.NoError(err)
 
 	testCases := []struct {
 		name      string
-		input     *stakingtypes.MsgUndelegate
+		input     *types.MsgUndelegate
 		expErr    bool
 		expErrMsg string
 	}{
 		{
 			name: "invalid validator",
-			input: &stakingtypes.MsgUndelegate{
+			input: &types.MsgUndelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: sdk.AccAddress([]byte("invalid")).String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, shares.RoundInt()),
@@ -776,7 +778,7 @@ func (s *KeeperTestSuite) TestMsgUndelegate() {
 		},
 		{
 			name: "empty delegator",
-			input: &stakingtypes.MsgUndelegate{
+			input: &types.MsgUndelegate{
 				DelegatorAddress: "",
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: shares.RoundInt()},
@@ -786,7 +788,7 @@ func (s *KeeperTestSuite) TestMsgUndelegate() {
 		},
 		{
 			name: "invalid delegator",
-			input: &stakingtypes.MsgUndelegate{
+			input: &types.MsgUndelegate{
 				DelegatorAddress: "invalid",
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: shares.RoundInt()},
@@ -796,7 +798,7 @@ func (s *KeeperTestSuite) TestMsgUndelegate() {
 		},
 		{
 			name: "validator does not exist",
-			input: &stakingtypes.MsgUndelegate{
+			input: &types.MsgUndelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: sdk.ValAddress([]byte("invalid")).String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, shares.RoundInt()),
@@ -806,7 +808,7 @@ func (s *KeeperTestSuite) TestMsgUndelegate() {
 		},
 		{
 			name: "amount greater than delegated shares amount",
-			input: &stakingtypes.MsgUndelegate{
+			input: &types.MsgUndelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, math.NewInt(101)),
@@ -816,7 +818,7 @@ func (s *KeeperTestSuite) TestMsgUndelegate() {
 		},
 		{
 			name: "zero amount",
-			input: &stakingtypes.MsgUndelegate{
+			input: &types.MsgUndelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, math.NewInt(0)),
@@ -826,7 +828,7 @@ func (s *KeeperTestSuite) TestMsgUndelegate() {
 		},
 		{
 			name: "invalid coin denom",
-			input: &stakingtypes.MsgUndelegate{
+			input: &types.MsgUndelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin("test", shares.RoundInt()),
@@ -836,7 +838,7 @@ func (s *KeeperTestSuite) TestMsgUndelegate() {
 		},
 		{
 			name: "valid msg",
-			input: &stakingtypes.MsgUndelegate{
+			input: &types.MsgUndelegate{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, shares.RoundInt()),
@@ -866,25 +868,25 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 	pk := ed25519.GenPrivKey().PubKey()
 	require.NotNil(pk)
 
-	comm := stakingtypes.NewCommissionRates(math.LegacyNewDec(0), math.LegacyNewDec(0), math.LegacyNewDec(0))
+	comm := types.NewCommissionRates(math.LegacyNewDec(0), math.LegacyNewDec(0), math.LegacyNewDec(0))
 	amt := sdk.Coin{Denom: sdk.DefaultBondDenom, Amount: keeper.TokensFromConsensusPower(s.ctx, int64(100))}
 
-	s.bankKeeper.EXPECT().DelegateCoinsFromAccountToModule(gomock.Any(), Addr, stakingtypes.NotBondedPoolName, gomock.Any()).AnyTimes()
+	s.bankKeeper.EXPECT().DelegateCoinsFromAccountToModule(gomock.Any(), Addr, types.NotBondedPoolName, gomock.Any()).AnyTimes()
 
-	msg, err := stakingtypes.NewMsgCreateValidator(ValAddr.String(), pk, amt, stakingtypes.Description{Moniker: "NewVal"}, comm, math.OneInt())
+	msg, err := types.NewMsgCreateValidator(ValAddr.String(), pk, amt, types.Description{Moniker: "NewVal"}, comm, math.OneInt())
 	require.NoError(err)
 	res, err := msgServer.CreateValidator(ctx, msg)
 	require.NoError(err)
 	require.NotNil(res)
 
 	shares := math.LegacyNewDec(100)
-	del := stakingtypes.NewDelegation(Addr.String(), ValAddr.String(), shares)
+	del := types.NewDelegation(Addr.String(), ValAddr.String(), shares)
 	require.NoError(keeper.SetDelegation(ctx, del))
 	resDel, err := keeper.Delegations.Get(ctx, collections.Join(Addr, ValAddr))
 	require.NoError(err)
 	require.Equal(del, resDel)
 
-	ubd := stakingtypes.NewUnbondingDelegation(Addr, ValAddr, 10, ctx.HeaderInfo().Time.Add(time.Minute*10), shares.RoundInt(), 0, keeper.ValidatorAddressCodec(), ak.AddressCodec())
+	ubd := types.NewUnbondingDelegation(Addr, ValAddr, 10, ctx.HeaderInfo().Time.Add(time.Minute*10), shares.RoundInt(), 0, keeper.ValidatorAddressCodec(), ak.AddressCodec())
 	require.NoError(keeper.SetUnbondingDelegation(ctx, ubd))
 	resUnbond, err := keeper.GetUnbondingDelegation(ctx, Addr, ValAddr)
 	require.NoError(err)
@@ -892,13 +894,13 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 
 	testCases := []struct {
 		name      string
-		input     *stakingtypes.MsgCancelUnbondingDelegation
+		input     *types.MsgCancelUnbondingDelegation
 		expErr    bool
 		expErrMsg string
 	}{
 		{
 			name: "invalid validator",
-			input: &stakingtypes.MsgCancelUnbondingDelegation{
+			input: &types.MsgCancelUnbondingDelegation{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: sdk.AccAddress([]byte("invalid")).String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, shares.RoundInt()),
@@ -909,7 +911,7 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 		},
 		{
 			name: "empty delegator",
-			input: &stakingtypes.MsgCancelUnbondingDelegation{
+			input: &types.MsgCancelUnbondingDelegation{
 				DelegatorAddress: "",
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, shares.RoundInt()),
@@ -920,7 +922,7 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 		},
 		{
 			name: "invalid delegator",
-			input: &stakingtypes.MsgCancelUnbondingDelegation{
+			input: &types.MsgCancelUnbondingDelegation{
 				DelegatorAddress: "invalid",
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, shares.RoundInt()),
@@ -931,7 +933,7 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 		},
 		{
 			name: "entry not found at height",
-			input: &stakingtypes.MsgCancelUnbondingDelegation{
+			input: &types.MsgCancelUnbondingDelegation{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, shares.RoundInt()),
@@ -942,7 +944,7 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 		},
 		{
 			name: "invalid height",
-			input: &stakingtypes.MsgCancelUnbondingDelegation{
+			input: &types.MsgCancelUnbondingDelegation{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, shares.RoundInt()),
@@ -953,7 +955,7 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 		},
 		{
 			name: "invalid coin",
-			input: &stakingtypes.MsgCancelUnbondingDelegation{
+			input: &types.MsgCancelUnbondingDelegation{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin("test", shares.RoundInt()),
@@ -964,7 +966,7 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 		},
 		{
 			name: "validator does not exist",
-			input: &stakingtypes.MsgCancelUnbondingDelegation{
+			input: &types.MsgCancelUnbondingDelegation{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: sdk.ValAddress([]byte("invalid")).String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, shares.RoundInt()),
@@ -975,7 +977,7 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 		},
 		{
 			name: "amount is greater than balance",
-			input: &stakingtypes.MsgCancelUnbondingDelegation{
+			input: &types.MsgCancelUnbondingDelegation{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, math.NewInt(101)),
@@ -986,7 +988,7 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 		},
 		{
 			name: "zero amount",
-			input: &stakingtypes.MsgCancelUnbondingDelegation{
+			input: &types.MsgCancelUnbondingDelegation{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, math.NewInt(0)),
@@ -997,7 +999,7 @@ func (s *KeeperTestSuite) TestMsgCancelUnbondingDelegation() {
 		},
 		{
 			name: "valid msg",
-			input: &stakingtypes.MsgCancelUnbondingDelegation{
+			input: &types.MsgCancelUnbondingDelegation{
 				DelegatorAddress: Addr.String(),
 				ValidatorAddress: ValAddr.String(),
 				Amount:           sdk.NewCoin(sdk.DefaultBondDenom, shares.RoundInt()),
@@ -1027,38 +1029,38 @@ func (s *KeeperTestSuite) TestMsgUpdateParams() {
 
 	testCases := []struct {
 		name      string
-		input     *stakingtypes.MsgUpdateParams
+		input     *types.MsgUpdateParams
 		expErr    bool
 		expErrMsg string
 	}{
 		{
 			name: "valid params",
-			input: &stakingtypes.MsgUpdateParams{
+			input: &types.MsgUpdateParams{
 				Authority: keeper.GetAuthority(),
-				Params:    stakingtypes.DefaultParams(),
+				Params:    types.DefaultParams(),
 			},
 			expErr: false,
 		},
 		{
 			name: "invalid authority",
-			input: &stakingtypes.MsgUpdateParams{
+			input: &types.MsgUpdateParams{
 				Authority: "invalid",
-				Params:    stakingtypes.DefaultParams(),
+				Params:    types.DefaultParams(),
 			},
 			expErr:    true,
 			expErrMsg: "invalid authority",
 		},
 		{
 			name: "negative commission rate",
-			input: &stakingtypes.MsgUpdateParams{
+			input: &types.MsgUpdateParams{
 				Authority: keeper.GetAuthority(),
-				Params: stakingtypes.Params{
+				Params: types.Params{
 					MinCommissionRate: math.LegacyNewDec(-10),
-					UnbondingTime:     stakingtypes.DefaultUnbondingTime,
-					MaxValidators:     stakingtypes.DefaultMaxValidators,
-					MaxEntries:        stakingtypes.DefaultMaxEntries,
-					HistoricalEntries: stakingtypes.DefaultHistoricalEntries,
-					BondDenom:         stakingtypes.BondStatusBonded,
+					UnbondingTime:     types.DefaultUnbondingTime,
+					MaxValidators:     types.DefaultMaxValidators,
+					MaxEntries:        types.DefaultMaxEntries,
+					HistoricalEntries: types.DefaultHistoricalEntries,
+					BondDenom:         types.BondStatusBonded,
 				},
 			},
 			expErr:    true,
@@ -1066,15 +1068,15 @@ func (s *KeeperTestSuite) TestMsgUpdateParams() {
 		},
 		{
 			name: "commission rate cannot be bigger than 100",
-			input: &stakingtypes.MsgUpdateParams{
+			input: &types.MsgUpdateParams{
 				Authority: keeper.GetAuthority(),
-				Params: stakingtypes.Params{
+				Params: types.Params{
 					MinCommissionRate: math.LegacyNewDec(2),
-					UnbondingTime:     stakingtypes.DefaultUnbondingTime,
-					MaxValidators:     stakingtypes.DefaultMaxValidators,
-					MaxEntries:        stakingtypes.DefaultMaxEntries,
-					HistoricalEntries: stakingtypes.DefaultHistoricalEntries,
-					BondDenom:         stakingtypes.BondStatusBonded,
+					UnbondingTime:     types.DefaultUnbondingTime,
+					MaxValidators:     types.DefaultMaxValidators,
+					MaxEntries:        types.DefaultMaxEntries,
+					HistoricalEntries: types.DefaultHistoricalEntries,
+					BondDenom:         types.BondStatusBonded,
 				},
 			},
 			expErr:    true,
@@ -1082,14 +1084,14 @@ func (s *KeeperTestSuite) TestMsgUpdateParams() {
 		},
 		{
 			name: "invalid bond denom",
-			input: &stakingtypes.MsgUpdateParams{
+			input: &types.MsgUpdateParams{
 				Authority: keeper.GetAuthority(),
-				Params: stakingtypes.Params{
-					MinCommissionRate: stakingtypes.DefaultMinCommissionRate,
-					UnbondingTime:     stakingtypes.DefaultUnbondingTime,
-					MaxValidators:     stakingtypes.DefaultMaxValidators,
-					MaxEntries:        stakingtypes.DefaultMaxEntries,
-					HistoricalEntries: stakingtypes.DefaultHistoricalEntries,
+				Params: types.Params{
+					MinCommissionRate: types.DefaultMinCommissionRate,
+					UnbondingTime:     types.DefaultUnbondingTime,
+					MaxValidators:     types.DefaultMaxValidators,
+					MaxEntries:        types.DefaultMaxEntries,
+					HistoricalEntries: types.DefaultHistoricalEntries,
 					BondDenom:         "",
 				},
 			},
@@ -1098,15 +1100,15 @@ func (s *KeeperTestSuite) TestMsgUpdateParams() {
 		},
 		{
 			name: "max validators must be positive",
-			input: &stakingtypes.MsgUpdateParams{
+			input: &types.MsgUpdateParams{
 				Authority: keeper.GetAuthority(),
-				Params: stakingtypes.Params{
-					MinCommissionRate: stakingtypes.DefaultMinCommissionRate,
-					UnbondingTime:     stakingtypes.DefaultUnbondingTime,
+				Params: types.Params{
+					MinCommissionRate: types.DefaultMinCommissionRate,
+					UnbondingTime:     types.DefaultUnbondingTime,
 					MaxValidators:     0,
-					MaxEntries:        stakingtypes.DefaultMaxEntries,
-					HistoricalEntries: stakingtypes.DefaultHistoricalEntries,
-					BondDenom:         stakingtypes.BondStatusBonded,
+					MaxEntries:        types.DefaultMaxEntries,
+					HistoricalEntries: types.DefaultHistoricalEntries,
+					BondDenom:         types.BondStatusBonded,
 				},
 			},
 			expErr:    true,
@@ -1114,15 +1116,15 @@ func (s *KeeperTestSuite) TestMsgUpdateParams() {
 		},
 		{
 			name: "max entries most be positive",
-			input: &stakingtypes.MsgUpdateParams{
+			input: &types.MsgUpdateParams{
 				Authority: keeper.GetAuthority(),
-				Params: stakingtypes.Params{
-					MinCommissionRate: stakingtypes.DefaultMinCommissionRate,
-					UnbondingTime:     stakingtypes.DefaultUnbondingTime,
-					MaxValidators:     stakingtypes.DefaultMaxValidators,
+				Params: types.Params{
+					MinCommissionRate: types.DefaultMinCommissionRate,
+					UnbondingTime:     types.DefaultUnbondingTime,
+					MaxValidators:     types.DefaultMaxValidators,
 					MaxEntries:        0,
-					HistoricalEntries: stakingtypes.DefaultHistoricalEntries,
-					BondDenom:         stakingtypes.BondStatusBonded,
+					HistoricalEntries: types.DefaultHistoricalEntries,
+					BondDenom:         types.BondStatusBonded,
 				},
 			},
 			expErr:    true,
@@ -1130,14 +1132,14 @@ func (s *KeeperTestSuite) TestMsgUpdateParams() {
 		},
 		{
 			name: "negative unbounding time",
-			input: &stakingtypes.MsgUpdateParams{
+			input: &types.MsgUpdateParams{
 				Authority: keeper.GetAuthority(),
-				Params: stakingtypes.Params{
+				Params: types.Params{
 					UnbondingTime:     time.Hour * 24 * 7 * 3 * -1,
-					MaxEntries:        stakingtypes.DefaultMaxEntries,
-					MaxValidators:     stakingtypes.DefaultMaxValidators,
-					HistoricalEntries: stakingtypes.DefaultHistoricalEntries,
-					MinCommissionRate: stakingtypes.DefaultMinCommissionRate,
+					MaxEntries:        types.DefaultMaxEntries,
+					MaxValidators:     types.DefaultMaxValidators,
+					HistoricalEntries: types.DefaultHistoricalEntries,
+					MinCommissionRate: types.DefaultMinCommissionRate,
 					BondDenom:         "denom",
 				},
 			},
@@ -1159,3 +1161,190 @@ func (s *KeeperTestSuite) TestMsgUpdateParams() {
 		})
 	}
 }
+
+func (s *KeeperTestSuite) TestConsKeyRotn() {
+	stakingKeeper, ctx, accountKeeper, bankKeeper := s.stakingKeeper, s.ctx, s.accountKeeper, s.bankKeeper
+
+	msgServer := stakingkeeper.NewMsgServerImpl(stakingKeeper)
+	s.setValidators(6)
+	validators, err := stakingKeeper.GetAllValidators(ctx)
+	s.Require().NoError(err)
+
+	s.Require().Len(validators, 6)
+
+	existingPubkey, ok := validators[1].ConsensusPubkey.GetCachedValue().(cryptotypes.PubKey)
+	s.Require().True(ok)
+
+	bondedPool := authtypes.NewEmptyModuleAccount(types.BondedPoolName)
+	accountKeeper.EXPECT().GetModuleAccount(gomock.Any(), types.BondedPoolName).Return(bondedPool).AnyTimes()
+	bankKeeper.EXPECT().GetBalance(gomock.Any(), bondedPool.GetAddress(), sdk.DefaultBondDenom).Return(sdk.NewInt64Coin(sdk.DefaultBondDenom, 1000000)).AnyTimes()
+
+	testCases := []struct {
+		name      string
+		malleate  func() sdk.Context
+		validator string
+		newPubKey cryptotypes.PubKey
+		isErr     bool
+		errMsg    string
+	}{
+		{
+			name: "1st iteration no error",
+			malleate: func() sdk.Context {
+				val, err := stakingKeeper.ValidatorAddressCodec().StringToBytes(validators[0].GetOperator())
+				s.Require().NoError(err)
+
+				bankKeeper.EXPECT().SendCoinsFromAccountToModule(gomock.Any(), sdk.AccAddress(val), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
+				return ctx
+			},
+			isErr:     false,
+			errMsg:    "",
+			newPubKey: PKs[499],
+			validator: validators[0].GetOperator(),
+		},
+		{
+			name:      "pubkey already associated with another validator",
+			malleate:  func() sdk.Context { return ctx },
+			isErr:     true,
+			errMsg:    "consensus pubkey is already used for a validator",
+			newPubKey: existingPubkey,
+			validator: validators[0].GetOperator(),
+		},
+		{
+			name:      "non existing validator",
+			malleate:  func() sdk.Context { return ctx },
+			isErr:     true,
+			errMsg:    "decoding bech32 failed",
+			newPubKey: PKs[498],
+			validator: "non_existing_val",
+		},
+		{
+			name: "limit exceeding",
+			malleate: func() sdk.Context {
+				val, err := stakingKeeper.ValidatorAddressCodec().StringToBytes(validators[2].GetOperator())
+				s.Require().NoError(err)
+				bankKeeper.EXPECT().SendCoinsFromAccountToModule(gomock.Any(), sdk.AccAddress(val), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
+
+				req, err := types.NewMsgRotateConsPubKey(validators[2].GetOperator(), PKs[495])
+				s.Require().NoError(err)
+				_, err = msgServer.RotateConsPubKey(ctx, req)
+				s.Require().NoError(err)
+
+				return ctx
+			},
+			isErr:     true,
+			errMsg:    "exceeding maximum consensus pubkey rotations within unbonding period",
+			newPubKey: PKs[494],
+			validator: validators[2].GetOperator(),
+		},
+		{
+			name: "limit exceeding, but it should rotate after unbonding period",
+			malleate: func() sdk.Context {
+				params, err := stakingKeeper.Params.Get(ctx)
+				s.Require().NoError(err)
+				val, err := stakingKeeper.ValidatorAddressCodec().StringToBytes(validators[3].GetOperator())
+				s.Require().NoError(err)
+				bankKeeper.EXPECT().SendCoinsFromAccountToModule(gomock.Any(), sdk.AccAddress(val), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
+
+				// 1st rotation should pass, since limit is 1
+				req, err := types.NewMsgRotateConsPubKey(validators[3].GetOperator(), PKs[494])
+				s.Require().NoError(err)
+				_, err = msgServer.RotateConsPubKey(ctx, req)
+				s.Require().NoError(err)
+
+				// this shouldn't mature the recent rotation since unbonding period isn't reached
+				s.Require().NoError(stakingKeeper.PurgeAllMaturedConsKeyRotatedKeys(ctx, ctx.BlockTime()))
+
+				// 2nd rotation should fail since limit exceeding
+				req, err = types.NewMsgRotateConsPubKey(validators[3].GetOperator(), PKs[493])
+				s.Require().NoError(err)
+				_, err = msgServer.RotateConsPubKey(ctx, req)
+				s.Require().Error(err, "exceeding maximum consensus pubkey rotations within unbonding period")
+
+				// This should remove the keys from queue
+				// after setting the blocktime to reach the unbonding period
+				newCtx := ctx.WithHeaderInfo(header.Info{Time: ctx.BlockTime().Add(params.UnbondingTime)})
+				s.Require().NoError(stakingKeeper.PurgeAllMaturedConsKeyRotatedKeys(newCtx, newCtx.BlockTime()))
+				return newCtx
+			},
+			isErr:     false,
+			newPubKey: PKs[493],
+			validator: validators[3].GetOperator(),
+		},
+		{
+			name: "verify other validator rotation blocker",
+			malleate: func() sdk.Context {
+				params, err := stakingKeeper.Params.Get(ctx)
+				s.Require().NoError(err)
+				valStr4 := validators[4].GetOperator()
+				valStr5 := validators[5].GetOperator()
+				valAddr4, err := stakingKeeper.ValidatorAddressCodec().StringToBytes(valStr4)
+				s.Require().NoError(err)
+
+				valAddr5, err := stakingKeeper.ValidatorAddressCodec().StringToBytes(valStr5)
+				s.Require().NoError(err)
+
+				bankKeeper.EXPECT().SendCoinsFromAccountToModule(gomock.Any(), sdk.AccAddress(valAddr4), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
+				bankKeeper.EXPECT().SendCoinsFromAccountToModule(gomock.Any(), sdk.AccAddress(valAddr5), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
+
+				// add 2 days to the current time and add rotate key, it should allow to rotate.
+				newCtx := ctx.WithHeaderInfo(header.Info{Time: ctx.BlockTime().Add(2 * 24 * time.Hour)})
+				req1, err := types.NewMsgRotateConsPubKey(valStr5, PKs[491])
+				s.Require().NoError(err)
+				_, err = msgServer.RotateConsPubKey(newCtx, req1)
+				s.Require().NoError(err)
+
+				// 1st rotation should pass, since limit is 1
+				req, err := types.NewMsgRotateConsPubKey(valStr4, PKs[490])
+				s.Require().NoError(err)
+				_, err = msgServer.RotateConsPubKey(ctx, req)
+				s.Require().NoError(err)
+
+				// this shouldn't mature the recent rotation since unbonding period isn't reached
+				s.Require().NoError(stakingKeeper.PurgeAllMaturedConsKeyRotatedKeys(ctx, ctx.BlockTime()))
+
+				// 2nd rotation should fail since limit exceeding
+				req, err = types.NewMsgRotateConsPubKey(valStr4, PKs[489])
+				s.Require().NoError(err)
+				_, err = msgServer.RotateConsPubKey(ctx, req)
+				s.Require().Error(err, "exceeding maximum consensus pubkey rotations within unbonding period")
+
+				// This should remove the keys from queue
+				// after setting the blocktime to reach the unbonding period,
+				// but other validator which rotated with addition of 2 days shouldn't be removed, so it should stop the rotation of valStr5.
+				newCtx1 := ctx.WithHeaderInfo(header.Info{Time: ctx.BlockTime().Add(params.UnbondingTime).Add(time.Hour)})
+				s.Require().NoError(stakingKeeper.PurgeAllMaturedConsKeyRotatedKeys(newCtx1, newCtx1.BlockTime()))
+				return newCtx1
+			},
+			isErr:     true,
+			newPubKey: PKs[492],
+			errMsg:    "exceeding maximum consensus pubkey rotations within unbonding period",
+			validator: validators[5].GetOperator(),
+		},
+	}
+
+	for _, tc := range testCases {
+		s.T().Run(tc.name, func(t *testing.T) {
+			newCtx := tc.malleate()
+
+			req, err := types.NewMsgRotateConsPubKey(tc.validator, tc.newPubKey)
+			s.Require().NoError(err)
+
+			_, err = msgServer.RotateConsPubKey(newCtx, req)
+			if tc.isErr {
+				s.Require().Error(err)
+				s.Require().Contains(err.Error(), tc.errMsg)
+			} else {
+				s.Require().NoError(err)
+				_, err = stakingKeeper.EndBlocker(newCtx)
+				s.Require().NoError(err)
+
+				addr, err := stakingKeeper.ValidatorAddressCodec().StringToBytes(tc.validator)
+				s.Require().NoError(err)
+
+				valInfo, err := stakingKeeper.GetValidator(newCtx, addr)
+				s.Require().NoError(err)
+				s.Require().Equal(valInfo.ConsensusPubkey, req.NewPubkey)
+			}
+		})
+	}
+}
diff --git a/x/staking/keeper/val_state_change.go b/x/staking/keeper/val_state_change.go
index 3d09937c1ef4..cbebd10410c9 100644
--- a/x/staking/keeper/val_state_change.go
+++ b/x/staking/keeper/val_state_change.go
@@ -10,10 +10,14 @@ import (
 	gogotypes "github.com/cosmos/gogoproto/types"
 
 	"cosmossdk.io/core/address"
+	errorsmod "cosmossdk.io/errors"
 	"cosmossdk.io/math"
 	"cosmossdk.io/x/staking/types"
 
+	cryptocodec "github.com/cosmos/cosmos-sdk/crypto/codec"
+	cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types"
 	sdk "github.com/cosmos/cosmos-sdk/types"
+	sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
 )
 
 // BlockValidatorUpdates calculates the ValidatorUpdates for the current block
@@ -112,6 +116,11 @@ func (k Keeper) BlockValidatorUpdates(ctx context.Context) ([]abci.ValidatorUpda
 		)
 	}
 
+	err = k.PurgeAllMaturedConsKeyRotatedKeys(sdkCtx, sdkCtx.HeaderInfo().Time)
+	if err != nil {
+		return nil, err
+	}
+
 	return validatorUpdates, nil
 }
 
@@ -241,6 +250,63 @@ func (k Keeper) ApplyAndReturnValidatorSetUpdates(ctx context.Context) (updates
 		updates = append(updates, validator.ABCIValidatorUpdateZero())
 	}
 
+	// ApplyAndReturnValidatorSetUpdates checks if there is ConsPubKeyRotationHistory
+	// with ConsPubKeyRotationHistory.RotatedHeight == ctx.BlockHeight() and if so, generates 2 ValidatorUpdate,
+	// one for a remove validator and one for create new validator
+	historyObjects, err := k.GetBlockConsPubKeyRotationHistory(ctx)
+	if err != nil {
+		return nil, err
+	}
+
+	for _, history := range historyObjects {
+		valAddr := history.OperatorAddress
+		if err != nil {
+			return nil, err
+		}
+
+		validator, err := k.GetValidator(ctx, valAddr)
+		if err != nil {
+			return nil, err
+		}
+
+		oldPk, ok := history.OldConsPubkey.GetCachedValue().(cryptotypes.PubKey)
+		if !ok {
+			return nil, errorsmod.Wrapf(sdkerrors.ErrInvalidType, "Expecting cryptotypes.PubKey, got %T", oldPk)
+		}
+		oldCmtPk, err := cryptocodec.ToCmtProtoPublicKey(oldPk)
+		if err != nil {
+			return nil, err
+		}
+
+		newPk, ok := history.NewConsPubkey.GetCachedValue().(cryptotypes.PubKey)
+		if !ok {
+			return nil, errorsmod.Wrapf(sdkerrors.ErrInvalidType, "Expecting cryptotypes.PubKey, got %T", oldPk)
+		}
+		newCmtPk, err := cryptocodec.ToCmtProtoPublicKey(newPk)
+		if err != nil {
+			return nil, err
+		}
+
+		// a validator cannot rotate keys if it's not bonded or if it's jailed
+		// - a validator can be unbonding state but jailed status false
+		// - a validator can be jailed and status can be unbonding
+		if !(validator.Jailed || validator.Status != types.Bonded) {
+			updates = append(updates, abci.ValidatorUpdate{
+				PubKey: oldCmtPk,
+				Power:  0,
+			})
+
+			updates = append(updates, abci.ValidatorUpdate{
+				PubKey: newCmtPk,
+				Power:  validator.ConsensusPower(powerReduction),
+			})
+
+			if err := k.updateToNewPubkey(ctx, validator, history.OldConsPubkey, history.NewConsPubkey, history.Fee); err != nil {
+				return nil, err
+			}
+		}
+	}
+
 	// Update the pools based on the recent updates in the validator set:
 	// - The tokens from the non-bonded candidates that enter the new validator set need to be transferred
 	// to the Bonded pool.
diff --git a/x/staking/keeper/validator.go b/x/staking/keeper/validator.go
index b95dd87a7742..ca3f0391fa8f 100644
--- a/x/staking/keeper/validator.go
+++ b/x/staking/keeper/validator.go
@@ -38,11 +38,21 @@ func (k Keeper) GetValidator(ctx context.Context, addr sdk.ValAddress) (validato
 func (k Keeper) GetValidatorByConsAddr(ctx context.Context, consAddr sdk.ConsAddress) (validator types.Validator, err error) {
 	opAddr, err := k.ValidatorByConsensusAddress.Get(ctx, consAddr)
 	if err != nil && !errors.Is(err, collections.ErrNotFound) {
-		return validator, err
+		// if the validator not found try to find it in the map of `OldToNewConsKeyMap`` because validator may've rotated it's key.
+		if !errors.Is(err, collections.ErrNotFound) {
+			return types.Validator{}, err
+		}
+
+		newConsAddr, err := k.OldToNewConsKeyMap.Get(ctx, consAddr)
+		if err != nil {
+			return types.Validator{}, err
+		}
+
+		opAddr = newConsAddr
 	}
 
 	if opAddr == nil {
-		return validator, types.ErrNoValidatorFound
+		return types.Validator{}, types.ErrNoValidatorFound
 	}
 
 	return k.GetValidator(ctx, opAddr)
diff --git a/x/staking/testutil/expected_keepers_mocks.go b/x/staking/testutil/expected_keepers_mocks.go
index 57ebb96eb140..fdf54c9085b2 100644
--- a/x/staking/testutil/expected_keepers_mocks.go
+++ b/x/staking/testutil/expected_keepers_mocks.go
@@ -13,7 +13,8 @@ import (
 	math "cosmossdk.io/math"
 	types "cosmossdk.io/x/staking/types"
 	crypto "github.com/cometbft/cometbft/proto/tendermint/crypto"
-	types0 "github.com/cosmos/cosmos-sdk/types"
+	types0 "github.com/cosmos/cosmos-sdk/crypto/types"
+	types1 "github.com/cosmos/cosmos-sdk/types"
 	gomock "github.com/golang/mock/gomock"
 )
 
@@ -55,10 +56,10 @@ func (mr *MockAccountKeeperMockRecorder) AddressCodec() *gomock.Call {
 }
 
 // GetAccount mocks base method.
-func (m *MockAccountKeeper) GetAccount(ctx context.Context, addr types0.AccAddress) types0.AccountI {
+func (m *MockAccountKeeper) GetAccount(ctx context.Context, addr types1.AccAddress) types1.AccountI {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "GetAccount", ctx, addr)
-	ret0, _ := ret[0].(types0.AccountI)
+	ret0, _ := ret[0].(types1.AccountI)
 	return ret0
 }
 
@@ -69,10 +70,10 @@ func (mr *MockAccountKeeperMockRecorder) GetAccount(ctx, addr interface{}) *gomo
 }
 
 // GetModuleAccount mocks base method.
-func (m *MockAccountKeeper) GetModuleAccount(ctx context.Context, moduleName string) types0.ModuleAccountI {
+func (m *MockAccountKeeper) GetModuleAccount(ctx context.Context, moduleName string) types1.ModuleAccountI {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "GetModuleAccount", ctx, moduleName)
-	ret0, _ := ret[0].(types0.ModuleAccountI)
+	ret0, _ := ret[0].(types1.ModuleAccountI)
 	return ret0
 }
 
@@ -83,10 +84,10 @@ func (mr *MockAccountKeeperMockRecorder) GetModuleAccount(ctx, moduleName interf
 }
 
 // GetModuleAddress mocks base method.
-func (m *MockAccountKeeper) GetModuleAddress(name string) types0.AccAddress {
+func (m *MockAccountKeeper) GetModuleAddress(name string) types1.AccAddress {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "GetModuleAddress", name)
-	ret0, _ := ret[0].(types0.AccAddress)
+	ret0, _ := ret[0].(types1.AccAddress)
 	return ret0
 }
 
@@ -97,7 +98,7 @@ func (mr *MockAccountKeeperMockRecorder) GetModuleAddress(name interface{}) *gom
 }
 
 // IterateAccounts mocks base method.
-func (m *MockAccountKeeper) IterateAccounts(ctx context.Context, process func(types0.AccountI) bool) {
+func (m *MockAccountKeeper) IterateAccounts(ctx context.Context, process func(types1.AccountI) bool) {
 	m.ctrl.T.Helper()
 	m.ctrl.Call(m, "IterateAccounts", ctx, process)
 }
@@ -109,7 +110,7 @@ func (mr *MockAccountKeeperMockRecorder) IterateAccounts(ctx, process interface{
 }
 
 // SetModuleAccount mocks base method.
-func (m *MockAccountKeeper) SetModuleAccount(arg0 context.Context, arg1 types0.ModuleAccountI) {
+func (m *MockAccountKeeper) SetModuleAccount(arg0 context.Context, arg1 types1.ModuleAccountI) {
 	m.ctrl.T.Helper()
 	m.ctrl.Call(m, "SetModuleAccount", arg0, arg1)
 }
@@ -144,7 +145,7 @@ func (m *MockBankKeeper) EXPECT() *MockBankKeeperMockRecorder {
 }
 
 // BurnCoins mocks base method.
-func (m *MockBankKeeper) BurnCoins(arg0 context.Context, arg1 []byte, arg2 types0.Coins) error {
+func (m *MockBankKeeper) BurnCoins(arg0 context.Context, arg1 []byte, arg2 types1.Coins) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "BurnCoins", arg0, arg1, arg2)
 	ret0, _ := ret[0].(error)
@@ -158,7 +159,7 @@ func (mr *MockBankKeeperMockRecorder) BurnCoins(arg0, arg1, arg2 interface{}) *g
 }
 
 // DelegateCoinsFromAccountToModule mocks base method.
-func (m *MockBankKeeper) DelegateCoinsFromAccountToModule(ctx context.Context, senderAddr types0.AccAddress, recipientModule string, amt types0.Coins) error {
+func (m *MockBankKeeper) DelegateCoinsFromAccountToModule(ctx context.Context, senderAddr types1.AccAddress, recipientModule string, amt types1.Coins) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "DelegateCoinsFromAccountToModule", ctx, senderAddr, recipientModule, amt)
 	ret0, _ := ret[0].(error)
@@ -172,10 +173,10 @@ func (mr *MockBankKeeperMockRecorder) DelegateCoinsFromAccountToModule(ctx, send
 }
 
 // GetAllBalances mocks base method.
-func (m *MockBankKeeper) GetAllBalances(ctx context.Context, addr types0.AccAddress) types0.Coins {
+func (m *MockBankKeeper) GetAllBalances(ctx context.Context, addr types1.AccAddress) types1.Coins {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "GetAllBalances", ctx, addr)
-	ret0, _ := ret[0].(types0.Coins)
+	ret0, _ := ret[0].(types1.Coins)
 	return ret0
 }
 
@@ -186,10 +187,10 @@ func (mr *MockBankKeeperMockRecorder) GetAllBalances(ctx, addr interface{}) *gom
 }
 
 // GetBalance mocks base method.
-func (m *MockBankKeeper) GetBalance(ctx context.Context, addr types0.AccAddress, denom string) types0.Coin {
+func (m *MockBankKeeper) GetBalance(ctx context.Context, addr types1.AccAddress, denom string) types1.Coin {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "GetBalance", ctx, addr, denom)
-	ret0, _ := ret[0].(types0.Coin)
+	ret0, _ := ret[0].(types1.Coin)
 	return ret0
 }
 
@@ -200,10 +201,10 @@ func (mr *MockBankKeeperMockRecorder) GetBalance(ctx, addr, denom interface{}) *
 }
 
 // GetSupply mocks base method.
-func (m *MockBankKeeper) GetSupply(ctx context.Context, denom string) types0.Coin {
+func (m *MockBankKeeper) GetSupply(ctx context.Context, denom string) types1.Coin {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "GetSupply", ctx, denom)
-	ret0, _ := ret[0].(types0.Coin)
+	ret0, _ := ret[0].(types1.Coin)
 	return ret0
 }
 
@@ -214,10 +215,10 @@ func (mr *MockBankKeeperMockRecorder) GetSupply(ctx, denom interface{}) *gomock.
 }
 
 // LockedCoins mocks base method.
-func (m *MockBankKeeper) LockedCoins(ctx context.Context, addr types0.AccAddress) types0.Coins {
+func (m *MockBankKeeper) LockedCoins(ctx context.Context, addr types1.AccAddress) types1.Coins {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "LockedCoins", ctx, addr)
-	ret0, _ := ret[0].(types0.Coins)
+	ret0, _ := ret[0].(types1.Coins)
 	return ret0
 }
 
@@ -228,7 +229,7 @@ func (mr *MockBankKeeperMockRecorder) LockedCoins(ctx, addr interface{}) *gomock
 }
 
 // SendCoinsFromAccountToModule mocks base method.
-func (m *MockBankKeeper) SendCoinsFromAccountToModule(ctx context.Context, senderAddr types0.AccAddress, recipientModule string, amt types0.Coins) error {
+func (m *MockBankKeeper) SendCoinsFromAccountToModule(ctx context.Context, senderAddr types1.AccAddress, recipientModule string, amt types1.Coins) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "SendCoinsFromAccountToModule", ctx, senderAddr, recipientModule, amt)
 	ret0, _ := ret[0].(error)
@@ -242,7 +243,7 @@ func (mr *MockBankKeeperMockRecorder) SendCoinsFromAccountToModule(ctx, senderAd
 }
 
 // SendCoinsFromModuleToModule mocks base method.
-func (m *MockBankKeeper) SendCoinsFromModuleToModule(ctx context.Context, senderPool, recipientPool string, amt types0.Coins) error {
+func (m *MockBankKeeper) SendCoinsFromModuleToModule(ctx context.Context, senderPool, recipientPool string, amt types1.Coins) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "SendCoinsFromModuleToModule", ctx, senderPool, recipientPool, amt)
 	ret0, _ := ret[0].(error)
@@ -256,10 +257,10 @@ func (mr *MockBankKeeperMockRecorder) SendCoinsFromModuleToModule(ctx, senderPoo
 }
 
 // SpendableCoins mocks base method.
-func (m *MockBankKeeper) SpendableCoins(ctx context.Context, addr types0.AccAddress) types0.Coins {
+func (m *MockBankKeeper) SpendableCoins(ctx context.Context, addr types1.AccAddress) types1.Coins {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "SpendableCoins", ctx, addr)
-	ret0, _ := ret[0].(types0.Coins)
+	ret0, _ := ret[0].(types1.Coins)
 	return ret0
 }
 
@@ -270,7 +271,7 @@ func (mr *MockBankKeeperMockRecorder) SpendableCoins(ctx, addr interface{}) *gom
 }
 
 // UndelegateCoinsFromModuleToAccount mocks base method.
-func (m *MockBankKeeper) UndelegateCoinsFromModuleToAccount(ctx context.Context, senderModule string, recipientAddr types0.AccAddress, amt types0.Coins) error {
+func (m *MockBankKeeper) UndelegateCoinsFromModuleToAccount(ctx context.Context, senderModule string, recipientAddr types1.AccAddress, amt types1.Coins) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "UndelegateCoinsFromModuleToAccount", ctx, senderModule, recipientAddr, amt)
 	ret0, _ := ret[0].(error)
@@ -307,10 +308,10 @@ func (m *MockValidatorSet) EXPECT() *MockValidatorSetMockRecorder {
 }
 
 // Delegation mocks base method.
-func (m *MockValidatorSet) Delegation(arg0 context.Context, arg1 types0.AccAddress, arg2 types0.ValAddress) (types0.DelegationI, error) {
+func (m *MockValidatorSet) Delegation(arg0 context.Context, arg1 types1.AccAddress, arg2 types1.ValAddress) (types1.DelegationI, error) {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "Delegation", arg0, arg1, arg2)
-	ret0, _ := ret[0].(types0.DelegationI)
+	ret0, _ := ret[0].(types1.DelegationI)
 	ret1, _ := ret[1].(error)
 	return ret0, ret1
 }
@@ -322,7 +323,7 @@ func (mr *MockValidatorSetMockRecorder) Delegation(arg0, arg1, arg2 interface{})
 }
 
 // GetPubKeyByConsAddr mocks base method.
-func (m *MockValidatorSet) GetPubKeyByConsAddr(arg0 context.Context, arg1 types0.ConsAddress) (crypto.PublicKey, error) {
+func (m *MockValidatorSet) GetPubKeyByConsAddr(arg0 context.Context, arg1 types1.ConsAddress) (crypto.PublicKey, error) {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "GetPubKeyByConsAddr", arg0, arg1)
 	ret0, _ := ret[0].(crypto.PublicKey)
@@ -337,7 +338,7 @@ func (mr *MockValidatorSetMockRecorder) GetPubKeyByConsAddr(arg0, arg1 interface
 }
 
 // IterateBondedValidatorsByPower mocks base method.
-func (m *MockValidatorSet) IterateBondedValidatorsByPower(arg0 context.Context, arg1 func(int64, types0.ValidatorI) bool) error {
+func (m *MockValidatorSet) IterateBondedValidatorsByPower(arg0 context.Context, arg1 func(int64, types1.ValidatorI) bool) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "IterateBondedValidatorsByPower", arg0, arg1)
 	ret0, _ := ret[0].(error)
@@ -351,7 +352,7 @@ func (mr *MockValidatorSetMockRecorder) IterateBondedValidatorsByPower(arg0, arg
 }
 
 // IterateValidators mocks base method.
-func (m *MockValidatorSet) IterateValidators(arg0 context.Context, arg1 func(int64, types0.ValidatorI) bool) error {
+func (m *MockValidatorSet) IterateValidators(arg0 context.Context, arg1 func(int64, types1.ValidatorI) bool) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "IterateValidators", arg0, arg1)
 	ret0, _ := ret[0].(error)
@@ -365,7 +366,7 @@ func (mr *MockValidatorSetMockRecorder) IterateValidators(arg0, arg1 interface{}
 }
 
 // Jail mocks base method.
-func (m *MockValidatorSet) Jail(arg0 context.Context, arg1 types0.ConsAddress) error {
+func (m *MockValidatorSet) Jail(arg0 context.Context, arg1 types1.ConsAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "Jail", arg0, arg1)
 	ret0, _ := ret[0].(error)
@@ -394,7 +395,7 @@ func (mr *MockValidatorSetMockRecorder) MaxValidators(arg0 interface{}) *gomock.
 }
 
 // Slash mocks base method.
-func (m *MockValidatorSet) Slash(arg0 context.Context, arg1 types0.ConsAddress, arg2, arg3 int64, arg4 math.LegacyDec) (math.Int, error) {
+func (m *MockValidatorSet) Slash(arg0 context.Context, arg1 types1.ConsAddress, arg2, arg3 int64, arg4 math.LegacyDec) (math.Int, error) {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "Slash", arg0, arg1, arg2, arg3, arg4)
 	ret0, _ := ret[0].(math.Int)
@@ -409,7 +410,7 @@ func (mr *MockValidatorSetMockRecorder) Slash(arg0, arg1, arg2, arg3, arg4 inter
 }
 
 // SlashWithInfractionReason mocks base method.
-func (m *MockValidatorSet) SlashWithInfractionReason(arg0 context.Context, arg1 types0.ConsAddress, arg2, arg3 int64, arg4 math.LegacyDec, arg5 stakingv1beta1.Infraction) (math.Int, error) {
+func (m *MockValidatorSet) SlashWithInfractionReason(arg0 context.Context, arg1 types1.ConsAddress, arg2, arg3 int64, arg4 math.LegacyDec, arg5 stakingv1beta1.Infraction) (math.Int, error) {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "SlashWithInfractionReason", arg0, arg1, arg2, arg3, arg4, arg5)
 	ret0, _ := ret[0].(math.Int)
@@ -454,7 +455,7 @@ func (mr *MockValidatorSetMockRecorder) TotalBondedTokens(arg0 interface{}) *gom
 }
 
 // Unjail mocks base method.
-func (m *MockValidatorSet) Unjail(arg0 context.Context, arg1 types0.ConsAddress) error {
+func (m *MockValidatorSet) Unjail(arg0 context.Context, arg1 types1.ConsAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "Unjail", arg0, arg1)
 	ret0, _ := ret[0].(error)
@@ -468,10 +469,10 @@ func (mr *MockValidatorSetMockRecorder) Unjail(arg0, arg1 interface{}) *gomock.C
 }
 
 // Validator mocks base method.
-func (m *MockValidatorSet) Validator(arg0 context.Context, arg1 types0.ValAddress) (types0.ValidatorI, error) {
+func (m *MockValidatorSet) Validator(arg0 context.Context, arg1 types1.ValAddress) (types1.ValidatorI, error) {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "Validator", arg0, arg1)
-	ret0, _ := ret[0].(types0.ValidatorI)
+	ret0, _ := ret[0].(types1.ValidatorI)
 	ret1, _ := ret[1].(error)
 	return ret0, ret1
 }
@@ -483,10 +484,10 @@ func (mr *MockValidatorSetMockRecorder) Validator(arg0, arg1 interface{}) *gomoc
 }
 
 // ValidatorByConsAddr mocks base method.
-func (m *MockValidatorSet) ValidatorByConsAddr(arg0 context.Context, arg1 types0.ConsAddress) (types0.ValidatorI, error) {
+func (m *MockValidatorSet) ValidatorByConsAddr(arg0 context.Context, arg1 types1.ConsAddress) (types1.ValidatorI, error) {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "ValidatorByConsAddr", arg0, arg1)
-	ret0, _ := ret[0].(types0.ValidatorI)
+	ret0, _ := ret[0].(types1.ValidatorI)
 	ret1, _ := ret[1].(error)
 	return ret0, ret1
 }
@@ -535,7 +536,7 @@ func (mr *MockDelegationSetMockRecorder) GetValidatorSet() *gomock.Call {
 }
 
 // IterateDelegations mocks base method.
-func (m *MockDelegationSet) IterateDelegations(ctx context.Context, delegator types0.AccAddress, fn func(int64, types0.DelegationI) bool) error {
+func (m *MockDelegationSet) IterateDelegations(ctx context.Context, delegator types1.AccAddress, fn func(int64, types1.DelegationI) bool) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "IterateDelegations", ctx, delegator, fn)
 	ret0, _ := ret[0].(error)
@@ -571,8 +572,22 @@ func (m *MockStakingHooks) EXPECT() *MockStakingHooksMockRecorder {
 	return m.recorder
 }
 
+// AfterConsensusPubKeyUpdate mocks base method.
+func (m *MockStakingHooks) AfterConsensusPubKeyUpdate(ctx context.Context, oldPubKey, newPubKey types0.PubKey, rotationFee types1.Coin) error {
+	m.ctrl.T.Helper()
+	ret := m.ctrl.Call(m, "AfterConsensusPubKeyUpdate", ctx, oldPubKey, newPubKey, rotationFee)
+	ret0, _ := ret[0].(error)
+	return ret0
+}
+
+// AfterConsensusPubKeyUpdate indicates an expected call of AfterConsensusPubKeyUpdate.
+func (mr *MockStakingHooksMockRecorder) AfterConsensusPubKeyUpdate(ctx, oldPubKey, newPubKey, rotationFee interface{}) *gomock.Call {
+	mr.mock.ctrl.T.Helper()
+	return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AfterConsensusPubKeyUpdate", reflect.TypeOf((*MockStakingHooks)(nil).AfterConsensusPubKeyUpdate), ctx, oldPubKey, newPubKey, rotationFee)
+}
+
 // AfterDelegationModified mocks base method.
-func (m *MockStakingHooks) AfterDelegationModified(ctx context.Context, delAddr types0.AccAddress, valAddr types0.ValAddress) error {
+func (m *MockStakingHooks) AfterDelegationModified(ctx context.Context, delAddr types1.AccAddress, valAddr types1.ValAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "AfterDelegationModified", ctx, delAddr, valAddr)
 	ret0, _ := ret[0].(error)
@@ -600,7 +615,7 @@ func (mr *MockStakingHooksMockRecorder) AfterUnbondingInitiated(ctx, id interfac
 }
 
 // AfterValidatorBeginUnbonding mocks base method.
-func (m *MockStakingHooks) AfterValidatorBeginUnbonding(ctx context.Context, consAddr types0.ConsAddress, valAddr types0.ValAddress) error {
+func (m *MockStakingHooks) AfterValidatorBeginUnbonding(ctx context.Context, consAddr types1.ConsAddress, valAddr types1.ValAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "AfterValidatorBeginUnbonding", ctx, consAddr, valAddr)
 	ret0, _ := ret[0].(error)
@@ -614,7 +629,7 @@ func (mr *MockStakingHooksMockRecorder) AfterValidatorBeginUnbonding(ctx, consAd
 }
 
 // AfterValidatorBonded mocks base method.
-func (m *MockStakingHooks) AfterValidatorBonded(ctx context.Context, consAddr types0.ConsAddress, valAddr types0.ValAddress) error {
+func (m *MockStakingHooks) AfterValidatorBonded(ctx context.Context, consAddr types1.ConsAddress, valAddr types1.ValAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "AfterValidatorBonded", ctx, consAddr, valAddr)
 	ret0, _ := ret[0].(error)
@@ -628,7 +643,7 @@ func (mr *MockStakingHooksMockRecorder) AfterValidatorBonded(ctx, consAddr, valA
 }
 
 // AfterValidatorCreated mocks base method.
-func (m *MockStakingHooks) AfterValidatorCreated(ctx context.Context, valAddr types0.ValAddress) error {
+func (m *MockStakingHooks) AfterValidatorCreated(ctx context.Context, valAddr types1.ValAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "AfterValidatorCreated", ctx, valAddr)
 	ret0, _ := ret[0].(error)
@@ -642,7 +657,7 @@ func (mr *MockStakingHooksMockRecorder) AfterValidatorCreated(ctx, valAddr inter
 }
 
 // AfterValidatorRemoved mocks base method.
-func (m *MockStakingHooks) AfterValidatorRemoved(ctx context.Context, consAddr types0.ConsAddress, valAddr types0.ValAddress) error {
+func (m *MockStakingHooks) AfterValidatorRemoved(ctx context.Context, consAddr types1.ConsAddress, valAddr types1.ValAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "AfterValidatorRemoved", ctx, consAddr, valAddr)
 	ret0, _ := ret[0].(error)
@@ -656,7 +671,7 @@ func (mr *MockStakingHooksMockRecorder) AfterValidatorRemoved(ctx, consAddr, val
 }
 
 // BeforeDelegationCreated mocks base method.
-func (m *MockStakingHooks) BeforeDelegationCreated(ctx context.Context, delAddr types0.AccAddress, valAddr types0.ValAddress) error {
+func (m *MockStakingHooks) BeforeDelegationCreated(ctx context.Context, delAddr types1.AccAddress, valAddr types1.ValAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "BeforeDelegationCreated", ctx, delAddr, valAddr)
 	ret0, _ := ret[0].(error)
@@ -670,7 +685,7 @@ func (mr *MockStakingHooksMockRecorder) BeforeDelegationCreated(ctx, delAddr, va
 }
 
 // BeforeDelegationRemoved mocks base method.
-func (m *MockStakingHooks) BeforeDelegationRemoved(ctx context.Context, delAddr types0.AccAddress, valAddr types0.ValAddress) error {
+func (m *MockStakingHooks) BeforeDelegationRemoved(ctx context.Context, delAddr types1.AccAddress, valAddr types1.ValAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "BeforeDelegationRemoved", ctx, delAddr, valAddr)
 	ret0, _ := ret[0].(error)
@@ -684,7 +699,7 @@ func (mr *MockStakingHooksMockRecorder) BeforeDelegationRemoved(ctx, delAddr, va
 }
 
 // BeforeDelegationSharesModified mocks base method.
-func (m *MockStakingHooks) BeforeDelegationSharesModified(ctx context.Context, delAddr types0.AccAddress, valAddr types0.ValAddress) error {
+func (m *MockStakingHooks) BeforeDelegationSharesModified(ctx context.Context, delAddr types1.AccAddress, valAddr types1.ValAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "BeforeDelegationSharesModified", ctx, delAddr, valAddr)
 	ret0, _ := ret[0].(error)
@@ -698,7 +713,7 @@ func (mr *MockStakingHooksMockRecorder) BeforeDelegationSharesModified(ctx, delA
 }
 
 // BeforeValidatorModified mocks base method.
-func (m *MockStakingHooks) BeforeValidatorModified(ctx context.Context, valAddr types0.ValAddress) error {
+func (m *MockStakingHooks) BeforeValidatorModified(ctx context.Context, valAddr types1.ValAddress) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "BeforeValidatorModified", ctx, valAddr)
 	ret0, _ := ret[0].(error)
@@ -712,7 +727,7 @@ func (mr *MockStakingHooksMockRecorder) BeforeValidatorModified(ctx, valAddr int
 }
 
 // BeforeValidatorSlashed mocks base method.
-func (m *MockStakingHooks) BeforeValidatorSlashed(ctx context.Context, valAddr types0.ValAddress, fraction math.LegacyDec) error {
+func (m *MockStakingHooks) BeforeValidatorSlashed(ctx context.Context, valAddr types1.ValAddress, fraction math.LegacyDec) error {
 	m.ctrl.T.Helper()
 	ret := m.ctrl.Call(m, "BeforeValidatorSlashed", ctx, valAddr, fraction)
 	ret0, _ := ret[0].(error)
diff --git a/x/staking/types/codec.go b/x/staking/types/codec.go
index 5f5b48e9afb7..9f3bec47a742 100644
--- a/x/staking/types/codec.go
+++ b/x/staking/types/codec.go
@@ -18,6 +18,7 @@ func RegisterLegacyAminoCodec(cdc *codec.LegacyAmino) {
 	legacy.RegisterAminoMsg(cdc, &MsgBeginRedelegate{}, "cosmos-sdk/MsgBeginRedelegate")
 	legacy.RegisterAminoMsg(cdc, &MsgCancelUnbondingDelegation{}, "cosmos-sdk/MsgCancelUnbondingDelegation")
 	legacy.RegisterAminoMsg(cdc, &MsgUpdateParams{}, "cosmos-sdk/x/staking/MsgUpdateParams")
+	legacy.RegisterAminoMsg(cdc, &MsgRotateConsPubKey{}, "cosmos-sdk/MsgRotateConsPubKey")
 
 	cdc.RegisterInterface((*isStakeAuthorization_Validators)(nil), nil)
 	cdc.RegisterConcrete(&StakeAuthorization_AllowList{}, "cosmos-sdk/StakeAuthorization/AllowList", nil)
diff --git a/x/staking/types/expected_keepers.go b/x/staking/types/expected_keepers.go
index 9f3c318471e0..e6b6b98956d6 100644
--- a/x/staking/types/expected_keepers.go
+++ b/x/staking/types/expected_keepers.go
@@ -9,6 +9,7 @@ import (
 	"cosmossdk.io/core/address"
 	"cosmossdk.io/math"
 
+	cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types"
 	sdk "github.com/cosmos/cosmos-sdk/types"
 )
 
@@ -107,6 +108,7 @@ type StakingHooks interface {
 	AfterDelegationModified(ctx context.Context, delAddr sdk.AccAddress, valAddr sdk.ValAddress) error
 	BeforeValidatorSlashed(ctx context.Context, valAddr sdk.ValAddress, fraction math.LegacyDec) error
 	AfterUnbondingInitiated(ctx context.Context, id uint64) error
+	AfterConsensusPubKeyUpdate(ctx context.Context, oldPubKey, newPubKey cryptotypes.PubKey, rotationFee sdk.Coin) error
 }
 
 // StakingHooksWrapper is a wrapper for modules to inject StakingHooks using depinject.
diff --git a/x/staking/types/hooks.go b/x/staking/types/hooks.go
index 8a052d9514a1..9fad002b0170 100644
--- a/x/staking/types/hooks.go
+++ b/x/staking/types/hooks.go
@@ -5,6 +5,7 @@ import (
 
 	sdkmath "cosmossdk.io/math"
 
+	cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types"
 	sdk "github.com/cosmos/cosmos-sdk/types"
 )
 
@@ -116,3 +117,12 @@ func (h MultiStakingHooks) AfterUnbondingInitiated(ctx context.Context, id uint6
 	}
 	return nil
 }
+
+func (h MultiStakingHooks) AfterConsensusPubKeyUpdate(ctx context.Context, oldPubKey, newPubKey cryptotypes.PubKey, rotationFee sdk.Coin) error {
+	for i := range h {
+		if err := h[i].AfterConsensusPubKeyUpdate(ctx, oldPubKey, newPubKey, rotationFee); err != nil {
+			return err
+		}
+	}
+	return nil
+}
diff --git a/x/staking/types/keys.go b/x/staking/types/keys.go
index b7d99c9db2ef..e764c9c86169 100644
--- a/x/staking/types/keys.go
+++ b/x/staking/types/keys.go
@@ -25,10 +25,10 @@ const (
 	// GovModuleName is the name of the gov module
 	GovModuleName = "gov"
 
-	// distributionModuleName duplicates the distribution module's name to avoid a cyclic dependency with x/distribution.
+	// PoolModuleName duplicates the Protocolpool module's name to avoid a cyclic dependency with x/protocolpool.
 	// It should be synced with the distribution module's name if it is ever changed.
 	// See: https://github.com/cosmos/cosmos-sdk/blob/912390d5fc4a32113ea1aacc98b77b2649aea4c2/x/distribution/types/keys.go#L15
-	DistributionModuleName = "distribution"
+	PoolModuleName = "protocolpool"
 )
 
 var (
@@ -68,7 +68,8 @@ var (
 	BlockConsPubKeyRotationHistoryKey           = collections.NewPrefix(102) // prefix for consPubkey rotation history by height
 	ValidatorConsensusKeyRotationRecordQueueKey = collections.NewPrefix(103) // this key is used to set the unbonding period time on each rotation
 	ValidatorConsensusKeyRotationRecordIndexKey = collections.NewPrefix(104) // this key is used to restrict the validator next rotation within waiting (unbonding) period
-	RotatedConsKeyMapIndex                      = collections.NewPrefix(105) // prefix for rotated cons address to new cons address
+	NewToOldConsKeyMap                          = collections.NewPrefix(105) // prefix for rotated cons address to new cons address
+	OldToNewConsKeyMap                          = collections.NewPrefix(106) // prefix for rotated cons address to new cons address
 )
 
 // UnbondingType defines the type of unbonding operation
diff --git a/x/staking/types/msg.go b/x/staking/types/msg.go
index 0119bc82adcc..683d8495a673 100644
--- a/x/staking/types/msg.go
+++ b/x/staking/types/msg.go
@@ -142,3 +142,35 @@ func NewMsgCancelUnbondingDelegation(delAddr, valAddr string, creationHeight int
 		CreationHeight:   creationHeight,
 	}
 }
+
+// NewMsgRotateConsPubKey creates a new MsgRotateConsPubKey instance.
+func NewMsgRotateConsPubKey(valAddr string, pubKey cryptotypes.PubKey) (*MsgRotateConsPubKey, error) {
+	var pkAny *codectypes.Any
+	if pubKey != nil {
+		var err error
+		if pkAny, err = codectypes.NewAnyWithValue(pubKey); err != nil {
+			return nil, err
+		}
+	}
+	return &MsgRotateConsPubKey{
+		ValidatorAddress: valAddr,
+		NewPubkey:        pkAny,
+	}, nil
+}
+
+// UnpackInterfaces implements UnpackInterfacesMessage.UnpackInterfaces
+func (msg MsgRotateConsPubKey) UnpackInterfaces(unpacker codectypes.AnyUnpacker) error {
+	var pubKey cryptotypes.PubKey
+	return unpacker.UnpackAny(msg.NewPubkey, &pubKey)
+}
+
+// UnpackInterfaces implements UnpackInterfacesMessage.UnpackInterfaces
+func (hi ConsPubKeyRotationHistory) UnpackInterfaces(unpacker codectypes.AnyUnpacker) error {
+	var oldPubKey cryptotypes.PubKey
+	err := unpacker.UnpackAny(hi.OldConsPubkey, &oldPubKey)
+	if err != nil {
+		return err
+	}
+	var newPubKey cryptotypes.PubKey
+	return unpacker.UnpackAny(hi.NewConsPubkey, &newPubKey)
+}