From 314003c1ec74acad6e645e9370c5e9f43cfc7f0a Mon Sep 17 00:00:00 2001 From: Sergi Rene Date: Mon, 28 Oct 2024 13:41:43 +0100 Subject: [PATCH] feat(manager): state update mishbehavior detection (#1130) --- block/block.go | 5 + block/manager.go | 47 +-- block/pruning_test.go | 2 +- ...tateupdate_validator.go => slvalidator.go} | 40 +- block/slvalidator_test.go | 390 ++++++++++++++++++ block/state.go | 8 +- block/stateupdate_validator_test.go | 214 ---------- block/submit.go | 7 + block/sync.go | 3 - block/validate.go | 4 +- da/celestia/celestia.go | 70 +--- da/celestia/celestia_test.go | 26 -- da/errors.go | 2 + .../dymint/settlement/mock_ClientI.go | 87 +++- .../dymensionxyz/dymint/store/mock_Store.go | 144 ++++++- .../dymensionxyz/dymension/rollapp/tx.proto | 2 - proto/types/dymint/dymint.proto | 3 +- proto/types/dymint/state.proto | 5 +- rpc/client/client_test.go | 16 +- settlement/dymension/dymension.go | 65 ++- settlement/grpc/grpc.go | 15 +- settlement/local/local.go | 11 +- settlement/settlement.go | 6 +- store/pruning.go | 14 + store/pruning_test.go | 24 ++ store/store.go | 24 ++ store/storeIface.go | 8 +- testutil/mocks.go | 85 ++++ testutil/types.go | 31 +- types/batch.go | 3 +- types/errors.go | 78 +++- .../dymensionxyz/dymension/rollapp/tx.pb.go | 165 +++----- types/pb/dymint/state.pb.go | 156 +++---- types/serialization.go | 24 +- types/serialization_test.go | 5 +- types/state.go | 3 +- 36 files changed, 1153 insertions(+), 639 deletions(-) rename block/{stateupdate_validator.go => slvalidator.go} (80%) create mode 100644 block/slvalidator_test.go delete mode 100644 block/stateupdate_validator_test.go diff --git a/block/block.go b/block/block.go index 825bced82..a35afba05 100644 --- a/block/block.go +++ b/block/block.go @@ -105,6 +105,11 @@ func (m *Manager) applyBlock(block *types.Block, commit *types.Commit, blockMeta return fmt.Errorf("save block source: %w", err) } + _, err = m.Store.SaveDRSVersion(block.Header.Height, responses.EndBlock.RollappParamUpdates.Version, nil) + if err != nil { + return fmt.Errorf("add drs version: %w", err) + } + // Commit block to app appHash, retainHeight, err = m.Executor.Commit(m.State, block, responses) if err != nil { diff --git a/block/manager.go b/block/manager.go index e065b139b..8e315d5b5 100644 --- a/block/manager.go +++ b/block/manager.go @@ -226,6 +226,12 @@ func (m *Manager) Start(ctx context.Context) error { /* ----------------------------- full node mode ----------------------------- */ if !isProposer { + // update latest finalized height + err = m.updateLastFinalizedHeightFromSettlement() + if err != nil { + return fmt.Errorf("sync block manager from settlement: %w", err) + } + // Start the settlement validation loop in the background uerrors.ErrGroupGoLog(eg, m.logger, func() error { return m.SettlementValidateLoop(ctx) @@ -321,7 +327,8 @@ func (m *Manager) updateFromLastSettlementState() error { m.logger.Error("update bonded sequencer set", "error", err) } - res, err := m.SLClient.GetLatestBatch() + // update latest height from SL + latestHeight, err := m.SLClient.GetLatestHeight() if errors.Is(err, gerrc.ErrNotFound) { // The SL hasn't got any batches for this chain yet. m.logger.Info("No batches for chain found in SL.") @@ -334,17 +341,24 @@ func (m *Manager) updateFromLastSettlementState() error { return err } - m.LastSettlementHeight.Store(res.EndHeight) + m.LastSettlementHeight.Store(latestHeight) - if res.EndHeight >= m.State.NextHeight() { - m.UpdateTargetHeight(res.EndHeight) + if latestHeight >= m.State.NextHeight() { + m.UpdateTargetHeight(latestHeight) } - // get the latest finalized height to know from where to start validating - err = m.UpdateFinalizedHeight() - if err != nil { - return err + return nil +} + +func (m *Manager) updateLastFinalizedHeightFromSettlement() error { + // update latest finalized height from SL + height, err := m.SLClient.GetLatestFinalizedHeight() + if errors.Is(err, gerrc.ErrNotFound) { + m.logger.Info("No finalized batches for chain found in SL.") + } else if err != nil { + return fmt.Errorf("getting finalized height. err: %w", err) } + m.SettlementValidator.UpdateLastValidatedHeight(height) return nil } @@ -362,23 +376,6 @@ func (m *Manager) UpdateTargetHeight(h uint64) { } } -// UpdateFinalizedHeight retrieves the latest finalized batch and updates validation height with it -func (m *Manager) UpdateFinalizedHeight() error { - res, err := m.SLClient.GetLatestFinalizedBatch() - if err != nil && !errors.Is(err, gerrc.ErrNotFound) { - // The SL hasn't got any batches for this chain yet. - return fmt.Errorf("getting finalized height. err: %w", err) - } - if errors.Is(err, gerrc.ErrNotFound) { - // The SL hasn't got any batches for this chain yet. - m.logger.Info("No finalized batches for chain found in SL.") - } else { - // update validation height with latest finalized height (it will be updated only of finalized height is higher) - m.SettlementValidator.UpdateLastValidatedHeight(res.EndHeight) - } - return nil -} - // ValidateConfigWithRollappParams checks the configuration params are consistent with the params in the dymint state (e.g. DA and version) func (m *Manager) ValidateConfigWithRollappParams() error { if version.Commit != m.State.RollappParams.Version { diff --git a/block/pruning_test.go b/block/pruning_test.go index d37b610c4..ecf073e07 100644 --- a/block/pruning_test.go +++ b/block/pruning_test.go @@ -18,7 +18,7 @@ import ( func TestPruningRetainHeight(t *testing.T) { require := require.New(t) - app := testutil.GetAppMock() + app := testutil.GetAppMock(testutil.EndBlock) app.On("EndBlock", mock.Anything).Return(abci.ResponseEndBlock{ RollappParamUpdates: &abci.RollappParams{ Da: "mock", diff --git a/block/stateupdate_validator.go b/block/slvalidator.go similarity index 80% rename from block/stateupdate_validator.go rename to block/slvalidator.go index d9efa5c69..343714261 100644 --- a/block/stateupdate_validator.go +++ b/block/slvalidator.go @@ -3,6 +3,8 @@ package block import ( "bytes" "crypto/sha256" + "encoding/hex" + "errors" "fmt" "sync/atomic" @@ -71,7 +73,19 @@ func (v *SettlementValidator) ValidateStateUpdate(batch *settlement.ResultRetrie if daBatch.Code == da.StatusSuccess { break } + + // fraud detected in case blob is retrieved but unable to get blocks from it. + if errors.Is(daBatch.BaseResult.Error, da.ErrBlobNotParsed) { + return types.NewErrStateUpdateBlobCorruptedFraud(batch.StateIndex, string(batch.MetaData.DA.Client), batch.MetaData.DA.Height, hex.EncodeToString(batch.MetaData.DA.Commitment)) + } + + // fraud detected in case availability checks fail and therefore there certainty the blob, according to the state update DA path, is not available. + checkBatchResult := v.blockManager.Retriever.CheckBatchAvailability(batch.MetaData.DA) + if errors.Is(checkBatchResult.Error, da.ErrBlobNotIncluded) { + return types.NewErrStateUpdateBlobNotAvailableFraud(batch.StateIndex, string(batch.MetaData.DA.Client), batch.MetaData.DA.Height, hex.EncodeToString(batch.MetaData.DA.Commitment)) + } } + for _, batch := range daBatch.Batches { daBlocks = append(daBlocks, batch.Blocks...) } @@ -127,10 +141,9 @@ func (v *SettlementValidator) ValidateP2PBlocks(daBlocks []*types.Block, p2pBloc func (v *SettlementValidator) ValidateDaBlocks(slBatch *settlement.ResultRetrieveBatch, daBlocks []*types.Block) error { // we first verify the numblocks included in the state info match the block descriptors and the blocks obtained from DA numSlBDs := uint64(len(slBatch.BlockDescriptors)) - numDABlocks := uint64(len(daBlocks)) numSLBlocks := slBatch.NumBlocks - if numSLBlocks != numDABlocks || numSLBlocks != numSlBDs { - return types.NewErrStateUpdateNumBlocksNotMatchingFraud(slBatch.EndHeight, numSLBlocks, numDABlocks, numSLBlocks) + if numSLBlocks != numSlBDs { + return types.NewErrStateUpdateNumBlocksNotMatchingFraud(slBatch.EndHeight, numSLBlocks, numSLBlocks) } // we compare all DA blocks against the information included in the state info block descriptors @@ -148,6 +161,12 @@ func (v *SettlementValidator) ValidateDaBlocks(slBatch *settlement.ResultRetriev if !bd.Timestamp.Equal(daBlocks[i].Header.GetTimestamp()) { return types.NewErrStateUpdateTimestampNotMatchingFraud(slBatch.StateIndex, bd.Height, bd.Timestamp, daBlocks[i].Header.GetTimestamp()) } + + // we validate block descriptor drs version per height + err := v.validateDRS(slBatch.StateIndex, bd.Height, bd.DrsVersion) + if err != nil { + return err + } } v.logger.Debug("DA blocks validated successfully", "start height", daBlocks[0].Header.Height, "end height", daBlocks[len(daBlocks)-1].Header.Height) return nil @@ -173,11 +192,24 @@ func (v *SettlementValidator) GetLastValidatedHeight() uint64 { return v.lastValidatedHeight.Load() } -// GetLastValidatedHeight returns the next height that needs to be validated with settlement state updates. +// NextValidationHeight returns the next height that needs to be validated with settlement state updates. func (v *SettlementValidator) NextValidationHeight() uint64 { return v.lastValidatedHeight.Load() + 1 } +// validateDRS compares the DRS version stored for the specific height, obtained from rollapp params. +func (v *SettlementValidator) validateDRS(stateIndex uint64, height uint64, version string) error { + drs, err := v.blockManager.Store.LoadDRSVersion(height) + if err != nil { + return err + } + if drs != version { + return types.NewErrStateUpdateDRSVersionFraud(stateIndex, height, drs, version) + } + + return nil +} + // blockHash generates a hash from the block bytes to compare them func blockHash(block *types.Block) ([]byte, error) { blockBytes, err := block.MarshalBinary() diff --git a/block/slvalidator_test.go b/block/slvalidator_test.go new file mode 100644 index 000000000..d8f2826ab --- /dev/null +++ b/block/slvalidator_test.go @@ -0,0 +1,390 @@ +package block_test + +import ( + "crypto/rand" + "reflect" + "testing" + "time" + + "github.com/celestiaorg/celestia-openrpc/types/blob" + "github.com/dymensionxyz/dymint/block" + "github.com/dymensionxyz/dymint/da" + "github.com/dymensionxyz/dymint/p2p" + "github.com/dymensionxyz/dymint/settlement" + "github.com/dymensionxyz/dymint/testutil" + "github.com/dymensionxyz/dymint/types" + "github.com/dymensionxyz/dymint/types/pb/dymensionxyz/dymension/rollapp" + "github.com/dymensionxyz/dymint/version" + "github.com/libp2p/go-libp2p/core/crypto" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + abci "github.com/tendermint/tendermint/abci/types" + "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/pubsub" + "github.com/tendermint/tendermint/proxy" +) + +func TestStateUpdateValidator_ValidateStateUpdate(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) + proposerKey, _, err := crypto.GenerateEd25519Key(rand.Reader) + require.NoError(t, err) + + doubleSigned, err := testutil.GenerateBlocks(1, 10, proposerKey, [32]byte{}) + require.NoError(t, err) + + // Test cases + testCases := []struct { + name string + p2pBlocks bool + doubleSignedBlocks []*types.Block + stateUpdateFraud string + expectedErrType error + }{ + { + name: "Successful validation applied from DA", + p2pBlocks: false, + doubleSignedBlocks: nil, + stateUpdateFraud: "", + expectedErrType: nil, + }, + { + name: "Successful validation applied from P2P", + p2pBlocks: true, + doubleSignedBlocks: nil, + stateUpdateFraud: "", + expectedErrType: nil, + }, + { + name: "Failed validation blocks not matching", + p2pBlocks: true, + stateUpdateFraud: "", + doubleSignedBlocks: doubleSigned, + expectedErrType: &types.ErrStateUpdateDoubleSigningFraud{}, + }, + { + name: "Failed validation wrong state roots", + p2pBlocks: true, + stateUpdateFraud: "stateroot", + doubleSignedBlocks: nil, + expectedErrType: &types.ErrStateUpdateStateRootNotMatchingFraud{}, + }, + { + name: "Failed validation batch num blocks", + p2pBlocks: true, + stateUpdateFraud: "batchnumblocks", + doubleSignedBlocks: nil, + expectedErrType: &types.ErrStateUpdateNumBlocksNotMatchingFraud{}, + }, + { + name: "Failed validation batch num bds", + p2pBlocks: true, + stateUpdateFraud: "batchnumbds", + doubleSignedBlocks: nil, + expectedErrType: &types.ErrStateUpdateNumBlocksNotMatchingFraud{}, + }, + { + name: "Failed validation wrong timestamps", + p2pBlocks: true, + stateUpdateFraud: "timestamp", + doubleSignedBlocks: doubleSigned, + expectedErrType: &types.ErrStateUpdateTimestampNotMatchingFraud{}, + }, + { + name: "Failed validation wrong height", + p2pBlocks: true, + stateUpdateFraud: "height", + doubleSignedBlocks: doubleSigned, + expectedErrType: &types.ErrStateUpdateHeightNotMatchingFraud{}, + }, + { + name: "Failed validation drs version", + p2pBlocks: true, + stateUpdateFraud: "drs", + doubleSignedBlocks: nil, + expectedErrType: &types.ErrStateUpdateDRSVersionFraud{}, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + + // Create manager + manager, err := testutil.GetManagerWithProposerKey(testutil.GetManagerConfig(), proposerKey, nil, 1, 1, 0, proxyApp, nil) + require.NoError(t, err) + require.NotNil(t, manager) + + // Create DA + manager.DAClient = testutil.GetMockDALC(log.TestingLogger()) + manager.Retriever = manager.DAClient.(da.BatchRetriever) + + // Generate batch + batch, err := testutil.GenerateBatch(1, 10, proposerKey, [32]byte{}) + assert.NoError(t, err) + + // Submit batch to DA + daResultSubmitBatch := manager.DAClient.SubmitBatch(batch) + assert.Equal(t, daResultSubmitBatch.Code, da.StatusSuccess) + + // Create block descriptors + bds, err := getBlockDescriptors(batch) + require.NoError(t, err) + + // create the batch in settlement + slBatch := getSLBatch(bds, daResultSubmitBatch.SubmitMetaData, 1, 10) + + // Create the StateUpdateValidator + validator := block.NewSettlementValidator(testutil.NewLogger(t), manager) + + // in case double signing generate commits for these blocks + if tc.doubleSignedBlocks != nil { + batch.Blocks = tc.doubleSignedBlocks + batch.Commits, err = testutil.GenerateCommits(batch.Blocks, proposerKey) + require.NoError(t, err) + } + + // call manager flow for p2p received blocks + if tc.p2pBlocks { + for i, block := range batch.Blocks { + blockData := p2p.BlockData{Block: *block, Commit: *batch.Commits[i]} + msg := pubsub.NewMessage(blockData, map[string][]string{p2p.EventTypeKey: {p2p.EventNewGossipedBlock}}) + manager.OnReceivedBlock(msg) + } + // otherwise load them from DA + } else { + manager.ApplyBatchFromSL(slBatch.MetaData.DA) + } + + for _, bd := range bds { + manager.Store.SaveDRSVersion(bd.Height, bd.DrsVersion, nil) + } + + // set fraud data + switch tc.stateUpdateFraud { + case "drs": + // set different bd drs version + version, err := testutil.CreateRandomVersionCommit() + require.NoError(t, err) + slBatch.BlockDescriptors[0].DrsVersion = version + case "batchnumblocks": + // set wrong numblocks in state update + slBatch.NumBlocks = 11 + case "batchnumbds": + // add more block descriptors than blocks + bds = append(bds, rollapp.BlockDescriptor{}) + slBatch.BlockDescriptors = bds + case "stateroot": + // post empty state root + slBatch.BlockDescriptors[0].StateRoot = []byte{} + case "timestamp": + // add wrong timestamp + slBatch.BlockDescriptors[0].Timestamp = slBatch.BlockDescriptors[0].Timestamp.Add(time.Second) + case "height": + // add blockdescriptor with wrong height + slBatch.BlockDescriptors[0].Height = 2 + } + + // validate the state update + err = validator.ValidateStateUpdate(slBatch) + + // Check the result + if tc.expectedErrType == nil { + assert.NoError(t, err) + } else { + require.Equal(t, reflect.ValueOf(tc.expectedErrType).Type(), reflect.TypeOf(err)) + } + + }) + } + +} + +func TestStateUpdateValidator_ValidateDAFraud(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) + proposerKey, _, err := crypto.GenerateEd25519Key(rand.Reader) + require.NoError(t, err) + + // Generate batch + batch, err := testutil.GenerateBatch(1, 10, proposerKey, [32]byte{}) + require.NoError(t, err) + + // Batch data to be included in a blob + batchData, err := batch.MarshalBinary() + require.NoError(t, err) + + // Batch data to be included in a fraud blob + randomData := []byte{1, 2, 3, 4} + + // Test cases + testCases := []struct { + name string + checkAvailability bool + blobData []byte + expectedErrType error + }{ + { + name: "Successful DA Blob", + checkAvailability: false, + blobData: batchData, + expectedErrType: nil, + }, + { + name: "Blob not valid", + checkAvailability: false, + blobData: randomData, + expectedErrType: &types.ErrStateUpdateBlobCorruptedFraud{}, + }, + { + name: "Blob unavailable", + checkAvailability: true, + blobData: batchData, + expectedErrType: &types.ErrStateUpdateBlobNotAvailableFraud{}, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + + // Create manager + manager, err := testutil.GetManagerWithProposerKey(testutil.GetManagerConfig(), proposerKey, nil, 1, 1, 0, proxyApp, nil) + require.NoError(t, err) + require.NotNil(t, manager) + + // Create Mock DA + mockDA, err := testutil.NewMockDA(t) + require.NoError(t, err) + + // Start DA client + manager.DAClient = mockDA.DaClient + err = manager.DAClient.Start() + require.NoError(t, err) + manager.Retriever = manager.DAClient.(da.BatchRetriever) + + // Generate blob from batch data + require.NoError(t, err) + batchBlob, err := blob.NewBlobV0(mockDA.NID, tc.blobData) + require.NoError(t, err) + + // RPC calls necessary for blob submission + mockDA.MockRPC.On("Submit", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(uint64(1234), nil).Once().Run(func(args mock.Arguments) { time.Sleep(10 * time.Millisecond) }) + mockDA.MockRPC.On("GetProof", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&mockDA.BlobProof, nil).Once().Run(func(args mock.Arguments) { time.Sleep(10 * time.Millisecond) }) + mockDA.MockRPC.On("Included", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(true, nil).Once().Run(func(args mock.Arguments) { time.Sleep(10 * time.Millisecond) }) + mockDA.MockRPC.On("GetByHeight", mock.Anything, mock.Anything).Return(mockDA.Header, nil).Once().Run(func(args mock.Arguments) { time.Sleep(10 * time.Millisecond) }) + + // Submit batch to DA + daResultSubmitBatch := manager.DAClient.SubmitBatch(batch) + assert.Equal(t, daResultSubmitBatch.Code, da.StatusSuccess) + + // RPC calls for successful blob retrieval + if !tc.checkAvailability { + mockDA.MockRPC.On("Get", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(batchBlob, nil).Run(func(args mock.Arguments) {}) + } + + // RPC calls for unavailable blobs + if tc.checkAvailability { + mockDA.MockRPC.On("Get", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil, nil).Run(func(args mock.Arguments) {}) + mockDA.MockRPC.On("GetProof", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(&mockDA.BlobProof, nil).Once().Run(func(args mock.Arguments) { time.Sleep(10 * time.Millisecond) }) + mockDA.MockRPC.On("Included", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(false, nil).Once().Run(func(args mock.Arguments) { time.Sleep(10 * time.Millisecond) }) + mockDA.MockRPC.On("GetByHeight", mock.Anything, mock.Anything).Return(mockDA.Header, nil).Once().Run(func(args mock.Arguments) { time.Sleep(10 * time.Millisecond) }) + } + + // Create the StateUpdateValidator + validator := block.NewSettlementValidator(testutil.NewLogger(t), manager) + + bds, err := getBlockDescriptors(batch) + require.NoError(t, err) + // Generate batch with block descriptors + slBatch := getSLBatch(bds, daResultSubmitBatch.SubmitMetaData, 1, 10) + + for _, bd := range bds { + manager.Store.SaveDRSVersion(bd.Height, bd.DrsVersion, nil) + } + + // Validate state + err = validator.ValidateStateUpdate(slBatch) + + // Check the result + if tc.expectedErrType == nil { + assert.NoError(t, err) + } else { + require.Equal(t, reflect.ValueOf(tc.expectedErrType).Type(), reflect.TypeOf(err)) + } + }) + } + +} + +func getBlockDescriptors(batch *types.Batch) ([]rollapp.BlockDescriptor, error) { + // Create block descriptors + var bds []rollapp.BlockDescriptor + for _, block := range batch.Blocks { + version, err := testutil.CreateRandomVersionCommit() + if err != nil { + return nil, err + } + bd := rollapp.BlockDescriptor{ + Height: block.Header.Height, + StateRoot: block.Header.AppHash[:], + Timestamp: block.Header.GetTimestamp(), + DrsVersion: version, + } + bds = append(bds, bd) + } + return bds, nil +} + +func getSLBatch(bds []rollapp.BlockDescriptor, daMetaData *da.DASubmitMetaData, startHeight uint64, endHeight uint64) *settlement.ResultRetrieveBatch { + // create the batch in settlement + return &settlement.ResultRetrieveBatch{ + Batch: &settlement.Batch{ + BlockDescriptors: bds, + MetaData: &settlement.BatchMetaData{ + DA: daMetaData, + }, + StartHeight: startHeight, + EndHeight: endHeight, + NumBlocks: endHeight - startHeight + 1, + }, + ResultBase: settlement.ResultBase{ + StateIndex: 1, + }, + } +} diff --git a/block/state.go b/block/state.go index 53bd498df..780e64f76 100644 --- a/block/state.go +++ b/block/state.go @@ -32,6 +32,7 @@ func (m *Manager) LoadStateOnInit(store store.Store, genesis *tmtypes.GenesisDoc } m.State = s + return nil } @@ -59,10 +60,9 @@ func NewStateFromGenesis(genDoc *tmtypes.GenesisDoc) (*types.State, error) { Version: InitStateVersion, ChainID: genDoc.ChainID, - InitialHeight: uint64(genDoc.InitialHeight), - BaseHeight: uint64(genDoc.InitialHeight), - ConsensusParams: *genDoc.ConsensusParams, - LastHeightConsensusParamsChanged: genDoc.InitialHeight, + InitialHeight: uint64(genDoc.InitialHeight), + BaseHeight: uint64(genDoc.InitialHeight), + ConsensusParams: *genDoc.ConsensusParams, } s.SetHeight(0) copy(s.AppHash[:], genDoc.AppHash) diff --git a/block/stateupdate_validator_test.go b/block/stateupdate_validator_test.go deleted file mode 100644 index 2c6734a1a..000000000 --- a/block/stateupdate_validator_test.go +++ /dev/null @@ -1,214 +0,0 @@ -package block_test - -import ( - "crypto/rand" - "errors" - "testing" - "time" - - "github.com/dymensionxyz/dymint/block" - "github.com/dymensionxyz/dymint/da" - "github.com/dymensionxyz/dymint/p2p" - "github.com/dymensionxyz/dymint/settlement" - "github.com/dymensionxyz/dymint/testutil" - "github.com/dymensionxyz/dymint/types" - "github.com/dymensionxyz/dymint/types/pb/dymensionxyz/dymension/rollapp" - "github.com/dymensionxyz/dymint/version" - "github.com/libp2p/go-libp2p/core/crypto" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - abci "github.com/tendermint/tendermint/abci/types" - "github.com/tendermint/tendermint/libs/log" - "github.com/tendermint/tendermint/libs/pubsub" - "github.com/tendermint/tendermint/proxy" -) - -func TestSettlementValidator_ValidateStateUpdate(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) - proposerKey, _, err := crypto.GenerateEd25519Key(rand.Reader) - require.NoError(t, err) - - doubleSigned, err := testutil.GenerateBlocks(1, 10, proposerKey, [32]byte{}) - require.NoError(t, err) - - // Test cases - testCases := []struct { - name string - p2pBlocks bool - doubleSignedBlocks []*types.Block - stateUpdateFraud string - expectedErrType interface{} - }{ - { - name: "Successful validation applied from DA", - p2pBlocks: false, - doubleSignedBlocks: nil, - stateUpdateFraud: "", - expectedErrType: nil, - }, - { - name: "Successful validation applied from P2P", - p2pBlocks: true, - doubleSignedBlocks: nil, - stateUpdateFraud: "", - expectedErrType: nil, - }, - { - name: "Failed validation blocks not matching", - p2pBlocks: true, - stateUpdateFraud: "", - doubleSignedBlocks: doubleSigned, - expectedErrType: types.ErrStateUpdateDoubleSigningFraud{}, - }, - { - name: "Failed validation wrong state roots", - p2pBlocks: true, - stateUpdateFraud: "stateroot", - doubleSignedBlocks: nil, - expectedErrType: types.ErrStateUpdateStateRootNotMatchingFraud{}, - }, - { - name: "Failed validation batch num blocks", - p2pBlocks: true, - stateUpdateFraud: "batchnumblocks", - doubleSignedBlocks: nil, - expectedErrType: types.ErrStateUpdateNumBlocksNotMatchingFraud{}, - }, - { - name: "Failed validation batch num bds", - p2pBlocks: true, - stateUpdateFraud: "batchnumbds", - doubleSignedBlocks: nil, - expectedErrType: types.ErrStateUpdateNumBlocksNotMatchingFraud{}, - }, - { - name: "Failed validation wrong timestamps", - p2pBlocks: true, - stateUpdateFraud: "timestamp", - doubleSignedBlocks: doubleSigned, - expectedErrType: types.ErrStateUpdateTimestampNotMatchingFraud{}, - }, - { - name: "Failed validation wrong height", - p2pBlocks: true, - stateUpdateFraud: "height", - doubleSignedBlocks: doubleSigned, - expectedErrType: types.ErrStateUpdateHeightNotMatchingFraud{}, - }, - } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - - // Create manager - manager, err := testutil.GetManagerWithProposerKey(testutil.GetManagerConfig(), proposerKey, nil, 1, 1, 0, proxyApp, nil) - require.NoError(t, err) - require.NotNil(t, manager) - - // Create DA - manager.DAClient = testutil.GetMockDALC(log.TestingLogger()) - manager.Retriever = manager.DAClient.(da.BatchRetriever) - - // Generate batch - batch, err := testutil.GenerateBatch(1, 10, proposerKey, [32]byte{}) - assert.NoError(t, err) - - // Submit batch to DA - daResultSubmitBatch := manager.DAClient.SubmitBatch(batch) - assert.Equal(t, daResultSubmitBatch.Code, da.StatusSuccess) - - // Create block descriptors - var bds []rollapp.BlockDescriptor - for _, block := range batch.Blocks { - bd := rollapp.BlockDescriptor{ - Height: block.Header.Height, - StateRoot: block.Header.AppHash[:], - Timestamp: block.Header.GetTimestamp(), - } - bds = append(bds, bd) - } - - // create the batch in settlement - slBatch := &settlement.ResultRetrieveBatch{ - Batch: &settlement.Batch{ - BlockDescriptors: bds, - MetaData: &settlement.BatchMetaData{ - DA: daResultSubmitBatch.SubmitMetaData, - }, - StartHeight: 1, - EndHeight: 10, - NumBlocks: 10, - }, - ResultBase: settlement.ResultBase{ - StateIndex: 1, - }, - } - - // Create the SettlementValidator - validator := block.NewSettlementValidator(testutil.NewLogger(t), manager) - - // set fraud data - switch tc.stateUpdateFraud { - case "batchnumblocks": - slBatch.NumBlocks = 11 - case "batchnumbds": - bds = append(bds, rollapp.BlockDescriptor{}) - slBatch.BlockDescriptors = bds - case "stateroot": - slBatch.BlockDescriptors[0].StateRoot = []byte{} - case "timestamp": - slBatch.BlockDescriptors[0].Timestamp = slBatch.BlockDescriptors[0].Timestamp.Add(time.Second) - case "height": - slBatch.BlockDescriptors[0].Height = 2 - } - - if tc.doubleSignedBlocks != nil { - batch.Blocks = tc.doubleSignedBlocks - batch.Commits, err = testutil.GenerateCommits(batch.Blocks, proposerKey) - require.NoError(t, err) - } - - if tc.p2pBlocks { - for i, block := range batch.Blocks { - blockData := p2p.BlockData{Block: *block, Commit: *batch.Commits[i]} - msg := pubsub.NewMessage(blockData, map[string][]string{p2p.EventTypeKey: {p2p.EventNewGossipedBlock}}) - manager.OnReceivedBlock(msg) - } - } else { - manager.ApplyBatchFromSL(slBatch.MetaData.DA) - } - - // validate the state update - err = validator.ValidateStateUpdate(slBatch) - - // Check the result - if tc.expectedErrType == nil { - assert.NoError(t, err) - } else { - assert.True(t, errors.As(err, &tc.expectedErrType), - "expected error of type %T, got %T", tc.expectedErrType, err) - } - - }) - } - -} diff --git a/block/submit.go b/block/submit.go index 9e852e831..d2d6d6e03 100644 --- a/block/submit.go +++ b/block/submit.go @@ -201,8 +201,13 @@ func (m *Manager) CreateBatch(maxBatchSize uint64, startHeight uint64, endHeight return nil, fmt.Errorf("load commit: h: %d: %w", h, err) } + drsVersion, err := m.Store.LoadDRSVersion(block.Header.Height) + if err != nil { + return nil, fmt.Errorf("load drs version: h: %d: %w", h, err) + } batch.Blocks = append(batch.Blocks, block) batch.Commits = append(batch.Commits, commit) + batch.DRSVersion = append(batch.DRSVersion, drsVersion) totalSize := batch.SizeBytes() if int(maxBatchSize) < totalSize { @@ -210,6 +215,7 @@ func (m *Manager) CreateBatch(maxBatchSize uint64, startHeight uint64, endHeight // Remove the last block and commit from the batch batch.Blocks = batch.Blocks[:len(batch.Blocks)-1] batch.Commits = batch.Commits[:len(batch.Commits)-1] + batch.DRSVersion = batch.DRSVersion[:len(batch.DRSVersion)-1] if h == startHeight { return nil, fmt.Errorf("block size exceeds max batch size: h %d: size: %d: %w", h, totalSize, gerrc.ErrOutOfRange) @@ -237,6 +243,7 @@ func (m *Manager) SubmitBatch(batch *types.Batch) error { types.RollappHubHeightGauge.Set(float64(batch.EndHeight())) m.LastSettlementHeight.Store(batch.EndHeight()) + return nil } diff --git a/block/sync.go b/block/sync.go index 216c5cac9..10c9cea1d 100644 --- a/block/sync.go +++ b/block/sync.go @@ -23,8 +23,6 @@ func (m *Manager) onNewStateUpdate(event pubsub.Message) { // Update heights based on state update end height m.LastSettlementHeight.Store(eventData.EndHeight) - m.logger.Error("syncing") - // Update sequencers list from SL err := m.UpdateSequencerSetFromSL() if err != nil { @@ -36,7 +34,6 @@ func (m *Manager) onNewStateUpdate(event pubsub.Message) { m.triggerSettlementSyncing() // update target height used for syncing status rpc m.UpdateTargetHeight(eventData.EndHeight) - } else { // trigger validation of the last state update available in settlement m.triggerSettlementValidation() diff --git a/block/validate.go b/block/validate.go index 9eb5c4bda..a7f0e0df2 100644 --- a/block/validate.go +++ b/block/validate.go @@ -30,9 +30,9 @@ func (m *Manager) SettlementValidateLoop(ctx context.Context) error { return ctx.Err() case <-m.settlementValidationC: - m.logger.Info("validating state updates to target height", "targetHeight", m.State.Height()) + m.logger.Info("validating state updates to target height", "targetHeight", m.LastSettlementHeight.Load()) - for currH := m.SettlementValidator.NextValidationHeight(); currH <= m.State.Height(); currH = m.SettlementValidator.NextValidationHeight() { + for currH := m.SettlementValidator.NextValidationHeight(); currH <= m.LastSettlementHeight.Load(); currH = m.SettlementValidator.NextValidationHeight() { // get next batch that needs to be validated from SL batch, err := m.SLClient.GetBatchAtHeight(currH) diff --git a/da/celestia/celestia.go b/da/celestia/celestia.go index 0120678ec..0f29ad4cf 100644 --- a/da/celestia/celestia.go +++ b/da/celestia/celestia.go @@ -17,7 +17,6 @@ import ( "github.com/tendermint/tendermint/libs/pubsub" openrpc "github.com/celestiaorg/celestia-openrpc" - "github.com/celestiaorg/celestia-openrpc/types/share" "github.com/dymensionxyz/dymint/da" celtypes "github.com/dymensionxyz/dymint/da/celestia/types" @@ -270,18 +269,11 @@ func (c *DataAvailabilityLayerClient) RetrieveBatches(daMetaData *da.DASubmitMet var resultRetrieveBatch da.ResultRetrieveBatch err := retry.Do( func() error { - var result da.ResultRetrieveBatch - // Just for backward compatibility, in case no commitments are sent from the Hub, batch can be retrieved using previous implementation. - if daMetaData.Commitment == nil { - result = c.retrieveBatchesNoCommitment(daMetaData.Height) - } else { - result = c.retrieveBatches(daMetaData) - } - resultRetrieveBatch = result + resultRetrieveBatch = c.retrieveBatches(daMetaData) - if errors.Is(result.Error, da.ErrRetrieval) { - c.logger.Error("Retrieve batch.", "error", result.Error) - return result.Error + if errors.Is(resultRetrieveBatch.Error, da.ErrRetrieval) { + c.logger.Error("Retrieve batch.", "error", resultRetrieveBatch.Error) + return resultRetrieveBatch.Error } return nil @@ -328,7 +320,14 @@ func (c *DataAvailabilityLayerClient) retrieveBatches(daMetaData *da.DASubmitMet var batch pb.Batch err = proto.Unmarshal(blob.Data, &batch) if err != nil { - c.logger.Error("Unmarshal block.", "daHeight", daMetaData.Height, "error", err) + c.logger.Error("Unmarshal blob.", "daHeight", daMetaData.Height, "error", err) + return da.ResultRetrieveBatch{ + BaseResult: da.BaseResult{ + Code: da.StatusError, + Message: err.Error(), + Error: da.ErrBlobNotParsed, + }, + } } c.logger.Debug("Blob retrieved successfully from DA.", "DA height", daMetaData.Height, "lastBlockHeight", batch.EndHeight) @@ -340,7 +339,7 @@ func (c *DataAvailabilityLayerClient) retrieveBatches(daMetaData *da.DASubmitMet BaseResult: da.BaseResult{ Code: da.StatusError, Message: err.Error(), - Error: err, + Error: da.ErrBlobNotParsed, }, } } @@ -354,49 +353,6 @@ func (c *DataAvailabilityLayerClient) retrieveBatches(daMetaData *da.DASubmitMet } } -// RetrieveBatches gets a batch of blocks from DA layer. -func (c *DataAvailabilityLayerClient) retrieveBatchesNoCommitment(dataLayerHeight uint64) da.ResultRetrieveBatch { - ctx, cancel := context.WithTimeout(c.ctx, c.config.Timeout) - defer cancel() - blobs, err := c.rpc.GetAll(ctx, dataLayerHeight, []share.Namespace{c.config.NamespaceID.Bytes()}) - if err != nil { - return da.ResultRetrieveBatch{ - BaseResult: da.BaseResult{ - Code: da.StatusError, - Message: err.Error(), - }, - } - } - - var batches []*types.Batch - for i, blob := range blobs { - var batch pb.Batch - err = proto.Unmarshal(blob.Data, &batch) - if err != nil { - c.logger.Error("Unmarshal block.", "daHeight", dataLayerHeight, "position", i, "error", err) - continue - } - parsedBatch := new(types.Batch) - err := parsedBatch.FromProto(&batch) - if err != nil { - return da.ResultRetrieveBatch{ - BaseResult: da.BaseResult{ - Code: da.StatusError, - Message: err.Error(), - }, - } - } - batches = append(batches, parsedBatch) - } - - return da.ResultRetrieveBatch{ - BaseResult: da.BaseResult{ - Code: da.StatusSuccess, - }, - Batches: batches, - } -} - func (c *DataAvailabilityLayerClient) CheckBatchAvailability(daMetaData *da.DASubmitMetaData) da.ResultCheckBatch { var availabilityResult da.ResultCheckBatch for { diff --git a/da/celestia/celestia_test.go b/da/celestia/celestia_test.go index 862832e3b..34eab5c8d 100644 --- a/da/celestia/celestia_test.go +++ b/da/celestia/celestia_test.go @@ -133,32 +133,6 @@ func TestRetrievalNotFound(t *testing.T) { require.True(len(retreiveRes.Batches) == 0) } -func TestRetrievalNoCommitment(t *testing.T) { - assert := assert.New(t) - require := require.New(t) - - mockRPCClient, dalc, nID, _ := setDAandMock(t) - block1 := getRandomBlock(1, 10) - batch1 := &types.Batch{ - Blocks: []*types.Block{block1}, - } - // only blocks b1 and b2 will be submitted to DA - data1, _ := batch1.MarshalBinary() - blob1, _ := blob.NewBlobV0(nID, data1) - - mockRPCClient.On("GetAll", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*blob.Blob{blob1}, nil).Run(func(args mock.Arguments) { - }) - - retriever := dalc.(da.BatchRetriever) - - h1 := &da.DASubmitMetaData{ - Height: 1, - } - retreiveRes := retriever.RetrieveBatches(h1) - assert.Equal(da.StatusSuccess, retreiveRes.Code) - require.True(len(retreiveRes.Batches) == 1) -} - func TestAvalabilityOK(t *testing.T) { assert := assert.New(t) // require := require.New(t) diff --git a/da/errors.go b/da/errors.go index 8301d05c0..ba02343a8 100644 --- a/da/errors.go +++ b/da/errors.go @@ -21,6 +21,8 @@ var ( ErrBlobNotFound = errors.New("blob not found") // ErrBlobNotIncluded is returned when blob is not included. ErrBlobNotIncluded = errors.New("blob not included") + // ErrBlobNotParsed is returned when blob cannot be parsed + ErrBlobNotParsed = errors.New("unable to parse blob to batch") // ErrProofNotMatching is returned when proof does not match. ErrProofNotMatching = errors.New("proof not matching") // ErrNameSpace is returned when wrong namespace used diff --git a/mocks/github.com/dymensionxyz/dymint/settlement/mock_ClientI.go b/mocks/github.com/dymensionxyz/dymint/settlement/mock_ClientI.go index a55004c6e..5aa37eb41 100644 --- a/mocks/github.com/dymensionxyz/dymint/settlement/mock_ClientI.go +++ b/mocks/github.com/dymensionxyz/dymint/settlement/mock_ClientI.go @@ -370,25 +370,78 @@ func (_c *MockClientI_GetLatestBatch_Call) RunAndReturn(run func() (*settlement. return _c } -// GetLatestFinalizedBatch provides a mock function with given fields: -func (_m *MockClientI) GetLatestFinalizedBatch() (*settlement.ResultRetrieveBatch, error) { +// GetLatestFinalizedHeight provides a mock function with given fields: +func (_m *MockClientI) GetLatestFinalizedHeight() (uint64, error) { ret := _m.Called() if len(ret) == 0 { - panic("no return value specified for GetLatestFinalizedBatch") + panic("no return value specified for GetLatestFinalizedHeight") } - var r0 *settlement.ResultRetrieveBatch + var r0 uint64 var r1 error - if rf, ok := ret.Get(0).(func() (*settlement.ResultRetrieveBatch, error)); ok { + if rf, ok := ret.Get(0).(func() (uint64, error)); ok { return rf() } - if rf, ok := ret.Get(0).(func() *settlement.ResultRetrieveBatch); ok { + if rf, ok := ret.Get(0).(func() uint64); ok { r0 = rf() } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*settlement.ResultRetrieveBatch) - } + r0 = ret.Get(0).(uint64) + } + + if rf, ok := ret.Get(1).(func() error); ok { + r1 = rf() + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockClientI_GetLatestFinalizedHeight_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetLatestFinalizedHeight' +type MockClientI_GetLatestFinalizedHeight_Call struct { + *mock.Call +} + +// GetLatestFinalizedHeight is a helper method to define mock.On call +func (_e *MockClientI_Expecter) GetLatestFinalizedHeight() *MockClientI_GetLatestFinalizedHeight_Call { + return &MockClientI_GetLatestFinalizedHeight_Call{Call: _e.mock.On("GetLatestFinalizedHeight")} +} + +func (_c *MockClientI_GetLatestFinalizedHeight_Call) Run(run func()) *MockClientI_GetLatestFinalizedHeight_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockClientI_GetLatestFinalizedHeight_Call) Return(_a0 uint64, _a1 error) *MockClientI_GetLatestFinalizedHeight_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockClientI_GetLatestFinalizedHeight_Call) RunAndReturn(run func() (uint64, error)) *MockClientI_GetLatestFinalizedHeight_Call { + _c.Call.Return(run) + return _c +} + +// GetLatestHeight provides a mock function with given fields: +func (_m *MockClientI) GetLatestHeight() (uint64, error) { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetLatestHeight") + } + + var r0 uint64 + var r1 error + if rf, ok := ret.Get(0).(func() (uint64, error)); ok { + return rf() + } + if rf, ok := ret.Get(0).(func() uint64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(uint64) } if rf, ok := ret.Get(1).(func() error); ok { @@ -400,29 +453,29 @@ func (_m *MockClientI) GetLatestFinalizedBatch() (*settlement.ResultRetrieveBatc return r0, r1 } -// MockClientI_GetLatestFinalizedBatch_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetLatestFinalizedBatch' -type MockClientI_GetLatestFinalizedBatch_Call struct { +// MockClientI_GetLatestHeight_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetLatestHeight' +type MockClientI_GetLatestHeight_Call struct { *mock.Call } -// GetLatestFinalizedBatch is a helper method to define mock.On call -func (_e *MockClientI_Expecter) GetLatestFinalizedBatch() *MockClientI_GetLatestFinalizedBatch_Call { - return &MockClientI_GetLatestFinalizedBatch_Call{Call: _e.mock.On("GetLatestFinalizedBatch")} +// GetLatestHeight is a helper method to define mock.On call +func (_e *MockClientI_Expecter) GetLatestHeight() *MockClientI_GetLatestHeight_Call { + return &MockClientI_GetLatestHeight_Call{Call: _e.mock.On("GetLatestHeight")} } -func (_c *MockClientI_GetLatestFinalizedBatch_Call) Run(run func()) *MockClientI_GetLatestFinalizedBatch_Call { +func (_c *MockClientI_GetLatestHeight_Call) Run(run func()) *MockClientI_GetLatestHeight_Call { _c.Call.Run(func(args mock.Arguments) { run() }) return _c } -func (_c *MockClientI_GetLatestFinalizedBatch_Call) Return(_a0 *settlement.ResultRetrieveBatch, _a1 error) *MockClientI_GetLatestFinalizedBatch_Call { +func (_c *MockClientI_GetLatestHeight_Call) Return(_a0 uint64, _a1 error) *MockClientI_GetLatestHeight_Call { _c.Call.Return(_a0, _a1) return _c } -func (_c *MockClientI_GetLatestFinalizedBatch_Call) RunAndReturn(run func() (*settlement.ResultRetrieveBatch, error)) *MockClientI_GetLatestFinalizedBatch_Call { +func (_c *MockClientI_GetLatestHeight_Call) RunAndReturn(run func() (uint64, error)) *MockClientI_GetLatestHeight_Call { _c.Call.Return(run) return _c } diff --git a/mocks/github.com/dymensionxyz/dymint/store/mock_Store.go b/mocks/github.com/dymensionxyz/dymint/store/mock_Store.go index 7d4ec56d6..a555c6612 100644 --- a/mocks/github.com/dymensionxyz/dymint/store/mock_Store.go +++ b/mocks/github.com/dymensionxyz/dymint/store/mock_Store.go @@ -302,22 +302,22 @@ func (_c *MockStore_LoadBlockResponses_Call) RunAndReturn(run func(uint64) (*sta } // LoadBlockSource provides a mock function with given fields: height -func (_m *MockStore) LoadBlockSource(height uint64) (string, error) { +func (_m *MockStore) LoadBlockSource(height uint64) (types.BlockSource, error) { ret := _m.Called(height) if len(ret) == 0 { panic("no return value specified for LoadBlockSource") } - var r0 string + var r0 types.BlockSource var r1 error - if rf, ok := ret.Get(0).(func(uint64) (string, error)); ok { + if rf, ok := ret.Get(0).(func(uint64) (types.BlockSource, error)); ok { return rf(height) } - if rf, ok := ret.Get(0).(func(uint64) string); ok { + if rf, ok := ret.Get(0).(func(uint64) types.BlockSource); ok { r0 = rf(height) } else { - r0 = ret.Get(0).(string) + r0 = ret.Get(0).(types.BlockSource) } if rf, ok := ret.Get(1).(func(uint64) error); ok { @@ -347,12 +347,12 @@ func (_c *MockStore_LoadBlockSource_Call) Run(run func(height uint64)) *MockStor return _c } -func (_c *MockStore_LoadBlockSource_Call) Return(_a0 string, _a1 error) *MockStore_LoadBlockSource_Call { +func (_c *MockStore_LoadBlockSource_Call) Return(_a0 types.BlockSource, _a1 error) *MockStore_LoadBlockSource_Call { _c.Call.Return(_a0, _a1) return _c } -func (_c *MockStore_LoadBlockSource_Call) RunAndReturn(run func(uint64) (string, error)) *MockStore_LoadBlockSource_Call { +func (_c *MockStore_LoadBlockSource_Call) RunAndReturn(run func(uint64) (types.BlockSource, error)) *MockStore_LoadBlockSource_Call { _c.Call.Return(run) return _c } @@ -588,6 +588,61 @@ func (_c *MockStore_LoadState_Call) RunAndReturn(run func() (*types.State, error return _c } +// LoadValidationHeight provides a mock function with given fields: +func (_m *MockStore) LoadValidationHeight() (uint64, error) { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for LoadValidationHeight") + } + + var r0 uint64 + var r1 error + if rf, ok := ret.Get(0).(func() (uint64, error)); ok { + return rf() + } + if rf, ok := ret.Get(0).(func() uint64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(uint64) + } + + if rf, ok := ret.Get(1).(func() error); ok { + r1 = rf() + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockStore_LoadValidationHeight_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'LoadValidationHeight' +type MockStore_LoadValidationHeight_Call struct { + *mock.Call +} + +// LoadValidationHeight is a helper method to define mock.On call +func (_e *MockStore_Expecter) LoadValidationHeight() *MockStore_LoadValidationHeight_Call { + return &MockStore_LoadValidationHeight_Call{Call: _e.mock.On("LoadValidationHeight")} +} + +func (_c *MockStore_LoadValidationHeight_Call) Run(run func()) *MockStore_LoadValidationHeight_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockStore_LoadValidationHeight_Call) Return(_a0 uint64, _a1 error) *MockStore_LoadValidationHeight_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockStore_LoadValidationHeight_Call) RunAndReturn(run func() (uint64, error)) *MockStore_LoadValidationHeight_Call { + _c.Call.Return(run) + return _c +} + // NewBatch provides a mock function with given fields: func (_m *MockStore) NewBatch() store.KVBatch { ret := _m.Called() @@ -874,7 +929,7 @@ func (_c *MockStore_SaveBlockResponses_Call) RunAndReturn(run func(uint64, *stat } // SaveBlockSource provides a mock function with given fields: height, source, batch -func (_m *MockStore) SaveBlockSource(height uint64, source string, batch store.KVBatch) (store.KVBatch, error) { +func (_m *MockStore) SaveBlockSource(height uint64, source types.BlockSource, batch store.KVBatch) (store.KVBatch, error) { ret := _m.Called(height, source, batch) if len(ret) == 0 { @@ -883,10 +938,10 @@ func (_m *MockStore) SaveBlockSource(height uint64, source string, batch store.K var r0 store.KVBatch var r1 error - if rf, ok := ret.Get(0).(func(uint64, string, store.KVBatch) (store.KVBatch, error)); ok { + if rf, ok := ret.Get(0).(func(uint64, types.BlockSource, store.KVBatch) (store.KVBatch, error)); ok { return rf(height, source, batch) } - if rf, ok := ret.Get(0).(func(uint64, string, store.KVBatch) store.KVBatch); ok { + if rf, ok := ret.Get(0).(func(uint64, types.BlockSource, store.KVBatch) store.KVBatch); ok { r0 = rf(height, source, batch) } else { if ret.Get(0) != nil { @@ -894,7 +949,7 @@ func (_m *MockStore) SaveBlockSource(height uint64, source string, batch store.K } } - if rf, ok := ret.Get(1).(func(uint64, string, store.KVBatch) error); ok { + if rf, ok := ret.Get(1).(func(uint64, types.BlockSource, store.KVBatch) error); ok { r1 = rf(height, source, batch) } else { r1 = ret.Error(1) @@ -910,15 +965,15 @@ type MockStore_SaveBlockSource_Call struct { // SaveBlockSource is a helper method to define mock.On call // - height uint64 -// - source string +// - source types.BlockSource // - batch store.KVBatch func (_e *MockStore_Expecter) SaveBlockSource(height interface{}, source interface{}, batch interface{}) *MockStore_SaveBlockSource_Call { return &MockStore_SaveBlockSource_Call{Call: _e.mock.On("SaveBlockSource", height, source, batch)} } -func (_c *MockStore_SaveBlockSource_Call) Run(run func(height uint64, source string, batch store.KVBatch)) *MockStore_SaveBlockSource_Call { +func (_c *MockStore_SaveBlockSource_Call) Run(run func(height uint64, source types.BlockSource, batch store.KVBatch)) *MockStore_SaveBlockSource_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(uint64), args[1].(string), args[2].(store.KVBatch)) + run(args[0].(uint64), args[1].(types.BlockSource), args[2].(store.KVBatch)) }) return _c } @@ -928,7 +983,7 @@ func (_c *MockStore_SaveBlockSource_Call) Return(_a0 store.KVBatch, _a1 error) * return _c } -func (_c *MockStore_SaveBlockSource_Call) RunAndReturn(run func(uint64, string, store.KVBatch) (store.KVBatch, error)) *MockStore_SaveBlockSource_Call { +func (_c *MockStore_SaveBlockSource_Call) RunAndReturn(run func(uint64, types.BlockSource, store.KVBatch) (store.KVBatch, error)) *MockStore_SaveBlockSource_Call { _c.Call.Return(run) return _c } @@ -1052,6 +1107,65 @@ func (_c *MockStore_SaveState_Call) RunAndReturn(run func(*types.State, store.KV return _c } +// SaveValidationHeight provides a mock function with given fields: height, batch +func (_m *MockStore) SaveValidationHeight(height uint64, batch store.KVBatch) (store.KVBatch, error) { + ret := _m.Called(height, batch) + + if len(ret) == 0 { + panic("no return value specified for SaveValidationHeight") + } + + var r0 store.KVBatch + var r1 error + if rf, ok := ret.Get(0).(func(uint64, store.KVBatch) (store.KVBatch, error)); ok { + return rf(height, batch) + } + if rf, ok := ret.Get(0).(func(uint64, store.KVBatch) store.KVBatch); ok { + r0 = rf(height, batch) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(store.KVBatch) + } + } + + if rf, ok := ret.Get(1).(func(uint64, store.KVBatch) error); ok { + r1 = rf(height, batch) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockStore_SaveValidationHeight_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SaveValidationHeight' +type MockStore_SaveValidationHeight_Call struct { + *mock.Call +} + +// SaveValidationHeight is a helper method to define mock.On call +// - height uint64 +// - batch store.KVBatch +func (_e *MockStore_Expecter) SaveValidationHeight(height interface{}, batch interface{}) *MockStore_SaveValidationHeight_Call { + return &MockStore_SaveValidationHeight_Call{Call: _e.mock.On("SaveValidationHeight", height, batch)} +} + +func (_c *MockStore_SaveValidationHeight_Call) Run(run func(height uint64, batch store.KVBatch)) *MockStore_SaveValidationHeight_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(uint64), args[1].(store.KVBatch)) + }) + return _c +} + +func (_c *MockStore_SaveValidationHeight_Call) Return(_a0 store.KVBatch, _a1 error) *MockStore_SaveValidationHeight_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockStore_SaveValidationHeight_Call) RunAndReturn(run func(uint64, store.KVBatch) (store.KVBatch, error)) *MockStore_SaveValidationHeight_Call { + _c.Call.Return(run) + return _c +} + // NewMockStore creates a new instance of MockStore. 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 NewMockStore(t interface { diff --git a/proto/types/dymensionxyz/dymension/rollapp/tx.proto b/proto/types/dymensionxyz/dymension/rollapp/tx.proto index 34d6adde5..e4d44f9f7 100644 --- a/proto/types/dymensionxyz/dymension/rollapp/tx.proto +++ b/proto/types/dymensionxyz/dymension/rollapp/tx.proto @@ -80,8 +80,6 @@ message MsgUpdateState { BlockDescriptors BDs = 7 [(gogoproto.nullable) = false]; // last is true if this is the last batch of the sequencer bool last = 8; - // DrsVersion is a DRS version used by the rollapp. - string drs_version = 9; } message MsgUpdateStateResponse { diff --git a/proto/types/dymint/dymint.proto b/proto/types/dymint/dymint.proto index dd6e051f7..baa4436a7 100755 --- a/proto/types/dymint/dymint.proto +++ b/proto/types/dymint/dymint.proto @@ -104,4 +104,5 @@ message SequencerSet { message EventKeys { repeated bytes keys = 1; -} \ No newline at end of file +} + diff --git a/proto/types/dymint/state.proto b/proto/types/dymint/state.proto index 95bb4cb51..e773b34f0 100755 --- a/proto/types/dymint/state.proto +++ b/proto/types/dymint/state.proto @@ -33,7 +33,7 @@ message State { tendermint.types.ConsensusParams consensus_params = 12 [(gogoproto.nullable) = false]; - int64 last_height_consensus_params_changed = 13; + reserved 13; bytes last_results_hash = 14; @@ -55,5 +55,4 @@ message RollappParams { string da = 1 ; //commit used for the rollapp executable string version = 2 ; - -} \ No newline at end of file +} diff --git a/rpc/client/client_test.go b/rpc/client/client_test.go index d52fe3144..d103bbb34 100644 --- a/rpc/client/client_test.go +++ b/rpc/client/client_test.go @@ -823,8 +823,19 @@ func TestValidatorSetHandling(t *testing.T) { // dummy pubkey, we don't care about the actual key pbValKey, err := encoding.PubKeyToProto(vKeys[0].PubKey()) require.NoError(err) - app.On("EndBlock", mock.Anything).Return(abci.ResponseEndBlock{ValidatorUpdates: []abci.ValidatorUpdate{{PubKey: pbValKey, Power: 100}}}) - + app.On("EndBlock", mock.Anything).Return(abci.ResponseEndBlock{ + RollappParamUpdates: &abci.RollappParams{ + Da: "mock", + Version: version.Commit, + }, + ConsensusParamUpdates: &abci.ConsensusParams{ + Block: &abci.BlockParams{ + MaxGas: 100, + MaxBytes: 100, + }, + }, + ValidatorUpdates: []abci.ValidatorUpdate{{PubKey: pbValKey, Power: 100}}, + }) waitCh := make(chan interface{}) app.On("Commit", mock.Anything).Return(abci.ResponseCommit{}).Times(5) @@ -862,6 +873,7 @@ func TestValidatorSetHandling(t *testing.T) { log.TestingLogger(), mempool.NopMetrics(), ) + require.NoError(err) require.NotNil(node) diff --git a/settlement/dymension/dymension.go b/settlement/dymension/dymension.go index ea2bbba13..7179d0884 100644 --- a/settlement/dymension/dymension.go +++ b/settlement/dymension/dymension.go @@ -28,7 +28,6 @@ import ( rollapptypes "github.com/dymensionxyz/dymint/types/pb/dymensionxyz/dymension/rollapp" sequencertypes "github.com/dymensionxyz/dymint/types/pb/dymensionxyz/dymension/sequencer" protoutils "github.com/dymensionxyz/dymint/utils/proto" - "github.com/dymensionxyz/dymint/version" ) const ( @@ -218,7 +217,7 @@ func (c *Client) SubmitBatch(batch *types.Batch, daClient da.Client, daResult *d } } -func (c *Client) getStateInfo(index, height *uint64, finalized bool) (res *rollapptypes.QueryGetStateInfoResponse, err error) { +func (c *Client) getStateInfo(index, height *uint64) (res *rollapptypes.QueryGetStateInfoResponse, err error) { req := &rollapptypes.QueryGetStateInfoRequest{ RollappId: c.rollappId, } @@ -228,9 +227,6 @@ func (c *Client) getStateInfo(index, height *uint64, finalized bool) (res *rolla if height != nil { req.Height = *height } - if finalized { - req.Finalized = finalized - } err = c.RunWithRetry(func() error { res, err = c.rollappQueryClient.StateInfo(c.ctx, req) @@ -248,18 +244,31 @@ func (c *Client) getStateInfo(index, height *uint64, finalized bool) (res *rolla return } -// GetLatestBatch returns the latest batch from the Dymension Hub. -func (c *Client) GetLatestBatch() (*settlement.ResultRetrieveBatch, error) { - res, err := c.getStateInfo(nil, nil, false) +func (c *Client) getLatestHeight(finalized bool) (res *rollapptypes.QueryGetLatestHeightResponse, err error) { + req := &rollapptypes.QueryGetLatestHeightRequest{ + RollappId: c.rollappId, + Finalized: finalized, + } + err = c.RunWithRetry(func() error { + res, err = c.rollappQueryClient.LatestHeight(c.ctx, req) + + if status.Code(err) == codes.NotFound { + return retry.Unrecoverable(errors.Join(gerrc.ErrNotFound, err)) + } + return err + }) if err != nil { - return nil, fmt.Errorf("get state info: %w", err) + return nil, fmt.Errorf("query state info: %w", err) } - return convertStateInfoToResultRetrieveBatch(&res.StateInfo) + if res == nil { // not supposed to happen + return nil, fmt.Errorf("empty response with nil err: %w", gerrc.ErrUnknown) + } + return } -// GetLatestFinalizedBatch returns the latest finalized batch from the Dymension Hub. -func (c *Client) GetLatestFinalizedBatch() (*settlement.ResultRetrieveBatch, error) { - res, err := c.getStateInfo(nil, nil, true) +// GetLatestBatch returns the latest batch from the Dymension Hub. +func (c *Client) GetLatestBatch() (*settlement.ResultRetrieveBatch, error) { + res, err := c.getStateInfo(nil, nil) if err != nil { return nil, fmt.Errorf("get state info: %w", err) } @@ -268,7 +277,7 @@ func (c *Client) GetLatestFinalizedBatch() (*settlement.ResultRetrieveBatch, err // GetBatchAtIndex returns the batch at the given index from the Dymension Hub. func (c *Client) GetBatchAtIndex(index uint64) (*settlement.ResultRetrieveBatch, error) { - res, err := c.getStateInfo(&index, nil, false) + res, err := c.getStateInfo(&index, nil) if err != nil { return nil, fmt.Errorf("get state info: %w", err) } @@ -277,13 +286,31 @@ func (c *Client) GetBatchAtIndex(index uint64) (*settlement.ResultRetrieveBatch, // GetBatchAtHeight returns the batch at the given height from the Dymension Hub. func (c *Client) GetBatchAtHeight(height uint64) (*settlement.ResultRetrieveBatch, error) { - res, err := c.getStateInfo(nil, &height, false) + res, err := c.getStateInfo(nil, &height) if err != nil { return nil, fmt.Errorf("get state info: %w", err) } return convertStateInfoToResultRetrieveBatch(&res.StateInfo) } +// GetLatestHeight returns the latest state update height from the settlement layer. +func (c *Client) GetLatestHeight() (uint64, error) { + res, err := c.getLatestHeight(false) + if err != nil { + return uint64(0), fmt.Errorf("get latest height: %w", err) + } + return res.Height, nil +} + +// GetLatestFinalizedHeight returns the latest finalized height from the settlement layer. +func (c *Client) GetLatestFinalizedHeight() (uint64, error) { + res, err := c.getLatestHeight(true) + if err != nil { + return uint64(0), fmt.Errorf("get latest height: %w", err) + } + return res.Height, nil +} + // GetProposer implements settlement.ClientI. func (c *Client) GetProposer() *types.Sequencer { // return cached proposer @@ -541,9 +568,10 @@ func (c *Client) convertBatchToMsgUpdateState(batch *types.Batch, daResult *da.R blockDescriptors := make([]rollapptypes.BlockDescriptor, len(batch.Blocks)) for index, block := range batch.Blocks { blockDescriptor := rollapptypes.BlockDescriptor{ - Height: block.Header.Height, - StateRoot: block.Header.AppHash[:], - Timestamp: block.Header.GetTimestamp(), + Height: block.Header.Height, + StateRoot: block.Header.AppHash[:], + Timestamp: block.Header.GetTimestamp(), + DrsVersion: batch.DRSVersion[index], } blockDescriptors[index] = blockDescriptor } @@ -556,7 +584,6 @@ func (c *Client) convertBatchToMsgUpdateState(batch *types.Batch, daResult *da.R DAPath: daResult.SubmitMetaData.ToPath(), BDs: rollapptypes.BlockDescriptors{BD: blockDescriptors}, Last: batch.LastBatch, - DrsVersion: version.Commit, } return settlementBatch, nil } diff --git a/settlement/grpc/grpc.go b/settlement/grpc/grpc.go index 29cee91ad..2b7e82fa1 100644 --- a/settlement/grpc/grpc.go +++ b/settlement/grpc/grpc.go @@ -199,11 +199,6 @@ func (c *Client) GetLatestBatch() (*settlement.ResultRetrieveBatch, error) { return batchResult, nil } -// GetLatestFinalizedBatch returns the latest finalized batch from the kv store. batches are never finalized for grpc settlement -func (c *Client) GetLatestFinalizedBatch() (*settlement.ResultRetrieveBatch, error) { - return nil, gerrc.ErrNotFound -} - // GetBatchAtIndex returns the batch at the given index func (c *Client) GetBatchAtIndex(index uint64) (*settlement.ResultRetrieveBatch, error) { batchResult, err := c.retrieveBatchAtStateIndex(index) @@ -277,6 +272,16 @@ func (c *Client) CheckRotationInProgress() (*types.Sequencer, error) { return nil, nil } +// GetLatestHeight returns the latest state update height from the settlement layer. +func (c *Client) GetLatestHeight() (uint64, error) { + return c.latestHeight.Load(), nil +} + +// GetLatestFinalizedHeight returns the latest finalized height from the settlement layer. +func (c *Client) GetLatestFinalizedHeight() (uint64, error) { + return uint64(0), gerrc.ErrNotFound +} + func (c *Client) saveBatch(batch *settlement.Batch) error { c.logger.Debug("Saving batch to grpc settlement layer", "start height", batch.StartHeight, "end height", batch.EndHeight) diff --git a/settlement/local/local.go b/settlement/local/local.go index 81001eba7..b10850c53 100644 --- a/settlement/local/local.go +++ b/settlement/local/local.go @@ -160,9 +160,14 @@ func (c *Client) GetLatestBatch() (*settlement.ResultRetrieveBatch, error) { return batchResult, nil } -// GetLatestFinalizedBatch returns the latest finalized batch from the kv store. batches are never finalized for local settlement -func (c *Client) GetLatestFinalizedBatch() (*settlement.ResultRetrieveBatch, error) { - return nil, gerrc.ErrNotFound // TODO: need to return a cosmos specific error? +// GetLatestHeight returns the latest state update height from the settlement layer. +func (c *Client) GetLatestHeight() (uint64, error) { + return c.latestHeight, nil +} + +// GetLatestFinalizedHeight returns the latest finalized height from the settlement layer. +func (c *Client) GetLatestFinalizedHeight() (uint64, error) { + return uint64(0), gerrc.ErrNotFound } // GetBatchAtIndex returns the batch at the given index diff --git a/settlement/settlement.go b/settlement/settlement.go index f29eb2c0f..3a851a416 100644 --- a/settlement/settlement.go +++ b/settlement/settlement.go @@ -79,8 +79,10 @@ type ClientI interface { GetSequencerByAddress(address string) (types.Sequencer, error) // GetBatchAtHeight returns the batch at the given height. GetBatchAtHeight(index uint64) (*ResultRetrieveBatch, error) - // GetLatestFinalizedBatch returns the latest finalized batch from the settlement layer. - GetLatestFinalizedBatch() (*ResultRetrieveBatch, error) + // GetLatestHeight returns the latest state update height from the settlement layer. + GetLatestHeight() (uint64, error) + // GetLatestFinalizedHeight returns the latest finalized height from the settlement layer. + GetLatestFinalizedHeight() (uint64, error) // GetAllSequencers returns all sequencers for this rollapp (bonded and not bonded). GetAllSequencers() ([]types.Sequencer, error) // GetBondedSequencers returns the list of the bonded sequencers for this rollapp. diff --git a/store/pruning.go b/store/pruning.go index bb242cb5c..b1f97facb 100644 --- a/store/pruning.go +++ b/store/pruning.go @@ -37,6 +37,11 @@ func (s *DefaultStore) PruneStore(from, to uint64, logger types.Logger) (uint64, logger.Error("pruning block sync identifiers", "from", from, "to", to, "cids pruned", prunedCids, "err", err) } + prunedDRS, err := s.pruneDRSVersion(from, to, logger) + if err != nil { + logger.Error("pruning drs version", "from", from, "to", to, "drs pruned", prunedDRS, "err", err) + } + return prunedBlocks, nil } @@ -91,6 +96,15 @@ func (s *DefaultStore) pruneCids(from, to uint64, logger types.Logger) (uint64, return prunedCids, err } +// pruneDRSVersion prunes drs version info from store +func (s *DefaultStore) pruneDRSVersion(from, to uint64, logger types.Logger) (uint64, error) { + pruneDRS := func(batch KVBatch, height uint64) error { + return batch.Delete(getDRSVersionKey(height)) + } + prunedSequencers, err := s.pruneHeights(from, to, pruneDRS, logger) + return prunedSequencers, err +} + // pruneHeights is the common function for all pruning that iterates through all heights and prunes according to the pruning function set func (s *DefaultStore) pruneHeights(from, to uint64, prune func(batch KVBatch, height uint64) error, logger types.Logger) (uint64, error) { pruned := uint64(0) diff --git a/store/pruning_test.go b/store/pruning_test.go index f24423f77..f9c79f41a 100644 --- a/store/pruning_test.go +++ b/store/pruning_test.go @@ -67,6 +67,7 @@ func TestStorePruning(t *testing.T) { savedRespHeights := make(map[uint64]bool) savedSeqHeights := make(map[uint64]bool) savedCidHeights := make(map[uint64]bool) + savedDRSHeights := make(map[uint64]bool) for _, block := range c.blocks { @@ -107,6 +108,13 @@ func TestStorePruning(t *testing.T) { savedCidHeights[block.Header.Height] = true } + // generate and store drs version randomly for block heights + if randBool() { + _, err = bstore.SaveDRSVersion(block.Header.Height, "", nil) + savedDRSHeights[block.Header.Height] = true + assert.NoError(err) + } + } // Validate everything is saved @@ -130,6 +138,11 @@ func TestStorePruning(t *testing.T) { assert.NoError(err) } + for k := range savedDRSHeights { + _, err := bstore.LoadDRSVersion(k) + assert.NoError(err) + } + _, err := bstore.PruneStore(c.from, c.to, log.NewNopLogger()) if c.shouldError { assert.Error(err) @@ -189,6 +202,17 @@ func TestStorePruning(t *testing.T) { assert.NoError(err) } } + + // Validate only block drs in the range are pruned + for k := range savedDRSHeights { + if k >= c.from && k < c.to { // k < c.to is the exclusion test + _, err = bstore.LoadDRSVersion(k) + assert.Error(err, "DRS version at height %d should be pruned", k) + } else { + _, err = bstore.LoadDRSVersion(k) + assert.NoError(err) + } + } }) } } diff --git a/store/store.go b/store/store.go index 2133435f6..3638d1ef5 100644 --- a/store/store.go +++ b/store/store.go @@ -25,6 +25,7 @@ var ( cidPrefix = [1]byte{7} sourcePrefix = [1]byte{8} validatedHeightPrefix = [1]byte{9} + drsVersionPrefix = [1]byte{10} ) // DefaultStore is a default store implementation. @@ -337,6 +338,23 @@ func (s *DefaultStore) LoadValidationHeight() (uint64, error) { return binary.LittleEndian.Uint64(b), nil } +func (s *DefaultStore) LoadDRSVersion(height uint64) (string, error) { + versionBytes, err := s.db.Get(getDRSVersionKey(height)) + if err != nil { + return "", fmt.Errorf("load drs version for height %v: %w", height, err) + } + return string(versionBytes), nil +} + +func (s *DefaultStore) SaveDRSVersion(height uint64, version string, batch KVBatch) (KVBatch, error) { + versionBytes := []byte(version) + if batch == nil { + return nil, s.db.Set(getDRSVersionKey(height), versionBytes) + } + err := batch.Set(getDRSVersionKey(height), versionBytes) + return batch, err +} + func getBlockKey(hash [32]byte) []byte { return append(blockPrefix[:], hash[:]...) } @@ -382,3 +400,9 @@ func getSourceKey(height uint64) []byte { func getValidatedHeightKey() []byte { return validatedHeightPrefix[:] } + +func getDRSVersionKey(height uint64) []byte { + buf := make([]byte, 8) + binary.BigEndian.PutUint64(buf, height) + return append(drsVersionPrefix[:], buf[:]...) +} diff --git a/store/storeIface.go b/store/storeIface.go index 015802f6b..494b04ef8 100644 --- a/store/storeIface.go +++ b/store/storeIface.go @@ -73,8 +73,6 @@ type Store interface { PruneStore(from, to uint64, logger types.Logger) (uint64, error) - Close() error - SaveBlockCid(height uint64, cid cid.Cid, batch KVBatch) (KVBatch, error) LoadBlockCid(height uint64) (cid.Cid, error) @@ -86,4 +84,10 @@ type Store interface { SaveValidationHeight(height uint64, batch KVBatch) (KVBatch, error) LoadValidationHeight() (uint64, error) + + LoadDRSVersion(height uint64) (string, error) + + SaveDRSVersion(height uint64, version string, batch KVBatch) (KVBatch, error) + + Close() error } diff --git a/testutil/mocks.go b/testutil/mocks.go index fdfb9fc9f..01bed53bb 100644 --- a/testutil/mocks.go +++ b/testutil/mocks.go @@ -1,8 +1,17 @@ package testutil import ( + "bytes" + "crypto/sha256" + "encoding/json" "errors" + "fmt" + "testing" + "time" + "github.com/celestiaorg/celestia-openrpc/types/blob" + "github.com/celestiaorg/celestia-openrpc/types/header" + "github.com/celestiaorg/nmt" tmmocks "github.com/dymensionxyz/dymint/mocks/github.com/tendermint/tendermint/abci/types" "github.com/dymensionxyz/dymint/types" "github.com/stretchr/testify/mock" @@ -11,8 +20,12 @@ import ( "github.com/tendermint/tendermint/proxy" "github.com/dymensionxyz/dymint/da" + "github.com/dymensionxyz/dymint/da/celestia" localda "github.com/dymensionxyz/dymint/da/local" + "github.com/dymensionxyz/dymint/da/registry" "github.com/dymensionxyz/dymint/store" + + damocks "github.com/dymensionxyz/dymint/mocks/github.com/dymensionxyz/dymint/da/celestia/types" ) // ABCIMethod is a string representing an ABCI method @@ -169,3 +182,75 @@ func (s *SubscribeMock) Chan() <-chan interface{} { func (s *SubscribeMock) Unsubscribe() { close(s.messageCh) } + +type MockDA struct { + DaClient da.DataAvailabilityLayerClient + MockRPC *damocks.MockCelestiaRPCClient + NID []byte + Header *header.ExtendedHeader + BlobProof blob.Proof +} + +func NewMockDA(t *testing.T) (*MockDA, error) { + mockDA := &MockDA{} + // Create DA + // init celestia DA with mock RPC client + mockDA.DaClient = registry.GetClient("celestia") + + config := celestia.Config{ + BaseURL: "http://localhost:26658", + Timeout: 30 * time.Second, + GasPrices: celestia.DefaultGasPrices, + NamespaceIDStr: "0000000000000000ffff", + } + err := config.InitNamespaceID() + if err != nil { + return nil, err + } + conf, err := json.Marshal(config) + if err != nil { + return nil, err + } + + mockDA.MockRPC = damocks.NewMockCelestiaRPCClient(t) + options := []da.Option{ + celestia.WithRPCClient(mockDA.MockRPC), + celestia.WithRPCAttempts(1), + celestia.WithRPCRetryDelay(time.Second * 2), + } + roots := [][]byte{[]byte("apple"), []byte("watermelon"), []byte("kiwi")} + dah := &header.DataAvailabilityHeader{ + RowRoots: roots, + ColumnRoots: roots, + } + mockDA.Header = &header.ExtendedHeader{ + DAH: dah, + } + + mockDA.NID = config.NamespaceID.Bytes() + + nIDSize := 1 + tree := exampleNMT(nIDSize, true, 1, 2, 3, 4) + // build a proof for an NID that is within the namespace range of the tree + proof, _ := tree.ProveNamespace(mockDA.NID) + mockDA.BlobProof = blob.Proof([]*nmt.Proof{&proof}) + + err = mockDA.DaClient.Init(conf, nil, store.NewDefaultInMemoryKVStore(), log.TestingLogger(), options...) + if err != nil { + return nil, err + } + return mockDA, nil +} + +// exampleNMT creates a new NamespacedMerkleTree with the given namespace ID size and leaf namespace IDs. Each byte in the leavesNIDs parameter corresponds to one leaf's namespace ID. If nidSize is greater than 1, the function repeats each NID in leavesNIDs nidSize times before prepending it to the leaf data. +func exampleNMT(nidSize int, ignoreMaxNamespace bool, leavesNIDs ...byte) *nmt.NamespacedMerkleTree { + tree := nmt.New(sha256.New(), nmt.NamespaceIDSize(nidSize), nmt.IgnoreMaxNamespace(ignoreMaxNamespace)) + for i, nid := range leavesNIDs { + namespace := bytes.Repeat([]byte{nid}, nidSize) + d := append(namespace, []byte(fmt.Sprintf("leaf_%d", i))...) + if err := tree.Push(d); err != nil { + panic(fmt.Sprintf("unexpected error: %v", err)) + } + } + return tree +} diff --git a/testutil/types.go b/testutil/types.go index 568ebc629..dcc8deb8f 100644 --- a/testutil/types.go +++ b/testutil/types.go @@ -169,6 +169,14 @@ func GenerateCommits(blocks []*types.Block, proposerKey crypto.PrivKey) ([]*type return commits, nil } +func GenerateDRS(blocks int) []string { + drs := make([]string, blocks) + for i := 0; i < blocks; i++ { + drs[i] = dymintversion.Commit + } + return drs +} + func generateSignature(proposerKey crypto.PrivKey, header *types.Header) ([]byte, error) { abciHeaderPb := types.ToABCIHeaderPB(header) abciHeaderBytes, err := abciHeaderPb.Marshal() @@ -193,8 +201,9 @@ func GenerateBatch(startHeight uint64, endHeight uint64, proposerKey crypto.Priv return nil, err } batch := &types.Batch{ - Blocks: blocks, - Commits: commits, + Blocks: blocks, + Commits: commits, + DRSVersion: GenerateDRS(len(blocks)), } return batch, nil } @@ -209,8 +218,9 @@ func MustGenerateBatch(startHeight uint64, endHeight uint64, proposerKey crypto. panic(err) } return &types.Batch{ - Blocks: blocks, - Commits: commits, + Blocks: blocks, + Commits: commits, + DRSVersion: GenerateDRS(len(blocks)), } } @@ -310,3 +320,16 @@ func GetRandomBlock(height uint64, nTxs int) *types.Block { return block } + +func CreateRandomVersionCommit() (string, error) { + letterRunes := []rune("abcdefghijklmnopqrstuvwxyz") + b := make([]rune, 40) + for i := range b { + num, err := rand.Int(rand.Reader, big.NewInt(int64(len(letterRunes)))) + if err != nil { + return "", err + } + b[i] = letterRunes[num.Int64()] + } + return string(b), nil +} diff --git a/types/batch.go b/types/batch.go index 2476df5a3..2912f0a28 100644 --- a/types/batch.go +++ b/types/batch.go @@ -10,7 +10,8 @@ type Batch struct { Blocks []*Block Commits []*Commit // LastBatch is true if this is the last batch of the sequencer (i.e completes it's rotation flow). - LastBatch bool + LastBatch bool + DRSVersion []string } // StartHeight is the height of the first block in the batch. diff --git a/types/errors.go b/types/errors.go index d44fbfa64..7c4104854 100644 --- a/types/errors.go +++ b/types/errors.go @@ -324,21 +324,19 @@ func (e ErrInvalidHeaderDataHashFraud) Unwrap() error { type ErrStateUpdateNumBlocksNotMatchingFraud struct { StateIndex uint64 SLNumBlocks uint64 - DAblocks uint64 NumBds uint64 } -func NewErrStateUpdateNumBlocksNotMatchingFraud(stateIndex uint64, slNumBlocks uint64, daBlocks uint64, numbds uint64) error { +func NewErrStateUpdateNumBlocksNotMatchingFraud(stateIndex uint64, slNumBlocks uint64, numbds uint64) error { return &ErrStateUpdateNumBlocksNotMatchingFraud{ StateIndex: stateIndex, SLNumBlocks: slNumBlocks, - DAblocks: daBlocks, NumBds: numbds, } } func (e ErrStateUpdateNumBlocksNotMatchingFraud) Error() string { - return fmt.Sprintf("numblocks not matching. StateIndex: %d Batch numblocks: %d Blocks in DA: %d Num of block descriptors: %d", e.StateIndex, e.SLNumBlocks, e.DAblocks, e.NumBds) + return fmt.Sprintf("numblocks not matching. StateIndex: %d Batch numblocks: %d Num of block descriptors: %d", e.StateIndex, e.SLNumBlocks, e.NumBds) } func (e ErrStateUpdateNumBlocksNotMatchingFraud) Unwrap() error { @@ -470,3 +468,75 @@ func getJsonFromBlock(block *Block) ([]byte, error) { } return jsonBlock, nil } + +type ErrStateUpdateBlobNotAvailableFraud struct { + StateIndex uint64 + DA string + DAHeight uint64 + Commitment string +} + +func NewErrStateUpdateBlobNotAvailableFraud(stateIndex uint64, da string, daHeight uint64, commitment string) error { + return &ErrStateUpdateBlobNotAvailableFraud{ + StateIndex: stateIndex, + DA: da, + DAHeight: daHeight, + Commitment: commitment, + } +} + +func (e ErrStateUpdateBlobNotAvailableFraud) Error() string { + return fmt.Sprintf("blob not available in DA. StateIndex: %d DA: %s DA Height: %d Commitment: %s", e.StateIndex, e.DA, e.DAHeight, e.Commitment) +} + +func (e ErrStateUpdateBlobNotAvailableFraud) Unwrap() error { + return gerrc.ErrFault +} + +type ErrStateUpdateBlobCorruptedFraud struct { + StateIndex uint64 + DA string + DAHeight uint64 + Commitment string +} + +func NewErrStateUpdateBlobCorruptedFraud(stateIndex uint64, da string, daHeight uint64, commitment string) error { + return &ErrStateUpdateBlobCorruptedFraud{ + StateIndex: stateIndex, + DA: da, + DAHeight: daHeight, + Commitment: commitment, + } +} + +func (e ErrStateUpdateBlobCorruptedFraud) Error() string { + return fmt.Sprintf("blob not parsable in DA. StateIndex: %d DA: %s DA Height: %d Commitment: %s", e.StateIndex, e.DA, e.DAHeight, e.Commitment) +} + +func (e ErrStateUpdateBlobCorruptedFraud) Unwrap() error { + return gerrc.ErrFault +} + +type ErrStateUpdateDRSVersionFraud struct { + StateIndex uint64 + Height uint64 + BlockVersion string + SLVersion string +} + +func NewErrStateUpdateDRSVersionFraud(stateIndex uint64, height uint64, blockVersion string, slVersion string) error { + return &ErrStateUpdateDRSVersionFraud{ + StateIndex: stateIndex, + Height: height, + BlockVersion: blockVersion, + SLVersion: slVersion, + } +} + +func (e ErrStateUpdateDRSVersionFraud) Error() string { + return fmt.Sprintf("drs version not matching. StateIndex: %d Height: %d Block DRS: %s SL DRS: %s", e.StateIndex, e.Height, e.BlockVersion, e.SLVersion) +} + +func (e ErrStateUpdateDRSVersionFraud) Unwrap() error { + return gerrc.ErrFault +} diff --git a/types/pb/dymensionxyz/dymension/rollapp/tx.pb.go b/types/pb/dymensionxyz/dymension/rollapp/tx.pb.go index c8c6899dc..dba4ff173 100644 --- a/types/pb/dymensionxyz/dymension/rollapp/tx.pb.go +++ b/types/pb/dymensionxyz/dymension/rollapp/tx.pb.go @@ -301,8 +301,6 @@ type MsgUpdateState struct { BDs BlockDescriptors `protobuf:"bytes,7,opt,name=BDs,proto3" json:"BDs"` // last is true if this is the last batch of the sequencer Last bool `protobuf:"varint,8,opt,name=last,proto3" json:"last,omitempty"` - // DrsVersion is a DRS version used by the rollapp. - DrsVersion string `protobuf:"bytes,9,opt,name=drs_version,json=drsVersion,proto3" json:"drs_version,omitempty"` } func (m *MsgUpdateState) Reset() { *m = MsgUpdateState{} } @@ -387,13 +385,6 @@ func (m *MsgUpdateState) GetLast() bool { return false } -func (m *MsgUpdateState) GetDrsVersion() string { - if m != nil { - return m.DrsVersion - } - return "" -} - type MsgUpdateStateResponse struct { } @@ -1027,64 +1018,63 @@ func init() { } var fileDescriptor_fa9b86052c2a43cd = []byte{ - // 901 bytes of a gzipped FileDescriptorProto + // 889 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x56, 0x4f, 0x6f, 0x1b, 0x45, - 0x14, 0xcf, 0xda, 0x1b, 0xff, 0x79, 0x76, 0x8d, 0x3b, 0x44, 0xd1, 0x92, 0xb6, 0xae, 0xeb, 0x08, - 0x29, 0xa2, 0xc2, 0x86, 0xf4, 0x16, 0x4e, 0x09, 0x11, 0x6d, 0x41, 0x56, 0x61, 0x1b, 0x72, 0xe0, - 0x62, 0xc6, 0xde, 0xc9, 0x7a, 0xd4, 0xdd, 0x99, 0x65, 0x66, 0xec, 0xd4, 0x1c, 0xb9, 0x22, 0x21, - 0x3e, 0x0a, 0x12, 0x7c, 0x06, 0xd4, 0x63, 0x8f, 0x70, 0x41, 0x28, 0x39, 0xf0, 0x35, 0xd0, 0xcc, - 0x8e, 0xd7, 0x4e, 0xe3, 0xd8, 0xa6, 0xe2, 0xb4, 0xf3, 0xde, 0xcc, 0x7b, 0xf3, 0x7e, 0xbf, 0xdf, - 0x7b, 0xa3, 0x85, 0x0f, 0xd5, 0x24, 0x21, 0xb2, 0x13, 0x4c, 0x62, 0xc2, 0x24, 0xe5, 0xec, 0xe5, - 0xe4, 0xfb, 0x99, 0xd1, 0x11, 0x3c, 0x8a, 0x70, 0x92, 0x74, 0xd4, 0xcb, 0x76, 0x22, 0xb8, 0xe2, - 0xa8, 0x31, 0x7f, 0xb0, 0x9d, 0x19, 0x6d, 0x7b, 0x70, 0xe7, 0x5e, 0x9a, 0x6e, 0xc0, 0x65, 0xcc, - 0x65, 0x27, 0x96, 0x61, 0x67, 0xfc, 0xb1, 0xfe, 0xa4, 0xe1, 0x3b, 0x9f, 0xac, 0x75, 0x5b, 0x3f, - 0xe2, 0x83, 0x17, 0xbd, 0x80, 0xc8, 0x81, 0xa0, 0x89, 0xe2, 0xc2, 0x06, 0xef, 0xaf, 0x15, 0x6c, - 0xbf, 0x36, 0xe6, 0xd1, 0x5a, 0x31, 0x31, 0x51, 0x38, 0xc0, 0x0a, 0xdb, 0xa0, 0xad, 0x90, 0x87, - 0xdc, 0x2c, 0x3b, 0x7a, 0x95, 0x7a, 0x5b, 0x3f, 0xe5, 0xa1, 0xde, 0x95, 0xe1, 0xa7, 0x82, 0x60, - 0x45, 0xfc, 0x34, 0x12, 0x79, 0x50, 0x1c, 0x68, 0x07, 0x17, 0x9e, 0xd3, 0x74, 0xf6, 0xca, 0xfe, - 0xd4, 0x44, 0xf7, 0x00, 0x6c, 0xfa, 0x1e, 0x0d, 0xbc, 0x9c, 0xd9, 0x2c, 0x5b, 0xcf, 0xd3, 0x00, - 0x3d, 0x84, 0xdb, 0x94, 0x51, 0x45, 0x71, 0xd4, 0x93, 0xe4, 0xbb, 0x11, 0x61, 0x03, 0x22, 0xbc, - 0x8a, 0x39, 0x55, 0xb7, 0x1b, 0xcf, 0xa7, 0x7e, 0xb4, 0x05, 0x9b, 0x38, 0xa2, 0x58, 0x7a, 0x55, - 0x73, 0x20, 0x35, 0xd0, 0x17, 0x50, 0x9a, 0x16, 0xee, 0xdd, 0x6a, 0x3a, 0x7b, 0x95, 0xfd, 0x4e, - 0x7b, 0xb9, 0x3c, 0x6d, 0x5b, 0x76, 0xd7, 0x86, 0xf9, 0x59, 0x02, 0x74, 0x02, 0xd5, 0x90, 0x30, - 0x22, 0xa9, 0xec, 0x51, 0x76, 0xc6, 0xbd, 0x9a, 0x49, 0xf8, 0x70, 0x55, 0xc2, 0xc7, 0x69, 0xcc, - 0x53, 0x76, 0xc6, 0x8f, 0xdc, 0x57, 0x7f, 0xdd, 0xdf, 0xf0, 0x2b, 0xe1, 0xcc, 0x85, 0x1e, 0x43, - 0x71, 0x1c, 0xf7, 0xb4, 0x06, 0xde, 0x3b, 0x4d, 0x67, 0xaf, 0xb6, 0xdf, 0x5e, 0xb3, 0xc2, 0xf6, - 0x69, 0xf7, 0x64, 0x92, 0x10, 0xbf, 0x30, 0x8e, 0xf5, 0xf7, 0xa0, 0xfa, 0xc3, 0x3f, 0xbf, 0x7c, - 0x30, 0xe5, 0xf6, 0x73, 0xb7, 0x94, 0xaf, 0x57, 0x5a, 0x3b, 0xe0, 0xbd, 0xa9, 0x87, 0x4f, 0x64, - 0xc2, 0x99, 0x24, 0xad, 0xdf, 0x72, 0x70, 0xa7, 0x2b, 0xc3, 0xaf, 0x93, 0x60, 0xb6, 0xa9, 0x2b, - 0x12, 0x31, 0x56, 0x94, 0x33, 0xcd, 0x28, 0x3f, 0x67, 0x64, 0xaa, 0x5a, 0x6a, 0xbc, 0x95, 0x66, - 0xf9, 0x1b, 0x34, 0xfb, 0x6a, 0x4e, 0x9d, 0xcd, 0xb7, 0x52, 0xc7, 0x10, 0xea, 0x2c, 0xd1, 0xa8, - 0xf0, 0x7f, 0x68, 0x74, 0x00, 0x9a, 0xda, 0x94, 0x80, 0xd6, 0xfb, 0xb0, 0xbb, 0x84, 0xb5, 0x8c, - 0xdd, 0x5f, 0x73, 0x50, 0xcb, 0xce, 0x3d, 0x57, 0x58, 0x91, 0x25, 0x83, 0x70, 0x17, 0x66, 0x14, - 0x5e, 0xe7, 0xb4, 0x09, 0x15, 0xa9, 0xb0, 0x50, 0x4f, 0x08, 0x0d, 0x87, 0xca, 0xb0, 0xe9, 0xfa, - 0xf3, 0x2e, 0x1d, 0xcf, 0x46, 0xf1, 0x91, 0x7e, 0x13, 0xa4, 0xe7, 0x9a, 0xfd, 0x99, 0x03, 0x6d, - 0x43, 0xe1, 0xf8, 0xf0, 0x4b, 0xac, 0x86, 0x86, 0xe4, 0xb2, 0x6f, 0x2d, 0xf4, 0x04, 0xf2, 0x47, - 0xc7, 0xd2, 0x2b, 0x1a, 0x8a, 0x3e, 0x5a, 0x45, 0x91, 0x49, 0x76, 0x9c, 0x3d, 0x38, 0xd2, 0xf2, - 0xa4, 0x53, 0x20, 0x04, 0x6e, 0x84, 0xa5, 0xf2, 0x4a, 0x4d, 0x67, 0xaf, 0xe4, 0x9b, 0x35, 0xba, - 0x0f, 0x95, 0x40, 0xc8, 0xde, 0x98, 0x08, 0x9d, 0xc6, 0x2b, 0x9b, 0xab, 0x21, 0x10, 0xf2, 0x34, - 0xf5, 0x5c, 0xeb, 0xd7, 0x42, 0xbd, 0xd8, 0xf2, 0x60, 0xfb, 0x2a, 0x69, 0x19, 0x9f, 0x3f, 0x3a, - 0xb0, 0xd5, 0x95, 0xe1, 0x89, 0xc0, 0x4c, 0x9e, 0x11, 0xf1, 0x4c, 0x6b, 0x21, 0x87, 0x34, 0x41, - 0xbb, 0x70, 0x6b, 0x30, 0x12, 0x82, 0x30, 0xd5, 0x9b, 0x6f, 0xd7, 0xaa, 0x75, 0x9a, 0x83, 0xe8, - 0x0e, 0x94, 0x19, 0x39, 0xb7, 0x07, 0x52, 0x82, 0x4b, 0x8c, 0x9c, 0x3f, 0x5b, 0xd0, 0xd2, 0xf9, - 0x37, 0xe8, 0x3f, 0x40, 0xba, 0xce, 0xab, 0x77, 0xb4, 0x1a, 0x70, 0x77, 0x51, 0x31, 0x59, 0xb5, - 0xbf, 0x3b, 0x50, 0xee, 0xca, 0xf0, 0x30, 0x08, 0x0e, 0x97, 0xbe, 0x80, 0x08, 0x5c, 0x86, 0x63, - 0x62, 0x4b, 0x32, 0xeb, 0x15, 0xe5, 0xe8, 0x6e, 0x98, 0x3e, 0xfb, 0x9a, 0x57, 0xd7, 0xec, 0xcf, - 0xbb, 0xf4, 0xe0, 0xd2, 0x18, 0x87, 0xc4, 0xca, 0x9d, 0x1a, 0xa8, 0x0e, 0xf9, 0x91, 0x88, 0xcc, - 0x40, 0x94, 0x7d, 0xbd, 0x34, 0x03, 0x2e, 0x02, 0x22, 0x4c, 0x07, 0x6c, 0xfa, 0xa9, 0x71, 0x55, - 0x96, 0xd6, 0xbb, 0x70, 0x3b, 0xc3, 0x91, 0xa1, 0xfb, 0xd3, 0x81, 0x6a, 0x26, 0xd3, 0x72, 0x80, - 0x35, 0xc8, 0xd9, 0x67, 0xc2, 0xf5, 0x73, 0x34, 0xc8, 0x00, 0xe7, 0x6f, 0x04, 0xec, 0xae, 0x00, - 0xbc, 0xb9, 0x04, 0x70, 0x61, 0x01, 0xe0, 0xe2, 0x02, 0xc0, 0xa5, 0x9b, 0x01, 0x6f, 0x9b, 0x36, - 0xcb, 0xa0, 0x65, 0x98, 0x89, 0x81, 0xec, 0x93, 0x98, 0x8f, 0xff, 0x23, 0xe4, 0x15, 0xed, 0xb5, - 0xe8, 0xfa, 0xec, 0x9a, 0xec, 0xfa, 0x08, 0xde, 0xeb, 0xca, 0xb0, 0x8b, 0xc5, 0x8b, 0xd3, 0x51, - 0xc4, 0x88, 0xc0, 0xfd, 0x68, 0xfa, 0xfa, 0x48, 0x3d, 0xfe, 0x78, 0xa4, 0x86, 0x5c, 0x50, 0x35, - 0xb1, 0xd5, 0xcc, 0x1c, 0xe8, 0x01, 0x54, 0xe7, 0x06, 0x51, 0x7a, 0xb9, 0x66, 0xde, 0x10, 0x98, - 0x4d, 0xa2, 0x3c, 0xa8, 0xe9, 0x1a, 0x66, 0x21, 0xad, 0x5d, 0x78, 0x70, 0xe3, 0x6d, 0xd3, 0x92, - 0x8e, 0xbe, 0x7d, 0x75, 0xd1, 0x70, 0x5e, 0x5f, 0x34, 0x9c, 0xbf, 0x2f, 0x1a, 0xce, 0xcf, 0x97, - 0x8d, 0x8d, 0xd7, 0x97, 0x8d, 0x8d, 0x3f, 0x2e, 0x1b, 0x1b, 0xdf, 0x7c, 0x16, 0x52, 0x35, 0x1c, - 0xf5, 0xdb, 0x03, 0x1e, 0x5f, 0xfb, 0xad, 0xa0, 0x4c, 0x75, 0xd2, 0x1f, 0x8e, 0xa4, 0xbf, 0xe2, - 0x9f, 0xa3, 0x5f, 0x30, 0x7f, 0x15, 0x8f, 0xfe, 0x0d, 0x00, 0x00, 0xff, 0xff, 0xf7, 0x26, 0x1d, - 0x3c, 0x81, 0x09, 0x00, 0x00, + 0x14, 0xcf, 0xda, 0x8e, 0xff, 0x3c, 0xbb, 0xc6, 0x1d, 0xa2, 0x68, 0x49, 0x5b, 0xe3, 0x3a, 0x42, + 0x8a, 0xa8, 0xb0, 0x21, 0xbd, 0x85, 0x53, 0x42, 0x44, 0x5b, 0x90, 0x55, 0xd8, 0x86, 0x1c, 0xb8, + 0x98, 0xb1, 0x77, 0xb2, 0x1e, 0x75, 0x77, 0x66, 0x99, 0x19, 0x3b, 0x35, 0x47, 0xae, 0x48, 0x08, + 0xf1, 0x49, 0x38, 0xf0, 0x19, 0x50, 0x8f, 0x3d, 0xc2, 0x05, 0xa1, 0xe4, 0xc0, 0xd7, 0x40, 0x33, + 0x3b, 0xbb, 0x76, 0x1a, 0xc7, 0x36, 0x55, 0x4f, 0x33, 0xef, 0xcd, 0x7b, 0x6f, 0xde, 0xef, 0xf7, + 0x9b, 0x19, 0x0d, 0x7c, 0xa4, 0xa6, 0x31, 0x91, 0x5d, 0x7f, 0x1a, 0x11, 0x26, 0x29, 0x67, 0x2f, + 0xa6, 0x3f, 0xcc, 0x8c, 0xae, 0xe0, 0x61, 0x88, 0xe3, 0xb8, 0xab, 0x5e, 0x74, 0x62, 0xc1, 0x15, + 0x47, 0xcd, 0xf9, 0xc0, 0x4e, 0x66, 0x74, 0x6c, 0xe0, 0xce, 0xbd, 0xa4, 0xdc, 0x90, 0xcb, 0x88, + 0xcb, 0x6e, 0x24, 0x83, 0xee, 0xe4, 0x13, 0x3d, 0x24, 0xe9, 0x3b, 0x9f, 0xae, 0xb5, 0xdb, 0x20, + 0xe4, 0xc3, 0xe7, 0x7d, 0x9f, 0xc8, 0xa1, 0xa0, 0xb1, 0xe2, 0xc2, 0x26, 0xef, 0xaf, 0x95, 0x6c, + 0x47, 0x9b, 0xf3, 0x70, 0xad, 0x9c, 0x88, 0x28, 0xec, 0x63, 0x85, 0x6d, 0xd2, 0x56, 0xc0, 0x03, + 0x6e, 0xa6, 0x5d, 0x3d, 0x4b, 0xbc, 0xed, 0x9f, 0xf3, 0xd0, 0xe8, 0xc9, 0xe0, 0x33, 0x41, 0xb0, + 0x22, 0x5e, 0x92, 0x89, 0x5c, 0x28, 0x0d, 0xb5, 0x83, 0x0b, 0xd7, 0x69, 0x39, 0x7b, 0x15, 0x2f, + 0x35, 0xd1, 0x3d, 0x00, 0x5b, 0xbe, 0x4f, 0x7d, 0x37, 0x67, 0x16, 0x2b, 0xd6, 0xf3, 0xc4, 0x47, + 0x0f, 0xe0, 0x36, 0x65, 0x54, 0x51, 0x1c, 0xf6, 0x25, 0xf9, 0x7e, 0x4c, 0xd8, 0x90, 0x08, 0xb7, + 0x6a, 0xa2, 0x1a, 0x76, 0xe1, 0x59, 0xea, 0x47, 0x5b, 0xb0, 0x89, 0x43, 0x8a, 0xa5, 0x5b, 0x33, + 0x01, 0x89, 0x81, 0xbe, 0x84, 0x72, 0xda, 0xb8, 0x7b, 0xab, 0xe5, 0xec, 0x55, 0xf7, 0xbb, 0x9d, + 0xe5, 0xf2, 0x74, 0x6c, 0xdb, 0x3d, 0x9b, 0xe6, 0x65, 0x05, 0xd0, 0x09, 0xd4, 0x02, 0xc2, 0x88, + 0xa4, 0xb2, 0x4f, 0xd9, 0x19, 0x77, 0xeb, 0xa6, 0xe0, 0x83, 0x55, 0x05, 0x1f, 0x25, 0x39, 0x4f, + 0xd8, 0x19, 0x3f, 0x2a, 0xbc, 0xfc, 0xfb, 0xfd, 0x0d, 0xaf, 0x1a, 0xcc, 0x5c, 0xe8, 0x11, 0x94, + 0x26, 0x51, 0x5f, 0x6b, 0xe0, 0xbe, 0xd3, 0x72, 0xf6, 0xea, 0xfb, 0x9d, 0x35, 0x3b, 0xec, 0x9c, + 0xf6, 0x4e, 0xa6, 0x31, 0xf1, 0x8a, 0x93, 0x48, 0x8f, 0x07, 0xb5, 0x1f, 0xff, 0xfd, 0xed, 0xc3, + 0x94, 0xdb, 0x2f, 0x0a, 0xe5, 0x7c, 0xa3, 0xda, 0xde, 0x01, 0xf7, 0x75, 0x3d, 0x3c, 0x22, 0x63, + 0xce, 0x24, 0x69, 0xff, 0x9e, 0x83, 0x3b, 0x3d, 0x19, 0x7c, 0x13, 0xfb, 0xb3, 0x45, 0xdd, 0x91, + 0x88, 0xb0, 0xa2, 0x9c, 0x69, 0x46, 0xf9, 0x39, 0x23, 0xa9, 0x6a, 0x89, 0xf1, 0x46, 0x9a, 0xe5, + 0x6f, 0xd0, 0xec, 0xeb, 0x39, 0x75, 0x36, 0xdf, 0x48, 0x1d, 0x43, 0xa8, 0xb3, 0x44, 0xa3, 0xe2, + 0xdb, 0xd0, 0xe8, 0x00, 0x34, 0xb5, 0x09, 0x01, 0xed, 0x0f, 0x60, 0x77, 0x09, 0x6b, 0x19, 0xbb, + 0xbf, 0xe6, 0xa0, 0x9e, 0xc5, 0x3d, 0x53, 0x58, 0x91, 0x25, 0x17, 0xe1, 0x2e, 0xcc, 0x28, 0xbc, + 0xce, 0x69, 0x0b, 0xaa, 0x52, 0x61, 0xa1, 0x1e, 0x13, 0x1a, 0x8c, 0x94, 0x61, 0xb3, 0xe0, 0xcd, + 0xbb, 0x74, 0x3e, 0x1b, 0x47, 0x47, 0xfa, 0x4d, 0x90, 0x6e, 0xc1, 0xac, 0xcf, 0x1c, 0x68, 0x1b, + 0x8a, 0xc7, 0x87, 0x5f, 0x61, 0x35, 0x32, 0x24, 0x57, 0x3c, 0x6b, 0xa1, 0xc7, 0x90, 0x3f, 0x3a, + 0x96, 0x6e, 0xc9, 0x50, 0xf4, 0xf1, 0x2a, 0x8a, 0x4c, 0xb1, 0xe3, 0xec, 0xc1, 0x91, 0x96, 0x27, + 0x5d, 0x02, 0x21, 0x28, 0x84, 0x58, 0x2a, 0xb7, 0xdc, 0x72, 0xf6, 0xca, 0x9e, 0x99, 0x5f, 0x3b, + 0x8e, 0xc5, 0x46, 0xa9, 0xed, 0xc2, 0xf6, 0x55, 0x4e, 0x32, 0xba, 0x7e, 0x72, 0x60, 0xab, 0x27, + 0x83, 0x13, 0x81, 0x99, 0x3c, 0x23, 0xe2, 0xa9, 0xa6, 0x5a, 0x8e, 0x68, 0x8c, 0x76, 0xe1, 0xd6, + 0x70, 0x2c, 0x04, 0x61, 0xaa, 0x3f, 0x7f, 0x1a, 0x6b, 0xd6, 0x69, 0x02, 0xd1, 0x1d, 0xa8, 0x30, + 0x72, 0x6e, 0x03, 0x12, 0xfe, 0xca, 0x8c, 0x9c, 0x3f, 0x5d, 0x70, 0x62, 0xf3, 0xaf, 0xb1, 0x7b, + 0x80, 0x74, 0x9f, 0x57, 0xf7, 0x68, 0x37, 0xe1, 0xee, 0xa2, 0x66, 0xb2, 0x6e, 0xff, 0x70, 0xa0, + 0xd2, 0x93, 0xc1, 0xa1, 0xef, 0x1f, 0x2e, 0x7d, 0xe0, 0x10, 0x14, 0x18, 0x8e, 0x88, 0x6d, 0xc9, + 0xcc, 0x57, 0xb4, 0xa3, 0xc5, 0x4e, 0x5f, 0x75, 0xca, 0x99, 0x11, 0xb3, 0xe2, 0xcd, 0xbb, 0xf4, + 0xbd, 0xa4, 0x11, 0x0e, 0x88, 0x55, 0x33, 0x31, 0x50, 0x03, 0xf2, 0x63, 0x11, 0x9a, 0xf3, 0x5e, + 0xf1, 0xf4, 0xd4, 0xdc, 0x5f, 0xe1, 0x13, 0x61, 0x04, 0xde, 0xf4, 0x12, 0xe3, 0xaa, 0x2c, 0xed, + 0x77, 0xe1, 0x76, 0x86, 0x23, 0x43, 0xf7, 0x97, 0x03, 0xb5, 0x4c, 0xa6, 0xe5, 0x00, 0xeb, 0x90, + 0xb3, 0xaf, 0x40, 0xc1, 0xcb, 0x51, 0x3f, 0x03, 0x9c, 0xbf, 0x11, 0x70, 0x61, 0x05, 0xe0, 0xcd, + 0x25, 0x80, 0x8b, 0x0b, 0x00, 0x97, 0x16, 0x00, 0x2e, 0xdf, 0x0c, 0x78, 0xdb, 0x1c, 0xb3, 0x0c, + 0x5a, 0x86, 0x99, 0x18, 0xc8, 0x1e, 0x89, 0xf8, 0xe4, 0x7f, 0x42, 0x5e, 0x71, 0xbc, 0x16, 0x6d, + 0x9f, 0x6d, 0x93, 0x6d, 0x1f, 0xc2, 0x7b, 0x3d, 0x19, 0xf4, 0xb0, 0x78, 0x7e, 0x3a, 0x0e, 0x19, + 0x11, 0x78, 0x10, 0xa6, 0x8f, 0x8b, 0xd4, 0xb7, 0x1b, 0x8f, 0xd5, 0x88, 0x0b, 0xaa, 0xa6, 0xb6, + 0x9b, 0x99, 0x03, 0xdd, 0x87, 0x9a, 0x2f, 0x64, 0x7f, 0x42, 0x84, 0xbe, 0xae, 0xd2, 0xcd, 0xb5, + 0xf2, 0x86, 0x40, 0x21, 0x4f, 0xad, 0xeb, 0xa0, 0xae, 0x7b, 0x98, 0xa5, 0xb4, 0x77, 0xe1, 0xfe, + 0x8d, 0xbb, 0xa5, 0x2d, 0x1d, 0x7d, 0xf7, 0xf2, 0xa2, 0xe9, 0xbc, 0xba, 0x68, 0x3a, 0xff, 0x5c, + 0x34, 0x9d, 0x5f, 0x2e, 0x9b, 0x1b, 0xaf, 0x2e, 0x9b, 0x1b, 0x7f, 0x5e, 0x36, 0x37, 0xbe, 0xfd, + 0x3c, 0xa0, 0x6a, 0x34, 0x1e, 0x74, 0x86, 0x3c, 0xba, 0xf6, 0x6b, 0xa0, 0x4c, 0x75, 0x93, 0xff, + 0x44, 0x3c, 0x58, 0xf1, 0xa5, 0x18, 0x14, 0xcd, 0xa7, 0xe1, 0xe1, 0x7f, 0x01, 0x00, 0x00, 0xff, + 0xff, 0xd3, 0x61, 0x82, 0xbe, 0x60, 0x09, 0x00, 0x00, } func (m *MsgCreateRollapp) Marshal() (dAtA []byte, err error) { @@ -1297,13 +1287,6 @@ func (m *MsgUpdateState) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.DrsVersion) > 0 { - i -= len(m.DrsVersion) - copy(dAtA[i:], m.DrsVersion) - i = encodeVarintTx(dAtA, i, uint64(len(m.DrsVersion))) - i-- - dAtA[i] = 0x4a - } if m.Last { i-- if m.Last { @@ -1885,10 +1868,6 @@ func (m *MsgUpdateState) Size() (n int) { if m.Last { n += 2 } - l = len(m.DrsVersion) - if l > 0 { - n += 1 + l + sovTx(uint64(l)) - } return n } @@ -2884,38 +2863,6 @@ func (m *MsgUpdateState) Unmarshal(dAtA []byte) error { } } m.Last = bool(v != 0) - case 9: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DrsVersion", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowTx - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthTx - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthTx - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.DrsVersion = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipTx(dAtA[iNdEx:]) diff --git a/types/pb/dymint/state.pb.go b/types/pb/dymint/state.pb.go index 33c2a1de3..8c3bf173d 100644 --- a/types/pb/dymint/state.pb.go +++ b/types/pb/dymint/state.pb.go @@ -31,23 +31,22 @@ var _ = time.Kitchen const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package type State struct { - Version *state.Version `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` - ChainId string `protobuf:"bytes,2,opt,name=chain_id,json=chainId,proto3" json:"chain_id,omitempty"` - InitialHeight int64 `protobuf:"varint,3,opt,name=initial_height,json=initialHeight,proto3" json:"initial_height,omitempty"` - LastBlockHeight int64 `protobuf:"varint,4,opt,name=last_block_height,json=lastBlockHeight,proto3" json:"last_block_height,omitempty"` - LastBlockID types.BlockID `protobuf:"bytes,5,opt,name=last_block_id,json=lastBlockId,proto3" json:"last_block_id"` - LastBlockTime time.Time `protobuf:"bytes,6,opt,name=last_block_time,json=lastBlockTime,proto3,stdtime" json:"last_block_time"` - Validators *types.ValidatorSet `protobuf:"bytes,9,opt,name=validators,proto3" json:"validators,omitempty"` // Deprecated: Do not use. - LastHeightValidatorsChanged int64 `protobuf:"varint,11,opt,name=last_height_validators_changed,json=lastHeightValidatorsChanged,proto3" json:"last_height_validators_changed,omitempty"` - ConsensusParams types.ConsensusParams `protobuf:"bytes,12,opt,name=consensus_params,json=consensusParams,proto3" json:"consensus_params"` - LastHeightConsensusParamsChanged int64 `protobuf:"varint,13,opt,name=last_height_consensus_params_changed,json=lastHeightConsensusParamsChanged,proto3" json:"last_height_consensus_params_changed,omitempty"` - LastResultsHash []byte `protobuf:"bytes,14,opt,name=last_results_hash,json=lastResultsHash,proto3" json:"last_results_hash,omitempty"` - AppHash []byte `protobuf:"bytes,15,opt,name=app_hash,json=appHash,proto3" json:"app_hash,omitempty"` - LastStoreHeight uint64 `protobuf:"varint,16,opt,name=last_store_height,json=lastStoreHeight,proto3" json:"last_store_height,omitempty"` - 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"` + Version *state.Version `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` + ChainId string `protobuf:"bytes,2,opt,name=chain_id,json=chainId,proto3" json:"chain_id,omitempty"` + InitialHeight int64 `protobuf:"varint,3,opt,name=initial_height,json=initialHeight,proto3" json:"initial_height,omitempty"` + LastBlockHeight int64 `protobuf:"varint,4,opt,name=last_block_height,json=lastBlockHeight,proto3" json:"last_block_height,omitempty"` + LastBlockID types.BlockID `protobuf:"bytes,5,opt,name=last_block_id,json=lastBlockId,proto3" json:"last_block_id"` + LastBlockTime time.Time `protobuf:"bytes,6,opt,name=last_block_time,json=lastBlockTime,proto3,stdtime" json:"last_block_time"` + Validators *types.ValidatorSet `protobuf:"bytes,9,opt,name=validators,proto3" json:"validators,omitempty"` // Deprecated: Do not use. + LastHeightValidatorsChanged int64 `protobuf:"varint,11,opt,name=last_height_validators_changed,json=lastHeightValidatorsChanged,proto3" json:"last_height_validators_changed,omitempty"` + ConsensusParams types.ConsensusParams `protobuf:"bytes,12,opt,name=consensus_params,json=consensusParams,proto3" json:"consensus_params"` + LastResultsHash []byte `protobuf:"bytes,14,opt,name=last_results_hash,json=lastResultsHash,proto3" json:"last_results_hash,omitempty"` + AppHash []byte `protobuf:"bytes,15,opt,name=app_hash,json=appHash,proto3" json:"app_hash,omitempty"` + LastStoreHeight uint64 `protobuf:"varint,16,opt,name=last_store_height,json=lastStoreHeight,proto3" json:"last_store_height,omitempty"` + 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{} } @@ -147,13 +146,6 @@ func (m *State) GetConsensusParams() types.ConsensusParams { return types.ConsensusParams{} } -func (m *State) GetLastHeightConsensusParamsChanged() int64 { - if m != nil { - return m.LastHeightConsensusParamsChanged - } - return 0 -} - func (m *State) GetLastResultsHash() []byte { if m != nil { return m.LastResultsHash @@ -266,51 +258,50 @@ func init() { func init() { proto.RegisterFile("types/dymint/state.proto", fileDescriptor_4b679420add07272) } var fileDescriptor_4b679420add07272 = []byte{ - // 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, + // 686 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x94, 0x4d, 0x4f, 0xdb, 0x3e, + 0x1c, 0xc7, 0x9b, 0x12, 0x68, 0xea, 0xd2, 0x36, 0x18, 0xfe, 0x52, 0xe0, 0x2f, 0xa5, 0x85, 0x69, + 0x53, 0xb5, 0x43, 0x2a, 0x8d, 0xd3, 0x2e, 0x9b, 0x14, 0x38, 0xd0, 0x8a, 0xc3, 0x94, 0x4e, 0x1c, + 0x76, 0x89, 0x9c, 0xc4, 0x4b, 0xa2, 0xa5, 0x71, 0x16, 0xbb, 0x68, 0xec, 0x55, 0x70, 0xdc, 0x4b, + 0xe2, 0xc8, 0x71, 0x27, 0x36, 0x95, 0x37, 0x32, 0xf9, 0x21, 0x25, 0x55, 0xd5, 0x53, 0xeb, 0xef, + 0xef, 0xe3, 0x6f, 0xfc, 0x7b, 0xb0, 0x81, 0xc5, 0xee, 0x0a, 0x4c, 0xc7, 0xd1, 0xdd, 0x3c, 0xcd, + 0xd9, 0x98, 0x32, 0xc4, 0xb0, 0x53, 0x94, 0x84, 0x11, 0xb8, 0x27, 0xb5, 0x93, 0xa3, 0x98, 0xc4, + 0x44, 0x48, 0x63, 0xfe, 0x4f, 0x46, 0x4f, 0x06, 0x31, 0x21, 0x71, 0x86, 0xc7, 0x62, 0x15, 0x2c, + 0xbe, 0x8e, 0x59, 0x3a, 0xc7, 0x94, 0xa1, 0x79, 0xa1, 0x80, 0x53, 0x69, 0xcc, 0x70, 0x1e, 0xe1, + 0x52, 0x98, 0xa3, 0x20, 0x4c, 0xc7, 0x42, 0x55, 0xc8, 0xd9, 0x06, 0xa2, 0x84, 0x1a, 0xf3, 0x66, + 0x0b, 0x73, 0x8b, 0xb2, 0x34, 0x42, 0x8c, 0x94, 0x8a, 0x7b, 0xb5, 0x85, 0x2b, 0x50, 0x89, 0xe6, + 0xdb, 0x3f, 0x28, 0x12, 0x5e, 0xfb, 0xe0, 0xf1, 0x5a, 0x41, 0xe4, 0x8f, 0x0c, 0x9d, 0xfd, 0x6a, + 0x81, 0xdd, 0x19, 0xdf, 0x00, 0xcf, 0x41, 0xeb, 0x16, 0x97, 0x34, 0x25, 0xb9, 0xa5, 0x0d, 0xb5, + 0x51, 0xe7, 0xdd, 0xb1, 0xf3, 0x62, 0xea, 0xc8, 0x2a, 0xde, 0x48, 0xc0, 0xab, 0x48, 0x78, 0x0c, + 0x8c, 0x30, 0x41, 0x69, 0xee, 0xa7, 0x91, 0xd5, 0x1c, 0x6a, 0xa3, 0xb6, 0xd7, 0x12, 0xeb, 0x49, + 0x04, 0x5f, 0x83, 0x5e, 0x9a, 0xa7, 0x2c, 0x45, 0x99, 0x9f, 0xe0, 0x34, 0x4e, 0x98, 0xb5, 0x33, + 0xd4, 0x46, 0x3b, 0x5e, 0x57, 0xa9, 0x57, 0x42, 0x84, 0x6f, 0xc1, 0x41, 0x86, 0x28, 0xf3, 0x83, + 0x8c, 0x84, 0xdf, 0x2a, 0x52, 0x17, 0x64, 0x9f, 0x07, 0x5c, 0xae, 0x2b, 0xd6, 0x03, 0xdd, 0x1a, + 0x9b, 0x46, 0xd6, 0xee, 0xe6, 0x41, 0x65, 0xde, 0x62, 0xd7, 0xe4, 0xd2, 0x3d, 0x7c, 0x78, 0x1a, + 0x34, 0x96, 0x4f, 0x83, 0xce, 0x75, 0x65, 0x35, 0xb9, 0xf4, 0x3a, 0x2b, 0xdf, 0x49, 0x04, 0xaf, + 0x41, 0xbf, 0xe6, 0xc9, 0x3b, 0x6e, 0xed, 0x09, 0xd7, 0x13, 0x47, 0x8e, 0x83, 0x53, 0x8d, 0x83, + 0xf3, 0xb9, 0x1a, 0x07, 0xd7, 0xe0, 0xb6, 0xf7, 0x7f, 0x06, 0x9a, 0xd7, 0x5d, 0x79, 0xf1, 0x28, + 0x74, 0x01, 0x58, 0x75, 0x91, 0x5a, 0x6d, 0x61, 0x64, 0x6f, 0x1e, 0xef, 0xa6, 0x62, 0x66, 0x98, + 0xb9, 0x4d, 0x4b, 0xf3, 0x6a, 0xbb, 0xe0, 0x05, 0xb0, 0xc5, 0x89, 0x64, 0x2d, 0xfc, 0x97, 0x88, + 0x1f, 0x26, 0x28, 0x8f, 0x71, 0x64, 0x75, 0x44, 0x79, 0xfe, 0xe7, 0x94, 0xac, 0xcc, 0xca, 0x8f, + 0x5e, 0x48, 0x04, 0x7a, 0xc0, 0x0c, 0x49, 0x4e, 0x71, 0x4e, 0x17, 0xd4, 0x97, 0x03, 0x63, 0xed, + 0x8b, 0xe3, 0x9c, 0x6e, 0x1e, 0xe7, 0xa2, 0x22, 0x3f, 0x09, 0xd0, 0xd5, 0x79, 0x7a, 0x5e, 0x3f, + 0x5c, 0x97, 0x57, 0xad, 0x2a, 0x31, 0x5d, 0x64, 0x8c, 0xfa, 0x09, 0xa2, 0x89, 0xd5, 0x1b, 0x6a, + 0xa3, 0x7d, 0xd9, 0x2a, 0x4f, 0xea, 0x57, 0x88, 0x26, 0x7c, 0x30, 0x50, 0x51, 0x48, 0xa4, 0x2f, + 0x90, 0x16, 0x2a, 0x0a, 0x11, 0xfa, 0xa8, 0x6c, 0x28, 0x23, 0x25, 0xae, 0x3a, 0x6e, 0x0e, 0xb5, + 0x91, 0xee, 0x1e, 0x2e, 0x9f, 0x06, 0x7d, 0xde, 0xaa, 0x19, 0x8f, 0xc9, 0xdc, 0xa4, 0x77, 0x4d, + 0x80, 0x03, 0xd0, 0x09, 0x10, 0x5d, 0x6d, 0x3d, 0xe0, 0x5b, 0x3d, 0xc0, 0x25, 0x05, 0x7c, 0x00, + 0xfb, 0x14, 0x7f, 0x5f, 0xe0, 0x3c, 0xc4, 0xbc, 0xc2, 0x16, 0x14, 0x89, 0x1f, 0x39, 0x6a, 0xf2, + 0x67, 0xb5, 0x98, 0xca, 0x75, 0x8d, 0x87, 0x2e, 0xe8, 0x95, 0x24, 0xcb, 0x78, 0x02, 0xaa, 0x74, + 0x87, 0xc2, 0xe1, 0xbf, 0xca, 0xc1, 0x93, 0xd1, 0xb5, 0x72, 0x75, 0xcb, 0xba, 0x08, 0x47, 0xc0, + 0x54, 0x5d, 0x44, 0x11, 0x2e, 0x65, 0x21, 0x8e, 0x44, 0x21, 0x7a, 0xb2, 0x6f, 0x5c, 0xe6, 0xf5, + 0x98, 0xea, 0x46, 0xcb, 0x34, 0xa6, 0xba, 0x61, 0x98, 0xed, 0xa9, 0x6e, 0x00, 0xb3, 0x33, 0xd5, + 0x8d, 0xae, 0xd9, 0x3b, 0x7b, 0x0f, 0xba, 0x6b, 0xdf, 0x81, 0x3d, 0xd0, 0x8c, 0x90, 0xb8, 0x9c, + 0x6d, 0xaf, 0x19, 0x21, 0x68, 0xbd, 0xdc, 0x58, 0x75, 0xf7, 0xd4, 0xd2, 0xbd, 0x7a, 0x58, 0xda, + 0xda, 0xe3, 0xd2, 0xd6, 0xfe, 0x2e, 0x6d, 0xed, 0xfe, 0xd9, 0x6e, 0x3c, 0x3e, 0xdb, 0x8d, 0xdf, + 0xcf, 0x76, 0xe3, 0x8b, 0x13, 0xa7, 0x2c, 0x59, 0x04, 0x4e, 0x48, 0xe6, 0xfc, 0x21, 0xc0, 0x39, + 0xe7, 0x7f, 0xdc, 0xfd, 0xac, 0x1e, 0x07, 0xf5, 0xc2, 0x04, 0x6a, 0x1d, 0xec, 0x89, 0xe9, 0x3f, + 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0x34, 0x50, 0x06, 0x1e, 0x54, 0x05, 0x00, 0x00, } func (m *State) Marshal() (dAtA []byte, err error) { @@ -394,11 +385,6 @@ func (m *State) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x72 } - if m.LastHeightConsensusParamsChanged != 0 { - i = encodeVarintState(dAtA, i, uint64(m.LastHeightConsensusParamsChanged)) - i-- - dAtA[i] = 0x68 - } { size, err := m.ConsensusParams.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -557,9 +543,6 @@ func (m *State) Size() (n int) { } l = m.ConsensusParams.Size() n += 1 + l + sovState(uint64(l)) - if m.LastHeightConsensusParamsChanged != 0 { - n += 1 + sovState(uint64(m.LastHeightConsensusParamsChanged)) - } l = len(m.LastResultsHash) if l > 0 { n += 1 + l + sovState(uint64(l)) @@ -897,25 +880,6 @@ func (m *State) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 13: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LastHeightConsensusParamsChanged", wireType) - } - m.LastHeightConsensusParamsChanged = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowState - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.LastHeightConsensusParamsChanged |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } case 14: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field LastResultsHash", wireType) diff --git a/types/serialization.go b/types/serialization.go index f3b1b21d9..6dcdb08ac 100644 --- a/types/serialization.go +++ b/types/serialization.go @@ -256,18 +256,17 @@ func (s *State) ToProto() (*pb.State, error) { } return &pb.State{ - Version: &s.Version, - ChainId: s.ChainID, - InitialHeight: int64(s.InitialHeight), - LastBlockHeight: int64(s.Height()), - SequencerSet: *seqsProto, - BaseHeight: s.BaseHeight, - ConsensusParams: s.ConsensusParams, - LastHeightConsensusParamsChanged: s.LastHeightConsensusParamsChanged, - LastResultsHash: s.LastResultsHash[:], - LastHeaderHash: s.LastHeaderHash[:], - AppHash: s.AppHash[:], - RollappParams: s.RollappParams, + Version: &s.Version, + ChainId: s.ChainID, + InitialHeight: int64(s.InitialHeight), + LastBlockHeight: int64(s.Height()), + SequencerSet: *seqsProto, + BaseHeight: s.BaseHeight, + ConsensusParams: s.ConsensusParams, + LastResultsHash: s.LastResultsHash[:], + LastHeaderHash: s.LastHeaderHash[:], + AppHash: s.AppHash[:], + RollappParams: s.RollappParams, }, nil } @@ -286,7 +285,6 @@ 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) diff --git a/types/serialization_test.go b/types/serialization_test.go index ca1218c2e..ff876cee7 100644 --- a/types/serialization_test.go +++ b/types/serialization_test.go @@ -139,9 +139,8 @@ func TestStateRoundTrip(t *testing.T) { Da: "mock", Version: version.Commit, }, - LastHeightConsensusParamsChanged: 12345, - LastResultsHash: [32]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2}, - AppHash: [32]byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1}, + LastResultsHash: [32]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2}, + AppHash: [32]byte{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1}, }, }, } diff --git a/types/state.go b/types/state.go index 1f4959a50..0bfd6d093 100644 --- a/types/state.go +++ b/types/state.go @@ -31,8 +31,7 @@ type State struct { // Consensus parameters used for validating blocks. // Changes returned by EndBlock and updated after Commit. - ConsensusParams tmproto.ConsensusParams - LastHeightConsensusParamsChanged int64 + ConsensusParams tmproto.ConsensusParams // Merkle root of the results from executing prev block LastResultsHash [32]byte