From 9aef1d0f0c665d548845d69001d4a8dc7ed6742d Mon Sep 17 00:00:00 2001 From: faultytolly <137398096+faultytolly@users.noreply.github.com> Date: Tue, 22 Oct 2024 15:20:51 +0200 Subject: [PATCH] feat: block production misbehavior detection (#1071) Co-authored-by: Faulty Tolly <@faulttolerance.net> --- .mockery.yaml | 8 +- block/block.go | 49 +- block/executor.go | 14 +- block/executor_test.go | 26 +- block/fraud.go | 32 ++ block/fraud_test.go | 35 ++ block/manager.go | 21 +- block/manager_test.go | 285 +++++++++- block/p2p.go | 2 +- block/produce.go | 5 +- block/retriever.go | 11 +- block/state.go | 7 +- block/submit_test.go | 2 +- .../dymint/block/mock_ExecutorI.go | 488 +++++++++++++++++ .../dymint/block/mock_FraudHandler.go | 70 +++ p2p/validator_test.go | 2 +- proto/types/dymint/state.proto | 1 + settlement/dymension/dymension_test.go | 2 +- settlement/local/local_test.go | 6 +- store/store.go | 2 +- testutil/types.go | 22 +- types/block.go | 2 +- types/errors.go | 301 ++++++++++ types/pb/dymint/state.pb.go | 144 +++-- types/serialization.go | 2 + types/state.go | 3 + types/validation.go | 66 ++- types/validation_test.go | 516 ++++++++++++++++++ 28 files changed, 2005 insertions(+), 119 deletions(-) create mode 100644 block/fraud.go create mode 100644 block/fraud_test.go create mode 100644 mocks/github.com/dymensionxyz/dymint/block/mock_ExecutorI.go create mode 100644 mocks/github.com/dymensionxyz/dymint/block/mock_FraudHandler.go create mode 100644 types/validation_test.go diff --git a/.mockery.yaml b/.mockery.yaml index d0ef54a13..8a0ac5738 100644 --- a/.mockery.yaml +++ b/.mockery.yaml @@ -35,7 +35,7 @@ packages: github.com/dymensionxyz/dymint/p2p: interfaces: GetProposerI: - - - - + github.com/dymensionxyz/dymint/block: + interfaces: + ExecutorI: + FraudHandler: diff --git a/block/block.go b/block/block.go index ecd544146..15b1f4e85 100644 --- a/block/block.go +++ b/block/block.go @@ -1,13 +1,50 @@ package block import ( + "context" + "errors" "fmt" + "github.com/dymensionxyz/gerr-cosmos/gerrc" + errorsmod "cosmossdk.io/errors" "github.com/dymensionxyz/dymint/types" ) +// applyBlockWithFraudHandling calls applyBlock and validateBlockBeforeApply with fraud handling. +func (m *Manager) applyBlockWithFraudHandling(block *types.Block, commit *types.Commit, blockMetaData types.BlockMetaData) error { + validateWithFraud := func() error { + if err := m.validateBlockBeforeApply(block, commit); err != nil { + if err != nil { + m.blockCache.Delete(block.Header.Height) + // TODO: can we take an action here such as dropping the peer / reducing their reputation? + + return fmt.Errorf("block not valid at height %d, dropping it: err:%w", block.Header.Height, err) + } + } + + if err := m.applyBlock(block, commit, blockMetaData); err != nil { + return fmt.Errorf("apply block: %w", err) + } + + return nil + } + + err := validateWithFraud() + if errors.Is(err, gerrc.ErrFault) { + // Here we handle the fault by calling the fraud handler. + // FraudHandler is an interface that defines a method to handle faults. Implement this interface to handle faults + // in specific ways. For example, once a fault is detected, it publishes a DataHealthStatus event to the + // pubsub which sets the node in a frozen state. + m.FraudHandler.HandleFault(context.Background(), err) + + return err + } + + return nil +} + // applyBlock applies the block to the store and the abci app. // Contract: block and commit must be validated before calling this function! // steps: save block -> execute block with app -> update state -> commit block to app -> update state's height and commit result. @@ -35,7 +72,7 @@ func (m *Manager) applyBlock(block *types.Block, commit *types.Commit, blockMeta // In case the following true, it means we crashed after the app commit but before updating the state // In that case we'll want to align the state with the app commit result, as if the block was applied. if isBlockAlreadyApplied { - err := m.UpdateStateFromApp() + err := m.UpdateStateFromApp(block.Header.Hash()) if err != nil { return fmt.Errorf("update state from app: %w", err) } @@ -83,7 +120,7 @@ func (m *Manager) applyBlock(block *types.Block, commit *types.Commit, blockMeta // Update the state with the new app hash, and store height from the commit. // Every one of those, if happens before commit, prevents us from re-executing the block in case failed during commit. - m.Executor.UpdateStateAfterCommit(m.State, responses, appHash, block.Header.Height) + m.Executor.UpdateStateAfterCommit(m.State, responses, appHash, block.Header.Height, block.Header.Hash()) } // check if the proposer needs to be changed @@ -150,18 +187,12 @@ func (m *Manager) attemptApplyCachedBlocks() error { if !blockExists { break } - if err := m.validateBlockBeforeApply(cachedBlock.Block, cachedBlock.Commit); err != nil { - m.blockCache.Delete(cachedBlock.Block.Header.Height) - // TODO: can we take an action here such as dropping the peer / reducing their reputation? - return fmt.Errorf("block not valid at height %d, dropping it: err:%w", cachedBlock.Block.Header.Height, err) - } - err := m.applyBlock(cachedBlock.Block, cachedBlock.Commit, types.BlockMetaData{Source: cachedBlock.Source}) + err := m.applyBlockWithFraudHandling(cachedBlock.Block, cachedBlock.Commit, types.BlockMetaData{Source: cachedBlock.Source}) if err != nil { return fmt.Errorf("apply cached block: expected height: %d: %w", expectedHeight, err) } m.logger.Info("Block applied", "height", expectedHeight) - } return nil diff --git a/block/executor.go b/block/executor.go index b5484c040..96a037ea6 100644 --- a/block/executor.go +++ b/block/executor.go @@ -22,6 +22,18 @@ import ( // default minimum block max size allowed. not specific reason to set it to 10K, but we need to avoid no transactions can be included in a block. const minBlockMaxBytes = 10000 +type ExecutorI interface { + InitChain(genesis *tmtypes.GenesisDoc, valset []*tmtypes.Validator) (*abci.ResponseInitChain, error) + CreateBlock(height uint64, lastCommit *types.Commit, lastHeaderHash, nextSeqHash [32]byte, state *types.State, maxBlockDataSizeBytes uint64) *types.Block + Commit(state *types.State, block *types.Block, resp *tmstate.ABCIResponses) ([]byte, int64, error) + GetAppInfo() (*abci.ResponseInfo, error) + ExecuteBlock(state *types.State, block *types.Block) (*tmstate.ABCIResponses, error) + UpdateStateAfterInitChain(s *types.State, res *abci.ResponseInitChain) + UpdateMempoolAfterInitChain(s *types.State) + UpdateStateAfterCommit(s *types.State, resp *tmstate.ABCIResponses, appHash []byte, height uint64, lastHeaderHash [32]byte) + UpdateProposerFromBlock(s *types.State, block *types.Block) bool +} + // Executor creates and applies blocks and maintains state. type Executor struct { localAddress []byte @@ -46,7 +58,7 @@ func NewExecutor( eventBus *tmtypes.EventBus, consensusMessagesStream ConsensusMessagesStream, logger types.Logger, -) (*Executor, error) { +) (ExecutorI, error) { be := Executor{ localAddress: localAddress, chainID: chainID, diff --git a/block/executor_test.go b/block/executor_test.go index e7888576e..fb4e28dc5 100644 --- a/block/executor_test.go +++ b/block/executor_test.go @@ -183,6 +183,11 @@ func TestApplyBlock(t *testing.T) { logger := log.TestingLogger() + // Create a valid proposer for the block + proposerKey := ed25519.GenPrivKey() + tmPubKey, err := cryptocodec.ToTmPubKeyInterface(proposerKey.PubKey()) + require.NoError(err) + // Mock ABCI app app := &tmmocks.MockApplication{} app.On("CheckTx", mock.Anything).Return(abci.ResponseCheckTx{}) @@ -201,7 +206,7 @@ func TestApplyBlock(t *testing.T) { }, }) var mockAppHash [32]byte - _, err := rand.Read(mockAppHash[:]) + _, err = rand.Read(mockAppHash[:]) require.NoError(err) app.On("Commit", mock.Anything).Return(abci.ResponseCommit{ Data: mockAppHash[:], @@ -229,7 +234,7 @@ func TestApplyBlock(t *testing.T) { appConns := &tmmocksproxy.MockAppConns{} appConns.On("Consensus").Return(abciClient) appConns.On("Query").Return(abciClient) - executor, err := block.NewExecutor([]byte("test address"), chainID, mpool, appConns, eventBus, nil, logger) + executor, err := block.NewExecutor(proposerKey.PubKey().Address(), chainID, mpool, appConns, eventBus, nil, logger) assert.NoError(err) // Subscribe to tx events @@ -246,25 +251,22 @@ func TestApplyBlock(t *testing.T) { require.NoError(err) require.NotNil(headerSub) - // Create a valid proposer for the block - proposerKey := ed25519.GenPrivKey() - tmPubKey, err := cryptocodec.ToTmPubKeyInterface(proposerKey.PubKey()) - require.NoError(err) - // Init state state := &types.State{} state.Sequencers.SetProposer(types.NewSequencerFromValidator(*tmtypes.NewValidator(tmPubKey, 1))) state.InitialHeight = 1 + state.ChainID = chainID state.SetHeight(0) maxBytes := uint64(10000) state.ConsensusParams.Block.MaxBytes = int64(maxBytes) state.ConsensusParams.Block.MaxGas = 100000 state.RollappParams.Da = "mock" + state.LastHeaderHash = [32]byte{0x01} // Create first block with one Tx from mempool _ = mpool.CheckTx([]byte{1, 2, 3, 4}, func(r *abci.Response) {}, mempool.TxInfo{}) require.NoError(err) - block := executor.CreateBlock(1, &types.Commit{Height: 0}, [32]byte{}, [32]byte(state.Sequencers.ProposerHash()), state, maxBytes) + block := executor.CreateBlock(1, &types.Commit{Height: 0}, [32]byte{0x01}, [32]byte(state.Sequencers.ProposerHash()), state, maxBytes) require.NotNil(block) assert.Equal(uint64(1), block.Header.Height) assert.Len(block.Data.Txs, 1) @@ -290,7 +292,7 @@ func TestApplyBlock(t *testing.T) { require.NotNil(resp) appHash, _, err := executor.Commit(state, block, resp) require.NoError(err) - executor.UpdateStateAfterCommit(state, resp, appHash, block.Header.Height) + executor.UpdateStateAfterCommit(state, resp, appHash, block.Header.Height, block.Header.Hash()) assert.Equal(uint64(1), state.Height()) assert.Equal(mockAppHash, state.AppHash) @@ -299,7 +301,7 @@ func TestApplyBlock(t *testing.T) { require.NoError(mpool.CheckTx([]byte{5, 6, 7, 8, 9}, func(r *abci.Response) {}, mempool.TxInfo{})) require.NoError(mpool.CheckTx([]byte{1, 2, 3, 4, 5}, func(r *abci.Response) {}, mempool.TxInfo{})) require.NoError(mpool.CheckTx(make([]byte, 9990), func(r *abci.Response) {}, mempool.TxInfo{})) - block = executor.CreateBlock(2, commit, [32]byte{}, [32]byte(state.Sequencers.ProposerHash()), state, maxBytes) + block = executor.CreateBlock(2, commit, block.Header.Hash(), [32]byte(state.Sequencers.ProposerHash()), state, maxBytes) require.NotNil(block) assert.Equal(uint64(2), block.Header.Height) assert.Len(block.Data.Txs, 3) @@ -321,7 +323,7 @@ func TestApplyBlock(t *testing.T) { // Apply the block with an invalid commit err = types.ValidateProposedTransition(state, block, invalidCommit, state.Sequencers.GetProposerPubKey()) - require.ErrorIs(err, types.ErrInvalidSignature) + require.ErrorContains(err, types.ErrInvalidSignature.Error()) // Create a valid commit for the block signature, err = proposerKey.Sign(abciHeaderBytes) @@ -340,7 +342,7 @@ func TestApplyBlock(t *testing.T) { require.NotNil(resp) _, _, err = executor.Commit(state, block, resp) require.NoError(err) - executor.UpdateStateAfterCommit(state, resp, appHash, block.Header.Height) + executor.UpdateStateAfterCommit(state, resp, appHash, block.Header.Height, block.Header.Hash()) assert.Equal(uint64(2), state.Height()) // check rollapp params update diff --git a/block/fraud.go b/block/fraud.go new file mode 100644 index 000000000..b3533e275 --- /dev/null +++ b/block/fraud.go @@ -0,0 +1,32 @@ +package block + +import ( + "context" + + "github.com/dymensionxyz/dymint/node/events" + uevent "github.com/dymensionxyz/dymint/utils/event" +) + +// FraudHandler is an interface that defines a method to handle faults. +// Contract: should not be blocking. +type FraudHandler interface { + // HandleFault handles a fault that occurred in the system. + // The fault is passed as an error type. + HandleFault(ctx context.Context, fault error) +} + +// FreezeHandler is used to handle faults coming from executing and validating blocks. +// once a fault is detected, it publishes a DataHealthStatus event to the pubsub which sets the node in a frozen state. +type FreezeHandler struct { + m *Manager +} + +func (f FreezeHandler) HandleFault(ctx context.Context, fault error) { + uevent.MustPublish(ctx, f.m.Pubsub, &events.DataHealthStatus{Error: fault}, events.HealthStatusList) +} + +func NewFreezeHandler(manager *Manager) *FreezeHandler { + return &FreezeHandler{ + m: manager, + } +} diff --git a/block/fraud_test.go b/block/fraud_test.go new file mode 100644 index 000000000..179b95324 --- /dev/null +++ b/block/fraud_test.go @@ -0,0 +1,35 @@ +package block_test + +import ( + "errors" + "testing" + + "github.com/dymensionxyz/gerr-cosmos/gerrc" +) + +type mockError struct { + name string + data string +} + +func (m mockError) Error() string { + return "some string" +} + +func (mockError) Unwrap() error { + return gerrc.ErrFault +} + +func TestErrorIsErrFault(t *testing.T) { + err := mockError{name: "test", data: "test"} + + if !errors.Is(err, gerrc.ErrFault) { + t.Error("Expected Is to return true") + } + + anotherErr := errors.New("some error") + + if errors.Is(anotherErr, gerrc.ErrFault) { + t.Error("Expected Is to return false") + } +} diff --git a/block/manager.go b/block/manager.go index 89ee16df0..e3b16b84e 100644 --- a/block/manager.go +++ b/block/manager.go @@ -21,15 +21,13 @@ import ( tmcrypto "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/libs/log" "github.com/tendermint/tendermint/libs/pubsub" - tmtypes "github.com/tendermint/tendermint/types" - - "github.com/dymensionxyz/dymint/p2p" - "github.com/tendermint/tendermint/proxy" + tmtypes "github.com/tendermint/tendermint/types" "github.com/dymensionxyz/dymint/config" "github.com/dymensionxyz/dymint/da" "github.com/dymensionxyz/dymint/mempool" + "github.com/dymensionxyz/dymint/p2p" "github.com/dymensionxyz/dymint/settlement" "github.com/dymensionxyz/dymint/types" ) @@ -46,7 +44,7 @@ type Manager struct { // Store and execution Store store.Store State *types.State - Executor *Executor + Executor ExecutorI // Clients and servers Pubsub *pubsub.Server @@ -79,6 +77,9 @@ type Manager struct { // TargetHeight holds the value of the current highest block seen from either p2p (probably higher) or the DA TargetHeight atomic.Uint64 + // Fraud handler + FraudHandler FraudHandler + // channel used to send the retain height to the pruning background loop pruningC chan int64 @@ -135,6 +136,7 @@ func NewManager( }, pruningC: make(chan int64, 10), // use of buffered channel to avoid blocking applyBlock thread. In case channel is full, pruning will be skipped, but the retain height can be pruned in the next iteration. } + m.setFraudHandler(NewFreezeHandler(m)) err = m.LoadStateOnInit(store, genesis, logger) if err != nil { @@ -197,8 +199,8 @@ func (m *Manager) Start(ctx context.Context) error { }() // P2P Sync. Subscribe to P2P received blocks events - go uevent.MustSubscribe(ctx, m.Pubsub, "applyGossipedBlocksLoop", p2p.EventQueryNewGossipedBlock, m.onReceivedBlock, m.logger) - go uevent.MustSubscribe(ctx, m.Pubsub, "applyBlockSyncBlocksLoop", p2p.EventQueryNewBlockSyncBlock, m.onReceivedBlock, m.logger) + go uevent.MustSubscribe(ctx, m.Pubsub, "applyGossipedBlocksLoop", p2p.EventQueryNewGossipedBlock, m.OnReceivedBlock, m.logger) + go uevent.MustSubscribe(ctx, m.Pubsub, "applyBlockSyncBlocksLoop", p2p.EventQueryNewBlockSyncBlock, m.OnReceivedBlock, m.logger) return nil } @@ -352,3 +354,8 @@ func (m *Manager) setDA(daconfig string, dalcKV store.KV, logger log.Logger) err m.Retriever = retriever return nil } + +// setFraudHandler sets the fraud handler for the block manager. +func (m *Manager) setFraudHandler(handler *FreezeHandler) { + m.FraudHandler = handler +} diff --git a/block/manager_test.go b/block/manager_test.go index 287757189..f153a19b2 100644 --- a/block/manager_test.go +++ b/block/manager_test.go @@ -3,6 +3,8 @@ package block_test import ( "context" "crypto/rand" + "errors" + "github.com/dymensionxyz/gerr-cosmos/gerrc" "sync/atomic" "testing" "time" @@ -30,8 +32,11 @@ import ( "github.com/dymensionxyz/dymint/config" "github.com/dymensionxyz/dymint/da" + blockmocks "github.com/dymensionxyz/dymint/mocks/github.com/dymensionxyz/dymint/block" + "github.com/dymensionxyz/dymint/node/events" slregistry "github.com/dymensionxyz/dymint/settlement/registry" "github.com/dymensionxyz/dymint/store" + "github.com/dymensionxyz/dymint/utils/event" ) // TODO: test loading sequencer while rotation in progress @@ -154,15 +159,20 @@ func TestProduceOnlyAfterSynced(t *testing.T) { numBatchesToAdd := 2 nextBatchStartHeight := manager.NextHeightToSubmit() + var lastBlockHeaderHash [32]byte var batch *types.Batch for i := 0; i < numBatchesToAdd; i++ { - batch, err = testutil.GenerateBatch(nextBatchStartHeight, nextBatchStartHeight+uint64(testutil.DefaultTestBatchSize-1), manager.LocalKey) + batch, err = testutil.GenerateBatch( + nextBatchStartHeight, nextBatchStartHeight+uint64(testutil.DefaultTestBatchSize-1), manager.LocalKey, + lastBlockHeaderHash, + ) assert.NoError(t, err) daResultSubmitBatch := manager.DAClient.SubmitBatch(batch) assert.Equal(t, daResultSubmitBatch.Code, da.StatusSuccess) err = manager.SLClient.SubmitBatch(batch, manager.DAClient.GetClientType(), &daResultSubmitBatch) require.NoError(t, err) nextBatchStartHeight = batch.EndHeight() + 1 + lastBlockHeaderHash = batch.Blocks[len(batch.Blocks)-1].Header.Hash() // Wait until daHeight is updated time.Sleep(time.Millisecond * 500) } @@ -188,6 +198,194 @@ func TestProduceOnlyAfterSynced(t *testing.T) { assert.Greater(t, manager.State.Height(), batch.EndHeight()) } +// TestApplyCachedBlocks checks the flow that happens when we are receiving blocks from p2p and some of the blocks +// are already cached. This means blocks that were gossiped but are bigger than the expected next block height. +func TestApplyCachedBlocks_WithFraudCheck(t *testing.T) { + // Init app + app := testutil.GetAppMock(testutil.EndBlock) + app.On("EndBlock", mock.Anything).Return(abci.ResponseEndBlock{ + RollappParamUpdates: &abci.RollappParams{ + Da: "mock", + Version: version.Commit, + }, + ConsensusParamUpdates: &abci.ConsensusParams{ + Block: &abci.BlockParams{ + MaxGas: 40000000, + MaxBytes: 500000, + }, + }, + }) + // Create proxy app + clientCreator := proxy.NewLocalClientCreator(app) + proxyApp := proxy.NewAppConns(clientCreator) + err := proxyApp.Start() + require.NoError(t, err) + manager, err := testutil.GetManager(testutil.GetManagerConfig(), nil, 1, 1, 0, proxyApp, nil) + require.NoError(t, err) + require.NotNil(t, manager) + + t.Log("Taking the manager out of sync by submitting a batch") + manager.DAClient = testutil.GetMockDALC(log.TestingLogger()) + manager.Retriever = manager.DAClient.(da.BatchRetriever) + + mockExecutor := &blockmocks.MockExecutorI{} + manager.Executor = mockExecutor + mockExecutor.On("GetAppInfo").Return(&abci.ResponseInfo{ + LastBlockHeight: int64(0), + }, nil) + mockExecutor.On("ExecuteBlock", mock.Anything, mock.Anything).Return(nil, gerrc.ErrFault) + + // Check that handle fault is called + manager.FraudHandler = block.NewFreezeHandler(manager) + + fraudEventReceived := make(chan *events.DataHealthStatus, 1) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + go event.MustSubscribe( + ctx, + manager.Pubsub, + "testFraudClient", + events.QueryHealthStatus, + func(msg pubsub.Message) { + event, ok := msg.Data().(*events.DataHealthStatus) + if !ok { + t.Errorf("Unexpected event type received: %T", msg.Data()) + return + } + fraudEventReceived <- event + }, + log.NewNopLogger(), + ) + + numBatchesToAdd := 1 + nextBatchStartHeight := manager.NextHeightToSubmit() + var batch *types.Batch + for i := 0; i < numBatchesToAdd; i++ { + batch, err = testutil.GenerateBatch(nextBatchStartHeight, nextBatchStartHeight, manager.LocalKey, [32]byte{}) + assert.NoError(t, err) + blockData := p2p.BlockData{Block: *batch.Blocks[0], Commit: *batch.Commits[0]} + msg := pubsub.NewMessage(blockData, map[string][]string{p2p.EventTypeKey: {p2p.EventNewGossipedBlock}}) + manager.OnReceivedBlock(msg) + + // Wait until daHeight is updated + time.Sleep(time.Millisecond * 500) + } + + select { + case receivedEvent := <-fraudEventReceived: + if receivedEvent.Error == nil { + t.Error("there should be an error in the event") + } else if !errors.Is(receivedEvent.Error, gerrc.ErrFault) { + t.Errorf("Unexpected error received, expected: %v, got: %v", gerrc.ErrFault, receivedEvent.Error) + } + case <-time.After(5 * time.Second): + t.Error("timeout waiting for fraud event") + } + + mockExecutor.AssertExpectations(t) +} + +// TestApplyLocalBlock checks the flow that happens when there is a block saved on the Store and we apply it locally. +func TestApplyLocalBlock_WithFraudCheck(t *testing.T) { + // Init app + app := testutil.GetAppMock(testutil.EndBlock) + app.On("EndBlock", mock.Anything).Return(abci.ResponseEndBlock{ + RollappParamUpdates: &abci.RollappParams{ + Da: "mock", + Version: version.Commit, + }, + ConsensusParamUpdates: &abci.ConsensusParams{ + Block: &abci.BlockParams{ + MaxGas: 40000000, + MaxBytes: 500000, + }, + }, + }) + + // Create proxy app + clientCreator := proxy.NewLocalClientCreator(app) + proxyApp := proxy.NewAppConns(clientCreator) + err := proxyApp.Start() + require.NoError(t, err) + + manager, err := testutil.GetManager(testutil.GetManagerConfig(), nil, 1, 1, 0, proxyApp, nil) + require.NoError(t, err) + require.NotNil(t, manager) + + t.Log("Taking the manager out of sync by submitting a batch") + manager.DAClient = testutil.GetMockDALC(log.TestingLogger()) + manager.Retriever = manager.DAClient.(da.BatchRetriever) + + numBatchesToAdd := 2 + nextBatchStartHeight := manager.NextHeightToSubmit() + + var batch *types.Batch + for i := 0; i < numBatchesToAdd; i++ { + batch, err = testutil.GenerateBatch( + nextBatchStartHeight, + nextBatchStartHeight+uint64(testutil.DefaultTestBatchSize-1), + manager.LocalKey, + [32]byte{}, + ) + assert.NoError(t, err) + + // Save one block on state to enforce local block application + _, err := manager.Store.SaveBlock(batch.Blocks[0], batch.Commits[0], nil) + require.NoError(t, err) + + daResultSubmitBatch := manager.DAClient.SubmitBatch(batch) + assert.Equal(t, daResultSubmitBatch.Code, da.StatusSuccess) + + err = manager.SLClient.SubmitBatch(batch, manager.DAClient.GetClientType(), &daResultSubmitBatch) + require.NoError(t, err) + + nextBatchStartHeight = batch.EndHeight() + 1 + + time.Sleep(time.Millisecond * 500) + } + + mockExecutor := &blockmocks.MockExecutorI{} + manager.Executor = mockExecutor + mockExecutor.On("InitChain", mock.Anything, mock.Anything).Return(&abci.ResponseInitChain{}, nil) + mockExecutor.On("GetAppInfo").Return(&abci.ResponseInfo{ + LastBlockHeight: int64(batch.EndHeight()), + }, nil) + mockExecutor.On("UpdateStateAfterInitChain", mock.Anything, mock.Anything).Return(nil) + mockExecutor.On("UpdateMempoolAfterInitChain", mock.Anything).Return(nil) + mockExecutor.On("ExecuteBlock", mock.Anything, mock.Anything).Return(nil, gerrc.ErrFault) + + // Check that handle fault is called + mockFraudHandler := &blockmocks.MockFraudHandler{} + manager.FraudHandler = mockFraudHandler + + mockFraudHandler.On("HandleFault", mock.Anything, mock.MatchedBy(func(err error) bool { + return errors.Is(err, gerrc.ErrFault) + })).Return(nil) + + // Initially sync target is 0 + assert.Zero(t, manager.LastSubmittedHeight.Load()) + assert.True(t, manager.State.Height() == 0) + + // enough time to sync and produce blocks + ctx, cancel := context.WithTimeout(context.Background(), time.Second*4) + defer cancel() + // Capture the error returned by manager.Start. + + errChan := make(chan error, 1) + go func() { + errChan <- manager.Start(ctx) + err := <-errChan + require.True(t, errors.Is(err, gerrc.ErrFault)) + }() + <-ctx.Done() + assert.Equal(t, batch.EndHeight(), manager.LastSubmittedHeight.Load()) + + mockExecutor.AssertExpectations(t) + mockFraudHandler.AssertExpectations(t) +} + func TestRetrieveDaBatchesFailed(t *testing.T) { manager, err := testutil.GetManager(testutil.GetManagerConfig(), nil, 1, 1, 0, nil, nil) require.NoError(t, err) @@ -492,7 +690,7 @@ func TestDAFetch(t *testing.T) { // Init manager manager, err := testutil.GetManager(testutil.GetManagerConfig(), nil, 1, 1, 0, proxyApp, mockStore) require.NoError(err) - commitHash := [32]byte{1} + commitHash := [32]byte{} manager.DAClient = testutil.GetMockDALC(log.TestingLogger()) manager.Retriever = manager.DAClient.(da.BatchRetriever) @@ -500,7 +698,12 @@ func TestDAFetch(t *testing.T) { app.On("Commit", mock.Anything).Return(abci.ResponseCommit{Data: commitHash[:]}) nextBatchStartHeight := manager.NextHeightToSubmit() - batch, err := testutil.GenerateBatch(nextBatchStartHeight, nextBatchStartHeight+uint64(testutil.DefaultTestBatchSize-1), manager.LocalKey) + batch, err := testutil.GenerateBatch( + nextBatchStartHeight, + nextBatchStartHeight+uint64(testutil.DefaultTestBatchSize-1), + manager.LocalKey, + [32]byte{}, + ) require.NoError(err) daResultSubmitBatch := manager.DAClient.SubmitBatch(batch) require.Equal(daResultSubmitBatch.Code, da.StatusSuccess) @@ -543,6 +746,82 @@ func TestDAFetch(t *testing.T) { } } +// TestManager_ProcessNextDABatch_FraudHandling tests the case when the manager receives a fraud when the block is +// part of the batch received from the DA. +func TestManager_ProcessNextDABatch_FraudHandling(t *testing.T) { + require := require.New(t) + // Setup app + app := testutil.GetAppMock(testutil.Info, testutil.Commit, testutil.EndBlock) + app.On("EndBlock", mock.Anything).Return(abci.ResponseEndBlock{ + RollappParamUpdates: &abci.RollappParams{ + Da: "mock", + Version: version.Commit, + }, + ConsensusParamUpdates: &abci.ConsensusParams{ + Block: &abci.BlockParams{ + MaxGas: 40000000, + MaxBytes: 500000, + }, + }, + }) + // Create proxy app + clientCreator := proxy.NewLocalClientCreator(app) + proxyApp := proxy.NewAppConns(clientCreator) + err := proxyApp.Start() + require.NoError(err) + // Create a new mock store which should succeed to save the first block + mockStore := testutil.NewMockStore() + // Init manager + manager, err := testutil.GetManager(testutil.GetManagerConfig(), nil, 1, 1, 0, proxyApp, mockStore) + require.NoError(err) + commitHash := [32]byte{1} + manager.DAClient = testutil.GetMockDALC(log.TestingLogger()) + manager.Retriever = manager.DAClient.(da.BatchRetriever) + app.On("Commit", mock.Anything).Return(abci.ResponseCommit{Data: commitHash[:]}) + nextBatchStartHeight := manager.NextHeightToSubmit() + batch, err := testutil.GenerateBatch( + nextBatchStartHeight, + nextBatchStartHeight+uint64(testutil.DefaultTestBatchSize-1), + manager.LocalKey, + [32]byte{}, + ) + require.NoError(err) + daResultSubmitBatch := manager.DAClient.SubmitBatch(batch) + require.Equal(daResultSubmitBatch.Code, da.StatusSuccess) + err = manager.SLClient.SubmitBatch(batch, manager.DAClient.GetClientType(), &daResultSubmitBatch) + require.NoError(err) + + //// Mock Executor to return ErrFraud + mockExecutor := &blockmocks.MockExecutorI{} + manager.Executor = mockExecutor + mockExecutor.On("GetAppInfo").Return(&abci.ResponseInfo{ + LastBlockHeight: int64(batch.EndHeight()), + }, nil) + mockExecutor.On("ExecuteBlock", mock.Anything, mock.Anything).Return(nil, gerrc.ErrFault) + + // Check that handle fault is called + mockFraudHandler := &blockmocks.MockFraudHandler{} + manager.FraudHandler = mockFraudHandler + + mockFraudHandler.On("HandleFault", mock.Anything, mock.MatchedBy(func(err error) bool { + return errors.Is(err, gerrc.ErrFault) + })).Return(nil) + + app.On("Commit", mock.Anything).Return(abci.ResponseCommit{Data: commitHash[:]}).Once() + app.On("Info", mock.Anything).Return(abci.ResponseInfo{ + LastBlockHeight: int64(batch.EndHeight()), + LastBlockAppHash: commitHash[:], + }) + + // Call ProcessNextDABatch + err = manager.ProcessNextDABatch(daResultSubmitBatch.SubmitMetaData) + + // Verify + require.True(errors.Is(err, gerrc.ErrFault)) + mockExecutor.AssertExpectations(t) + mockFraudHandler.AssertExpectations(t) +} + func TestManager_updateTargetHeight(t *testing.T) { tests := []struct { name string diff --git a/block/p2p.go b/block/p2p.go index 1fc904bf9..5e7b92697 100644 --- a/block/p2p.go +++ b/block/p2p.go @@ -12,7 +12,7 @@ import ( ) // onReceivedBlock receives a block received event from P2P, saves the block to a cache and tries to apply the blocks from the cache. -func (m *Manager) onReceivedBlock(event pubsub.Message) { +func (m *Manager) OnReceivedBlock(event pubsub.Message) { eventData, ok := event.Data().(p2p.BlockData) if !ok { m.logger.Error("onReceivedBlock", "err", "wrong event data received") diff --git a/block/produce.go b/block/produce.go index e9c77a909..c904adb6a 100644 --- a/block/produce.go +++ b/block/produce.go @@ -241,7 +241,7 @@ func (m *Manager) createTMSignature(block *types.Block, proposerAddress []byte, // GetPreviousBlockHashes returns the hash of the last block and the commit for the last block // to be used as the previous block hash and commit for the next block func (m *Manager) GetPreviousBlockHashes(forHeight uint64) (lastHeaderHash [32]byte, lastCommit *types.Commit, err error) { - lastHeaderHash, lastCommit, err = loadPrevBlock(m.Store, forHeight-1) // prev height = forHeight - 1 + lastHeaderHash, lastCommit, err = getHeaderHashAndCommit(m.Store, forHeight-1) // prev height = forHeight - 1 if err != nil { if !m.State.IsGenesis() { // allow prevBlock not to be found only on genesis return [32]byte{}, nil, fmt.Errorf("load prev block: %w: %w", err, ErrNonRecoverable) @@ -252,7 +252,8 @@ func (m *Manager) GetPreviousBlockHashes(forHeight uint64) (lastHeaderHash [32]b return lastHeaderHash, lastCommit, nil } -func loadPrevBlock(store store.Store, height uint64) ([32]byte, *types.Commit, error) { +// getHeaderHashAndCommit returns the Header Hash and Commit for a given height +func getHeaderHashAndCommit(store store.Store, height uint64) ([32]byte, *types.Commit, error) { lastCommit, err := store.LoadCommit(height) if err != nil { return [32]byte{}, nil, fmt.Errorf("load commit: height: %d: %w", height, err) diff --git a/block/retriever.go b/block/retriever.go index 96c288c2a..0e5f412ff 100644 --- a/block/retriever.go +++ b/block/retriever.go @@ -119,11 +119,8 @@ func (m *Manager) applyLocalBlock(height uint64) error { if err != nil { return fmt.Errorf("load commit: %w", gerrc.ErrNotFound) } - if err := m.validateBlockBeforeApply(block, commit); err != nil { - return fmt.Errorf("validate block from local store: height: %d: %w", height, err) - } - err = m.applyBlock(block, commit, types.BlockMetaData{Source: types.LocalDb}) + err = m.applyBlockWithFraudHandling(block, commit, types.BlockMetaData{Source: types.LocalDb}) if err != nil { return fmt.Errorf("apply block from local store: height: %d: %w", height, err) } @@ -149,12 +146,8 @@ func (m *Manager) ProcessNextDABatch(daMetaData *da.DASubmitMetaData) error { if block.Header.Height != m.State.NextHeight() { continue } - if err := m.validateBlockBeforeApply(block, batch.Commits[i]); err != nil { - m.logger.Error("validate block from DA", "height", block.Header.Height, "err", err) - continue - } - err := m.applyBlock(block, batch.Commits[i], types.BlockMetaData{Source: types.DA, DAHeight: daMetaData.Height}) + err := m.applyBlockWithFraudHandling(block, batch.Commits[i], types.BlockMetaData{Source: types.DA, DAHeight: daMetaData.Height}) if err != nil { return fmt.Errorf("apply block: height: %d: %w", block.Header.Height, err) } diff --git a/block/state.go b/block/state.go index e13d8c3bc..53bd498df 100644 --- a/block/state.go +++ b/block/state.go @@ -76,7 +76,7 @@ func NewStateFromGenesis(genDoc *tmtypes.GenesisDoc) (*types.State, error) { } // UpdateStateFromApp is responsible for aligning the state of the store from the abci app -func (m *Manager) UpdateStateFromApp() error { +func (m *Manager) UpdateStateFromApp(blockHeaderHash [32]byte) error { proxyAppInfo, err := m.Executor.GetAppInfo() if err != nil { return errorsmod.Wrap(err, "get app info") @@ -89,7 +89,7 @@ func (m *Manager) UpdateStateFromApp() error { } // update the state with the app hashes created on the app commit - m.Executor.UpdateStateAfterCommit(m.State, resp, proxyAppInfo.LastBlockAppHash, appHeight) + m.Executor.UpdateStateAfterCommit(m.State, resp, proxyAppInfo.LastBlockAppHash, appHeight, blockHeaderHash) return nil } @@ -118,9 +118,10 @@ func (e *Executor) UpdateMempoolAfterInitChain(s *types.State) { } // UpdateStateAfterCommit updates the state with the app hash and last results hash -func (e *Executor) UpdateStateAfterCommit(s *types.State, resp *tmstate.ABCIResponses, appHash []byte, height uint64) { +func (e *Executor) UpdateStateAfterCommit(s *types.State, resp *tmstate.ABCIResponses, appHash []byte, height uint64, lastHeaderHash [32]byte) { copy(s.AppHash[:], appHash[:]) copy(s.LastResultsHash[:], tmtypes.NewResults(resp.DeliverTxs).Hash()) + copy(s.LastHeaderHash[:], lastHeaderHash[:]) s.SetHeight(height) diff --git a/block/submit_test.go b/block/submit_test.go index 7f2713787..fe0683804 100644 --- a/block/submit_test.go +++ b/block/submit_test.go @@ -58,7 +58,7 @@ func TestBatchOverhead(t *testing.T) { } for _, tcase := range tcases { - blocks, err := testutil.GenerateBlocks(1, 1, manager.LocalKey) + blocks, err := testutil.GenerateBlocks(1, 1, manager.LocalKey, [32]byte{}) require.NoError(t, err) block := blocks[0] diff --git a/mocks/github.com/dymensionxyz/dymint/block/mock_ExecutorI.go b/mocks/github.com/dymensionxyz/dymint/block/mock_ExecutorI.go new file mode 100644 index 000000000..32d2f6856 --- /dev/null +++ b/mocks/github.com/dymensionxyz/dymint/block/mock_ExecutorI.go @@ -0,0 +1,488 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package block + +import ( + abcitypes "github.com/tendermint/tendermint/abci/types" + + mock "github.com/stretchr/testify/mock" + + state "github.com/tendermint/tendermint/proto/tendermint/state" + + tenderminttypes "github.com/tendermint/tendermint/types" + + types "github.com/dymensionxyz/dymint/types" +) + +// MockExecutorI is an autogenerated mock type for the ExecutorI type +type MockExecutorI struct { + mock.Mock +} + +type MockExecutorI_Expecter struct { + mock *mock.Mock +} + +func (_m *MockExecutorI) EXPECT() *MockExecutorI_Expecter { + return &MockExecutorI_Expecter{mock: &_m.Mock} +} + +// Commit provides a mock function with given fields: _a0, _a1, resp +func (_m *MockExecutorI) Commit(_a0 *types.State, _a1 *types.Block, resp *state.ABCIResponses) ([]byte, int64, error) { + ret := _m.Called(_a0, _a1, resp) + + if len(ret) == 0 { + panic("no return value specified for Commit") + } + + var r0 []byte + var r1 int64 + var r2 error + if rf, ok := ret.Get(0).(func(*types.State, *types.Block, *state.ABCIResponses) ([]byte, int64, error)); ok { + return rf(_a0, _a1, resp) + } + if rf, ok := ret.Get(0).(func(*types.State, *types.Block, *state.ABCIResponses) []byte); ok { + r0 = rf(_a0, _a1, resp) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]byte) + } + } + + if rf, ok := ret.Get(1).(func(*types.State, *types.Block, *state.ABCIResponses) int64); ok { + r1 = rf(_a0, _a1, resp) + } else { + r1 = ret.Get(1).(int64) + } + + if rf, ok := ret.Get(2).(func(*types.State, *types.Block, *state.ABCIResponses) error); ok { + r2 = rf(_a0, _a1, resp) + } else { + r2 = ret.Error(2) + } + + return r0, r1, r2 +} + +// MockExecutorI_Commit_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Commit' +type MockExecutorI_Commit_Call struct { + *mock.Call +} + +// Commit is a helper method to define mock.On call +// - _a0 *types.State +// - _a1 *types.Block +// - resp *state.ABCIResponses +func (_e *MockExecutorI_Expecter) Commit(_a0 interface{}, _a1 interface{}, resp interface{}) *MockExecutorI_Commit_Call { + return &MockExecutorI_Commit_Call{Call: _e.mock.On("Commit", _a0, _a1, resp)} +} + +func (_c *MockExecutorI_Commit_Call) Run(run func(_a0 *types.State, _a1 *types.Block, resp *state.ABCIResponses)) *MockExecutorI_Commit_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(*types.State), args[1].(*types.Block), args[2].(*state.ABCIResponses)) + }) + return _c +} + +func (_c *MockExecutorI_Commit_Call) Return(_a0 []byte, _a1 int64, _a2 error) *MockExecutorI_Commit_Call { + _c.Call.Return(_a0, _a1, _a2) + return _c +} + +func (_c *MockExecutorI_Commit_Call) RunAndReturn(run func(*types.State, *types.Block, *state.ABCIResponses) ([]byte, int64, error)) *MockExecutorI_Commit_Call { + _c.Call.Return(run) + return _c +} + +// CreateBlock provides a mock function with given fields: height, lastCommit, lastHeaderHash, nextSeqHash, _a4, maxBlockDataSizeBytes +func (_m *MockExecutorI) CreateBlock(height uint64, lastCommit *types.Commit, lastHeaderHash [32]byte, nextSeqHash [32]byte, _a4 *types.State, maxBlockDataSizeBytes uint64) *types.Block { + ret := _m.Called(height, lastCommit, lastHeaderHash, nextSeqHash, _a4, maxBlockDataSizeBytes) + + if len(ret) == 0 { + panic("no return value specified for CreateBlock") + } + + var r0 *types.Block + if rf, ok := ret.Get(0).(func(uint64, *types.Commit, [32]byte, [32]byte, *types.State, uint64) *types.Block); ok { + r0 = rf(height, lastCommit, lastHeaderHash, nextSeqHash, _a4, maxBlockDataSizeBytes) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.Block) + } + } + + return r0 +} + +// MockExecutorI_CreateBlock_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CreateBlock' +type MockExecutorI_CreateBlock_Call struct { + *mock.Call +} + +// CreateBlock is a helper method to define mock.On call +// - height uint64 +// - lastCommit *types.Commit +// - lastHeaderHash [32]byte +// - nextSeqHash [32]byte +// - _a4 *types.State +// - maxBlockDataSizeBytes uint64 +func (_e *MockExecutorI_Expecter) CreateBlock(height interface{}, lastCommit interface{}, lastHeaderHash interface{}, nextSeqHash interface{}, _a4 interface{}, maxBlockDataSizeBytes interface{}) *MockExecutorI_CreateBlock_Call { + return &MockExecutorI_CreateBlock_Call{Call: _e.mock.On("CreateBlock", height, lastCommit, lastHeaderHash, nextSeqHash, _a4, maxBlockDataSizeBytes)} +} + +func (_c *MockExecutorI_CreateBlock_Call) Run(run func(height uint64, lastCommit *types.Commit, lastHeaderHash [32]byte, nextSeqHash [32]byte, _a4 *types.State, maxBlockDataSizeBytes uint64)) *MockExecutorI_CreateBlock_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(uint64), args[1].(*types.Commit), args[2].([32]byte), args[3].([32]byte), args[4].(*types.State), args[5].(uint64)) + }) + return _c +} + +func (_c *MockExecutorI_CreateBlock_Call) Return(_a0 *types.Block) *MockExecutorI_CreateBlock_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockExecutorI_CreateBlock_Call) RunAndReturn(run func(uint64, *types.Commit, [32]byte, [32]byte, *types.State, uint64) *types.Block) *MockExecutorI_CreateBlock_Call { + _c.Call.Return(run) + return _c +} + +// ExecuteBlock provides a mock function with given fields: _a0, _a1 +func (_m *MockExecutorI) ExecuteBlock(_a0 *types.State, _a1 *types.Block) (*state.ABCIResponses, error) { + ret := _m.Called(_a0, _a1) + + if len(ret) == 0 { + panic("no return value specified for ExecuteBlock") + } + + var r0 *state.ABCIResponses + var r1 error + if rf, ok := ret.Get(0).(func(*types.State, *types.Block) (*state.ABCIResponses, error)); ok { + return rf(_a0, _a1) + } + if rf, ok := ret.Get(0).(func(*types.State, *types.Block) *state.ABCIResponses); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*state.ABCIResponses) + } + } + + if rf, ok := ret.Get(1).(func(*types.State, *types.Block) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockExecutorI_ExecuteBlock_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ExecuteBlock' +type MockExecutorI_ExecuteBlock_Call struct { + *mock.Call +} + +// ExecuteBlock is a helper method to define mock.On call +// - _a0 *types.State +// - _a1 *types.Block +func (_e *MockExecutorI_Expecter) ExecuteBlock(_a0 interface{}, _a1 interface{}) *MockExecutorI_ExecuteBlock_Call { + return &MockExecutorI_ExecuteBlock_Call{Call: _e.mock.On("ExecuteBlock", _a0, _a1)} +} + +func (_c *MockExecutorI_ExecuteBlock_Call) Run(run func(_a0 *types.State, _a1 *types.Block)) *MockExecutorI_ExecuteBlock_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(*types.State), args[1].(*types.Block)) + }) + return _c +} + +func (_c *MockExecutorI_ExecuteBlock_Call) Return(_a0 *state.ABCIResponses, _a1 error) *MockExecutorI_ExecuteBlock_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockExecutorI_ExecuteBlock_Call) RunAndReturn(run func(*types.State, *types.Block) (*state.ABCIResponses, error)) *MockExecutorI_ExecuteBlock_Call { + _c.Call.Return(run) + return _c +} + +// GetAppInfo provides a mock function with given fields: +func (_m *MockExecutorI) GetAppInfo() (*abcitypes.ResponseInfo, error) { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetAppInfo") + } + + var r0 *abcitypes.ResponseInfo + var r1 error + if rf, ok := ret.Get(0).(func() (*abcitypes.ResponseInfo, error)); ok { + return rf() + } + if rf, ok := ret.Get(0).(func() *abcitypes.ResponseInfo); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*abcitypes.ResponseInfo) + } + } + + if rf, ok := ret.Get(1).(func() error); ok { + r1 = rf() + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockExecutorI_GetAppInfo_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetAppInfo' +type MockExecutorI_GetAppInfo_Call struct { + *mock.Call +} + +// GetAppInfo is a helper method to define mock.On call +func (_e *MockExecutorI_Expecter) GetAppInfo() *MockExecutorI_GetAppInfo_Call { + return &MockExecutorI_GetAppInfo_Call{Call: _e.mock.On("GetAppInfo")} +} + +func (_c *MockExecutorI_GetAppInfo_Call) Run(run func()) *MockExecutorI_GetAppInfo_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockExecutorI_GetAppInfo_Call) Return(_a0 *abcitypes.ResponseInfo, _a1 error) *MockExecutorI_GetAppInfo_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockExecutorI_GetAppInfo_Call) RunAndReturn(run func() (*abcitypes.ResponseInfo, error)) *MockExecutorI_GetAppInfo_Call { + _c.Call.Return(run) + return _c +} + +// InitChain provides a mock function with given fields: genesis, valset +func (_m *MockExecutorI) InitChain(genesis *tenderminttypes.GenesisDoc, valset []*tenderminttypes.Validator) (*abcitypes.ResponseInitChain, error) { + ret := _m.Called(genesis, valset) + + if len(ret) == 0 { + panic("no return value specified for InitChain") + } + + var r0 *abcitypes.ResponseInitChain + var r1 error + if rf, ok := ret.Get(0).(func(*tenderminttypes.GenesisDoc, []*tenderminttypes.Validator) (*abcitypes.ResponseInitChain, error)); ok { + return rf(genesis, valset) + } + if rf, ok := ret.Get(0).(func(*tenderminttypes.GenesisDoc, []*tenderminttypes.Validator) *abcitypes.ResponseInitChain); ok { + r0 = rf(genesis, valset) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*abcitypes.ResponseInitChain) + } + } + + if rf, ok := ret.Get(1).(func(*tenderminttypes.GenesisDoc, []*tenderminttypes.Validator) error); ok { + r1 = rf(genesis, valset) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockExecutorI_InitChain_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'InitChain' +type MockExecutorI_InitChain_Call struct { + *mock.Call +} + +// InitChain is a helper method to define mock.On call +// - genesis *tenderminttypes.GenesisDoc +// - valset []*tenderminttypes.Validator +func (_e *MockExecutorI_Expecter) InitChain(genesis interface{}, valset interface{}) *MockExecutorI_InitChain_Call { + return &MockExecutorI_InitChain_Call{Call: _e.mock.On("InitChain", genesis, valset)} +} + +func (_c *MockExecutorI_InitChain_Call) Run(run func(genesis *tenderminttypes.GenesisDoc, valset []*tenderminttypes.Validator)) *MockExecutorI_InitChain_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(*tenderminttypes.GenesisDoc), args[1].([]*tenderminttypes.Validator)) + }) + return _c +} + +func (_c *MockExecutorI_InitChain_Call) Return(_a0 *abcitypes.ResponseInitChain, _a1 error) *MockExecutorI_InitChain_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockExecutorI_InitChain_Call) RunAndReturn(run func(*tenderminttypes.GenesisDoc, []*tenderminttypes.Validator) (*abcitypes.ResponseInitChain, error)) *MockExecutorI_InitChain_Call { + _c.Call.Return(run) + return _c +} + +// UpdateMempoolAfterInitChain provides a mock function with given fields: s +func (_m *MockExecutorI) UpdateMempoolAfterInitChain(s *types.State) { + _m.Called(s) +} + +// MockExecutorI_UpdateMempoolAfterInitChain_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateMempoolAfterInitChain' +type MockExecutorI_UpdateMempoolAfterInitChain_Call struct { + *mock.Call +} + +// UpdateMempoolAfterInitChain is a helper method to define mock.On call +// - s *types.State +func (_e *MockExecutorI_Expecter) UpdateMempoolAfterInitChain(s interface{}) *MockExecutorI_UpdateMempoolAfterInitChain_Call { + return &MockExecutorI_UpdateMempoolAfterInitChain_Call{Call: _e.mock.On("UpdateMempoolAfterInitChain", s)} +} + +func (_c *MockExecutorI_UpdateMempoolAfterInitChain_Call) Run(run func(s *types.State)) *MockExecutorI_UpdateMempoolAfterInitChain_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(*types.State)) + }) + return _c +} + +func (_c *MockExecutorI_UpdateMempoolAfterInitChain_Call) Return() *MockExecutorI_UpdateMempoolAfterInitChain_Call { + _c.Call.Return() + return _c +} + +func (_c *MockExecutorI_UpdateMempoolAfterInitChain_Call) RunAndReturn(run func(*types.State)) *MockExecutorI_UpdateMempoolAfterInitChain_Call { + _c.Call.Return(run) + return _c +} + +// UpdateProposerFromBlock provides a mock function with given fields: s, _a1 +func (_m *MockExecutorI) UpdateProposerFromBlock(s *types.State, _a1 *types.Block) bool { + ret := _m.Called(s, _a1) + + if len(ret) == 0 { + panic("no return value specified for UpdateProposerFromBlock") + } + + var r0 bool + if rf, ok := ret.Get(0).(func(*types.State, *types.Block) bool); ok { + r0 = rf(s, _a1) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// MockExecutorI_UpdateProposerFromBlock_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateProposerFromBlock' +type MockExecutorI_UpdateProposerFromBlock_Call struct { + *mock.Call +} + +// UpdateProposerFromBlock is a helper method to define mock.On call +// - s *types.State +// - _a1 *types.Block +func (_e *MockExecutorI_Expecter) UpdateProposerFromBlock(s interface{}, _a1 interface{}) *MockExecutorI_UpdateProposerFromBlock_Call { + return &MockExecutorI_UpdateProposerFromBlock_Call{Call: _e.mock.On("UpdateProposerFromBlock", s, _a1)} +} + +func (_c *MockExecutorI_UpdateProposerFromBlock_Call) Run(run func(s *types.State, _a1 *types.Block)) *MockExecutorI_UpdateProposerFromBlock_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(*types.State), args[1].(*types.Block)) + }) + return _c +} + +func (_c *MockExecutorI_UpdateProposerFromBlock_Call) Return(_a0 bool) *MockExecutorI_UpdateProposerFromBlock_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockExecutorI_UpdateProposerFromBlock_Call) RunAndReturn(run func(*types.State, *types.Block) bool) *MockExecutorI_UpdateProposerFromBlock_Call { + _c.Call.Return(run) + return _c +} + +// UpdateStateAfterCommit provides a mock function with given fields: s, resp, appHash, height, lastHeaderHash +func (_m *MockExecutorI) UpdateStateAfterCommit(s *types.State, resp *state.ABCIResponses, appHash []byte, height uint64, lastHeaderHash [32]byte) { + _m.Called(s, resp, appHash, height, lastHeaderHash) +} + +// MockExecutorI_UpdateStateAfterCommit_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateStateAfterCommit' +type MockExecutorI_UpdateStateAfterCommit_Call struct { + *mock.Call +} + +// UpdateStateAfterCommit is a helper method to define mock.On call +// - s *types.State +// - resp *state.ABCIResponses +// - appHash []byte +// - height uint64 +// - lastHeaderHash [32]byte +func (_e *MockExecutorI_Expecter) UpdateStateAfterCommit(s interface{}, resp interface{}, appHash interface{}, height interface{}, lastHeaderHash interface{}) *MockExecutorI_UpdateStateAfterCommit_Call { + return &MockExecutorI_UpdateStateAfterCommit_Call{Call: _e.mock.On("UpdateStateAfterCommit", s, resp, appHash, height, lastHeaderHash)} +} + +func (_c *MockExecutorI_UpdateStateAfterCommit_Call) Run(run func(s *types.State, resp *state.ABCIResponses, appHash []byte, height uint64, lastHeaderHash [32]byte)) *MockExecutorI_UpdateStateAfterCommit_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(*types.State), args[1].(*state.ABCIResponses), args[2].([]byte), args[3].(uint64), args[4].([32]byte)) + }) + return _c +} + +func (_c *MockExecutorI_UpdateStateAfterCommit_Call) Return() *MockExecutorI_UpdateStateAfterCommit_Call { + _c.Call.Return() + return _c +} + +func (_c *MockExecutorI_UpdateStateAfterCommit_Call) RunAndReturn(run func(*types.State, *state.ABCIResponses, []byte, uint64, [32]byte)) *MockExecutorI_UpdateStateAfterCommit_Call { + _c.Call.Return(run) + return _c +} + +// UpdateStateAfterInitChain provides a mock function with given fields: s, res +func (_m *MockExecutorI) UpdateStateAfterInitChain(s *types.State, res *abcitypes.ResponseInitChain) { + _m.Called(s, res) +} + +// MockExecutorI_UpdateStateAfterInitChain_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateStateAfterInitChain' +type MockExecutorI_UpdateStateAfterInitChain_Call struct { + *mock.Call +} + +// UpdateStateAfterInitChain is a helper method to define mock.On call +// - s *types.State +// - res *abcitypes.ResponseInitChain +func (_e *MockExecutorI_Expecter) UpdateStateAfterInitChain(s interface{}, res interface{}) *MockExecutorI_UpdateStateAfterInitChain_Call { + return &MockExecutorI_UpdateStateAfterInitChain_Call{Call: _e.mock.On("UpdateStateAfterInitChain", s, res)} +} + +func (_c *MockExecutorI_UpdateStateAfterInitChain_Call) Run(run func(s *types.State, res *abcitypes.ResponseInitChain)) *MockExecutorI_UpdateStateAfterInitChain_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(*types.State), args[1].(*abcitypes.ResponseInitChain)) + }) + return _c +} + +func (_c *MockExecutorI_UpdateStateAfterInitChain_Call) Return() *MockExecutorI_UpdateStateAfterInitChain_Call { + _c.Call.Return() + return _c +} + +func (_c *MockExecutorI_UpdateStateAfterInitChain_Call) RunAndReturn(run func(*types.State, *abcitypes.ResponseInitChain)) *MockExecutorI_UpdateStateAfterInitChain_Call { + _c.Call.Return(run) + return _c +} + +// NewMockExecutorI creates a new instance of MockExecutorI. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewMockExecutorI(t interface { + mock.TestingT + Cleanup(func()) +}) *MockExecutorI { + mock := &MockExecutorI{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/mocks/github.com/dymensionxyz/dymint/block/mock_FraudHandler.go b/mocks/github.com/dymensionxyz/dymint/block/mock_FraudHandler.go new file mode 100644 index 000000000..b9ddec5ac --- /dev/null +++ b/mocks/github.com/dymensionxyz/dymint/block/mock_FraudHandler.go @@ -0,0 +1,70 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package block + +import ( + context "context" + + mock "github.com/stretchr/testify/mock" +) + +// MockFraudHandler is an autogenerated mock type for the FraudHandler type +type MockFraudHandler struct { + mock.Mock +} + +type MockFraudHandler_Expecter struct { + mock *mock.Mock +} + +func (_m *MockFraudHandler) EXPECT() *MockFraudHandler_Expecter { + return &MockFraudHandler_Expecter{mock: &_m.Mock} +} + +// HandleFault provides a mock function with given fields: ctx, fault +func (_m *MockFraudHandler) HandleFault(ctx context.Context, fault error) { + _m.Called(ctx, fault) +} + +// MockFraudHandler_HandleFault_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleFault' +type MockFraudHandler_HandleFault_Call struct { + *mock.Call +} + +// HandleFault is a helper method to define mock.On call +// - ctx context.Context +// - fault error +func (_e *MockFraudHandler_Expecter) HandleFault(ctx interface{}, fault interface{}) *MockFraudHandler_HandleFault_Call { + return &MockFraudHandler_HandleFault_Call{Call: _e.mock.On("HandleFault", ctx, fault)} +} + +func (_c *MockFraudHandler_HandleFault_Call) Run(run func(ctx context.Context, fault error)) *MockFraudHandler_HandleFault_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(error)) + }) + return _c +} + +func (_c *MockFraudHandler_HandleFault_Call) Return() *MockFraudHandler_HandleFault_Call { + _c.Call.Return() + return _c +} + +func (_c *MockFraudHandler_HandleFault_Call) RunAndReturn(run func(context.Context, error)) *MockFraudHandler_HandleFault_Call { + _c.Call.Return(run) + return _c +} + +// NewMockFraudHandler creates a new instance of MockFraudHandler. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewMockFraudHandler(t interface { + mock.TestingT + Cleanup(func()) +}) *MockFraudHandler { + mock := &MockFraudHandler{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/p2p/validator_test.go b/p2p/validator_test.go index 3533926b7..aba9372db 100644 --- a/p2p/validator_test.go +++ b/p2p/validator_test.go @@ -125,7 +125,7 @@ func TestValidator_BlockValidator(t *testing.T) { require.NotNil(t, clientCreator) require.NotNil(t, abciClient) mpool := mempoolv1.NewTxMempool(logger, cfg.DefaultMempoolConfig(), proxy.NewAppConnMempool(abciClient), 0) - executor, err := block.NewExecutor([]byte("test address"), "test", mpool, proxy.NewAppConns(clientCreator), nil, nil, logger) + executor, err := block.NewExecutor(proposerKey.PubKey().Address(), "test", mpool, proxy.NewAppConns(clientCreator), nil, nil, logger) assert.NoError(t, err) // Create state diff --git a/proto/types/dymint/state.proto b/proto/types/dymint/state.proto index dc3b1bbf1..09abbb38b 100755 --- a/proto/types/dymint/state.proto +++ b/proto/types/dymint/state.proto @@ -45,6 +45,7 @@ message State { SequencerSet sequencerSet = 18 [(gogoproto.nullable) = false]; RollappParams rollapp_params = 19 [(gogoproto.nullable) = false]; + bytes last_header_hash = 20; } //rollapp params defined in genesis and updated via gov proposal diff --git a/settlement/dymension/dymension_test.go b/settlement/dymension/dymension_test.go index fdd712914..76d383575 100644 --- a/settlement/dymension/dymension_test.go +++ b/settlement/dymension/dymension_test.go @@ -104,7 +104,7 @@ func TestPostBatch(t *testing.T) { // Create a batch which will be submitted propserKey, _, err := crypto.GenerateEd25519Key(nil) require.NoError(err) - batch, err := testutil.GenerateBatch(1, 1, propserKey) + batch, err := testutil.GenerateBatch(1, 1, propserKey, [32]byte{}) require.NoError(err) cases := []struct { diff --git a/settlement/local/local_test.go b/settlement/local/local_test.go index 76600c57b..e2f30e87d 100644 --- a/settlement/local/local_test.go +++ b/settlement/local/local_test.go @@ -59,9 +59,9 @@ func TestSubmitBatch(t *testing.T) { // Create a batches which will be submitted proposerKey, _, err := crypto.GenerateEd25519Key(nil) require.NoError(err) - batch1, err := testutil.GenerateBatch(1, 1, proposerKey) + batch1, err := testutil.GenerateBatch(1, 1, proposerKey, [32]byte{}) require.NoError(err) - batch2, err := testutil.GenerateBatch(2, 2, proposerKey) + batch2, err := testutil.GenerateBatch(2, 2, proposerKey, [32]byte{}) require.NoError(err) resultSubmitBatch := &da.ResultSubmitBatch{} resultSubmitBatch.SubmitMetaData = &da.DASubmitMetaData{} @@ -133,7 +133,7 @@ func TestPersistency(t *testing.T) { assert.Error(err) // no batch should be present // Create a batches which will be submitted - batch1, err := testutil.GenerateBatch(1, 1, proposerKey) + batch1, err := testutil.GenerateBatch(1, 1, proposerKey, [32]byte{}) require.NoError(err) resultSubmitBatch := &da.ResultSubmitBatch{} resultSubmitBatch.SubmitMetaData = &da.DASubmitMetaData{} diff --git a/store/store.go b/store/store.go index 1ab464dab..9bdb8075d 100644 --- a/store/store.go +++ b/store/store.go @@ -166,7 +166,7 @@ func (s *DefaultStore) LoadCommitByHash(hash [32]byte) (*types.Commit, error) { return commit, nil } -// UpdateState updates state saved in Store. Only one State is stored. +// SaveState updates state saved in Store. Only one State is stored. // If there is no State in Store, state will be saved. func (s *DefaultStore) SaveState(state *types.State, batch KVBatch) (KVBatch, error) { pbState, err := state.ToProto() diff --git a/testutil/types.go b/testutil/types.go index 519eee3f7..14214b589 100644 --- a/testutil/types.go +++ b/testutil/types.go @@ -51,7 +51,7 @@ func GetRandomBytes(n uint64) []byte { } // generateBlock generates random blocks. -func generateBlock(height uint64, proposerHash []byte) *types.Block { +func generateBlock(height uint64, proposerHash []byte, lastHeaderHash [32]byte) *types.Block { h := createRandomHashes() block := &types.Block{ Header: types.Header{ @@ -61,7 +61,7 @@ func generateBlock(height uint64, proposerHash []byte) *types.Block { }, Height: height, Time: 4567, - LastHeaderHash: h[0], + LastHeaderHash: lastHeaderHash, LastCommitHash: h[1], DataHash: h[2], ConsensusHash: h[3], @@ -70,6 +70,7 @@ func generateBlock(height uint64, proposerHash []byte) *types.Block { ProposerAddress: []byte{4, 3, 2, 1}, SequencerHash: [32]byte(proposerHash), NextSequencersHash: [32]byte(proposerHash), + ChainID: "test-chain", }, Data: types.Data{ Txs: nil, @@ -92,9 +93,10 @@ func GenerateBlocksWithTxs(startHeight uint64, num uint64, proposerKey crypto.Pr proposerHash := seq.MustHash() blocks := make([]*types.Block, num) + lastHeaderHash := [32]byte{} for i := uint64(0); i < num; i++ { - block := generateBlock(i+startHeight, proposerHash) + block := generateBlock(i+startHeight, proposerHash, lastHeaderHash) block.Data = types.Data{ Txs: make(types.Txs, nTxs), @@ -114,19 +116,21 @@ func GenerateBlocksWithTxs(startHeight uint64, num uint64, proposerKey crypto.Pr } block.LastCommit.Signatures = []types.Signature{signature} blocks[i] = block + lastHeaderHash = block.Header.Hash() } return blocks, nil } // GenerateBlocks generates random blocks. -func GenerateBlocks(startHeight uint64, num uint64, proposerKey crypto.PrivKey) ([]*types.Block, error) { +func GenerateBlocks(startHeight uint64, num uint64, proposerKey crypto.PrivKey, lastBlockHeader [32]byte) ([]*types.Block, error) { r, _ := proposerKey.Raw() seq := types.NewSequencerFromValidator(*tmtypes.NewValidator(ed25519.PrivKey(r).PubKey(), 1)) proposerHash := seq.MustHash() blocks := make([]*types.Block, num) + lastHeaderHash := lastBlockHeader for i := uint64(0); i < num; i++ { - block := generateBlock(i+startHeight, proposerHash) + block := generateBlock(i+startHeight, proposerHash, lastHeaderHash) copy(block.Header.DataHash[:], types.GetDataHash(block)) if i > 0 { copy(block.Header.LastCommitHash[:], types.GetLastCommitHash(&blocks[i-1].LastCommit, &block.Header)) @@ -137,7 +141,9 @@ func GenerateBlocks(startHeight uint64, num uint64, proposerKey crypto.PrivKey) return nil, err } block.LastCommit.Signatures = []types.Signature{signature} + block.Header.ProposerAddress = ed25519.PrivKey(r).PubKey().Address() blocks[i] = block + lastHeaderHash = block.Header.Hash() } return blocks, nil } @@ -176,8 +182,8 @@ func generateSignature(proposerKey crypto.PrivKey, header *types.Header) ([]byte } // GenerateBatch generates a batch out of random blocks -func GenerateBatch(startHeight uint64, endHeight uint64, proposerKey crypto.PrivKey) (*types.Batch, error) { - blocks, err := GenerateBlocks(startHeight, endHeight-startHeight+1, proposerKey) +func GenerateBatch(startHeight uint64, endHeight uint64, proposerKey crypto.PrivKey, lastBlockHeader [32]byte) (*types.Batch, error) { + blocks, err := GenerateBlocks(startHeight, endHeight-startHeight+1, proposerKey, lastBlockHeader) if err != nil { return nil, err } @@ -193,7 +199,7 @@ func GenerateBatch(startHeight uint64, endHeight uint64, proposerKey crypto.Priv } func MustGenerateBatch(startHeight uint64, endHeight uint64, proposerKey crypto.PrivKey) *types.Batch { - blocks, err := GenerateBlocks(startHeight, endHeight-startHeight+1, proposerKey) + blocks, err := GenerateBlocks(startHeight, endHeight-startHeight+1, proposerKey, [32]byte{}) if err != nil { panic(err) } diff --git a/types/block.go b/types/block.go index 060a7445f..8bd7b652b 100644 --- a/types/block.go +++ b/types/block.go @@ -14,7 +14,7 @@ type Header struct { Version Version Height uint64 - Time uint64 // time in tai64 format + Time uint64 // UNIX time in milliseconds // prev block info LastHeaderHash [32]byte diff --git a/types/errors.go b/types/errors.go index cfabbe0b1..fbb0f1bfb 100644 --- a/types/errors.go +++ b/types/errors.go @@ -3,8 +3,10 @@ package types import ( "errors" "fmt" + "time" "github.com/dymensionxyz/gerr-cosmos/gerrc" + tmcrypto "github.com/tendermint/tendermint/crypto" ) var ( @@ -14,4 +16,303 @@ var ( ErrInvalidBlockHeight = errors.New("invalid block height") ErrInvalidHeaderDataHash = errors.New("header not matching block data") ErrMissingProposerPubKey = fmt.Errorf("missing proposer public key: %w", gerrc.ErrNotFound) + ErrVersionMismatch = errors.New("version mismatch") + ErrEmptyProposerAddress = errors.New("no proposer address") ) + +// TimeFraudMaxDrift is the maximum allowed time drift between the block time and the local time. +var TimeFraudMaxDrift = 10 * time.Minute + +// ErrFraudHeightMismatch is the fraud that occurs when the height of the block is different from the expected +// next height of the state. +type ErrFraudHeightMismatch struct { + Expected uint64 + Actual uint64 + + HeaderHash [32]byte + Proposer []byte +} + +// NewErrFraudHeightMismatch creates a new ErrFraudHeightMismatch error. +func NewErrFraudHeightMismatch(expected uint64, actual uint64, block *Block) error { + return &ErrFraudHeightMismatch{ + Expected: expected, Actual: actual, + HeaderHash: block.Header.Hash(), Proposer: block.Header.ProposerAddress, + } +} + +func (e ErrFraudHeightMismatch) Error() string { + return fmt.Sprintf("possible fraud detected on height %d, with header hash %X, emitted by sequencer %X:"+ + " height mismatch: state expected %d, got %d", e.Expected, e.HeaderHash, e.Proposer, e.Expected, e.Actual) +} + +func (e ErrFraudHeightMismatch) Unwrap() error { + return gerrc.ErrFault +} + +type ErrFraudAppHashMismatch struct { + Expected [32]byte + + HeaderHeight uint64 + HeaderHash [32]byte + Proposer []byte +} + +// NewErrFraudAppHashMismatch creates a new ErrFraudAppHashMismatch error. +func NewErrFraudAppHashMismatch(expected [32]byte, actual [32]byte, block *Block) error { + return &ErrFraudAppHashMismatch{ + Expected: expected, + HeaderHeight: block.Header.Height, HeaderHash: block.Header.Hash(), Proposer: block.Header.ProposerAddress, + } +} + +func (e ErrFraudAppHashMismatch) Error() string { + return fmt.Sprintf("possible fraud detected on height %d, with header hash %X, emitted by sequencer %X:"+ + " AppHash mismatch: state expected %X, got %X", e.HeaderHeight, e.HeaderHash, e.Proposer, e.Expected, e.HeaderHash) +} + +func (e ErrFraudAppHashMismatch) Unwrap() error { + return gerrc.ErrFault +} + +type ErrLastResultsHashMismatch struct { + Expected [32]byte + + HeaderHeight uint64 + HeaderHash [32]byte + Proposer []byte + LastResultHash [32]byte +} + +// NewErrLastResultsHashMismatch creates a new ErrLastResultsHashMismatch error. +func NewErrLastResultsHashMismatch(expected [32]byte, block *Block) error { + return &ErrLastResultsHashMismatch{ + Expected: expected, + HeaderHeight: block.Header.Height, HeaderHash: block.Header.Hash(), Proposer: block.Header.ProposerAddress, + LastResultHash: block.Header.LastResultsHash, + } +} + +func (e ErrLastResultsHashMismatch) Error() string { + return fmt.Sprintf("possible fraud detected on height %d, with header hash %X, emitted by sequencer %X:"+ + " LastResultsHash mismatch: state expected %X, got %X", e.HeaderHeight, e.HeaderHash, e.Proposer, e.Expected, e.LastResultHash) +} + +func (e ErrLastResultsHashMismatch) Unwrap() error { + return gerrc.ErrFault +} + +type ErrTimeFraud struct { + Drift time.Duration + ProposerAddress []byte + HeaderHash [32]byte + HeaderHeight uint64 + HeaderTime time.Time + CurrentTime time.Time +} + +func NewErrTimeFraud(block *Block, currentTime time.Time) error { + drift := time.Unix(int64(block.Header.Time), 0).Sub(currentTime) + + return ErrTimeFraud{ + Drift: drift, + ProposerAddress: block.Header.ProposerAddress, + HeaderHash: block.Header.Hash(), + HeaderHeight: block.Header.Height, + HeaderTime: time.Unix(int64(block.Header.Time), 0), + CurrentTime: currentTime, + } +} + +func (e ErrTimeFraud) Error() string { + return fmt.Sprintf( + "sequencer posted a block with invalid time. "+ + "Max allowed drift exceeded. "+ + "proposerAddress=%s headerHash=%s headerHeight=%d drift=%s MaxDrift=%s headerTime=%s currentTime=%s", + e.ProposerAddress, e.HeaderHash, e.HeaderHeight, e.Drift, TimeFraudMaxDrift, e.HeaderTime, e.CurrentTime, + ) +} + +func (e ErrTimeFraud) Unwrap() error { + return gerrc.ErrFault +} + +type ErrLastHeaderHashMismatch struct { + Expected [32]byte + LastHeaderHash [32]byte +} + +func NewErrLastHeaderHashMismatch(expected [32]byte, block *Block) error { + return &ErrLastHeaderHashMismatch{ + Expected: expected, + LastHeaderHash: block.Header.LastHeaderHash, + } +} + +func (e ErrLastHeaderHashMismatch) Error() string { + return fmt.Sprintf("last header hash mismatch. expected=%X, got=%X", e.Expected, e.LastHeaderHash) +} + +func (e ErrLastHeaderHashMismatch) Unwrap() error { + return gerrc.ErrFault +} + +type ErrInvalidChainID struct { + Expected string + Block *Block +} + +func NewErrInvalidChainID(expected string, block *Block) error { + return &ErrInvalidChainID{ + Expected: expected, + Block: block, + } +} + +func (e ErrInvalidChainID) Error() string { + return fmt.Sprintf("invalid chain ID. expected=%s, got=%s", e.Expected, e.Block.Header.ChainID) +} + +func (e ErrInvalidChainID) Unwrap() error { + return gerrc.ErrFault +} + +// ErrInvalidBlockHeightFraud is the fraud that happens when the height that is on the commit header is +// different from the height of the block. +type ErrInvalidBlockHeightFraud struct { + Expected uint64 + ActualHeight uint64 +} + +func NewErrInvalidBlockHeightFraud(expected uint64, actualHeight uint64) error { + return &ErrInvalidBlockHeightFraud{ + Expected: expected, + ActualHeight: actualHeight, + } +} + +func (e ErrInvalidBlockHeightFraud) Error() string { + return fmt.Sprintf("invalid block height. expected=%d, got=%d", e.Expected, e.ActualHeight) +} + +func (e ErrInvalidBlockHeightFraud) Unwrap() error { + return gerrc.ErrFault +} + +type ErrInvalidHeaderHashFraud struct { + ExpectedHash [32]byte + ActualHash [32]byte +} + +func NewErrInvalidHeaderHashFraud(expectedHash [32]byte, actualHash [32]byte) error { + return &ErrInvalidHeaderHashFraud{ + ExpectedHash: expectedHash, + ActualHash: actualHash, + } +} + +func (e ErrInvalidHeaderHashFraud) Error() string { + return fmt.Sprintf("invalid header hash. expected=%X, got=%X", e.ExpectedHash, e.ActualHash) +} + +func (e ErrInvalidHeaderHashFraud) Unwrap() error { + return gerrc.ErrFault +} + +type ErrInvalidSignatureFraud struct { + Err error +} + +func NewErrInvalidSignatureFraud(err error) error { + return &ErrInvalidSignatureFraud{ + Err: err, + } +} + +func (e ErrInvalidSignatureFraud) Error() string { + return fmt.Sprintf("invalid signature: %s", e.Err) +} + +func (e ErrInvalidSignatureFraud) Unwrap() error { + return gerrc.ErrFault +} + +type ErrInvalidProposerAddressFraud struct { + ExpectedAddress []byte + ActualAddress tmcrypto.Address +} + +func NewErrInvalidProposerAddressFraud(address []byte, address2 tmcrypto.Address) error { + return &ErrInvalidProposerAddressFraud{ + ExpectedAddress: address, + ActualAddress: address2, + } +} + +func (e ErrInvalidProposerAddressFraud) Error() string { + return fmt.Sprintf("invalid proposer address. expected=%X, got=%X", e.ExpectedAddress, e.ActualAddress) +} + +func (e ErrInvalidProposerAddressFraud) Unwrap() error { + return gerrc.ErrFault +} + +type ErrInvalidSequencerHashFraud struct { + ExpectedHash [32]byte + ActualHash []byte +} + +func NewErrInvalidSequencerHashFraud(expectedHash [32]byte, actualHash []byte) error { + return &ErrInvalidSequencerHashFraud{ + ExpectedHash: expectedHash, + ActualHash: actualHash, + } +} + +func (e ErrInvalidSequencerHashFraud) Error() string { + return fmt.Sprintf("invalid sequencer hash. expected=%X, got=%X", e.ExpectedHash, e.ActualHash) +} + +func (e ErrInvalidSequencerHashFraud) Unwrap() error { + return gerrc.ErrFault +} + +type ErrInvalidNextSequencersHashFraud struct { + ExpectedHash [32]byte + ActualHash [32]byte +} + +func NewErrInvalidNextSequencersHashFraud(expectedHash [32]byte, actualHash [32]byte) error { + return &ErrInvalidNextSequencersHashFraud{ + ExpectedHash: expectedHash, + ActualHash: actualHash, + } +} + +func (e ErrInvalidNextSequencersHashFraud) Error() string { + return fmt.Sprintf("invalid next sequencers hash. expected=%X, got=%X", e.ExpectedHash, e.ActualHash) +} + +func (e ErrInvalidNextSequencersHashFraud) Unwrap() error { + return gerrc.ErrFault +} + +type ErrInvalidHeaderDataHashFraud struct { + Expected [32]byte + Actual [32]byte +} + +func NewErrInvalidHeaderDataHashFraud(expected [32]byte, actual [32]byte) error { + return &ErrInvalidHeaderDataHashFraud{ + Expected: expected, + Actual: actual, + } +} + +func (e ErrInvalidHeaderDataHashFraud) Error() string { + return fmt.Sprintf("invalid header data hash. expected=%X, got=%X", e.Expected, e.Actual) +} + +func (e ErrInvalidHeaderDataHashFraud) Unwrap() error { + return gerrc.ErrFault +} diff --git a/types/pb/dymint/state.pb.go b/types/pb/dymint/state.pb.go index 1cb1dfca9..33c2a1de3 100644 --- a/types/pb/dymint/state.pb.go +++ b/types/pb/dymint/state.pb.go @@ -47,6 +47,7 @@ type State struct { BaseHeight uint64 `protobuf:"varint,17,opt,name=base_height,json=baseHeight,proto3" json:"base_height,omitempty"` SequencerSet SequencerSet `protobuf:"bytes,18,opt,name=sequencerSet,proto3" json:"sequencerSet"` RollappParams RollappParams `protobuf:"bytes,19,opt,name=rollapp_params,json=rollappParams,proto3" json:"rollapp_params"` + LastHeaderHash []byte `protobuf:"bytes,20,opt,name=last_header_hash,json=lastHeaderHash,proto3" json:"last_header_hash,omitempty"` } func (m *State) Reset() { *m = State{} } @@ -195,6 +196,13 @@ func (m *State) GetRollappParams() RollappParams { return RollappParams{} } +func (m *State) GetLastHeaderHash() []byte { + if m != nil { + return m.LastHeaderHash + } + return nil +} + //rollapp params defined in genesis and updated via gov proposal type RollappParams struct { //data availability type (e.g. celestia) used in the rollapp @@ -258,50 +266,51 @@ func init() { func init() { proto.RegisterFile("types/dymint/state.proto", fileDescriptor_4b679420add07272) } var fileDescriptor_4b679420add07272 = []byte{ - // 685 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x94, 0xcb, 0x4e, 0xdb, 0x4c, - 0x14, 0xc7, 0xe3, 0x10, 0x88, 0x33, 0x21, 0x17, 0x06, 0x3e, 0xc9, 0xf0, 0x49, 0x4e, 0xe0, 0xbb, - 0x28, 0xea, 0xc2, 0x91, 0xca, 0xaa, 0x9b, 0x56, 0x32, 0x2c, 0x08, 0x42, 0x55, 0xe5, 0x54, 0x2c, - 0xba, 0xb1, 0xc6, 0xf6, 0xd4, 0x1e, 0xd5, 0xf1, 0xb8, 0x9e, 0x09, 0x2a, 0x7d, 0x0a, 0x1e, 0xa7, - 0x8f, 0xc0, 0x92, 0x65, 0x57, 0xb4, 0x0a, 0x2f, 0x52, 0xcd, 0xc5, 0x89, 0x43, 0xc4, 0x2a, 0x99, - 0xff, 0xf9, 0xcd, 0xdf, 0x67, 0xce, 0x39, 0x33, 0xc0, 0xe2, 0xb7, 0x39, 0x66, 0xe3, 0xe8, 0x76, - 0x46, 0x32, 0x3e, 0x66, 0x1c, 0x71, 0xec, 0xe4, 0x05, 0xe5, 0x14, 0xee, 0x28, 0xed, 0xe8, 0x20, - 0xa6, 0x31, 0x95, 0xd2, 0x58, 0xfc, 0x53, 0xd1, 0xa3, 0x41, 0x4c, 0x69, 0x9c, 0xe2, 0xb1, 0x5c, - 0x05, 0xf3, 0xcf, 0x63, 0x4e, 0x66, 0x98, 0x71, 0x34, 0xcb, 0x35, 0x70, 0xac, 0x8c, 0x39, 0xce, - 0x22, 0x5c, 0x48, 0x73, 0x14, 0x84, 0x64, 0x2c, 0x55, 0x8d, 0x9c, 0x6c, 0x20, 0x5a, 0xa8, 0x30, - 0xff, 0xbf, 0xc0, 0xdc, 0xa0, 0x94, 0x44, 0x88, 0xd3, 0x42, 0x73, 0xff, 0xbc, 0xc0, 0xe5, 0xa8, - 0x40, 0xb3, 0x97, 0x3f, 0x28, 0x0f, 0xbc, 0xf6, 0xc1, 0xc3, 0xb5, 0x82, 0xa8, 0x1f, 0x15, 0x3a, - 0xf9, 0xd1, 0x04, 0xdb, 0x53, 0xb1, 0x01, 0x9e, 0x82, 0xe6, 0x0d, 0x2e, 0x18, 0xa1, 0x99, 0x65, - 0x0c, 0x8d, 0x51, 0xfb, 0xf5, 0xa1, 0xb3, 0x32, 0x75, 0x54, 0x15, 0xaf, 0x15, 0xe0, 0x95, 0x24, - 0x3c, 0x04, 0x66, 0x98, 0x20, 0x92, 0xf9, 0x24, 0xb2, 0xea, 0x43, 0x63, 0xd4, 0xf2, 0x9a, 0x72, - 0x3d, 0x89, 0xe0, 0x7f, 0xa0, 0x4b, 0x32, 0xc2, 0x09, 0x4a, 0xfd, 0x04, 0x93, 0x38, 0xe1, 0xd6, - 0xd6, 0xd0, 0x18, 0x6d, 0x79, 0x1d, 0xad, 0x5e, 0x48, 0x11, 0xbe, 0x02, 0x7b, 0x29, 0x62, 0xdc, - 0x0f, 0x52, 0x1a, 0x7e, 0x29, 0xc9, 0x86, 0x24, 0x7b, 0x22, 0xe0, 0x0a, 0x5d, 0xb3, 0x1e, 0xe8, - 0x54, 0x58, 0x12, 0x59, 0xdb, 0x9b, 0x89, 0xaa, 0x73, 0xcb, 0x5d, 0x93, 0x73, 0x77, 0xff, 0xfe, - 0x71, 0x50, 0x5b, 0x3c, 0x0e, 0xda, 0x57, 0xa5, 0xd5, 0xe4, 0xdc, 0x6b, 0x2f, 0x7d, 0x27, 0x11, - 0xbc, 0x02, 0xbd, 0x8a, 0xa7, 0xe8, 0xb8, 0xb5, 0x23, 0x5d, 0x8f, 0x1c, 0x35, 0x0e, 0x4e, 0x39, - 0x0e, 0xce, 0xc7, 0x72, 0x1c, 0x5c, 0x53, 0xd8, 0xde, 0xfd, 0x1a, 0x18, 0x5e, 0x67, 0xe9, 0x25, - 0xa2, 0xd0, 0x05, 0x60, 0xd9, 0x45, 0x66, 0xb5, 0xa4, 0x91, 0xbd, 0x99, 0xde, 0x75, 0xc9, 0x4c, - 0x31, 0x77, 0xeb, 0x96, 0xe1, 0x55, 0x76, 0xc1, 0x33, 0x60, 0xcb, 0x8c, 0x54, 0x2d, 0xfc, 0x55, - 0xc4, 0x0f, 0x13, 0x94, 0xc5, 0x38, 0xb2, 0xda, 0xb2, 0x3c, 0x7f, 0x0b, 0x4a, 0x55, 0x66, 0xe9, - 0xc7, 0xce, 0x14, 0x02, 0x3d, 0xd0, 0x0f, 0x69, 0xc6, 0x70, 0xc6, 0xe6, 0xcc, 0x57, 0x03, 0x63, - 0xed, 0xca, 0x74, 0x8e, 0x37, 0xd3, 0x39, 0x2b, 0xc9, 0x0f, 0x12, 0x74, 0x1b, 0xe2, 0x78, 0x5e, - 0x2f, 0x5c, 0x97, 0xe1, 0x7b, 0xf0, 0x6f, 0x35, 0xb1, 0xe7, 0xfe, 0xcb, 0xf4, 0x3a, 0x32, 0xbd, - 0xe1, 0x2a, 0xbd, 0x67, 0xfe, 0x65, 0x8e, 0x65, 0xeb, 0x0b, 0xcc, 0xe6, 0x29, 0x67, 0x7e, 0x82, - 0x58, 0x62, 0x75, 0x87, 0xc6, 0x68, 0x57, 0xb5, 0xde, 0x53, 0xfa, 0x05, 0x62, 0x89, 0x18, 0x34, - 0x94, 0xe7, 0x0a, 0xe9, 0x49, 0xa4, 0x89, 0xf2, 0x5c, 0x86, 0xde, 0x69, 0x1b, 0xc6, 0x69, 0x81, - 0xcb, 0x09, 0xea, 0x0f, 0x8d, 0x51, 0xc3, 0xdd, 0x5f, 0x3c, 0x0e, 0x7a, 0xa2, 0xf5, 0x53, 0x11, - 0x53, 0xc9, 0x28, 0xef, 0x8a, 0x00, 0x07, 0xa0, 0x1d, 0x20, 0xb6, 0xdc, 0xba, 0x27, 0xb6, 0x7a, - 0x40, 0x48, 0x1a, 0x78, 0x0b, 0x76, 0x19, 0xfe, 0x3a, 0xc7, 0x59, 0x88, 0x45, 0xc7, 0x2c, 0x28, - 0x0b, 0x79, 0xe0, 0xe8, 0x9b, 0x34, 0xad, 0xc4, 0x74, 0xed, 0xd6, 0x78, 0xe8, 0x82, 0x6e, 0x41, - 0xd3, 0x54, 0x1c, 0x40, 0xb7, 0x62, 0x5f, 0x3a, 0xfc, 0x55, 0x3a, 0x78, 0x2a, 0xba, 0x56, 0xfe, - 0x4e, 0x51, 0x15, 0x2f, 0x1b, 0x66, 0xb3, 0x6f, 0x5e, 0x36, 0x4c, 0xb3, 0xdf, 0xba, 0x6c, 0x98, - 0xa0, 0xdf, 0x3e, 0x79, 0x03, 0x3a, 0x6b, 0xfb, 0x60, 0x17, 0xd4, 0x23, 0x24, 0x2f, 0x6f, 0xcb, - 0xab, 0x47, 0x08, 0x5a, 0xab, 0x1b, 0xad, 0xef, 0xa6, 0x5e, 0xba, 0x17, 0xf7, 0x0b, 0xdb, 0x78, - 0x58, 0xd8, 0xc6, 0xef, 0x85, 0x6d, 0xdc, 0x3d, 0xd9, 0xb5, 0x87, 0x27, 0xbb, 0xf6, 0xf3, 0xc9, - 0xae, 0x7d, 0x72, 0x62, 0xc2, 0x93, 0x79, 0xe0, 0x84, 0x74, 0x26, 0x1e, 0x0a, 0x9c, 0x09, 0xfe, - 0xdb, 0xed, 0xf7, 0xf2, 0xf1, 0xd0, 0x2f, 0x50, 0xa0, 0xd7, 0xc1, 0x8e, 0xbc, 0x1d, 0xa7, 0x7f, - 0x02, 0x00, 0x00, 0xff, 0xff, 0x9a, 0x4d, 0xfd, 0x88, 0x74, 0x05, 0x00, 0x00, + // 701 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x54, 0xcd, 0x4e, 0xdb, 0x4a, + 0x18, 0x8d, 0x43, 0x20, 0xce, 0x84, 0xfc, 0x30, 0x70, 0x25, 0xc3, 0x95, 0x9c, 0xc0, 0xbd, 0xad, + 0xa2, 0x2e, 0x1c, 0xa9, 0xac, 0xba, 0x69, 0x25, 0xc3, 0x82, 0x20, 0x54, 0x55, 0x4e, 0xc5, 0xa2, + 0x1b, 0x6b, 0x6c, 0x4f, 0xed, 0x51, 0x1d, 0x8f, 0xeb, 0x99, 0xa0, 0xd2, 0xa7, 0xe0, 0xb1, 0x58, + 0xb2, 0xec, 0x8a, 0x56, 0xe1, 0x45, 0xaa, 0xf9, 0x71, 0xe2, 0x10, 0xb1, 0x4a, 0xe6, 0x7c, 0x67, + 0x8e, 0xcf, 0x9c, 0xef, 0x9b, 0x01, 0x16, 0xbf, 0xcd, 0x31, 0x1b, 0x47, 0xb7, 0x33, 0x92, 0xf1, + 0x31, 0xe3, 0x88, 0x63, 0x27, 0x2f, 0x28, 0xa7, 0x70, 0x47, 0x61, 0x47, 0x07, 0x31, 0x8d, 0xa9, + 0x84, 0xc6, 0xe2, 0x9f, 0xaa, 0x1e, 0x0d, 0x62, 0x4a, 0xe3, 0x14, 0x8f, 0xe5, 0x2a, 0x98, 0x7f, + 0x1d, 0x73, 0x32, 0xc3, 0x8c, 0xa3, 0x59, 0xae, 0x09, 0xc7, 0x4a, 0x98, 0xe3, 0x2c, 0xc2, 0x85, + 0x14, 0x47, 0x41, 0x48, 0xc6, 0x12, 0xd5, 0x94, 0x93, 0x0d, 0x8a, 0x06, 0x2a, 0x9c, 0xd7, 0x2f, + 0x70, 0x6e, 0x50, 0x4a, 0x22, 0xc4, 0x69, 0xa1, 0x79, 0xff, 0xbd, 0xc0, 0xcb, 0x51, 0x81, 0x66, + 0x2f, 0x7f, 0x50, 0x1e, 0x78, 0xed, 0x83, 0x87, 0x6b, 0x81, 0xa8, 0x1f, 0x55, 0x3a, 0x59, 0x34, + 0xc1, 0xf6, 0x54, 0x6c, 0x80, 0xa7, 0xa0, 0x79, 0x83, 0x0b, 0x46, 0x68, 0x66, 0x19, 0x43, 0x63, + 0xd4, 0x7e, 0x7b, 0xe8, 0xac, 0x44, 0x1d, 0x95, 0xe2, 0xb5, 0x22, 0x78, 0x25, 0x13, 0x1e, 0x02, + 0x33, 0x4c, 0x10, 0xc9, 0x7c, 0x12, 0x59, 0xf5, 0xa1, 0x31, 0x6a, 0x79, 0x4d, 0xb9, 0x9e, 0x44, + 0xf0, 0x15, 0xe8, 0x92, 0x8c, 0x70, 0x82, 0x52, 0x3f, 0xc1, 0x24, 0x4e, 0xb8, 0xb5, 0x35, 0x34, + 0x46, 0x5b, 0x5e, 0x47, 0xa3, 0x17, 0x12, 0x84, 0x6f, 0xc0, 0x5e, 0x8a, 0x18, 0xf7, 0x83, 0x94, + 0x86, 0xdf, 0x4a, 0x66, 0x43, 0x32, 0x7b, 0xa2, 0xe0, 0x0a, 0x5c, 0x73, 0x3d, 0xd0, 0xa9, 0x70, + 0x49, 0x64, 0x6d, 0x6f, 0x1a, 0x55, 0xe7, 0x96, 0xbb, 0x26, 0xe7, 0xee, 0xfe, 0xfd, 0xe3, 0xa0, + 0xb6, 0x78, 0x1c, 0xb4, 0xaf, 0x4a, 0xa9, 0xc9, 0xb9, 0xd7, 0x5e, 0xea, 0x4e, 0x22, 0x78, 0x05, + 0x7a, 0x15, 0x4d, 0xd1, 0x71, 0x6b, 0x47, 0xaa, 0x1e, 0x39, 0x6a, 0x1c, 0x9c, 0x72, 0x1c, 0x9c, + 0xcf, 0xe5, 0x38, 0xb8, 0xa6, 0x90, 0xbd, 0xfb, 0x3d, 0x30, 0xbc, 0xce, 0x52, 0x4b, 0x54, 0xa1, + 0x0b, 0xc0, 0xb2, 0x8b, 0xcc, 0x6a, 0x49, 0x21, 0x7b, 0xd3, 0xde, 0x75, 0xc9, 0x99, 0x62, 0xee, + 0xd6, 0x2d, 0xc3, 0xab, 0xec, 0x82, 0x67, 0xc0, 0x96, 0x8e, 0x54, 0x16, 0xfe, 0xaa, 0xe2, 0x87, + 0x09, 0xca, 0x62, 0x1c, 0x59, 0x6d, 0x19, 0xcf, 0xbf, 0x82, 0xa5, 0x92, 0x59, 0xea, 0xb1, 0x33, + 0x45, 0x81, 0x1e, 0xe8, 0x87, 0x34, 0x63, 0x38, 0x63, 0x73, 0xe6, 0xab, 0x81, 0xb1, 0x76, 0xa5, + 0x9d, 0xe3, 0x4d, 0x3b, 0x67, 0x25, 0xf3, 0x93, 0x24, 0xba, 0x0d, 0x71, 0x3c, 0xaf, 0x17, 0xae, + 0xc3, 0xf0, 0x23, 0xf8, 0xbf, 0x6a, 0xec, 0xb9, 0xfe, 0xd2, 0x5e, 0x47, 0xda, 0x1b, 0xae, 0xec, + 0x3d, 0xd3, 0x2f, 0x3d, 0x96, 0xad, 0x2f, 0x30, 0x9b, 0xa7, 0x9c, 0xf9, 0x09, 0x62, 0x89, 0xd5, + 0x1d, 0x1a, 0xa3, 0x5d, 0xd5, 0x7a, 0x4f, 0xe1, 0x17, 0x88, 0x25, 0x62, 0xd0, 0x50, 0x9e, 0x2b, + 0x4a, 0x4f, 0x52, 0x9a, 0x28, 0xcf, 0x65, 0xe9, 0x83, 0x96, 0x61, 0x9c, 0x16, 0xb8, 0x9c, 0xa0, + 0xfe, 0xd0, 0x18, 0x35, 0xdc, 0xfd, 0xc5, 0xe3, 0xa0, 0x27, 0x5a, 0x3f, 0x15, 0x35, 0x65, 0x46, + 0x69, 0x57, 0x00, 0x38, 0x00, 0xed, 0x00, 0xb1, 0xe5, 0xd6, 0x3d, 0xb1, 0xd5, 0x03, 0x02, 0xd2, + 0x84, 0xf7, 0x60, 0x97, 0xe1, 0xef, 0x73, 0x9c, 0x85, 0x58, 0x74, 0xcc, 0x82, 0x32, 0xc8, 0x03, + 0x47, 0xdf, 0xa4, 0x69, 0xa5, 0xa6, 0xb3, 0x5b, 0xe3, 0x43, 0x17, 0x74, 0x0b, 0x9a, 0xa6, 0xe2, + 0x00, 0xba, 0x15, 0xfb, 0x52, 0xe1, 0x9f, 0x52, 0xc1, 0x53, 0xd5, 0xb5, 0xf8, 0x3b, 0x45, 0x15, + 0x84, 0x23, 0xd0, 0xd7, 0xe1, 0xa3, 0x08, 0x17, 0x2a, 0x88, 0x03, 0x19, 0x44, 0x57, 0x05, 0x2d, + 0x60, 0x91, 0xc7, 0x65, 0xc3, 0x6c, 0xf6, 0xcd, 0xcb, 0x86, 0x69, 0xf6, 0x5b, 0x97, 0x0d, 0x13, + 0xf4, 0xdb, 0x27, 0xef, 0x40, 0x67, 0xed, 0x0b, 0xb0, 0x0b, 0xea, 0x11, 0x92, 0xd7, 0xbc, 0xe5, + 0xd5, 0x23, 0x04, 0xad, 0xd5, 0xdd, 0xd7, 0xb7, 0x58, 0x2f, 0xdd, 0x8b, 0xfb, 0x85, 0x6d, 0x3c, + 0x2c, 0x6c, 0xe3, 0xcf, 0xc2, 0x36, 0xee, 0x9e, 0xec, 0xda, 0xc3, 0x93, 0x5d, 0xfb, 0xf5, 0x64, + 0xd7, 0xbe, 0x38, 0x31, 0xe1, 0xc9, 0x3c, 0x70, 0x42, 0x3a, 0x13, 0x4f, 0x0a, 0xce, 0x04, 0xff, + 0xc7, 0xed, 0xcf, 0xf2, 0x99, 0xd1, 0x6f, 0x55, 0xa0, 0xd7, 0xc1, 0x8e, 0xbc, 0x47, 0xa7, 0x7f, + 0x03, 0x00, 0x00, 0xff, 0xff, 0x76, 0xee, 0xb0, 0xc3, 0x9e, 0x05, 0x00, 0x00, } func (m *State) Marshal() (dAtA []byte, err error) { @@ -324,6 +333,15 @@ func (m *State) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.LastHeaderHash) > 0 { + i -= len(m.LastHeaderHash) + copy(dAtA[i:], m.LastHeaderHash) + i = encodeVarintState(dAtA, i, uint64(len(m.LastHeaderHash))) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xa2 + } { size, err := m.RollappParams.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -560,6 +578,10 @@ func (m *State) Size() (n int) { n += 2 + l + sovState(uint64(l)) l = m.RollappParams.Size() n += 2 + l + sovState(uint64(l)) + l = len(m.LastHeaderHash) + if l > 0 { + n += 2 + l + sovState(uint64(l)) + } return n } @@ -1066,6 +1088,40 @@ func (m *State) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 20: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LastHeaderHash", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowState + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthState + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthState + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LastHeaderHash = append(m.LastHeaderHash[:0], dAtA[iNdEx:postIndex]...) + if m.LastHeaderHash == nil { + m.LastHeaderHash = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipState(dAtA[iNdEx:]) diff --git a/types/serialization.go b/types/serialization.go index 1ccabd1ff..f3b1b21d9 100644 --- a/types/serialization.go +++ b/types/serialization.go @@ -265,6 +265,7 @@ func (s *State) ToProto() (*pb.State, error) { ConsensusParams: s.ConsensusParams, LastHeightConsensusParamsChanged: s.LastHeightConsensusParamsChanged, LastResultsHash: s.LastResultsHash[:], + LastHeaderHash: s.LastHeaderHash[:], AppHash: s.AppHash[:], RollappParams: s.RollappParams, }, nil @@ -287,6 +288,7 @@ func (s *State) FromProto(other *pb.State) error { s.ConsensusParams = other.ConsensusParams s.LastHeightConsensusParamsChanged = other.LastHeightConsensusParamsChanged copy(s.LastResultsHash[:], other.LastResultsHash) + copy(s.LastHeaderHash[:], other.LastHeaderHash) copy(s.AppHash[:], other.AppHash) s.RollappParams = other.RollappParams return nil diff --git a/types/state.go b/types/state.go index d4a6c87f0..1f4959a50 100644 --- a/types/state.go +++ b/types/state.go @@ -42,6 +42,9 @@ type State struct { // New rollapp parameters . RollappParams dymint.RollappParams + + // LastHeaderHash is the hash of the last block header. + LastHeaderHash [32]byte } func (s *State) IsGenesis() bool { diff --git a/types/validation.go b/types/validation.go index 100091807..8661fc914 100644 --- a/types/validation.go +++ b/types/validation.go @@ -4,6 +4,7 @@ import ( "bytes" "errors" "fmt" + "time" tmcrypto "github.com/tendermint/tendermint/crypto" tmtypes "github.com/tendermint/tendermint/types" @@ -46,22 +47,46 @@ func (b *Block) ValidateBasic() error { func (b *Block) ValidateWithState(state *State) error { err := b.ValidateBasic() if err != nil { + if errors.Is(err, ErrInvalidHeaderDataHash) { + return NewErrInvalidHeaderDataHashFraud(b.Header.DataHash, [32]byte(GetDataHash(b))) + } + return err } + + if b.Header.ChainID != state.ChainID { + return NewErrInvalidChainID(state.ChainID, b) + } + + if b.Header.LastHeaderHash != state.LastHeaderHash { + return NewErrLastHeaderHashMismatch(state.LastHeaderHash, b) + } + + currentTime := time.Now().UTC() + if currentTime.Add(TimeFraudMaxDrift).Before(time.Unix(0, int64(b.Header.Time))) { + return NewErrTimeFraud(b, currentTime) + } + if b.Header.Version.App != state.Version.Consensus.App || b.Header.Version.Block != state.Version.Consensus.Block { - return errors.New("b version mismatch") + return ErrVersionMismatch + } + + nextHeight := state.NextHeight() + if b.Header.Height != nextHeight { + return NewErrFraudHeightMismatch(state.NextHeight(), b.Header.Height, b) } - if b.Header.Height != state.NextHeight() { - return errors.New("height mismatch") + if !bytes.Equal(b.Header.NextSequencersHash[:], state.Sequencers.ProposerHash()) { + return NewErrInvalidNextSequencersHashFraud([32]byte(state.Sequencers.ProposerHash()), b.Header.NextSequencersHash) } if !bytes.Equal(b.Header.AppHash[:], state.AppHash[:]) { - return errors.New("AppHash mismatch") + return NewErrFraudAppHashMismatch(state.AppHash, b.Header.AppHash, b) } + if !bytes.Equal(b.Header.LastResultsHash[:], state.LastResultsHash[:]) { - return errors.New("LastResultsHash mismatch") + return NewErrLastResultsHashMismatch(state.LastResultsHash, b) } return nil @@ -70,7 +95,7 @@ func (b *Block) ValidateWithState(state *State) error { // ValidateBasic performs basic validation of a header. func (h *Header) ValidateBasic() error { if len(h.ProposerAddress) == 0 { - return errors.New("no proposer address") + return ErrEmptyProposerAddress } return nil @@ -97,16 +122,41 @@ func (c *Commit) ValidateBasic() error { func (c *Commit) ValidateWithHeader(proposerPubKey tmcrypto.PubKey, header *Header) error { if err := c.ValidateBasic(); err != nil { - return err + return NewErrInvalidSignatureFraud(err) } + abciHeaderPb := ToABCIHeaderPB(header) abciHeaderBytes, err := abciHeaderPb.Marshal() if err != nil { return err } + // commit is validated to have single signature if !proposerPubKey.VerifySignature(abciHeaderBytes, c.Signatures[0]) { - return ErrInvalidSignature + return NewErrInvalidSignatureFraud(ErrInvalidSignature) + } + + if c.Height != header.Height { + return NewErrInvalidBlockHeightFraud(c.Height, header.Height) + } + + if !bytes.Equal(header.ProposerAddress, proposerPubKey.Address()) { + return NewErrInvalidProposerAddressFraud(header.ProposerAddress, proposerPubKey.Address()) + } + + seq := NewSequencerFromValidator(*tmtypes.NewValidator(proposerPubKey, 1)) + proposerHash, err := seq.Hash() + if err != nil { + return err + } + + if !bytes.Equal(header.SequencerHash[:], proposerHash) { + return NewErrInvalidSequencerHashFraud(header.SequencerHash, proposerHash) } + + if c.HeaderHash != header.Hash() { + return NewErrInvalidHeaderHashFraud(c.HeaderHash, header.Hash()) + } + return nil } diff --git a/types/validation_test.go b/types/validation_test.go new file mode 100644 index 000000000..bf9e39498 --- /dev/null +++ b/types/validation_test.go @@ -0,0 +1,516 @@ +package types + +import ( + "errors" + "github.com/cometbft/cometbft/libs/math" + "github.com/dymensionxyz/gerr-cosmos/gerrc" + "github.com/tendermint/tendermint/crypto/ed25519" + tmtypes "github.com/tendermint/tendermint/types" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + tmstate "github.com/tendermint/tendermint/proto/tendermint/state" + "github.com/tendermint/tendermint/proto/tendermint/version" +) + +func TestBlock_ValidateWithState(t *testing.T) { + proposer := NewSequencerFromValidator(*tmtypes.NewValidator(ed25519.GenPrivKey().PubKey(), 1)) + proposerHash := proposer.MustHash() + currentTime := time.Now().UTC() + validState := &State{ + Version: tmstate.Version{ + Consensus: version.Consensus{ + Block: 1, + App: 1, + }, + }, + LastBlockHeight: atomic.Uint64{}, + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + LastHeaderHash: [32]byte{7, 8, 9}, + ChainID: "chainID", + Sequencers: SequencerSet{ + Proposer: proposer, + }, + } + validState.LastBlockHeight.Store(9) + + validBlock := &Block{ + Header: Header{ + Version: Version{ + Block: 1, + App: 1, + }, + Height: 10, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte{}, + LastHeaderHash: [32]byte{7, 8, 9}, + ChainID: "chainID", + NextSequencersHash: [32]byte(proposerHash), + }, + Data: Data{}, + LastCommit: Commit{}, + } + validBlock.Header.DataHash = [32]byte(GetDataHash(validBlock)) + + tests := []struct { + name string + block *Block + state *State + wantErr bool + theErr error + expectedErrType interface{} + isFraud bool + }{ + { + name: "Valid block", + block: validBlock, + state: validState, + wantErr: false, + isFraud: false, + }, + { + name: "Invalid block version", + block: &Block{ + Header: Header{ + Version: Version{ + Block: 2, + App: 1, + }, + Height: 10, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte(GetDataHash(validBlock)), + LastHeaderHash: [32]byte{7, 8, 9}, + ChainID: "chainID", + }, + }, + state: validState, + theErr: ErrVersionMismatch, + wantErr: true, + isFraud: false, + }, + { + name: "Invalid app version", + block: &Block{ + Header: Header{ + Version: Version{ + Block: 1, + App: 2, + }, + Height: 10, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte(GetDataHash(validBlock)), + LastHeaderHash: [32]byte{7, 8, 9}, + ChainID: "chainID", + }, + }, + state: validState, + wantErr: true, + theErr: ErrVersionMismatch, + isFraud: false, + }, + { + name: "Invalid height", + block: &Block{ + Header: Header{ + Version: validBlock.Header.Version, + Height: 11, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte(GetDataHash(validBlock)), + }, + }, + state: validState, + wantErr: true, + expectedErrType: &ErrFraudHeightMismatch{}, + isFraud: true, + }, + { + name: "Invalid AppHash", + block: &Block{ + Header: Header{ + Version: validBlock.Header.Version, + Height: 10, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{9, 9, 9}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte(GetDataHash(validBlock)), + }, + }, + state: validState, + expectedErrType: &ErrFraudAppHashMismatch{}, + wantErr: true, + isFraud: true, + }, + { + name: "Invalid LastResultsHash", + block: &Block{ + Header: Header{ + Version: validBlock.Header.Version, + Height: 10, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{9, 9, 9}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte(GetDataHash(validBlock)), + }, + }, + state: validState, + wantErr: true, + expectedErrType: &ErrLastResultsHashMismatch{}, + isFraud: true, + }, + { + name: "Future block time", + block: &Block{ + Header: Header{ + Version: validBlock.Header.Version, + Height: 10, + Time: uint64(currentTime.Add(2 * TimeFraudMaxDrift).UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte(GetDataHash(validBlock)), + }, + }, + state: validState, + wantErr: true, + expectedErrType: &ErrTimeFraud{}, + isFraud: true, + }, + { + name: "Invalid proposer address", + block: &Block{ + Header: Header{ + Version: validBlock.Header.Version, + Height: 10, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte{}, + }, + }, + state: validState, + wantErr: true, + expectedErrType: ErrEmptyProposerAddress, + isFraud: false, + }, + { + name: "invalid last header hash", + block: &Block{ + Header: Header{ + Version: validBlock.Header.Version, + Height: 10, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte(GetDataHash(validBlock)), + LastHeaderHash: [32]byte{1, 2, 3}, + }, + }, + state: validState, + wantErr: true, + expectedErrType: &ErrLastHeaderHashMismatch{}, + isFraud: true, + }, + { + name: "invalid chain ID", + block: &Block{ + Header: Header{ + Version: validBlock.Header.Version, + Height: 10, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte(GetDataHash(validBlock)), + LastHeaderHash: [32]byte{7, 8, 9}, + ChainID: "invalidChainID", + }, + }, + state: validState, + wantErr: true, + expectedErrType: &ErrInvalidChainID{}, + isFraud: true, + }, + { + name: "invalid NextSequencersHash", + block: &Block{ + Header: Header{ + Version: validBlock.Header.Version, + Height: 10, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte(GetDataHash(validBlock)), + LastHeaderHash: [32]byte{7, 8, 9}, + ChainID: "chainID", + NextSequencersHash: [32]byte{1, 2, 3}, + }, + }, + state: validState, + wantErr: true, + expectedErrType: &ErrInvalidNextSequencersHashFraud{}, + isFraud: true, + }, + { + name: "invalid header data hash", + block: &Block{ + Header: Header{ + Version: validBlock.Header.Version, + Height: 10, + Time: uint64(currentTime.UnixNano()), + AppHash: [32]byte{1, 2, 3}, + LastResultsHash: [32]byte{4, 5, 6}, + ProposerAddress: []byte("proposer"), + DataHash: [32]byte{1, 2, 3}, + LastHeaderHash: [32]byte{7, 8, 9}, + ChainID: "chainID", + }, + }, + state: validState, + wantErr: true, + expectedErrType: ErrInvalidHeaderDataHashFraud{}, + isFraud: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := tt.block.ValidateWithState(tt.state) + if tt.wantErr { + assert.Error(t, err) + if tt.isFraud { + require.True(t, errors.Is(err, gerrc.ErrFault)) + if tt.expectedErrType != nil { + assert.True(t, errors.As(err, &tt.expectedErrType), + "expected error of type %T, got %T", tt.expectedErrType, err) + } + } else { + require.False(t, errors.Is(err, gerrc.ErrFault)) + } + } else { + assert.NoError(t, err) + } + }) + } +} + +func TestCommit_ValidateWithHeader(t *testing.T) { + // Generate keys for the proposer and another actor for invalid signatures + proposerKey := ed25519.GenPrivKey() + anotherKey := ed25519.GenPrivKey() + + // Helper function to create a valid commit + createValidCommit := func() (*Commit, *Block, []byte, error) { + seq := NewSequencerFromValidator(*tmtypes.NewValidator(proposerKey.PubKey(), 1)) + proposerHash := seq.MustHash() + + block := &Block{ + Header: Header{ + Version: Version{ + Block: 1, + App: 1, + }, + ChainID: "test", + Height: 1, + Time: uint64(time.Now().UTC().UnixNano()), + LastHeaderHash: [32]byte{}, + DataHash: [32]byte{}, + ConsensusHash: [32]byte{}, + AppHash: [32]byte{}, + LastResultsHash: [32]byte{}, + ProposerAddress: proposerKey.PubKey().Address(), + SequencerHash: [32]byte(proposerHash), + }, + } + + abciHeaderBytes, signature, err := signBlock(block, proposerKey) + if err != nil { + return nil, nil, nil, err + } + + commit := &Commit{ + Height: block.Header.Height, + HeaderHash: block.Hash(), + Signatures: []Signature{signature}, + } + + return commit, block, abciHeaderBytes, nil + } + + t.Run("Valid commit", func(t *testing.T) { + commit, block, _, err := createValidCommit() + require.NoError(t, err, "Creating the valid commit should not fail") + + err = commit.ValidateWithHeader(proposerKey.PubKey(), &block.Header) + assert.NoError(t, err, "Validation should pass without errors") + }) + + t.Run("ValidateBasic fails - invalid height", func(t *testing.T) { + commit, block, _, err := createValidCommit() + require.NoError(t, err, "Creating the valid commit should not fail") + + commit.Height = 0 // Set an invalid height so ValidateBasic fails + + err = commit.ValidateWithHeader(proposerKey.PubKey(), &block.Header) + require.Error(t, err, "Validation should fail due to an invalid height") + require.Equal(t, &ErrInvalidBlockHeightFraud{0, 1}, err) + require.True(t, errors.Is(err, gerrc.ErrFault), "The error should be a fraud error") + }) + + t.Run("Invalid signature", func(t *testing.T) { + commit, block, abciHeaderBytes, err := createValidCommit() + require.NoError(t, err, "Creating the valid commit should not fail") + + // Generate an invalid signature using another key + invalidSignature, err := anotherKey.Sign(abciHeaderBytes) + require.NoError(t, err, "Generating the invalid signature should not fail") + + commit.Signatures = []Signature{invalidSignature} + + err = commit.ValidateWithHeader(proposerKey.PubKey(), &block.Header) + require.Error(t, err, "Validation should fail due to an invalid signature") + assert.Equal(t, NewErrInvalidSignatureFraud(ErrInvalidSignature), err) + assert.True(t, errors.Is(err, gerrc.ErrFault), "The error should be a fraud error") + }) + + t.Run("Fails with more than one signature", func(t *testing.T) { + commit, block, abciHeaderBytes, err := createValidCommit() + require.NoError(t, err, "Creating the valid commit should not fail") + + // Generate another valid signature using a different key + anotherSignature, err := anotherKey.Sign(abciHeaderBytes) + require.NoError(t, err, "Generating an additional signature should not fail") + + // Add the additional signature to the commit + commit.Signatures = append(commit.Signatures, anotherSignature) + + // Ensure there are more than one signature + require.Greater(t, len(commit.Signatures), 1, "Commit should have more than one signature") + + // Validate and expect an error due to multiple signatures + err = commit.ValidateWithHeader(proposerKey.PubKey(), &block.Header) + require.Error(t, err, "Validation should fail when there is more than one signature") + assert.Equal(t, NewErrInvalidSignatureFraud(errors.New("there should be 1 signature")), err) + assert.True(t, errors.Is(err, gerrc.ErrFault), "The error should be a fraud error") + }) + + t.Run("Fails when signature size exceeds MaxSignatureSize", func(t *testing.T) { + commit, block, _, err := createValidCommit() + require.NoError(t, err, "Creating the valid commit should not fail") + + // Define a signature that exceeds MaxSignatureSize + invalidSignature := make([]byte, math.MaxInt(ed25519.SignatureSize, 64)+1) + + // Replace the valid signature with the invalid oversized signature + commit.Signatures = []Signature{invalidSignature} + + // Ensure the signature size exceeds the maximum allowed size + require.Greater(t, len(invalidSignature), math.MaxInt(ed25519.SignatureSize, 64), "Signature size should exceed MaxSignatureSize") + + // Validate and expect an error due to oversized signature + err = commit.ValidateWithHeader(proposerKey.PubKey(), &block.Header) + require.Error(t, err, "Validation should fail when the signature size exceeds the maximum allowed size") + assert.Equal(t, NewErrInvalidSignatureFraud(errors.New("signature is too big")), err) + assert.True(t, errors.Is(err, gerrc.ErrFault), "The error should be a fraud error") + }) + + t.Run("Fails when proposerPubKey.Address() does not match Header.ProposerAddress", func(t *testing.T) { + commit, block, _, err := createValidCommit() + require.NoError(t, err, "Creating the valid commit should not fail") + + // Modify the block header's proposer address to simulate a mismatch + block.Header.ProposerAddress = anotherKey.PubKey().Address() // Set to a different proposer's address + + // resign the block with the new proposer address + _, signature, err := signBlock(block, proposerKey) + if err != nil { + return + } + + commit.Signatures = []Signature{signature} + + // Ensure the proposer's address does not match the block header's proposer address + require.NotEqual(t, proposerKey.PubKey().Address(), block.Header.ProposerAddress, "The proposer's public key address should not match the block header proposer address") + + // Validate and expect an error due to mismatching proposer addresses + err = commit.ValidateWithHeader(proposerKey.PubKey(), &block.Header) + require.Error(t, err, "Validation should fail when the proposer's address does not match the header's proposer address") + assert.Equal(t, NewErrInvalidProposerAddressFraud(block.Header.ProposerAddress, proposerKey.PubKey().Address()), err) + assert.True(t, errors.Is(err, gerrc.ErrFault), "The error should be a fraud error") + }) + + t.Run("Fails when SequencerHash does not match proposerHash", func(t *testing.T) { + commit, block, _, err := createValidCommit() + require.NoError(t, err, "Creating the valid commit should not fail") + + // Modify the block header's SequencerHash to simulate a mismatch + block.Header.SequencerHash = [32]byte{1, 2, 3} // Set to an invalid hash + + // resign the block with the new SequencerHash + _, signature, err := signBlock(block, proposerKey) + if err != nil { + return + } + + commit.Signatures = []Signature{signature} + + // Ensure the SequencerHash does not match the proposer's hash + hash := NewSequencerFromValidator(*tmtypes.NewValidator(proposerKey.PubKey(), 1)).MustHash() + require.NoError(t, err, "Generating the SequencerHash should not fail") + require.NotEqual(t, block.Header.SequencerHash, hash, "The SequencerHash should not match the proposer's hash") + + // Validate and expect an error due to mismatching SequencerHash + err = commit.ValidateWithHeader(proposerKey.PubKey(), &block.Header) + bytes := NewSequencerFromValidator(*tmtypes.NewValidator(proposerKey.PubKey(), 1)).MustHash() + + require.Equal(t, &ErrInvalidSequencerHashFraud{[32]byte{1, 2, 3}, bytes}, err) + require.True(t, errors.Is(err, gerrc.ErrFault), "The error should be a fraud error") + }) + + t.Run("HeaderHash does not match Block Hash", func(t *testing.T) { + commit, block, _, err := createValidCommit() + require.NoError(t, err, "Creating the valid commit should not fail") + + commit.HeaderHash = [32]byte{1, 2, 3} // Introduce an invalid hash + + assert.NotEqual(t, block.Hash(), commit.HeaderHash, "The commit header hash should not match the block header hash") + + err = commit.ValidateWithHeader(proposerKey.PubKey(), &block.Header) + require.Equal(t, &ErrInvalidHeaderHashFraud{[32]byte{1, 2, 3}, block.Hash()}, err) + require.True(t, errors.Is(err, gerrc.ErrFault), "The error should be a fraud error") + }) +} + +func signBlock(block *Block, proposerKey ed25519.PrivKey) ([]byte, []byte, error) { + abciHeaderPb := ToABCIHeaderPB(&block.Header) + abciHeaderBytes, err := abciHeaderPb.Marshal() + if err != nil { + return nil, nil, err + } + + signature, err := proposerKey.Sign(abciHeaderBytes) + if err != nil { + return nil, nil, err + } + + return abciHeaderBytes, signature, nil +}